From 072c16d9d281e6075497af10c604c84c9ea8d1ea Mon Sep 17 00:00:00 2001 From: irfan sharif Date: Fri, 7 Apr 2023 00:51:23 -0400 Subject: [PATCH 01/12] kvserver: setup flow token returns using the RaftTransport This commit integrates the kvflowcontrol.Dispatch with the kvserver-level RaftTransport. When log entries are admitted below raft, we'll want to inform the origin nodes of this fact, effectively returning the flow tokens that were deducted when replicating the log entry to us. We repurpose the existing RaftTransport for this communication -- we piggyback these flow token returns[^1] on raft messages already bound to nodes we're returning tokens to. We also guarantee delivery of tokens in the presence of idle RaftTransport connections[^2]. We had to to introduce some protocol changes here. When a client establishes a RaftMessageRequestBatch stream, it sends along to the server the set of all StoreIDs it has. It's populated on the first RaftMessageRequestBatch sent along MultiRaft.RaftMessageBatch gRPC stream identifying at least one store, and then populated once more if any additional stores have been initialized[^3]. This data is used by the kvflowcontrol machinery to track the exact set of stores on the client node. It uses this information to react to the gRPC streams breaking. Since these streams are used to piggy information about which log entries were admitted below raft[^4] in order for the server-side to free up flow tokens, if the stream breaks we possibly risk leaking these tokens. So when these streams break, we use information about the client's stores to release all held tokens[^5]. We're not using this code just yet, which is just the below-raft integration with kvflowcontrol. The subsequent commit will introduce the above-raft integration where we'll actually deduct flow tokens at the sender, encode proposals using EntryEncoding{Standard,Sideloaded}WithAC, which in turn enqueues virtual work items in below-raft admission queues for asynchronous admission. Once asynchronously admitted, using the changes in this commit, we'll return flow tokens using the now-wired-up kvflowcontrol.Dispatch interface. --- Suggested reading order for reviewers: - (*RaftTransport).kvflowControl Brief comment block which tries to give a lay of the land. - flow_control_stores.go Integration interface+implementation that's going to be used by the RaftTransport to return flow tokens to the specific locally held kvflowcontrol.Handles, after learning about admitted raft log entries from remote nodes. It's implemented more fully in the subsequent commit. - flow_control_raft_transport.go Contains the set of new dependencies now used in the RaftTransport code for flow token purposes. It also includes the interfaces that show how the RaftTransport informs individual replicas that its no longer connected to specific (remote) stores. They're used more fully in the subsequent commit. - raft_transport.go The actual code changes to the RaftTransport. - flow_token_transport_test.go and flow_token_transport/* Datadriven test to understand how the various pieces fit together. - kvflowdispatch/* Adds some metrics and unit testing for the canonical kvflowcontrol.Dispatch implementation (previously implemented). --- [^1]: In the form of kvflowcontrolpb.AdmittedRaftLogEntries. [^2]: See kvserver.TestFlowTokenTransport. [^3]: This two-step process is because of how and when we allocate StoreIDs. Ignoring nodes that are bootstrapping the cluster (which just picks the initial set of StoreIDs -- see pkg/server.bootstrapCluster), whenever a new node is added, it's assigned a node ID and store ID by an existing node in CRDB (see kvpb.JoinNodeResponse). Subsequent store IDs, for multi-store nodes, are generated by the joining node by incrementing a sequence ID generator (see pkg/server.(*Node).initializeAdditionalStores). All of which is to say that the very first time we issue a RaftMessageRequestBatch, we might not have all the StoreIDs. But we will very shortly after, and certainly before any replicas get allocated to the additional store. [^4]: See kvflowcontrolpb.AdmittedRaftLogEntries and its use in RaftMessageRequest. [^5]: See I1 from kvflowcontrol/doc.go. Release note: None --- pkg/kv/kvserver/BUILD.bazel | 10 + pkg/kv/kvserver/client_merge_test.go | 5 + pkg/kv/kvserver/client_raft_test.go | 9 + .../kvserver/flow_control_raft_transport.go | 146 +++++ .../flow_control_raft_transport_test.go | 508 ++++++++++++++++++ pkg/kv/kvserver/flow_control_stores.go | 155 ++++++ pkg/kv/kvserver/kvadmission/BUILD.bazel | 1 + pkg/kv/kvserver/kvadmission/kvadmission.go | 49 ++ .../kvserver/kvflowcontrol/kvflowcontrol.go | 9 +- .../kvflowcontrol/kvflowdispatch/BUILD.bazel | 12 +- .../kvflowcontrol/kvflowdispatch/dummy.go | 40 ++ .../kvflowdispatch/kvflowdispatch.go | 75 ++- .../kvflowdispatch/kvflowdispatch_metrics.go | 102 ++++ .../kvflowdispatch/kvflowdispatch_test.go | 56 +- .../testdata/local_remote_dispatch | 28 + .../testdata/log_position_ordering | 19 + .../testdata/multiple_nodes_priorities_stores | 23 + .../kvflowdispatch/testdata/single_dispatch | 16 + pkg/kv/kvserver/kvserverpb/raft.proto | 29 + pkg/kv/kvserver/raft_transport.go | 475 +++++++++++++++- pkg/kv/kvserver/raft_transport_metrics.go | 9 + pkg/kv/kvserver/raft_transport_test.go | 71 ++- pkg/kv/kvserver/raft_transport_unit_test.go | 5 + pkg/kv/kvserver/store_test.go | 14 +- .../basic_piggyback | 40 ++ .../fallback_dispatch | 48 ++ .../idle_connection | 92 ++++ .../flow_control_raft_transport/store_data | 83 +++ pkg/roachpb/metadata.go | 25 + pkg/server/BUILD.bazel | 1 + pkg/server/init.go | 30 ++ pkg/server/server.go | 35 +- pkg/storage/engine.go | 3 + pkg/storage/pebble.go | 15 + .../lint/passes/redactcheck/redactcheck.go | 1 + pkg/util/admission/granter_test.go | 14 + pkg/util/admission/work_queue.go | 20 +- 37 files changed, 2232 insertions(+), 41 deletions(-) create mode 100644 pkg/kv/kvserver/flow_control_raft_transport.go create mode 100644 pkg/kv/kvserver/flow_control_raft_transport_test.go create mode 100644 pkg/kv/kvserver/flow_control_stores.go create mode 100644 pkg/kv/kvserver/kvflowcontrol/kvflowdispatch/dummy.go create mode 100644 pkg/kv/kvserver/kvflowcontrol/kvflowdispatch/kvflowdispatch_metrics.go create mode 100644 pkg/kv/kvserver/kvflowcontrol/kvflowdispatch/testdata/local_remote_dispatch create mode 100644 pkg/kv/kvserver/testdata/flow_control_raft_transport/basic_piggyback create mode 100644 pkg/kv/kvserver/testdata/flow_control_raft_transport/fallback_dispatch create mode 100644 pkg/kv/kvserver/testdata/flow_control_raft_transport/idle_connection create mode 100644 pkg/kv/kvserver/testdata/flow_control_raft_transport/store_data diff --git a/pkg/kv/kvserver/BUILD.bazel b/pkg/kv/kvserver/BUILD.bazel index f65dd9a9c596..2225938169b8 100644 --- a/pkg/kv/kvserver/BUILD.bazel +++ b/pkg/kv/kvserver/BUILD.bazel @@ -12,6 +12,8 @@ go_library( "consistency_queue.go", "debug_print.go", "doc.go", + "flow_control_raft_transport.go", + "flow_control_stores.go", "lease_history.go", "markers.go", "merge_queue.go", @@ -136,6 +138,9 @@ go_library( "//pkg/kv/kvserver/idalloc", "//pkg/kv/kvserver/intentresolver", "//pkg/kv/kvserver/kvadmission", + "//pkg/kv/kvserver/kvflowcontrol", + "//pkg/kv/kvserver/kvflowcontrol/kvflowcontrolpb", + "//pkg/kv/kvserver/kvflowcontrol/kvflowdispatch", "//pkg/kv/kvserver/kvserverbase", "//pkg/kv/kvserver/kvserverpb", "//pkg/kv/kvserver/kvstorage", @@ -264,6 +269,7 @@ go_test( "consistency_queue_test.go", "debug_print_test.go", "errors_test.go", + "flow_control_raft_transport_test.go", "gossip_test.go", "helpers_test.go", "intent_resolver_integration_test.go", @@ -378,6 +384,10 @@ go_test( "//pkg/kv/kvserver/concurrency/lock", "//pkg/kv/kvserver/gc", "//pkg/kv/kvserver/intentresolver", + "//pkg/kv/kvserver/kvadmission", + "//pkg/kv/kvserver/kvflowcontrol", + "//pkg/kv/kvserver/kvflowcontrol/kvflowcontrolpb", + "//pkg/kv/kvserver/kvflowcontrol/kvflowdispatch", "//pkg/kv/kvserver/kvserverbase", "//pkg/kv/kvserver/kvserverpb", "//pkg/kv/kvserver/kvstorage", diff --git a/pkg/kv/kvserver/client_merge_test.go b/pkg/kv/kvserver/client_merge_test.go index cd74c2fcc228..b8d70ba85d1d 100644 --- a/pkg/kv/kvserver/client_merge_test.go +++ b/pkg/kv/kvserver/client_merge_test.go @@ -36,6 +36,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/isolation" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/lock" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol/kvflowdispatch" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/rditer" @@ -2474,6 +2475,10 @@ func TestStoreReplicaGCAfterMerge(t *testing.T) { nodedialer.New(tc.Servers[0].RPCContext(), gossip.AddressResolver(tc.Servers[0].Gossip())), nil, /* grpcServer */ tc.Servers[0].Stopper(), + kvflowdispatch.NewDummyDispatch(), + kvserver.NoopStoresFlowControlIntegration{}, + kvserver.NoopRaftTransportDisconnectListener{}, + nil, /* knobs */ ) errChan := errorChannelTestHandler(make(chan *kvpb.Error, 1)) transport.Listen(store0.StoreID(), errChan) diff --git a/pkg/kv/kvserver/client_raft_test.go b/pkg/kv/kvserver/client_raft_test.go index 2e8b1aba3595..72fdbe5b30b1 100644 --- a/pkg/kv/kvserver/client_raft_test.go +++ b/pkg/kv/kvserver/client_raft_test.go @@ -30,6 +30,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol/kvflowdispatch" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness" @@ -3199,6 +3200,10 @@ func TestReplicaGCRace(t *testing.T) { nodedialer.New(tc.Servers[0].RPCContext(), gossip.AddressResolver(fromStore.Gossip())), nil, /* grpcServer */ tc.Servers[0].Stopper(), + kvflowdispatch.NewDummyDispatch(), + kvserver.NoopStoresFlowControlIntegration{}, + kvserver.NoopRaftTransportDisconnectListener{}, + nil, /* knobs */ ) errChan := errorChannelTestHandler(make(chan *kvpb.Error, 1)) fromTransport.Listen(fromStore.StoreID(), errChan) @@ -3698,6 +3703,10 @@ func TestReplicateRemovedNodeDisruptiveElection(t *testing.T) { gossip.AddressResolver(tc.GetFirstStoreFromServer(t, 0).Gossip())), nil, /* grpcServer */ tc.Servers[0].Stopper(), + kvflowdispatch.NewDummyDispatch(), + kvserver.NoopStoresFlowControlIntegration{}, + kvserver.NoopRaftTransportDisconnectListener{}, + nil, /* knobs */ ) errChan := errorChannelTestHandler(make(chan *kvpb.Error, 1)) transport0.Listen(target0.StoreID, errChan) diff --git a/pkg/kv/kvserver/flow_control_raft_transport.go b/pkg/kv/kvserver/flow_control_raft_transport.go new file mode 100644 index 000000000000..f4535aa10f66 --- /dev/null +++ b/pkg/kv/kvserver/flow_control_raft_transport.go @@ -0,0 +1,146 @@ +// Copyright 2023 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package kvserver + +import ( + "context" + "fmt" + "sort" + "strings" + + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/rpc" +) + +// raftTransportForFlowControl abstracts the node-level raft transport, and is +// used by the canonical replicaFlowControlIntegration implementation. It +// exposes the set of (remote) stores the raft transport is connected to. If the +// underlying gRPC streams break and don't reconnect, this indicates as much. +// Ditto if they're reconnected to. Also see RaftTransportDisconnectListener, +// which is used to observe every instance of gRPC streams breaking. +type raftTransportForFlowControl interface { + isConnectedTo(storeID roachpb.StoreID) bool +} + +var _ raftTransportForFlowControl = &RaftTransport{} + +// isConnectedTo implements the raftTransportForFlowControl interface. +func (r *RaftTransport) isConnectedTo(storeID roachpb.StoreID) bool { + r.kvflowControl.mu.RLock() + defer r.kvflowControl.mu.RUnlock() + return r.kvflowControl.mu.connectionTracker.isStoreConnected(storeID) +} + +// RaftTransportDisconnectListener observes every instance of the raft +// transport disconnecting replication traffic to the given (remote) stores. +type RaftTransportDisconnectListener interface { + OnRaftTransportDisconnected(context.Context, ...roachpb.StoreID) +} + +// connectionTrackerForFlowControl tracks the set of client-side stores and +// server-side nodes the raft transport is connected to. The "client" and +// "server" refer to the client and server side of the RaftTransport stream (see +// flow_control_integration.go). Client-side stores return tokens, it's where +// work gets admitted. Server-side nodes are where tokens were originally +// deducted. +type connectionTrackerForFlowControl struct { + stores map[roachpb.StoreID]struct{} + nodes map[roachpb.NodeID]map[rpc.ConnectionClass]struct{} +} + +func newConnectionTrackerForFlowControl() *connectionTrackerForFlowControl { + return &connectionTrackerForFlowControl{ + stores: make(map[roachpb.StoreID]struct{}), + nodes: make(map[roachpb.NodeID]map[rpc.ConnectionClass]struct{}), + } +} + +// isStoreConnected returns whether we're connected to the given (client-side) +// store. +func (c *connectionTrackerForFlowControl) isStoreConnected(storeID roachpb.StoreID) bool { + _, found := c.stores[storeID] + return found +} + +// markStoresConnected is used to inform the tracker that we've received +// raft messages from nodes with the given set of stores. +func (c *connectionTrackerForFlowControl) markStoresConnected(storeIDs []roachpb.StoreID) { + for _, storeID := range storeIDs { + c.stores[storeID] = struct{}{} + } +} + +// markStoresDisconnected marks the given set of stores as disconnected. +func (c *connectionTrackerForFlowControl) markStoresDisconnected(storeIDs []roachpb.StoreID) { + for _, storeID := range storeIDs { + delete(c.stores, storeID) + } +} + +// isNodeConnected returns whether we're connected to the given (server-side) +// node, independent of the specific RPC connection class. +func (q *connectionTrackerForFlowControl) isNodeConnected(nodeID roachpb.NodeID) bool { + _, found := q.nodes[nodeID] + return found +} + +// markNodeConnected informs the tracker that we're connected to the given +// node using the given RPC connection class. +func (q *connectionTrackerForFlowControl) markNodeConnected( + nodeID roachpb.NodeID, class rpc.ConnectionClass, +) { + if len(q.nodes[nodeID]) == 0 { + q.nodes[nodeID] = map[rpc.ConnectionClass]struct{}{} + } + q.nodes[nodeID][class] = struct{}{} +} + +// markNodeDisconnected informs the tracker that a previous connection to +// the given node along the given RPC connection class is now broken. +func (q *connectionTrackerForFlowControl) markNodeDisconnected( + nodeID roachpb.NodeID, class rpc.ConnectionClass, +) { + delete(q.nodes[nodeID], class) + if len(q.nodes[nodeID]) == 0 { + delete(q.nodes, nodeID) + } +} + +func (c *connectionTrackerForFlowControl) testingPrint() string { + storeIDs := make([]roachpb.StoreID, 0, len(c.stores)) + nodeIDs := make([]roachpb.NodeID, 0, len(c.nodes)) + for storeID := range c.stores { + storeIDs = append(storeIDs, storeID) + } + for nodeID := range c.nodes { + nodeIDs = append(nodeIDs, nodeID) + } + + var buf strings.Builder + sort.Sort(roachpb.StoreIDSlice(storeIDs)) + sort.Sort(roachpb.NodeIDSlice(nodeIDs)) + buf.WriteString(fmt.Sprintf("connected-stores (server POV): %s\n", roachpb.StoreIDSlice(storeIDs))) + buf.WriteString(fmt.Sprintf("connected-nodes (client POV): %s\n", roachpb.NodeIDSlice(nodeIDs))) + return buf.String() +} + +// NoopRaftTransportDisconnectListener is a no-op implementation of the +// RaftTransportDisconnectListener interface. +type NoopRaftTransportDisconnectListener struct{} + +var _ RaftTransportDisconnectListener = NoopRaftTransportDisconnectListener{} + +// OnRaftTransportDisconnected implements the RaftTransportDisconnectListener +// interface. +func (n NoopRaftTransportDisconnectListener) OnRaftTransportDisconnected( + ctx context.Context, storeIDs ...roachpb.StoreID, +) { +} diff --git a/pkg/kv/kvserver/flow_control_raft_transport_test.go b/pkg/kv/kvserver/flow_control_raft_transport_test.go new file mode 100644 index 000000000000..c06720d372ca --- /dev/null +++ b/pkg/kv/kvserver/flow_control_raft_transport_test.go @@ -0,0 +1,508 @@ +// Copyright 2023 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package kvserver_test + +import ( + "context" + "fmt" + "sort" + "strconv" + "strings" + "testing" + "time" + + "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvadmission" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol/kvflowcontrolpb" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol/kvflowdispatch" + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/rpc" + "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/testutils" + "github.com/cockroachdb/cockroach/pkg/testutils/datapathutils" + "github.com/cockroachdb/cockroach/pkg/util" + "github.com/cockroachdb/cockroach/pkg/util/admission/admissionpb" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/cockroach/pkg/util/metric" + "github.com/cockroachdb/cockroach/pkg/util/syncutil" + "github.com/cockroachdb/datadriven" + "github.com/cockroachdb/errors" + "github.com/stretchr/testify/require" + "go.etcd.io/raft/v3/raftpb" +) + +// TestFlowControlRaftTransport tests the integration of flow tokens and the +// RaftTransport. It offers the following commands: +// +// - "init" +// Initializes the raft transport test harness. +// +// - "add" node=n store=s +// Add a transport link on the given node, and registers a given store on +// said node. +// +// - "send" range=r from=n/s/ to=n/s/ commit= +// Send a raft message with the given commit index for the named range, from +// the given node+store+replica to the given node+store+replica. +// +// - "dispatch" from=n +// node=n store=s range=r pri= up-to-log-position=/ +// ... +// Dispatch flow tokens (identified by a log prefix) from a given node to +// the named node+store, for the specific range and priority. +// +// - "client-mark-idle" from=n to=n +// Mark the transport connection between two nodes as idle. This is +// equivalent to the underlying RaftTransport stream being torn down. +// Specifically, the client initiated stream from the first node towards the +// second. So this is a unidirectional breakage. Of course the server side +// can observe it, and various tests showcase this fact. +// +// - "fallback-dispatch" from=n +// Dispatch all pending flow tokens from the named node. This is equivalent +// to the periodic, fallback dispatch the transport does to guarantee +// delivery even if streams are idle or no messages are being sent to +// piggyback on top of. +// +// - "drop-disconnected-tokens" from=n +// Drop tokens that are pending delivery to nodes we're disconnected from. +// This is equivalent to the periodic memory reclamation the transport does +// to ensure we don't accumulate dispatches unboundedly. +// +// - "set-initial-store-ids" from=n stores=s[,s]* +// Inform the raft transport on the given node of its initial set of store +// IDs. This is unrelated to the "add" command above - it's only used to +// test whether we ship over the set of right store IDs over the raft +// transport, as needed by the flow token protocol. +// +// - "set-additional-store-ids" from=n stores=s[,s]* +// Inform the raft transport on the given node of its additional set of +// store IDs. This is unrelated to the "add" command above - it's only used +// to test whether we ship over the set of right store IDs over the raft +// transport, as needed by the flow token protocol. +// +// - "connection-tracker" from=n +// Print out the exact nodes and stores the RaftTransport on the given node +// is connected to, as a server and as a client. +// +// - "disconnect-listener" from=n +// Print out the exact points at which the RaftTransportDisconnectListener +// was invoked by the RaftTransport on the given node. +// +// - "pending-dispatch" from=n to=n +// List the pending dispatches from one node to another. +// +// - "metrics" +// Print out transport metrics. +func TestFlowControlRaftTransport(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + ctx := context.Background() + st := cluster.MakeTestingClusterSettings() + kvadmission.FlowTokenDispatchInterval.Override(ctx, &st.SV, time.Hour) // we control this manually below + + datadriven.Walk(t, datapathutils.TestDataPath(t, "flow_control_raft_transport"), + func(t *testing.T, path string) { + var rttc *raftTransportTestContext + defer func() { + if rttc != nil { + rttc.Stop() + } + }() + + controlM := make(map[roachpb.NodeID]*transportControl) + datadriven.RunTest(t, path, + func(t *testing.T, d *datadriven.TestData) string { + switch d.Cmd { + case "init": + if rttc != nil { + rttc.Stop() + } + rttc = newRaftTransportTestContext(t, st) + return "" + + case "add": + nodeID := parseNodeID(t, d, "node") + storeID := parseStoreID(t, d, "store") + workerTeardownCh := make(chan roachpb.NodeID, 1) + controlM[nodeID] = &transportControl{ + dispatch: kvflowdispatch.New(metric.NewRegistry(), nil, &base.NodeIDContainer{}), + disconnectListener: &mockRaftTransportDisconnectListener{buf: &builderWithMu{}}, + workerTeardownCh: workerTeardownCh, + } + controlM[nodeID].knobs = &kvserver.RaftTransportTestingKnobs{ + TriggerFallbackDispatchCh: make(chan time.Time), + OnFallbackDispatch: func() { + controlM[nodeID].triggeredFallbackDispatch.Set(true) + }, + MarkSendQueueAsIdleCh: make(chan roachpb.NodeID), + OnWorkerTeardown: func(nodeID roachpb.NodeID) { + workerTeardownCh <- nodeID + }, + OnServerStreamDisconnected: func() { + controlM[nodeID].serverStreamDisconnected.Set(true) + }, + } + + transport, addr := rttc.AddNodeWithoutGossip( + nodeID, + util.TestAddr, + rttc.stopper, + controlM[nodeID].dispatch, + kvserver.NoopStoresFlowControlIntegration{}, + controlM[nodeID].disconnectListener, + controlM[nodeID].knobs, + ) + rttc.GossipNode(nodeID, addr) + controlM[nodeID].chanServer = rttc.ListenStore(nodeID, storeID) + require.NoError(t, transport.Start(ctx)) + return "" + + case "send": + // Parse range=r. + var arg string + d.ScanArgs(t, "range", &arg) + ri, err := strconv.Atoi(strings.TrimPrefix(arg, "r")) + require.NoError(t, err) + rangeID := roachpb.RangeID(ri) + + from := parseReplicaDescriptor(t, d, "from") // parse from=n/s/ + to := parseReplicaDescriptor(t, d, "to") // parse to=n/s/ + + // Parse commit=. + d.ScanArgs(t, "commit", &arg) + c, err := strconv.Atoi(arg) + require.NoError(t, err) + + testutils.SucceedsSoon(t, func() error { + if !rttc.Send(from, to, rangeID, raftpb.Message{Commit: uint64(c)}) { + breaker, ok := rttc.transports[from.NodeID].GetCircuitBreaker(to.NodeID, rpc.DefaultClass) + require.True(t, ok) + breaker.Reset() + } + select { + case req := <-controlM[to.NodeID].chanServer.ch: + if req.Message.Commit == uint64(c) { + return nil + } + case <-time.After(time.Second): + } + return errors.Errorf("expected message commit=%d", c) + }) + return "" + + case "dispatch": // cargo-culted from kvflowdispatch.TestDispatch + // Parse node=n. + fromNodeID := parseNodeID(t, d, "from") + control, found := controlM[fromNodeID] + require.True(t, found, "uninitialized node, did you use 'add node=n%s'?", fromNodeID) + + for _, line := range strings.Split(d.Input, "\n") { + parts := strings.Fields(line) + require.Len(t, parts, 5, "expected form 'node=n store=s range=r pri= up-to-log-position=/'") + var ( + entries kvflowcontrolpb.AdmittedRaftLogEntries + nodeID roachpb.NodeID + ) + for i := range parts { + parts[i] = strings.TrimSpace(parts[i]) + inner := strings.Split(parts[i], "=") + require.Len(t, inner, 2) + arg := strings.TrimSpace(inner[1]) + + switch { + case strings.HasPrefix(parts[i], "node="): + // Parse node=n. + ni, err := strconv.Atoi(strings.TrimPrefix(arg, "n")) + require.NoError(t, err) + nodeID = roachpb.NodeID(ni) + + case strings.HasPrefix(parts[i], "store="): + // Parse store=s. + si, err := strconv.Atoi(strings.TrimPrefix(arg, "s")) + require.NoError(t, err) + entries.StoreID = roachpb.StoreID(si) + + case strings.HasPrefix(parts[i], "range="): + // Parse range=r. + ri, err := strconv.Atoi(strings.TrimPrefix(arg, "r")) + require.NoError(t, err) + entries.RangeID = roachpb.RangeID(ri) + + case strings.HasPrefix(parts[i], "pri="): + // Parse pri=. + pri, found := admissionpb.TestingReverseWorkPriorityDict[arg] + require.True(t, found) + entries.AdmissionPriority = int32(pri) + + case strings.HasPrefix(parts[i], "up-to-log-position="): + // Parse up-to-log-position=/. + entries.UpToRaftLogPosition = parseLogPosition(t, arg) + + default: + t.Fatalf("unrecognized prefix: %s", parts[i]) + } + } + control.dispatch.Dispatch(ctx, nodeID, entries) + } + return "" + + case "client-mark-idle": + fromNodeID := parseNodeID(t, d, "from") + toNodeID := parseNodeID(t, d, "to") + + control, found := controlM[fromNodeID] + require.True(t, found, "uninitialized node, did you use 'add node=n%s'?", fromNodeID) + select { + case control.knobs.MarkSendQueueAsIdleCh <- toNodeID: + case <-time.After(time.Second): + return "timed out" + } + select { + case gotNodeID := <-control.workerTeardownCh: + require.Equal(t, gotNodeID, toNodeID) + case <-time.After(time.Second): + return "timed out" + } + + toControl, found := controlM[toNodeID] + require.True(t, found, "uninitialized node, did you use 'add node=n%s'?", toNodeID) + testutils.SucceedsSoon(t, func() error { + if toControl.serverStreamDisconnected.Get() { + return nil + } + return errors.Errorf("waiting for server-side stream to disconnect") + }) + toControl.serverStreamDisconnected.Set(false) // reset + return "" + + case "drop-disconnected-tokens": + nodeID := parseNodeID(t, d, "from") + transport, found := rttc.transports[nodeID] + require.True(t, found, "uninitialized node, did you use 'add node=n%s'?", nodeID) + transport.TestingDropFlowTokensForDisconnectedNodes() + return "" + + case "set-initial-store-ids": + nodeID := parseNodeID(t, d, "from") + transport, found := rttc.transports[nodeID] + require.True(t, found, "uninitialized node, did you use 'add node=n%s'?", nodeID) + transport.SetInitialStoreIDs(parseStoreIDs(t, d, "stores")) + return "" + + case "set-additional-store-ids": + nodeID := parseNodeID(t, d, "from") + transport, found := rttc.transports[nodeID] + require.True(t, found, "uninitialized node, did you use 'add node=n%s'?", nodeID) + transport.SetAdditionalStoreIDs(parseStoreIDs(t, d, "stores")) + return "" + + case "connection-tracker": + fromNodeID := parseNodeID(t, d, "from") + transport, found := rttc.transports[fromNodeID] + require.True(t, found, "uninitialized node, did you use 'add node=n%s'?", fromNodeID) + return transport.TestingPrintFlowControlConnectionTracker() + + case "disconnect-listener": + fromNodeID := parseNodeID(t, d, "from") + control, found := controlM[fromNodeID] + require.True(t, found, "uninitialized node, did you use 'add node=n%s'?", fromNodeID) + return control.disconnectListener.buf.stringAndReset() + + case "fallback-dispatch": + fromNodeID := parseNodeID(t, d, "from") + control, found := controlM[fromNodeID] + require.True(t, found, "uninitialized node, did you use 'add node=n%s'?", fromNodeID) + select { + case control.knobs.TriggerFallbackDispatchCh <- time.Time{}: + case <-time.After(time.Second): + return "timed out" + } + testutils.SucceedsSoon(t, func() error { + if control.triggeredFallbackDispatch.Get() { + return nil + } + return errors.Errorf("waiting for fallback mechanism to activate") + }) + control.triggeredFallbackDispatch.Set(false) // reset + return "" + + case "pending-dispatch": // cargo-culted from kvflowdispatch.TestDispatch + fromNodeID := parseNodeID(t, d, "from") + toNodeID := parseNodeID(t, d, "to") + + control, found := controlM[fromNodeID] + require.True(t, found, "uninitialized node, did you use 'add node=n%s'?", fromNodeID) + + var buf strings.Builder + es := control.dispatch.PendingDispatchFor(toNodeID) + sort.Slice(es, func(i, j int) bool { // for determinism + if es[i].RangeID != es[j].RangeID { + return es[i].RangeID < es[j].RangeID + } + if es[i].StoreID != es[j].StoreID { + return es[i].StoreID < es[j].StoreID + } + if es[i].AdmissionPriority != es[j].AdmissionPriority { + return es[i].AdmissionPriority < es[j].AdmissionPriority + } + return es[i].UpToRaftLogPosition.Less(es[j].UpToRaftLogPosition) + }) + for i, entries := range es { + if i != 0 { + buf.WriteString("\n") + } + buf.WriteString( + fmt.Sprintf("range=r%d pri=%s store=s%d up-to-log-position=%s", + entries.RangeID, + admissionpb.WorkPriority(entries.AdmissionPriority), + entries.StoreID, + entries.UpToRaftLogPosition, + ), + ) + control.dispatch.Dispatch(ctx, toNodeID, entries) // re-add to dispatch + } + return buf.String() + + case "metrics": + var buf strings.Builder + var nodeIDs roachpb.NodeIDSlice + for nodeID := range rttc.transports { + nodeIDs = append(nodeIDs, nodeID) + } + sort.Sort(nodeIDs) + + for _, nodeID := range nodeIDs { + transport := rttc.transports[nodeID] + buf.WriteString(fmt.Sprintf("node=n%s: dispatches-dropped=%d\n", + nodeID, + transport.Metrics().FlowTokenDispatchesDropped.Count(), + )) + } + return buf.String() + + default: + return "unknown command" + } + }) + }, + ) +} + +type transportControl struct { + dispatch *kvflowdispatch.Dispatch + disconnectListener *mockRaftTransportDisconnectListener + knobs *kvserver.RaftTransportTestingKnobs + serverStreamDisconnected syncutil.AtomicBool + triggeredFallbackDispatch syncutil.AtomicBool + workerTeardownCh chan roachpb.NodeID + chanServer channelServer +} + +func parseLogPosition(t *testing.T, input string) kvflowcontrolpb.RaftLogPosition { + inner := strings.Split(input, "/") + require.Len(t, inner, 2) + term, err := strconv.Atoi(inner[0]) + require.NoError(t, err) + index, err := strconv.Atoi(inner[1]) + require.NoError(t, err) + return kvflowcontrolpb.RaftLogPosition{ + Term: uint64(term), + Index: uint64(index), + } +} + +func parseNodeID(t *testing.T, d *datadriven.TestData, key string) roachpb.NodeID { + var arg string + d.ScanArgs(t, key, &arg) + ni, err := strconv.Atoi(strings.TrimPrefix(arg, "n")) + require.NoError(t, err) + return roachpb.NodeID(ni) +} + +func parseStoreID(t *testing.T, d *datadriven.TestData, key string) roachpb.StoreID { + var arg string + d.ScanArgs(t, key, &arg) + si, err := strconv.Atoi(strings.TrimPrefix(arg, "s")) + require.NoError(t, err) + return roachpb.StoreID(si) +} + +func parseStoreIDs(t *testing.T, d *datadriven.TestData, key string) []roachpb.StoreID { + var arg string + d.ScanArgs(t, key, &arg) + var storeIDs []roachpb.StoreID + for _, part := range strings.Split(arg, ",") { + si, err := strconv.Atoi(strings.TrimPrefix(part, "s")) + require.NoError(t, err) + storeIDs = append(storeIDs, roachpb.StoreID(si)) + } + return storeIDs +} + +func parseReplicaDescriptor( + t *testing.T, d *datadriven.TestData, key string, +) roachpb.ReplicaDescriptor { + var arg string + var desc roachpb.ReplicaDescriptor + d.ScanArgs(t, key, &arg) + parts := strings.Split(arg, "/") + require.Len(t, parts, 3) + ni, err := strconv.Atoi(strings.TrimPrefix(parts[0], "n")) + require.NoError(t, err) + store, err := strconv.Atoi(strings.TrimPrefix(parts[1], "s")) + require.NoError(t, err) + repl, err := strconv.Atoi(parts[2]) + require.NoError(t, err) + + desc.NodeID = roachpb.NodeID(ni) + desc.StoreID = roachpb.StoreID(store) + desc.ReplicaID = roachpb.ReplicaID(repl) + return desc +} + +type builderWithMu struct { + mu syncutil.Mutex + buf strings.Builder +} + +func (b *builderWithMu) printf(format string, a ...interface{}) { + b.mu.Lock() + defer b.mu.Unlock() + if b.buf.Len() > 0 { + fmt.Fprintf(&b.buf, "\n") + } + fmt.Fprintf(&b.buf, format, a...) +} + +func (b *builderWithMu) stringAndReset() string { + b.mu.Lock() + defer b.mu.Unlock() + str := b.buf.String() + b.buf.Reset() + return str +} + +type mockRaftTransportDisconnectListener struct { + buf *builderWithMu +} + +var _ kvserver.RaftTransportDisconnectListener = &mockRaftTransportDisconnectListener{} + +// OnRaftTransportDisconnected implements the kvserver.StoresForFlowControl interface. +func (m *mockRaftTransportDisconnectListener) OnRaftTransportDisconnected( + ctx context.Context, storeIDs ...roachpb.StoreID, +) { + m.buf.printf("disconnected-from: %s", roachpb.StoreIDSlice(storeIDs)) +} diff --git a/pkg/kv/kvserver/flow_control_stores.go b/pkg/kv/kvserver/flow_control_stores.go new file mode 100644 index 000000000000..fe0d14ad69ac --- /dev/null +++ b/pkg/kv/kvserver/flow_control_stores.go @@ -0,0 +1,155 @@ +// Copyright 2023 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package kvserver + +import ( + "context" + + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol" + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/util/log" +) + +// StoresForFlowControl is used to integrate with replication flow control. It +// exposes the underlying kvflowcontrol.Handles and is informed of (remote) +// stores we're no longer connected via the raft transport. +type StoresForFlowControl interface { + kvflowcontrol.Handles + RaftTransportDisconnectListener +} + +// storesForFlowControl is a concrete implementation of the +// StoresForFlowControl interface, backed by a set of Stores. +type storesForFlowControl Stores + +var _ StoresForFlowControl = &storesForFlowControl{} + +// MakeStoresForFlowControl returns the canonical StoresForFlowControl +// implementation. +func MakeStoresForFlowControl(stores *Stores) StoresForFlowControl { + return (*storesForFlowControl)(stores) +} + +// Lookup is part of the StoresForFlowControl interface. +func (sh *storesForFlowControl) Lookup( + rangeID roachpb.RangeID, +) (handle kvflowcontrol.Handle, found bool) { + ls := (*Stores)(sh) + if err := ls.VisitStores(func(s *Store) error { + if h, ok := makeStoreForFlowControl(s).Lookup(rangeID); ok { + handle = h + found = true + } + return nil + }); err != nil { + ctx := ls.AnnotateCtx(context.Background()) + log.Errorf(ctx, "unexpected error: %s", err) + return nil, false + } + return handle, found +} + +// ResetStreams is part of the StoresForFlowControl interface. +func (sh *storesForFlowControl) ResetStreams(ctx context.Context) { + ls := (*Stores)(sh) + if err := ls.VisitStores(func(s *Store) error { + makeStoreForFlowControl(s).ResetStreams(ctx) + return nil + }); err != nil { + ctx = ls.AnnotateCtx(ctx) + log.Errorf(ctx, "unexpected error: %s", err) + } +} + +// OnRaftTransportDisconnected is part of the StoresForFlowControl +// interface. +func (sh *storesForFlowControl) OnRaftTransportDisconnected( + ctx context.Context, storeIDs ...roachpb.StoreID, +) { + ls := (*Stores)(sh) + if err := ls.VisitStores(func(s *Store) error { + makeStoreForFlowControl(s).OnRaftTransportDisconnected(ctx, storeIDs...) + return nil + }); err != nil { + ctx := ls.AnnotateCtx(context.Background()) + log.Errorf(ctx, "unexpected error: %s", err) + } +} + +// storeForFlowControl is a concrete implementation of the +// StoresForFlowControl interface, backed by a single Store. +type storeForFlowControl Store + +var _ StoresForFlowControl = &storeForFlowControl{} + +// makeStoreForFlowControl returns a new storeForFlowControl instance. +func makeStoreForFlowControl(store *Store) *storeForFlowControl { + return (*storeForFlowControl)(store) +} + +// Lookup is part of the StoresForFlowControl interface. +func (sh *storeForFlowControl) Lookup( + rangeID roachpb.RangeID, +) (_ kvflowcontrol.Handle, found bool) { + s := (*Store)(sh) + repl := s.GetReplicaIfExists(rangeID) + if repl == nil { + return nil, false + } + return nil, false // TODO(irfansharif): Fill this in. +} + +// ResetStreams is part of the StoresForFlowControl interface. +func (sh *storeForFlowControl) ResetStreams(ctx context.Context) { + s := (*Store)(sh) + s.VisitReplicas(func(r *Replica) (wantMore bool) { + // TODO(irfansharif): Fill this in. + return true + }) +} + +// OnRaftTransportDisconnected is part of the StoresForFlowControl +// interface. +func (sh *storeForFlowControl) OnRaftTransportDisconnected( + ctx context.Context, storeIDs ...roachpb.StoreID, +) { + s := (*Store)(sh) + s.mu.replicasByRangeID.Range(func(replica *Replica) { + // TODO(irfansharif): Fill this in. + }) +} + +// NoopStoresFlowControlIntegration is a no-op implementation of the +// StoresForFlowControl interface. +type NoopStoresFlowControlIntegration struct{} + +var _ StoresForFlowControl = NoopStoresFlowControlIntegration{} + +// Lookup is part of the StoresForFlowControl interface. +func (l NoopStoresFlowControlIntegration) Lookup(roachpb.RangeID) (kvflowcontrol.Handle, bool) { + return nil, false +} + +// ResetStreams is part of the StoresForFlowControl interface. +func (l NoopStoresFlowControlIntegration) ResetStreams(context.Context) { +} + +// Inspect is part of the StoresForFlowControl interface. +func (l NoopStoresFlowControlIntegration) Inspect() []roachpb.RangeID { + return nil +} + +// OnRaftTransportDisconnected is part of the RaftTransportDisconnectListener +// interface. +func (NoopStoresFlowControlIntegration) OnRaftTransportDisconnected( + context.Context, ...roachpb.StoreID, +) { +} diff --git a/pkg/kv/kvserver/kvadmission/BUILD.bazel b/pkg/kv/kvserver/kvadmission/BUILD.bazel index c0446fd040ed..175b864bfe01 100644 --- a/pkg/kv/kvserver/kvadmission/BUILD.bazel +++ b/pkg/kv/kvserver/kvadmission/BUILD.bazel @@ -8,6 +8,7 @@ go_library( visibility = ["//visibility:public"], deps = [ "//pkg/kv/kvpb", + "//pkg/kv/kvserver/kvflowcontrol", "//pkg/kv/kvserver/raftlog", "//pkg/roachpb", "//pkg/settings", diff --git a/pkg/kv/kvserver/kvadmission/kvadmission.go b/pkg/kv/kvserver/kvadmission/kvadmission.go index def2d71c9507..0ebb62537f20 100644 --- a/pkg/kv/kvserver/kvadmission/kvadmission.go +++ b/pkg/kv/kvserver/kvadmission/kvadmission.go @@ -19,6 +19,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/kv/kvpb" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/raftlog" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings" @@ -92,6 +93,40 @@ var ProvisionedBandwidth = settings.RegisterByteSizeSetting( "for each store. It can be over-ridden on a per-store basis using the --store flag", 0).WithPublic() +// FlowTokenDropInterval determines the frequency at which we check for pending +// flow token dispatches to nodes we're no longer connected to, in order to drop +// them. +var FlowTokenDropInterval = settings.RegisterDurationSetting( + settings.SystemOnly, + "kvadmission.flow_token.drop_interval", + "the interval at which the raft transport checks for pending flow token dispatches "+ + "to nodes we're no longer connected to, in order to drop them; set to 0 to disable the mechanism", + 30*time.Second, + settings.NonNegativeDuration, +) + +// FlowTokenDispatchInterval determines the frequency at which we check for +// pending flow token dispatches from idle connections, in order to deliver +// them. +var FlowTokenDispatchInterval = settings.RegisterDurationSetting( + settings.SystemOnly, + "kvadmission.flow_token.dispatch_interval", + "the interval at which the raft transport checks for pending flow token dispatches "+ + "from idle connections and delivers them", + time.Second, + settings.PositiveDuration, settings.NonNegativeDurationWithMaximum(time.Minute), +) + +// ConnectedStoreExpiration controls how long the RaftTransport layers considers +// a stream connected without it having observed any messages from it. +var ConnectedStoreExpiration = settings.RegisterDurationSetting( + settings.SystemOnly, + "kvadmission.raft_transport.connected_store_expiration", + "the interval at which the raft transport prunes its set of connected stores; set to 0 to disable the mechanism", + 5*time.Minute, + settings.NonNegativeDuration, +) + // Controller provides admission control for the KV layer. type Controller interface { // AdmitKVWork must be called before performing KV work. @@ -430,6 +465,20 @@ func (n *controllerImpl) AdmitRaftEntry( return } + if log.V(1) { + log.Infof(ctx, "decoded raft admission meta below-raft: pri=%s create-time=%d proposer=n%s receiver=[n?,s%s] tenant=t%d tokensā‰ˆ%d sideloaded=%t raft-entry=%d/%d", + admissionpb.WorkPriority(meta.AdmissionPriority), + meta.AdmissionCreateTime, + meta.AdmissionOriginNode, + storeID, + tenantID.ToUint64(), + kvflowcontrol.Tokens(len(entry.Data)), + typ.IsSideloaded(), + entry.Term, + entry.Index, + ) + } + storeAdmissionQ := n.storeGrantCoords.TryGetQueueForStore(int32(storeID)) if storeAdmissionQ == nil { log.Errorf(ctx, "unable to find queue for store: %s", storeID) diff --git a/pkg/kv/kvserver/kvflowcontrol/kvflowcontrol.go b/pkg/kv/kvserver/kvflowcontrol/kvflowcontrol.go index de6641205a58..1e95f1277c8a 100644 --- a/pkg/kv/kvserver/kvflowcontrol/kvflowcontrol.go +++ b/pkg/kv/kvserver/kvflowcontrol/kvflowcontrol.go @@ -144,6 +144,13 @@ type Handle interface { Close(context.Context) } +// Handles represent a set of flow control handles. Note that handles are +// typically held on replicas initiating replication traffic, so on a given node +// they're uniquely identified by their range ID. +type Handles interface { + Lookup(roachpb.RangeID) (Handle, bool) +} + // Dispatch is used (i) to dispatch information about admitted raft log entries // to specific nodes, and (ii) to read pending dispatches. type Dispatch interface { @@ -155,7 +162,7 @@ type Dispatch interface { // entries to specific nodes (typically where said entries originated, where // flow tokens were deducted and waiting to be returned). type DispatchWriter interface { - Dispatch(roachpb.NodeID, kvflowcontrolpb.AdmittedRaftLogEntries) + Dispatch(context.Context, roachpb.NodeID, kvflowcontrolpb.AdmittedRaftLogEntries) } // DispatchReader is used to read pending dispatches. It's used in the raft diff --git a/pkg/kv/kvserver/kvflowcontrol/kvflowdispatch/BUILD.bazel b/pkg/kv/kvserver/kvflowcontrol/kvflowdispatch/BUILD.bazel index a286451bb7e5..3d45b91914a8 100644 --- a/pkg/kv/kvserver/kvflowcontrol/kvflowdispatch/BUILD.bazel +++ b/pkg/kv/kvserver/kvflowcontrol/kvflowdispatch/BUILD.bazel @@ -3,14 +3,21 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") go_library( name = "kvflowdispatch", - srcs = ["kvflowdispatch.go"], + srcs = [ + "dummy.go", + "kvflowdispatch.go", + "kvflowdispatch_metrics.go", + ], importpath = "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol/kvflowdispatch", visibility = ["//visibility:public"], deps = [ + "//pkg/base", "//pkg/kv/kvserver/kvflowcontrol", "//pkg/kv/kvserver/kvflowcontrol/kvflowcontrolpb", "//pkg/roachpb", "//pkg/util/admission/admissionpb", + "//pkg/util/log", + "//pkg/util/metric", "//pkg/util/syncutil", ], ) @@ -22,12 +29,15 @@ go_test( data = glob(["testdata/**"]), embed = [":kvflowdispatch"], deps = [ + "//pkg/base", + "//pkg/kv/kvserver/kvflowcontrol", "//pkg/kv/kvserver/kvflowcontrol/kvflowcontrolpb", "//pkg/roachpb", "//pkg/testutils/datapathutils", "//pkg/util/admission/admissionpb", "//pkg/util/leaktest", "//pkg/util/log", + "//pkg/util/metric", "@com_github_cockroachdb_datadriven//:datadriven", "@com_github_stretchr_testify//require", ], diff --git a/pkg/kv/kvserver/kvflowcontrol/kvflowdispatch/dummy.go b/pkg/kv/kvserver/kvflowcontrol/kvflowdispatch/dummy.go new file mode 100644 index 000000000000..a41ed702ad78 --- /dev/null +++ b/pkg/kv/kvserver/kvflowcontrol/kvflowdispatch/dummy.go @@ -0,0 +1,40 @@ +// Copyright 2023 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package kvflowdispatch + +import ( + "context" + + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol/kvflowcontrolpb" + "github.com/cockroachdb/cockroach/pkg/roachpb" +) + +type dummy struct{} + +var _ kvflowcontrol.Dispatch = &dummy{} + +// NewDummyDispatch returns a dummy implementation of kvflowcontrol.Dispatch, +// for use in tests. +func NewDummyDispatch() kvflowcontrol.Dispatch { + return &dummy{} +} + +func (d *dummy) Dispatch(context.Context, roachpb.NodeID, kvflowcontrolpb.AdmittedRaftLogEntries) { +} + +func (d *dummy) PendingDispatch() []roachpb.NodeID { + return nil +} + +func (d *dummy) PendingDispatchFor(id roachpb.NodeID) []kvflowcontrolpb.AdmittedRaftLogEntries { + return nil +} diff --git a/pkg/kv/kvserver/kvflowcontrol/kvflowdispatch/kvflowdispatch.go b/pkg/kv/kvserver/kvflowcontrol/kvflowdispatch/kvflowdispatch.go index fddb450425cb..0a3576ceb285 100644 --- a/pkg/kv/kvserver/kvflowcontrol/kvflowdispatch/kvflowdispatch.go +++ b/pkg/kv/kvserver/kvflowcontrol/kvflowdispatch/kvflowdispatch.go @@ -11,10 +11,15 @@ package kvflowdispatch import ( + "context" + + "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol/kvflowcontrolpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/util/admission/admissionpb" + "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/cockroach/pkg/util/metric" "github.com/cockroachdb/cockroach/pkg/util/syncutil" ) @@ -27,6 +32,11 @@ type Dispatch struct { // outbox maintains pending dispatches on a per-node basis. outbox map[roachpb.NodeID]dispatches } + metrics *metrics + // handles is used to dispatch tokens locally. Remote token dispatches are + // driven by the RaftTransport. + handles kvflowcontrol.Handles + nodeID *base.NodeIDContainer } // dispatchKey is used to coalesce dispatches bound for a given node. If @@ -44,29 +54,77 @@ type dispatches map[dispatchKey]kvflowcontrolpb.RaftLogPosition var _ kvflowcontrol.Dispatch = &Dispatch{} // New constructs a new Dispatch. -func New() *Dispatch { - d := &Dispatch{} +func New( + registry *metric.Registry, handles kvflowcontrol.Handles, nodeID *base.NodeIDContainer, +) *Dispatch { + d := &Dispatch{ + handles: handles, + nodeID: nodeID, + } d.mu.outbox = make(map[roachpb.NodeID]dispatches) + d.metrics = newMetrics() + registry.AddMetricStruct(d.metrics) return d } // Dispatch is part of the kvflowcontrol.Dispatch interface. -func (d *Dispatch) Dispatch(nodeID roachpb.NodeID, entries kvflowcontrolpb.AdmittedRaftLogEntries) { +func (d *Dispatch) Dispatch( + ctx context.Context, nodeID roachpb.NodeID, entries kvflowcontrolpb.AdmittedRaftLogEntries, +) { + if log.V(1) { + log.Infof(ctx, "dispatching %s to n%s", entries, nodeID) + } + pri := admissionpb.WorkPriority(entries.AdmissionPriority) + wc := admissionpb.WorkClassFromPri(pri) + if nodeID == d.nodeID.Get() { // local fast-path + handle, found := d.handles.Lookup(entries.RangeID) + if found { + handle.ReturnTokensUpto( + ctx, + admissionpb.WorkPriority(entries.AdmissionPriority), + entries.UpToRaftLogPosition, kvflowcontrol.Stream{ + StoreID: entries.StoreID, + }) + } + // If we've not found the local kvflowcontrol.Handle, it's because the + // range leaseholder/leader has recently been moved elsewhere. It's ok + // to drop these tokens on the floor since we already returned it when + // moving the leaseholder/leader. + d.metrics.LocalDispatch[wc].Inc(1) + return + } + d.metrics.RemoteDispatch[wc].Inc(1) + d.mu.Lock() defer d.mu.Unlock() if _, ok := d.mu.outbox[nodeID]; !ok { d.mu.outbox[nodeID] = dispatches{} + d.metrics.PendingNodes.Inc(1) } dk := dispatchKey{ entries.RangeID, entries.StoreID, - admissionpb.WorkPriority(entries.AdmissionPriority), + pri, } + existing, found := d.mu.outbox[nodeID][dk] if !found || existing.Less(entries.UpToRaftLogPosition) { d.mu.outbox[nodeID][dk] = entries.UpToRaftLogPosition + + if !found { + d.metrics.PendingDispatches[wc].Inc(1) + } else { + // We're replacing an existing dispatch with one with a higher log + // position. Increment the coalesced metric. + d.metrics.CoalescedDispatches[wc].Inc(1) + } + } + if found && !existing.Less(entries.UpToRaftLogPosition) { + // We're dropping a dispatch given we already have a pending one with a + // higher log position. Increment the coalesced metric. + d.metrics.CoalescedDispatches[wc].Inc(1) } } @@ -101,7 +159,16 @@ func (d *Dispatch) PendingDispatchFor( AdmissionPriority: int32(key.WorkPriority), UpToRaftLogPosition: dispatch, }) + wc := admissionpb.WorkClassFromPri(key.WorkPriority) + d.metrics.PendingDispatches[wc].Dec(1) } + delete(d.mu.outbox, nodeID) + d.metrics.PendingNodes.Dec(1) return entries } + +// testingMetrics returns the underlying metrics struct for testing purposes. +func (d *Dispatch) testingMetrics() *metrics { + return d.metrics +} diff --git a/pkg/kv/kvserver/kvflowcontrol/kvflowdispatch/kvflowdispatch_metrics.go b/pkg/kv/kvserver/kvflowcontrol/kvflowdispatch/kvflowdispatch_metrics.go new file mode 100644 index 000000000000..f5b59086043b --- /dev/null +++ b/pkg/kv/kvserver/kvflowcontrol/kvflowdispatch/kvflowdispatch_metrics.go @@ -0,0 +1,102 @@ +// Copyright 2023 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package kvflowdispatch + +import ( + "fmt" + + "github.com/cockroachdb/cockroach/pkg/util/admission/admissionpb" + "github.com/cockroachdb/cockroach/pkg/util/metric" +) + +var ( + pendingDispatches = metric.Metadata{ + Name: "kvadmission.flow_token_dispatch.pending_%s", + Help: "Number of pending %s flow token dispatches", + Measurement: "Dispatches", + Unit: metric.Unit_COUNT, + } + + pendingNodes = metric.Metadata{ + Name: "kvadmission.flow_token_dispatch.pending_nodes", + Help: "Number of nodes pending flow token dispatches", + Measurement: "Nodes", + Unit: metric.Unit_COUNT, + } + + localDispatches = metric.Metadata{ + Name: "kvadmission.flow_token_dispatch.local_%s", + Help: "Number of local %s flow token dispatches", + Measurement: "Dispatches", + Unit: metric.Unit_COUNT, + } + + remoteDispatches = metric.Metadata{ + Name: "kvadmission.flow_token_dispatch.remote_%s", + Help: "Number of remote %s flow token dispatches", + Measurement: "Dispatches", + Unit: metric.Unit_COUNT, + } + + coalescedDispatches = metric.Metadata{ + Name: "kvadmission.flow_token_dispatch.coalesced_%s", + Help: "Number of coalesced %s flow token dispatches (where we're informing the sender of a higher log entry being admitted)", + Measurement: "Dispatches", + Unit: metric.Unit_COUNT, + } +) + +type metrics struct { + PendingDispatches [admissionpb.NumWorkClasses]*metric.Gauge + CoalescedDispatches [admissionpb.NumWorkClasses]*metric.Counter + LocalDispatch [admissionpb.NumWorkClasses]*metric.Counter + RemoteDispatch [admissionpb.NumWorkClasses]*metric.Counter + PendingNodes *metric.Gauge +} + +var _ metric.Struct = &metrics{} + +func newMetrics() *metrics { + var m metrics + for _, wc := range []admissionpb.WorkClass{ + admissionpb.RegularWorkClass, + admissionpb.ElasticWorkClass, + } { + wc := wc // copy loop variable + m.PendingDispatches[wc] = metric.NewGauge( + annotateMetricTemplateWithWorkClass(wc, pendingDispatches), + ) + m.LocalDispatch[wc] = metric.NewCounter( + annotateMetricTemplateWithWorkClass(wc, localDispatches), + ) + m.RemoteDispatch[wc] = metric.NewCounter( + annotateMetricTemplateWithWorkClass(wc, remoteDispatches), + ) + m.CoalescedDispatches[wc] = metric.NewCounter( + annotateMetricTemplateWithWorkClass(wc, coalescedDispatches), + ) + } + m.PendingNodes = metric.NewGauge(pendingNodes) + return &m +} + +func (m *metrics) MetricStruct() {} + +// annotateMetricTemplateWithWorkClass uses the given metric template to build +// one suitable for the specific work class. +func annotateMetricTemplateWithWorkClass( + wc admissionpb.WorkClass, tmpl metric.Metadata, +) metric.Metadata { + rv := tmpl + rv.Name = fmt.Sprintf(tmpl.Name, wc) + rv.Help = fmt.Sprintf(tmpl.Help, wc) + return rv +} diff --git a/pkg/kv/kvserver/kvflowcontrol/kvflowdispatch/kvflowdispatch_test.go b/pkg/kv/kvserver/kvflowcontrol/kvflowdispatch/kvflowdispatch_test.go index 6610f902ba15..909d30e5c86b 100644 --- a/pkg/kv/kvserver/kvflowcontrol/kvflowdispatch/kvflowdispatch_test.go +++ b/pkg/kv/kvserver/kvflowcontrol/kvflowdispatch/kvflowdispatch_test.go @@ -11,18 +11,22 @@ package kvflowdispatch import ( + "context" "fmt" "sort" "strconv" "strings" "testing" + "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol/kvflowcontrolpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/testutils/datapathutils" "github.com/cockroachdb/cockroach/pkg/util/admission/admissionpb" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/cockroach/pkg/util/metric" "github.com/cockroachdb/datadriven" "github.com/stretchr/testify/require" ) @@ -31,12 +35,23 @@ func TestDispatch(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) + ctx := context.Background() datadriven.Walk(t, datapathutils.TestDataPath(t), func(t *testing.T, path string) { var dispatch *Dispatch datadriven.RunTest(t, path, func(t *testing.T, d *datadriven.TestData) string { switch d.Cmd { case "init": - dispatch = New() + nodeIDContainer := &base.NodeIDContainer{} + if d.HasArg("node") { + // Parse node=n. + var arg string + d.ScanArgs(t, "node", &arg) + ni, err := strconv.Atoi(strings.TrimPrefix(arg, "n")) + require.NoError(t, err) + nodeID := roachpb.NodeID(ni) + nodeIDContainer.Set(ctx, nodeID) + } + dispatch = New(metric.NewRegistry(), dummyHandles{}, nodeIDContainer) return "" case "dispatch": @@ -89,7 +104,7 @@ func TestDispatch(t *testing.T) { t.Fatalf("unrecognized prefix: %s", parts[i]) } } - dispatch.Dispatch(nodeID, entries) + dispatch.Dispatch(ctx, nodeID, entries) } return "" @@ -143,6 +158,9 @@ func TestDispatch(t *testing.T) { } return buf.String() + case "metrics": + return printMetrics(dispatch) + default: return fmt.Sprintf("unknown command: %s", d.Cmd) } @@ -162,3 +180,37 @@ func parseLogPosition(t *testing.T, input string) kvflowcontrolpb.RaftLogPositio Index: uint64(index), } } + +func printMetrics(d *Dispatch) string { + metrics := d.testingMetrics() + var buf strings.Builder + buf.WriteString(fmt.Sprintf(`pending-nodes=%d +[regular] pending=%d coalesced=%d dispatch{local=%d remote=%d} +[elastic] pending=%d coalesced=%d dispatch{local=%d remote=%d} +`, + metrics.PendingNodes.Value(), + metrics.PendingDispatches[admissionpb.RegularWorkClass].Value(), + metrics.CoalescedDispatches[admissionpb.RegularWorkClass].Count(), + metrics.LocalDispatch[admissionpb.RegularWorkClass].Count(), + metrics.RemoteDispatch[admissionpb.RegularWorkClass].Count(), + metrics.PendingDispatches[admissionpb.ElasticWorkClass].Value(), + metrics.CoalescedDispatches[admissionpb.ElasticWorkClass].Count(), + metrics.LocalDispatch[admissionpb.ElasticWorkClass].Count(), + metrics.RemoteDispatch[admissionpb.ElasticWorkClass].Count(), + )) + return buf.String() +} + +type dummyHandles struct{} + +func (d dummyHandles) Lookup(id roachpb.RangeID) (kvflowcontrol.Handle, bool) { + return nil, false +} + +func (d dummyHandles) ResetStreams(ctx context.Context) {} + +func (d dummyHandles) Inspect() []roachpb.RangeID { + return nil +} + +var _ kvflowcontrol.Handles = dummyHandles{} diff --git a/pkg/kv/kvserver/kvflowcontrol/kvflowdispatch/testdata/local_remote_dispatch b/pkg/kv/kvserver/kvflowcontrol/kvflowdispatch/testdata/local_remote_dispatch new file mode 100644 index 000000000000..52bcf5848fe0 --- /dev/null +++ b/pkg/kv/kvserver/kvflowcontrol/kvflowdispatch/testdata/local_remote_dispatch @@ -0,0 +1,28 @@ +# Verify that the local dispatch fast-path works as expected. + +# Initialize the dispatch on n1. +init node=n1 +---- + +# Issue dispatches to n1 and n2. +dispatch +node=n1 range=r1 pri=normal-pri store=s1 up-to-log-position=4/20 +node=n2 range=r2 pri=normal-pri store=s2 up-to-log-position=4/20 +---- + +# Verify that the metrics shows only 1 pending node (the remote one, n2) and +# increments dispatch{local=...} metric appopriately. +metrics +---- +pending-nodes=1 +[regular] pending=1 coalesced=0 dispatch{local=1 remote=1} +[elastic] pending=0 coalesced=0 dispatch{local=0 remote=0} + +pending-dispatch +---- +node=n2 + +pending-dispatch-for node=n1 +---- + +# vim:ft=sh diff --git a/pkg/kv/kvserver/kvflowcontrol/kvflowdispatch/testdata/log_position_ordering b/pkg/kv/kvserver/kvflowcontrol/kvflowdispatch/testdata/log_position_ordering index 947588448e74..e65891823f52 100644 --- a/pkg/kv/kvserver/kvflowcontrol/kvflowdispatch/testdata/log_position_ordering +++ b/pkg/kv/kvserver/kvflowcontrol/kvflowdispatch/testdata/log_position_ordering @@ -9,15 +9,34 @@ node=n1 range=r1 pri=normal-pri store=s1 up-to-log-position=4/20 node=n1 range=r1 pri=normal-pri store=s1 up-to-log-position=5/20 ---- +# Verify that the metrics indicate one pending and one coalesced flow tokens +# dispatch for 4/20. +metrics +---- +pending-nodes=1 +[regular] pending=1 coalesced=1 dispatch{local=0 remote=2} +[elastic] pending=0 coalesced=0 dispatch{local=0 remote=0} + +# Read off pending dispatchs, observing only the one with the higher log +# position. pending-dispatch-for node=n1 ---- range=r1 pri=normal-pri store=s1 up-to-log-position=log-position=5/20 +# Do the same as above, but dispatching them out of order. The higher position +# still takes precedence. dispatch node=n1 range=r1 pri=normal-pri store=s1 up-to-log-position=6/20 node=n1 range=r1 pri=normal-pri store=s1 up-to-log-position=6/19 ---- +# Verify that the metrics increment the coalesced count for 6/19. +metrics +---- +pending-nodes=1 +[regular] pending=1 coalesced=2 dispatch{local=0 remote=4} +[elastic] pending=0 coalesced=0 dispatch{local=0 remote=0} + pending-dispatch-for node=n1 ---- range=r1 pri=normal-pri store=s1 up-to-log-position=log-position=6/20 diff --git a/pkg/kv/kvserver/kvflowcontrol/kvflowdispatch/testdata/multiple_nodes_priorities_stores b/pkg/kv/kvserver/kvflowcontrol/kvflowdispatch/testdata/multiple_nodes_priorities_stores index 629b4be08ca3..1902ac1f36bb 100644 --- a/pkg/kv/kvserver/kvflowcontrol/kvflowdispatch/testdata/multiple_nodes_priorities_stores +++ b/pkg/kv/kvserver/kvflowcontrol/kvflowdispatch/testdata/multiple_nodes_priorities_stores @@ -15,6 +15,14 @@ node=n2 range=r3 pri=normal-pri store=s3 up-to-log-position=5/21 node=n2 range=r3 pri=high-pri store=s3 up-to-log-position=5/22 ---- +# Verify that the metrics show 2 pending nodes with 4 pending flow tokens +# dispatches (nothing gets coalesced). +metrics +---- +pending-nodes=2 +[regular] pending=4 coalesced=0 dispatch{local=0 remote=4} +[elastic] pending=0 coalesced=0 dispatch{local=0 remote=0} + pending-dispatch ---- node=n1 @@ -24,6 +32,14 @@ pending-dispatch-for node=n1 ---- range=r1 pri=normal-pri store=s1 up-to-log-position=log-position=4/20 +# Verify that there's only 1 pending node left (n1 was cleared) and pending flow +# tokens dispatch count was reduced by 1. +metrics +---- +pending-nodes=1 +[regular] pending=3 coalesced=0 dispatch{local=0 remote=4} +[elastic] pending=0 coalesced=0 dispatch{local=0 remote=0} + pending-dispatch ---- node=n2 @@ -37,4 +53,11 @@ range=r3 pri=high-pri store=s3 up-to-log-position=log-position=5/22 pending-dispatch ---- +# Verify no pending nodes or flow tokens dispatches. +metrics +---- +pending-nodes=0 +[regular] pending=0 coalesced=0 dispatch{local=0 remote=4} +[elastic] pending=0 coalesced=0 dispatch{local=0 remote=0} + # vim:ft=sh diff --git a/pkg/kv/kvserver/kvflowcontrol/kvflowdispatch/testdata/single_dispatch b/pkg/kv/kvserver/kvflowcontrol/kvflowdispatch/testdata/single_dispatch index c74a021ecce3..3e482c0bc681 100644 --- a/pkg/kv/kvserver/kvflowcontrol/kvflowdispatch/testdata/single_dispatch +++ b/pkg/kv/kvserver/kvflowcontrol/kvflowdispatch/testdata/single_dispatch @@ -8,6 +8,14 @@ dispatch node=n1 range=r1 pri=normal-pri store=s1 up-to-log-position=4/20 ---- +# Verify that the metrics show 1 pending node, with 1 pending flow tokens +# dispatch. +metrics +---- +pending-nodes=1 +[regular] pending=1 coalesced=0 dispatch{local=0 remote=1} +[elastic] pending=0 coalesced=0 dispatch{local=0 remote=0} + pending-dispatch ---- node=n1 @@ -22,4 +30,12 @@ pending-dispatch pending-dispatch-for node=n1 ---- +# Verify that the metrics show 0 pending nodes, with 0 pending flow +# tokens dispatches. +metrics +---- +pending-nodes=0 +[regular] pending=0 coalesced=0 dispatch{local=0 remote=1} +[elastic] pending=0 coalesced=0 dispatch{local=0 remote=0} + # vim:ft=sh diff --git a/pkg/kv/kvserver/kvserverpb/raft.proto b/pkg/kv/kvserver/kvserverpb/raft.proto index 37104d621bfa..ac8030298d90 100644 --- a/pkg/kv/kvserver/kvserverpb/raft.proto +++ b/pkg/kv/kvserver/kvserverpb/raft.proto @@ -100,6 +100,35 @@ message RaftMessageRequest { message RaftMessageRequestBatch { repeated RaftMessageRequest requests = 1 [(gogoproto.nullable) = false]; + + // StoreIDs identifies all the stores on the client node. It's populated on + // the first RaftMessageRequestBatch sent along MultiRaft.RaftMessageBatch + // gRPC stream identifying at least one store, and the populated once more + // if any additional stores have been initialized[^1]. This data is used by + // the kvflowcontrol machinery to track the exact set of stores on the client + // node. It uses this information to react to the gRPC streams breaking. Since + // these streams are used to piggy information about which log entries were + // admitted below raft[^2] in order for the server-side to free up flow + // tokens, if the stream breaks we possibly risk leaking these tokens. So + // when these streams break, we use information about the client's stores to + // release all held tokens[^3]. + // + // [^1]: This two-step process is because of how and when we allocate + // StoreIDs. Ignoring nodes that are bootstrapping the cluster (which + // just picks the initial set of StoreIDs -- see + // pkg/server.bootstrapCluster), whenever a new node is added, it's + // assigned a node ID and store ID by an existing node in CRDB (see + // kvpb.JoinNodeResponse). Subsequent store IDs, for multi-store nodes, + // are generated by the joining node by incrementing a sequence ID + // generator (see pkg/server.(*Node).initializeAdditionalStores). All of + // which is to say that the very first time we issue a + // RaftMessageRequestBatch, we might not have all the StoreIDs. But we + // will very shortly after, and certainly before and replicas get + // allocated to the additional store. + // [^2]: See kvflowcontrolpb.AdmittedRaftLogEntries and its use in + // RaftMessageRequest. + // [^3]: See I1 from kvflowcontrol/doc.go. + repeated uint64 store_ids = 2 [(gogoproto.customname) = "StoreIDs", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.StoreID"]; } message RaftMessageResponseUnion { diff --git a/pkg/kv/kvserver/raft_transport.go b/pkg/kv/kvserver/raft_transport.go index 1e7f5d150bbf..7892290fcb68 100644 --- a/pkg/kv/kvserver/raft_transport.go +++ b/pkg/kv/kvserver/raft_transport.go @@ -18,8 +18,13 @@ import ( "time" "unsafe" + "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/allocator/storepool" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvadmission" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol/kvflowcontrolpb" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol/kvflowdispatch" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/rpc" @@ -27,6 +32,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/storage" + "github.com/cockroachdb/cockroach/pkg/util/admission/admissionpb" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/stop" "github.com/cockroachdb/cockroach/pkg/util/syncutil" @@ -149,14 +155,105 @@ type RaftTransport struct { stopper *stop.Stopper metrics *RaftTransportMetrics - queues [rpc.NumConnectionClasses]syncutil.IntMap // map[roachpb.NodeID]*raftSendQueue + // Queues maintains a map[roachpb.NodeID]*raftSendQueue on a per rpc-class + // level. When new raftSendQueues are instantiated, or old ones deleted, we + // also maintain kvflowControl.mu.connectedTracker. So writing to this field + // is done while holding kvflowControl.mu. + queues [rpc.NumConnectionClasses]syncutil.IntMap + dialer *nodedialer.Dialer handlers syncutil.IntMap // map[roachpb.StoreID]*RaftMessageHandler + + kvflowControl struct { + // Everything nested under this struct is used to return flow tokens + // from the receiver (where work was admitted) up to the sender (where + // work originated and tokens were deducted). There are few parts to the + // protocol, and can be experimented with using TestFlowControlRaftTransport + // and various TestFlowControl* tests in this package. We briefly sketch + // how the various pieces fit below, repeating some of what is described + // in kvflowcontrol/doc.go (which details about how/why we integrate + // with the RaftTransport so intimately). + // + // Background: Each CRDB node acts as both the "server" and the "client" + // of bidirectional RaftTransport streams. That is, raft messages from + // n1->n2 are sent from the client code on n1 to server code on n2. + // Responses to those raft messages are sent from the client code on n2 + // to server code in n1. + // + // - When flow tokens are deducted where the MsgApps originate, and + // after they're admitted below-raft, the remote nodes send back + // kvflowcontrolpb.AdmittedRaftLogEntries through the RaftMessageBatch + // stream. This happens on the client side, and we read from + // dispatchReader and either attach the protos to already outbound + // messages, or fire off one-off messages periodically if there's + // little raft activity but still tokens to be returned. + // + // - On the server side, we intercept every message and look for these + // protos, and inform the storesForFlowControl integration interface + // of this fact. + // + // - The server side maintains the set of client-side stores it's + // currently connected to, from the POV of these server side streams. + // Since tokens are returned to the server along these streams, when + // they disconnect, it's possible for us to be leaking tokens in + // transit. So it uses information about the set of client-side stores + // it's no longer connected to, to simply free up all held tokens. See + // uses of connectionTracker below and how the storesForFlowControl + // interface is informed about stores we're no longer connected to. + // + // - There's some complexity in how this set of connected stores is + // tracked. For one it's a "connected" set, not a "disconnected" + // one. It's hard to do the latter. Ignoring memory maintenance + // issues that come from nodes that are no longer part of the + // cluster, we have multiple RPC classes from the client side and + // clients are also free to establish/disconnect many streams + // concurrently without synchronization on their end. The server is + // not guaranteed to learn about these streams connecting or + // disconnecting in any particular order, so it's difficult to track + // directly by looking at connection/disconnection events alone + // whether we're connected somehow to a given client. Some form of + // heartbeat scheme comes to mind, which fits more naturally with + // tracking the set of "connected" stores. We can still react to + // streams disconnecting by clearing the relevant stores from the + // tracked set, relying on a subsequent heartbeat (from, say, a + // different stream/RPC class) to re-track that client+its store as + // connected. + // + // - How does the server learn about what stores are present on each + // client? When establishing the stream, the client simply sends this + // information over. If the client learns of newly added stores + // (possible during early startup), it sends it again. See uses of + // localStoreIDs and setAdditionalStoreIDs below. + // + // - Idle streams are periodically culled from the client side. It's + // possible that we cull a stream without having delivered all flow + // tokens to the sender (for example, if below-raft admission happens + // after the stream is culled). The server side detected these + // disconnected streams and releases tokens, but on the client side we + // don't want to accumulate to-be-delivered flow tokens unboundedly. + // So we track the set of servers we're connected to, across all RPC + // classes, and periodically just clear our outbox if there are + // dispatches bound for nodes we're simply not connected to, across + // all RPC classes. See uses of connectedNodes below. + mu struct { + syncutil.RWMutex + localStoreIDs []roachpb.StoreID // sent to servers to track client-side stores + connectionTracker *connectionTrackerForFlowControl + } + setAdditionalStoreIDs atomic.Bool + dispatchReader kvflowcontrol.DispatchReader + handles kvflowcontrol.Handles + disconnectListener RaftTransportDisconnectListener + } + + knobs *RaftTransportTestingKnobs } // raftSendQueue is a queue of outgoing RaftMessageRequest messages. type raftSendQueue struct { reqs chan *kvserverpb.RaftMessageRequest + // The specific node this queue is sending RaftMessageRequests to. + nodeID roachpb.NodeID // The number of bytes in flight. Must be updated *atomically* on sending and // receiving from the reqs channel. bytes atomic.Int64 @@ -169,7 +266,11 @@ func NewDummyRaftTransport(st *cluster.Settings, tracer *tracing.Tracer) *RaftTr return nil, errors.New("dummy resolver") } return NewRaftTransport(log.MakeTestingAmbientContext(tracer), st, tracer, - nodedialer.New(nil, resolver), nil, nil) + nodedialer.New(nil, resolver), nil, nil, + kvflowdispatch.NewDummyDispatch(), NoopStoresFlowControlIntegration{}, + NoopRaftTransportDisconnectListener{}, + nil, + ) } // NewRaftTransport creates a new RaftTransport. @@ -180,14 +281,27 @@ func NewRaftTransport( dialer *nodedialer.Dialer, grpcServer *grpc.Server, stopper *stop.Stopper, + kvflowTokenDispatch kvflowcontrol.DispatchReader, + kvflowHandles kvflowcontrol.Handles, + disconnectListener RaftTransportDisconnectListener, + knobs *RaftTransportTestingKnobs, ) *RaftTransport { + if knobs == nil { + knobs = &RaftTransportTestingKnobs{} + } t := &RaftTransport{ AmbientContext: ambient, st: st, tracer: tracer, stopper: stopper, dialer: dialer, + knobs: knobs, } + t.kvflowControl.dispatchReader = kvflowTokenDispatch + t.kvflowControl.handles = kvflowHandles + t.kvflowControl.disconnectListener = disconnectListener + t.kvflowControl.mu.connectionTracker = newConnectionTrackerForFlowControl() + t.initMetrics() if grpcServer != nil { RegisterMultiRaftServer(grpcServer, t) @@ -195,6 +309,40 @@ func NewRaftTransport( return t } +// Start various internal goroutines needed for the RaftTransport's internal +// functioning. +func (t *RaftTransport) Start(ctx context.Context) error { + if err := t.startDroppingFlowTokensForDisconnectedNodes(ctx); err != nil { + return errors.Wrapf(err, "failed to run flow token dispatch loop") + } + return nil +} + +// SetInitialStoreIDs informs the RaftTransport of the initial set of store +// IDs the local node starts off with. If it's a restarting node, restarted with +// no additional stores, this is just all the local store IDs. For nodes newly +// added to the cluster, it's just the first initialized store. If there are +// additional stores post-restart, or stores other than the first for a newly +// added node, they're provided using (*RaftTransport).SetAdditionalStoreIDs. +func (t *RaftTransport) SetInitialStoreIDs(storeIDs []roachpb.StoreID) { + t.kvflowControl.mu.Lock() + defer t.kvflowControl.mu.Unlock() + t.kvflowControl.mu.localStoreIDs = storeIDs +} + +// SetAdditionalStoreIDs informs the RaftTransport of any additional stores the +// local node starts off with. See commentary on SetInitialStoreIDs for more +// details. +func (t *RaftTransport) SetAdditionalStoreIDs(storeIDs []roachpb.StoreID) { + if len(storeIDs) == 0 { + return // nothing to do + } + t.kvflowControl.mu.Lock() + defer t.kvflowControl.mu.Unlock() + t.kvflowControl.mu.localStoreIDs = append(t.kvflowControl.mu.localStoreIDs, storeIDs...) + t.kvflowControl.setAdditionalStoreIDs.Store(true) +} + // Metrics returns metrics tracking this transport. func (t *RaftTransport) Metrics() *RaftTransportMetrics { return t.metrics @@ -235,6 +383,33 @@ func (t *RaftTransport) getHandler(storeID roachpb.StoreID) (RaftMessageHandler, func (t *RaftTransport) handleRaftRequest( ctx context.Context, req *kvserverpb.RaftMessageRequest, respStream RaftMessageResponseStream, ) *kvpb.Error { + for i := range req.AdmittedRaftLogEntries { + // Process any flow tokens that were returned over the RaftTransport. Do + // this first thing, before these requests enter the receive queues + // which could drop them if full and cause token leaks, or bail after + // processing the raft heartbeats. See I8 from kvflowcontrol/doc.go. + admittedEntries := req.AdmittedRaftLogEntries[i] + handle, found := t.kvflowControl.handles.Lookup(admittedEntries.RangeID) + if found { + handle.ReturnTokensUpto( + ctx, + admissionpb.WorkPriority(admittedEntries.AdmissionPriority), + admittedEntries.UpToRaftLogPosition, + kvflowcontrol.Stream{StoreID: admittedEntries.StoreID}, + ) + } + + if log.V(1) { + log.Infof(ctx, "informed of below-raft %s", admittedEntries) + } + } + if req.ToReplica.StoreID == roachpb.StoreID(0) && len(req.AdmittedRaftLogEntries) > 0 { + // The fallback token dispatch mechanism does not specify a destination + // replica, and as such, there's no handler for it. We don't want to + // return StoreNotFoundErrors in such cases. + return nil + } + handler, ok := t.getHandler(req.ToReplica.StoreID) if !ok { log.Warningf(ctx, "unable to accept Raft message from %+v: no handler registered for %+v", @@ -263,12 +438,26 @@ func newRaftMessageResponse( } // RaftMessageBatch proxies the incoming requests to the listening server interface. -func (t *RaftTransport) RaftMessageBatch(stream MultiRaft_RaftMessageBatchServer) error { +func (t *RaftTransport) RaftMessageBatch(stream MultiRaft_RaftMessageBatchServer) (lastErr error) { errCh := make(chan error, 1) // Node stopping error is caught below in the select. taskCtx, cancel := t.stopper.WithCancelOnQuiesce(stream.Context()) + taskCtx = t.AnnotateCtx(taskCtx) defer cancel() + + var storeIDs []roachpb.StoreID + defer func() { + ctx := t.AnnotateCtx(context.Background()) + t.kvflowControl.mu.Lock() + t.kvflowControl.mu.connectionTracker.markStoresDisconnected(storeIDs) + t.kvflowControl.mu.Unlock() + t.kvflowControl.disconnectListener.OnRaftTransportDisconnected(ctx, storeIDs...) + if fn := t.knobs.OnServerStreamDisconnected; fn != nil { + fn() + } + }() + if err := t.stopper.RunAsyncTaskEx( taskCtx, stop.TaskOpts{ @@ -282,10 +471,18 @@ func (t *RaftTransport) RaftMessageBatch(stream MultiRaft_RaftMessageBatchServer if err != nil { return err } + if len(batch.StoreIDs) > 0 { + // Collect the set of store IDs from the client side to + // later free up relevant flow tokens once the gRPC + // stream breaks/disconnects. + storeIDs = batch.StoreIDs + } + t.kvflowControl.mu.Lock() + t.kvflowControl.mu.connectionTracker.markStoresConnected(storeIDs) + t.kvflowControl.mu.Unlock() if len(batch.Requests) == 0 { continue } - for i := range batch.Requests { req := &batch.Requests[i] t.metrics.MessagesRcvd.Inc(1) @@ -398,7 +595,7 @@ func (t *RaftTransport) Stop(storeID roachpb.StoreID) { // lost and a new instance of processQueue will be started by the next message // to be sent. func (t *RaftTransport) processQueue( - q *raftSendQueue, stream MultiRaft_RaftMessageBatchClient, + q *raftSendQueue, stream MultiRaft_RaftMessageBatchClient, class rpc.ConnectionClass, ) error { errCh := make(chan error, 1) @@ -429,25 +626,104 @@ func (t *RaftTransport) processQueue( return err } + maybeAnnotateWithAdmittedRaftLogEntries := func( + req *kvserverpb.RaftMessageRequest, + admitted []kvflowcontrolpb.AdmittedRaftLogEntries, + ) { + if len(admitted) == 0 { + return // nothing to do + } + req.AdmittedRaftLogEntries = append(req.AdmittedRaftLogEntries, admitted...) + flowTokenDispatchCount := len(req.AdmittedRaftLogEntries) + if log.V(2) && flowTokenDispatchCount > 0 { + for i, admittedEntries := range req.AdmittedRaftLogEntries { + log.Infof(ctx, "informing n%s of below-raft %s: %d out of %d dispatches", + q.nodeID, admittedEntries, + i+1, flowTokenDispatchCount, + ) + } + } + } + + var sentInitialStoreIDs, sentAdditionalStoreIDs bool + maybeAnnotateWithStoreIDs := func(batch *kvserverpb.RaftMessageRequestBatch) { + shouldSendAdditionalStoreIDs := t.kvflowControl.setAdditionalStoreIDs.Load() && !sentAdditionalStoreIDs + if !sentInitialStoreIDs || shouldSendAdditionalStoreIDs { + t.kvflowControl.mu.RLock() + batch.StoreIDs = nil + batch.StoreIDs = append(batch.StoreIDs, t.kvflowControl.mu.localStoreIDs...) + t.kvflowControl.mu.RUnlock() + // Unconditionally set sentInitialStoreIDs, since we always have + // the initial store IDs before the additional ones. + sentInitialStoreIDs = true + // Mark that we've sent the additional store IDs, to not need to + // re-send it again. + sentAdditionalStoreIDs = shouldSendAdditionalStoreIDs + log.VInfof(ctx, 1, "informing n%d of %d local store ID(s) (%s) over the raft transport[%s]", + q.nodeID, len(batch.StoreIDs), roachpb.StoreIDSlice(batch.StoreIDs), class) + } + } + + clearRequestBatch := func(batch *kvserverpb.RaftMessageRequestBatch) { + // Reuse the Requests slice, but zero out the contents to avoid delaying + // GC of memory referenced from within. + for i := range batch.Requests { + batch.Requests[i] = kvserverpb.RaftMessageRequest{} + } + batch.Requests = batch.Requests[:0] + batch.StoreIDs = nil + } + var raftIdleTimer timeutil.Timer defer raftIdleTimer.Stop() + + var dispatchPendingFlowTokensTimer timeutil.Timer + defer dispatchPendingFlowTokensTimer.Stop() + dispatchPendingFlowTokensTimer.Reset(kvadmission.FlowTokenDispatchInterval.Get(&t.st.SV)) + + dispatchPendingFlowTokensCh := dispatchPendingFlowTokensTimer.C + if t.knobs.TriggerFallbackDispatchCh != nil { + dispatchPendingFlowTokensCh = t.knobs.TriggerFallbackDispatchCh + } + batch := &kvserverpb.RaftMessageRequestBatch{} for { raftIdleTimer.Reset(raftIdleTimeout) + select { case <-t.stopper.ShouldQuiesce(): return nil + case <-raftIdleTimer.C: raftIdleTimer.Read = true return nil + case err := <-errCh: return err + case req := <-q.reqs: size := int64(req.Size()) q.bytes.Add(-size) budget := targetRaftOutgoingBatchSize.Get(&t.st.SV) - size + + // Piggyback any pending flow token dispatches on raft transport + // messages already bound for the remote node. If the stream + // over which we're returning these flow tokens breaks, this is + // detected by the remote node, where tokens were originally + // deducted, who then frees up all held tokens (see I1 from + // kvflowcontrol/doc.go). If the stream is culled because it's + // idle, that's deducted remotely using the same stream-break + // mechanism. If there are no open streams to a given node and + // there's still pending flow tokens, we'll drop those tokens to + // reclaim memory in dropFlowTokensForDisconnectedNodes. For + // idle-but-not-culled connections, we have a fallback timer to + // periodically transmit one-off RaftMessageRequests for timely + // token returns. + pendingDispatches := t.kvflowControl.dispatchReader.PendingDispatchFor(q.nodeID) + maybeAnnotateWithAdmittedRaftLogEntries(req, pendingDispatches) batch.Requests = append(batch.Requests, *req) releaseRaftMessageRequest(req) + // Pull off as many queued requests as possible, within reason. for budget > 0 { select { @@ -462,18 +738,54 @@ func (t *RaftTransport) processQueue( } } - err := stream.Send(batch) - if err != nil { + maybeAnnotateWithStoreIDs(batch) + if err := stream.Send(batch); err != nil { + t.metrics.FlowTokenDispatchesDropped.Inc(int64(len(pendingDispatches))) + return err + } + t.metrics.MessagesSent.Inc(int64(len(batch.Requests))) + clearRequestBatch(batch) + + case <-dispatchPendingFlowTokensCh: + dispatchPendingFlowTokensTimer.Read = true + dispatchPendingFlowTokensTimer.Reset(kvadmission.FlowTokenDispatchInterval.Get(&t.st.SV)) + + pendingDispatches := t.kvflowControl.dispatchReader.PendingDispatchFor(q.nodeID) + if len(pendingDispatches) == 0 { + continue // nothing to do + } + + // TODO(irfansharif): There's no limit on how many pending + // dispatches are going to be attached to the outgoing raft + // messages, both here and above. It can be excessive -- limit this + // by some count/byte policy as part of #95563. + req := newRaftMessageRequest() + maybeAnnotateWithAdmittedRaftLogEntries(req, pendingDispatches) + batch.Requests = append(batch.Requests, *req) + releaseRaftMessageRequest(req) + + maybeAnnotateWithStoreIDs(batch) + if err := stream.Send(batch); err != nil { + t.metrics.FlowTokenDispatchesDropped.Inc(int64(len(pendingDispatches))) return err } t.metrics.MessagesSent.Inc(int64(len(batch.Requests))) + clearRequestBatch(batch) - // Reuse the Requests slice, but zero out the contents to avoid delaying - // GC of memory referenced from within. - for i := range batch.Requests { - batch.Requests[i] = kvserverpb.RaftMessageRequest{} + if fn := t.knobs.OnFallbackDispatch; fn != nil { + fn() + } + + case gotNodeID := <-t.knobs.MarkSendQueueAsIdleCh: + if q.nodeID == gotNodeID { + return nil + } + select { + // Echo the node ID back into MarkSendQueueAsIdleCh until it reaches + // a raftSendQueue for this ID. Only used in tests. + case t.knobs.MarkSendQueueAsIdleCh <- gotNodeID: + default: } - batch.Requests = batch.Requests[:0] } } } @@ -486,8 +798,14 @@ func (t *RaftTransport) getQueue( queuesMap := &t.queues[class] value, ok := queuesMap.Load(int64(nodeID)) if !ok { - q := raftSendQueue{reqs: make(chan *kvserverpb.RaftMessageRequest, raftSendBufferSize)} + t.kvflowControl.mu.Lock() + q := raftSendQueue{ + reqs: make(chan *kvserverpb.RaftMessageRequest, raftSendBufferSize), + nodeID: nodeID, + } value, ok = queuesMap.LoadOrStore(int64(nodeID), unsafe.Pointer(&q)) + t.kvflowControl.mu.connectionTracker.markNodeConnected(nodeID, class) + t.kvflowControl.mu.Unlock() } return (*raftSendQueue)(value), ok } @@ -580,8 +898,19 @@ func (t *RaftTransport) startProcessNewQueue( if !existingQueue { log.Fatalf(ctx, "queue for n%d does not exist", toNodeID) } + defer func() { + if fn := t.knobs.OnWorkerTeardown; fn != nil { + fn(toNodeID) + } + }() defer cleanup(q) defer t.queues[class].Delete(int64(toNodeID)) + defer func() { + t.kvflowControl.mu.Lock() + t.queues[class].Delete(int64(toNodeID)) + t.kvflowControl.mu.connectionTracker.markNodeDisconnected(toNodeID, class) + t.kvflowControl.mu.Unlock() + }() conn, err := t.dialer.Dial(ctx, toNodeID, class) if err != nil { // DialNode already logs sufficiently, so just return. @@ -598,7 +927,7 @@ func (t *RaftTransport) startProcessNewQueue( return } - if err := t.processQueue(q, stream); err != nil { + if err := t.processQueue(q, stream, class); err != nil { log.Warningf(ctx, "while processing outgoing Raft queue to node %d: %s:", toNodeID, err) } } @@ -607,12 +936,102 @@ func (t *RaftTransport) startProcessNewQueue( pprof.Do(ctx, pprof.Labels("remote_node_id", toNodeID.String()), worker) }) if err != nil { + t.kvflowControl.mu.Lock() t.queues[class].Delete(int64(toNodeID)) + t.kvflowControl.mu.connectionTracker.markNodeDisconnected(toNodeID, class) + t.kvflowControl.mu.Unlock() return false } return true } +// startDroppingFlowTokensForDisconnectedNodes kicks of an asynchronous worker +// that periodically scans for nodes we're no longer connected to, and if there +// are any pending flow tokens bound for that node, it simply drops them. This +// "connected nodes" is a client-side view of the world. On the server-side when +// gRPC streams disconnect, we release all held tokens. So simply dropping these +// pending dispatches on the client side does not cause token leaks, and exists +// to prevent an unbounded accumulation of memory. +func (t *RaftTransport) startDroppingFlowTokensForDisconnectedNodes(ctx context.Context) error { + return t.stopper.RunAsyncTask( + ctx, + "kvserver.RaftTransport: drop flow tokens for disconnected nodes", + func(ctx context.Context) { + settingChangeCh := make(chan struct{}, 1) + kvadmission.FlowTokenDropInterval.SetOnChange( + &t.st.SV, func(ctx context.Context) { + select { + case settingChangeCh <- struct{}{}: + default: + } + }) + + timer := timeutil.NewTimer() + defer timer.Stop() + + for { + interval := kvadmission.FlowTokenDropInterval.Get(&t.st.SV) + if interval > 0 { + timer.Reset(interval) + } else { + // Disable the mechanism. + timer.Stop() + timer = timeutil.NewTimer() + } + select { + case <-timer.C: + timer.Read = true + t.dropFlowTokensForDisconnectedNodes() + continue + + case <-settingChangeCh: + // Loop around to use the updated timer. + continue + + case <-ctx.Done(): + return + + case <-t.stopper.ShouldQuiesce(): + return + } + } + }, + ) +} + +// TestingDropFlowTokensForDisconnectedNodes exports +// dropFlowTokensForDisconnectedNodes for testing purposes. +func (t *RaftTransport) TestingDropFlowTokensForDisconnectedNodes() { + t.dropFlowTokensForDisconnectedNodes() +} + +// TestingPrintFlowControlConnectionTracker renders the state of the underlying +// connection tracker. +func (t *RaftTransport) TestingPrintFlowControlConnectionTracker() string { + t.kvflowControl.mu.RLock() + defer t.kvflowControl.mu.RUnlock() + return t.kvflowControl.mu.connectionTracker.testingPrint() +} + +func (t *RaftTransport) dropFlowTokensForDisconnectedNodes() { + t.kvflowControl.mu.RLock() + defer t.kvflowControl.mu.RUnlock() + for _, nodeID := range t.kvflowControl.dispatchReader.PendingDispatch() { + if t.kvflowControl.mu.connectionTracker.isNodeConnected(nodeID) { + // If there's already a queue active, there's nothing to do. We rely + // on timely piggybacking of flow tokens on existing raft transport + // messages. It's only when all queues are idle/culled that we use + // this worker to drop any held tokens. The expectation is that on + // the server-side of the stream, we'll have returned all tokens + // when streams disconnect. + continue + } + // Drop any held tokens for that node. + pendingDispatches := t.kvflowControl.dispatchReader.PendingDispatchFor(nodeID) + t.metrics.FlowTokenDispatchesDropped.Inc(int64(len(pendingDispatches))) + } +} + // SendSnapshot streams the given outgoing snapshot. The caller is responsible // for closing the OutgoingSnapshot. func (t *RaftTransport) SendSnapshot( @@ -695,9 +1114,35 @@ func (t *RaftTransport) DelegateSnapshot( errors.Wrapf(resp.Error(), "error sending couldn't accept %v", req), errMarkSnapshotError) case kvserverpb.DelegateSnapshotResponse_APPLIED: // This is the response we're expecting. Snapshot successfully applied. - log.VEventf(ctx, 2, "%s: delegated snapshot was successfully applied", resp) + log.VEventf(ctx, 3, "%s: delegated snapshot was successfully applied", resp) return nil default: return err } } + +// RaftTransportTestingKnobs provide fine-grained control over the RaftTransport +// for tests. +type RaftTransportTestingKnobs struct { + // MarkSendQueueAsIdleCh is used to selectively mark a raft send queue as + // idle, identified by the remote node ID. The remote node ID must be + // connected to by this transport. + MarkSendQueueAsIdleCh chan roachpb.NodeID + // OnWorkerTeardown, if set, is invoked when a worker thread for a given + // send queue is tore down. + OnWorkerTeardown func(roachpb.NodeID) + // OnServerStreamDisconnected is invoked whenever the RaftMessageBatch + // stream is disconnected on the server side. + OnServerStreamDisconnected func() + // TriggerFallbackDispatchCh is used to manually trigger the fallback + // dispatch in tests. + TriggerFallbackDispatchCh chan time.Time + // OnFallbackDispatch is invoked whenever the fallback token dispatch + // mechanism is used. + OnFallbackDispatch func() +} + +// ModuleTestingKnobs is part of the base.ModuleTestingKnobs interface. +func (t *RaftTransportTestingKnobs) ModuleTestingKnobs() {} + +var _ base.ModuleTestingKnobs = (*RaftTransportTestingKnobs)(nil) diff --git a/pkg/kv/kvserver/raft_transport_metrics.go b/pkg/kv/kvserver/raft_transport_metrics.go index 971903c9503f..cea7d5eae8c6 100644 --- a/pkg/kv/kvserver/raft_transport_metrics.go +++ b/pkg/kv/kvserver/raft_transport_metrics.go @@ -23,6 +23,8 @@ type RaftTransportMetrics struct { ReverseSent *metric.Counter ReverseRcvd *metric.Counter + + FlowTokenDispatchesDropped *metric.Counter } func (t *RaftTransport) initMetrics() { @@ -91,5 +93,12 @@ responses to Raft messages. Responses are received over another stream.`, Measurement: "Messages", Unit: metric.Unit_COUNT, }), + + FlowTokenDispatchesDropped: metric.NewCounter(metric.Metadata{ + Name: "raft.transport.flow-token-dispatches-dropped", + Help: "Number of flow token dispatches dropped by the Raft Transport", + Measurement: "Dispatches", + Unit: metric.Unit_COUNT, + }), } } diff --git a/pkg/kv/kvserver/raft_transport_test.go b/pkg/kv/kvserver/raft_transport_test.go index c478056a11ea..fd9ed75045a7 100644 --- a/pkg/kv/kvserver/raft_transport_test.go +++ b/pkg/kv/kvserver/raft_transport_test.go @@ -22,6 +22,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/gossip" "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol/kvflowdispatch" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/rpc" @@ -112,15 +114,17 @@ type raftTransportTestContext struct { transports map[roachpb.NodeID]*kvserver.RaftTransport nodeRPCContext *rpc.Context gossip *gossip.Gossip + st *cluster.Settings } -func newRaftTransportTestContext(t testing.TB) *raftTransportTestContext { +func newRaftTransportTestContext(t testing.TB, st *cluster.Settings) *raftTransportTestContext { ctx := context.Background() tr := tracing.NewTracer() rttc := &raftTransportTestContext{ t: t, stopper: stop.NewStopper(stop.WithTracer(tr)), transports: map[roachpb.NodeID]*kvserver.RaftTransport{}, + st: st, } rttc.nodeRPCContext = rpc.NewContext(ctx, rpc.ContextOptions{ TenantID: roachpb.SystemTenantID, @@ -128,7 +132,7 @@ func newRaftTransportTestContext(t testing.TB) *raftTransportTestContext { Clock: &timeutil.DefaultTimeSource{}, ToleratedOffset: time.Nanosecond, Stopper: rttc.stopper, - Settings: cluster.MakeTestingClusterSettings(), + Settings: st, }) // Ensure that tests using this test context and restart/shut down // their servers do not inadvertently start talking to servers from @@ -152,7 +156,12 @@ func (rttc *raftTransportTestContext) Stop() { // before they can be used in other methods of // raftTransportTestContext. The node will be gossiped immediately. func (rttc *raftTransportTestContext) AddNode(nodeID roachpb.NodeID) *kvserver.RaftTransport { - transport, addr := rttc.AddNodeWithoutGossip(nodeID, util.TestAddr, rttc.stopper) + transport, addr := rttc.AddNodeWithoutGossip( + nodeID, util.TestAddr, rttc.stopper, + kvflowdispatch.NewDummyDispatch(), kvserver.NoopStoresFlowControlIntegration{}, + kvserver.NoopRaftTransportDisconnectListener{}, + nil, + ) rttc.GossipNode(nodeID, addr) return transport } @@ -162,18 +171,28 @@ func (rttc *raftTransportTestContext) AddNode(nodeID roachpb.NodeID) *kvserver.R // raftTransportTestContext. Unless you are testing the effects of // delaying gossip, use AddNode instead. func (rttc *raftTransportTestContext) AddNodeWithoutGossip( - nodeID roachpb.NodeID, addr net.Addr, stopper *stop.Stopper, + nodeID roachpb.NodeID, + addr net.Addr, + stopper *stop.Stopper, + kvflowTokenDispatch kvflowcontrol.DispatchReader, + kvflowHandles kvflowcontrol.Handles, + disconnectListener kvserver.RaftTransportDisconnectListener, + knobs *kvserver.RaftTransportTestingKnobs, ) (*kvserver.RaftTransport, net.Addr) { grpcServer, err := rpc.NewServer(rttc.nodeRPCContext) require.NoError(rttc.t, err) ctwWithTracer := log.MakeTestingAmbientCtxWithNewTracer() transport := kvserver.NewRaftTransport( ctwWithTracer, - cluster.MakeTestingClusterSettings(), + rttc.st, ctwWithTracer.Tracer, nodedialer.New(rttc.nodeRPCContext, gossip.AddressResolver(rttc.gossip)), grpcServer, rttc.stopper, + kvflowTokenDispatch, + kvflowHandles, + disconnectListener, + knobs, ) rttc.transports[nodeID] = transport ln, err := netutil.ListenAndServeGRPC(stopper, grpcServer, addr) @@ -223,7 +242,7 @@ func (rttc *raftTransportTestContext) Send( func TestSendAndReceive(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - rttc := newRaftTransportTestContext(t) + rttc := newRaftTransportTestContext(t, cluster.MakeTestingClusterSettings()) defer rttc.Stop() // Create several servers, each of which has two stores (A raft @@ -391,7 +410,7 @@ func TestSendAndReceive(t *testing.T) { func TestInOrderDelivery(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - rttc := newRaftTransportTestContext(t) + rttc := newRaftTransportTestContext(t, cluster.MakeTestingClusterSettings()) defer rttc.Stop() const numMessages = 100 @@ -429,7 +448,7 @@ func TestInOrderDelivery(t *testing.T) { func TestRaftTransportCircuitBreaker(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - rttc := newRaftTransportTestContext(t) + rttc := newRaftTransportTestContext(t, cluster.MakeTestingClusterSettings()) defer rttc.Stop() serverReplica := roachpb.ReplicaDescriptor{ @@ -437,7 +456,15 @@ func TestRaftTransportCircuitBreaker(t *testing.T) { StoreID: 2, ReplicaID: 2, } - _, serverAddr := rttc.AddNodeWithoutGossip(serverReplica.NodeID, util.TestAddr, rttc.stopper) + _, serverAddr := rttc.AddNodeWithoutGossip( + serverReplica.NodeID, + util.TestAddr, + rttc.stopper, + kvflowdispatch.NewDummyDispatch(), + kvserver.NoopStoresFlowControlIntegration{}, + kvserver.NoopRaftTransportDisconnectListener{}, + nil, + ) serverChannel := rttc.ListenStore(serverReplica.NodeID, serverReplica.StoreID) clientReplica := roachpb.ReplicaDescriptor{ @@ -484,7 +511,7 @@ func TestRaftTransportCircuitBreaker(t *testing.T) { func TestRaftTransportIndependentRanges(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - rttc := newRaftTransportTestContext(t) + rttc := newRaftTransportTestContext(t, cluster.MakeTestingClusterSettings()) defer rttc.Stop() server := roachpb.ReplicaDescriptor{ @@ -531,7 +558,7 @@ func TestRaftTransportIndependentRanges(t *testing.T) { func TestReopenConnection(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - rttc := newRaftTransportTestContext(t) + rttc := newRaftTransportTestContext(t, cluster.MakeTestingClusterSettings()) defer rttc.Stop() // Use a special stopper for the initial server so that we can fully stop it @@ -543,7 +570,15 @@ func TestReopenConnection(t *testing.T) { ReplicaID: 2, } serverTransport, serverAddr := - rttc.AddNodeWithoutGossip(serverReplica.NodeID, util.TestAddr, serverStopper) + rttc.AddNodeWithoutGossip( + serverReplica.NodeID, + util.TestAddr, + serverStopper, + kvflowdispatch.NewDummyDispatch(), + kvserver.NoopStoresFlowControlIntegration{}, + kvserver.NoopRaftTransportDisconnectListener{}, + nil, + ) rttc.GossipNode(serverReplica.NodeID, serverAddr) rttc.ListenStore(serverReplica.NodeID, serverReplica.StoreID) @@ -574,7 +609,15 @@ func TestReopenConnection(t *testing.T) { ReplicaID: 3, } - rttc.AddNodeWithoutGossip(replacementReplica.NodeID, serverAddr, rttc.stopper) + rttc.AddNodeWithoutGossip( + replacementReplica.NodeID, + serverAddr, + rttc.stopper, + kvflowdispatch.NewDummyDispatch(), + kvserver.NoopStoresFlowControlIntegration{}, + kvserver.NoopRaftTransportDisconnectListener{}, + nil, + ) replacementChannel := rttc.ListenStore(replacementReplica.NodeID, replacementReplica.StoreID) // Try sending a message to the old server's store (at the address its @@ -630,7 +673,7 @@ func TestReopenConnection(t *testing.T) { func TestSendFailureToConnectDoesNotHangRaft(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - rttc := newRaftTransportTestContext(t) + rttc := newRaftTransportTestContext(t, cluster.MakeTestingClusterSettings()) defer rttc.Stop() // Create a single server from which we're going to call send. diff --git a/pkg/kv/kvserver/raft_transport_unit_test.go b/pkg/kv/kvserver/raft_transport_unit_test.go index ec4f04f5a2e1..35907a3a7bd8 100644 --- a/pkg/kv/kvserver/raft_transport_unit_test.go +++ b/pkg/kv/kvserver/raft_transport_unit_test.go @@ -19,6 +19,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol/kvflowdispatch" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/rpc" "github.com/cockroachdb/cockroach/pkg/rpc/nodedialer" @@ -77,6 +78,10 @@ func TestRaftTransportStartNewQueue(t *testing.T) { nodedialer.New(rpcC, resolver), grpcServer, stopper, + kvflowdispatch.NewDummyDispatch(), + NoopStoresFlowControlIntegration{}, + NoopRaftTransportDisconnectListener{}, + nil, /* knobs */ ) ln, err := netutil.ListenAndServeGRPC(stopper, grpcServer, &util.UnresolvedAddr{NetworkField: "tcp", AddressField: "localhost:0"}) diff --git a/pkg/kv/kvserver/store_test.go b/pkg/kv/kvserver/store_test.go index 090acdbf7a65..c5e2f1e373d5 100644 --- a/pkg/kv/kvserver/store_test.go +++ b/pkg/kv/kvserver/store_test.go @@ -38,6 +38,7 @@ import ( aload "github.com/cockroachdb/cockroach/pkg/kv/kvserver/allocator/load" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/allocator/storepool" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/lock" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol/kvflowdispatch" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvstorage" @@ -232,7 +233,18 @@ func createTestStoreWithoutStart( // and it's important that this doesn't cause crashes. Just set up the // "real thing" since it's straightforward enough. cfg.NodeDialer = nodedialer.New(rpcContext, gossip.AddressResolver(cfg.Gossip)) - cfg.Transport = NewRaftTransport(cfg.AmbientCtx, cfg.Settings, cfg.Tracer(), cfg.NodeDialer, server, stopper) + cfg.Transport = NewRaftTransport( + cfg.AmbientCtx, + cfg.Settings, + cfg.Tracer(), + cfg.NodeDialer, + server, + stopper, + kvflowdispatch.NewDummyDispatch(), + NoopStoresFlowControlIntegration{}, + NoopRaftTransportDisconnectListener{}, + nil, /* knobs */ + ) stores := NewStores(cfg.AmbientCtx, cfg.Clock) nodeDesc := &roachpb.NodeDescriptor{NodeID: 1} diff --git a/pkg/kv/kvserver/testdata/flow_control_raft_transport/basic_piggyback b/pkg/kv/kvserver/testdata/flow_control_raft_transport/basic_piggyback new file mode 100644 index 000000000000..6f66b7b02425 --- /dev/null +++ b/pkg/kv/kvserver/testdata/flow_control_raft_transport/basic_piggyback @@ -0,0 +1,40 @@ +# Walk through the basics of the data-driven syntax. + +init +---- + +# Set up two nodes, each with one store. +add node=n1 store=s1 +---- + +add node=n2 store=s2 +---- + +# Dispatch flow tokens from n1 to n2. +dispatch from=n1 +node=n2 store=s1 range=r1 pri=normal-pri up-to-log-position=5/20 +node=n2 store=s1 range=r1 pri=low-pri up-to-log-position=5/21 +---- + +# Note that they're still pending delivery. +pending-dispatch from=n1 to=n2 +---- +range=r1 pri=low-pri store=s1 up-to-log-position=log-position=5/21 +range=r1 pri=normal-pri store=s1 up-to-log-position=log-position=5/20 + +# Send a raft message for r1 from n1 to n2, each node holding a replica with +# id=1,2 respectively. We expect the tokens to get piggybacked here. +send range=r1 from=n1/s1/1 to=n2/s2/2 commit=1 +---- + +# Note that there are no more pending dispatches, and not because they were +# dropped. +pending-dispatch from=n1 to=n2 +---- + +metrics +---- +node=n1: dispatches-dropped=0 +node=n2: dispatches-dropped=0 + +# vim:ft=sh diff --git a/pkg/kv/kvserver/testdata/flow_control_raft_transport/fallback_dispatch b/pkg/kv/kvserver/testdata/flow_control_raft_transport/fallback_dispatch new file mode 100644 index 000000000000..206ae3e53a5c --- /dev/null +++ b/pkg/kv/kvserver/testdata/flow_control_raft_transport/fallback_dispatch @@ -0,0 +1,48 @@ +# Walk through what happens in fallback path of token dispatch, i.e. we don't +# have raft messages to piggy token returns on top of but we then fire one-off +# messages to deliver tokens. + +init +---- + +# Set up two nodes, each with one store. +add node=n1 store=s1 +---- + +add node=n2 store=s2 +---- + +# Send a raft message for r1 from n1 to n2 and vice versa, each node holding a +# replica with id=1,2 respectively. We do this to create the non-idle connect +# between the two nodes (it's done on demand). +send range=r1 from=n1/s1/1 to=n2/s2/2 commit=1 +---- + +send range=r1 from=n2/s2/2 to=n1/s1/1 commit=1 +---- + +dispatch from=n2 +node=n1 store=s1 range=r1 pri=normal-pri up-to-log-position=6/20 +node=n1 store=s1 range=r1 pri=low-pri up-to-log-position=6/21 +---- + +# Note that they're still pending delivery. +pending-dispatch from=n2 to=n1 +---- +range=r1 pri=low-pri store=s1 up-to-log-position=log-position=6/21 +range=r1 pri=normal-pri store=s1 up-to-log-position=log-position=6/20 + +fallback-dispatch from=n2 +---- + +# Note that there are no more pending dispatches, and not because they were +# dropped. +pending-dispatch from=n2 to=n1 +---- + +metrics +---- +node=n1: dispatches-dropped=0 +node=n2: dispatches-dropped=0 + +# vim:ft=sh diff --git a/pkg/kv/kvserver/testdata/flow_control_raft_transport/idle_connection b/pkg/kv/kvserver/testdata/flow_control_raft_transport/idle_connection new file mode 100644 index 000000000000..eb9e5b24b189 --- /dev/null +++ b/pkg/kv/kvserver/testdata/flow_control_raft_transport/idle_connection @@ -0,0 +1,92 @@ +# Walk through what exactly happens when a client marks its outbound raft +# transport stream as idle. It's helpful to also read the store_data test in +# this directory. + +init +---- + +# Set up two nodes, each with one store. +add node=n1 store=s1 +---- + +add node=n2 store=s2 +---- + +# Send a raft message for r1 from n1 to n2 and vice versa, each node holding a +# replica with id=1,2 respectively. We do this to create the two non-idle +# client-> server connections between the two nodes, going both ways. It's done +# on demand. +send range=r1 from=n1/s1/1 to=n2/s2/2 commit=1 +---- + +send range=r1 from=n2/s2/2 to=n1/s1/1 commit=1 +---- + +# Verify that n1's marked n2 as something its connected to (as a client), and +# vice-versa. This test isn't making use of the +# set-{initial,additional}-store-ids directives used in the store_data test, +# which is triggered in deployments and showcases the server's POV. This test +# is narrowly showing what happens from the client's POV -- so ignore all +# server POV details. +connection-tracker from=n1 +---- +connected-stores (server POV): +connected-nodes (client POV): n2 + +connection-tracker from=n2 +---- +connected-stores (server POV): +connected-nodes (client POV): n1 + +# Return some flow tokens from n2 back to n1, where these tokens originally +# would have been deducted. +dispatch from=n2 +node=n1 store=s1 range=r1 pri=normal-pri up-to-log-position=6/20 +node=n1 store=s1 range=r1 pri=low-pri up-to-log-position=6/21 +---- + +# Note that they're pending delivery. +pending-dispatch from=n2 to=n1 +---- +range=r1 pri=low-pri store=s1 up-to-log-position=log-position=6/21 +range=r1 pri=normal-pri store=s1 up-to-log-position=log-position=6/20 + + +# Mark the client-initiated stream from n2->n1 as idle. +client-mark-idle from=n2 to=n1 +---- + +# n2's wound down its transport streams and has marked n1 as something it's +# disconnected from as a client. +connection-tracker from=n2 +---- +connected-stores (server POV): +connected-nodes (client POV): + +# n1's not done the same since it still is connected to n2 as a client. We +# don't actually care about n1's perspective in this test. +connection-tracker from=n1 +---- +connected-stores (server POV): +connected-nodes (client POV): n2 + +# Note that they're still pending delivery. They'll only get pruned out by the +# periodic pruning process, which is invoked next. +pending-dispatch from=n2 to=n1 +---- +range=r1 pri=low-pri store=s1 up-to-log-position=log-position=6/21 +range=r1 pri=normal-pri store=s1 up-to-log-position=log-position=6/20 + +drop-disconnected-tokens from=n2 +---- + +# Observe that there's nothing pending dispatch and the metrics indicate as much. +pending-dispatch from=n2 to=n1 +---- + +metrics +---- +node=n1: dispatches-dropped=0 +node=n2: dispatches-dropped=2 + +# vim:ft=sh diff --git a/pkg/kv/kvserver/testdata/flow_control_raft_transport/store_data b/pkg/kv/kvserver/testdata/flow_control_raft_transport/store_data new file mode 100644 index 000000000000..3e9474aadd47 --- /dev/null +++ b/pkg/kv/kvserver/testdata/flow_control_raft_transport/store_data @@ -0,0 +1,83 @@ +# Walk through the basics of the data-driven syntax. + +init +---- + +# Set up two nodes, each with one store. +add node=n1 store=s1 +---- + +add node=n2 store=s2 +---- + +set-initial-store-ids from=n1 stores=s1 +---- + +set-initial-store-ids from=n2 stores=s2,s3 +---- + +set-additional-store-ids from=n1 stores=s4 +---- + +# Send a raft message for r1 from n1 to n2 and vice versa, each node holding a +# replica with id=1,2 respectively. We do this to create the non-idle +# connection between the two nodes (it's done on demand). We should transmit +# (s2,s3) from n2->n1, and (s1,s4) from n1->n2. So effectively n1 should be +# connected to n2:s2,s3, and n2 to n1:s1,s4. +send range=r1 from=n1/s1/1 to=n2/s2/2 commit=1 +---- + +send range=r1 from=n2/s2/2 to=n1/s1/1 commit=1 +---- + +# Confirm the node+store connectivity, as seen by each node. The server POV is +# always store-oriented, since the server cares most about what remote stores +# got disconnected in order to release tokens that are identified partly by +# store IDs. The raft transport client cares most about what servers it's +# connected to, identified by node ID, since the client is the one managing the +# outbox of tokens to return to the server. If it's no longer connected to some +# server, two things happen: +# - The server detects this, and releases tokens (as described above). +# - The client can reclaim some memory in its outbox, where messages destined +# for the server just no longer need delivery. +connection-tracker from=n1 +---- +connected-stores (server POV): s2,s3 +connected-nodes (client POV): n2 + +connection-tracker from=n2 +---- +connected-stores (server POV): s1,s4 +connected-nodes (client POV): n1 + +# Mark the client-initiated stream from n2->n1 as idle. +client-mark-idle from=n2 to=n1 +---- + +# Confirm that n1 detects this on the server side, and marks that it's no +# longer connected to s2 and s3. It still show's that it's connected to n2 as a +# client, but this test primarily cares about the empty server POV. +# +# The set of connected nodes however is unchanged from n1's perspective, as +# there's still an n1->n2 link. The connected-nodes tracking is done on the +# client side. +connection-tracker from=n1 +---- +connected-stores (server POV): +connected-nodes (client POV): n2 + +# Confirm that the RaftTransport has informed the flow-control integration +# layer of this fact. +disconnect-listener from=n1 +---- +disconnected-from: s2,s3 + +# Sanity check that n2 marks that it's no longer connected to n1 as a client. +# Since it acts as the server for the n1->n2 link, it shows that it's connected +# to n1's stores. +connection-tracker from=n2 +---- +connected-stores (server POV): s1,s4 +connected-nodes (client POV): + +# vim:ft=sh diff --git a/pkg/roachpb/metadata.go b/pkg/roachpb/metadata.go index 45a0201023ab..3b482f9420b6 100644 --- a/pkg/roachpb/metadata.go +++ b/pkg/roachpb/metadata.go @@ -44,6 +44,17 @@ func (s NodeIDSlice) Len() int { return len(s) } func (s NodeIDSlice) Swap(i, j int) { s[i], s[j] = s[j], s[i] } func (s NodeIDSlice) Less(i, j int) bool { return s[i] < s[j] } +func (s NodeIDSlice) String() string { + var sb strings.Builder + for i, ni := range s { + if i > 0 { + sb.WriteRune(',') + } + fmt.Fprintf(&sb, "n%d", ni) + } + return sb.String() +} + // StoreID is a custom type for a cockroach store ID. type StoreID int32 @@ -54,6 +65,20 @@ func (s StoreIDSlice) Len() int { return len(s) } func (s StoreIDSlice) Swap(i, j int) { s[i], s[j] = s[j], s[i] } func (s StoreIDSlice) Less(i, j int) bool { return s[i] < s[j] } +func (s StoreIDSlice) String() string { + var sb strings.Builder + for i, st := range s { + if i > 0 { + sb.WriteRune(',') + } + fmt.Fprintf(&sb, "s%d", st) + } + return sb.String() +} + +// SafeValue implements the redact.SafeValue interface. +func (s StoreIDSlice) SafeValue() {} + // String implements the fmt.Stringer interface. // It is used to format the ID for use in Gossip keys. func (n StoreID) String() string { diff --git a/pkg/server/BUILD.bazel b/pkg/server/BUILD.bazel index 7386ed296d2d..219b1e095f71 100644 --- a/pkg/server/BUILD.bazel +++ b/pkg/server/BUILD.bazel @@ -126,6 +126,7 @@ go_library( "//pkg/kv/kvserver/closedts/ctpb", "//pkg/kv/kvserver/closedts/sidetransport", "//pkg/kv/kvserver/kvadmission", + "//pkg/kv/kvserver/kvflowcontrol/kvflowdispatch", "//pkg/kv/kvserver/kvserverbase", "//pkg/kv/kvserver/kvserverpb", "//pkg/kv/kvserver/kvstorage", diff --git a/pkg/server/init.go b/pkg/server/init.go index 90e40d2dbc47..9e406fffd2e1 100644 --- a/pkg/server/init.go +++ b/pkg/server/init.go @@ -158,6 +158,33 @@ func (i *initState) validate() error { return nil } +// initialStoreIDs returns the initial set of store IDs a node starts off with. +// If it's a restarting node, restarted with no additional stores, this is just +// all the local store IDs. If there's an additional store post-restart that's +// yet to be initialized, it's not found in this list. For nodes newly added to +// the cluster, it's just the first initialized store. For the remaining stores +// in all these cases, they're accessible through +// (*initState).additionalStoreIDs once they're initialized. +func (i *initState) initialStoreIDs() ([]roachpb.StoreID, error) { + return getStoreIDsInner(i.initializedEngines) +} + +func (i *initState) additionalStoreIDs() ([]roachpb.StoreID, error) { + return getStoreIDsInner(i.uninitializedEngines) +} + +func getStoreIDsInner(engines []storage.Engine) ([]roachpb.StoreID, error) { + storeIDs := make([]roachpb.StoreID, 0, len(engines)) + for _, eng := range engines { + storeID, err := eng.GetStoreID() + if err != nil { + return nil, err + } + storeIDs = append(storeIDs, roachpb.StoreID(storeID)) + } + return storeIDs, nil +} + // joinResult is used to represent the result of a node attempting to join // an already bootstrapped cluster. type joinResult struct { @@ -706,6 +733,9 @@ func inspectEngines( } nodeID = storeIdent.NodeID + if err := eng.SetStoreID(ctx, int32(storeIdent.StoreID)); err != nil { + return nil, err + } initializedEngines = append(initializedEngines, eng) } clusterVersion, err := kvstorage.SynthesizeClusterVersionFromEngines( diff --git a/pkg/server/server.go b/pkg/server/server.go index 7f0640c56ae2..9fd7b2dd7924 100644 --- a/pkg/server/server.go +++ b/pkg/server/server.go @@ -43,6 +43,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/allocator/storepool" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts/ctpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts/sidetransport" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol/kvflowdispatch" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvstorage" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness" @@ -547,8 +548,19 @@ func NewServer(cfg Config, stopper *stop.Stopper) (*Server, error) { /* deterministic */ false, ) + storesForFlowControl := kvserver.MakeStoresForFlowControl(stores) + kvflowTokenDispatch := kvflowdispatch.New(registry, storesForFlowControl, nodeIDContainer) raftTransport := kvserver.NewRaftTransport( - cfg.AmbientCtx, st, cfg.AmbientCtx.Tracer, nodeDialer, grpcServer.Server, stopper, + cfg.AmbientCtx, + st, + cfg.AmbientCtx.Tracer, + nodeDialer, + grpcServer.Server, + stopper, + kvflowTokenDispatch, + storesForFlowControl, + storesForFlowControl, + nil, /* knobs */ ) registry.AddMetricStruct(raftTransport.Metrics()) @@ -1662,6 +1674,14 @@ func (s *Server) PreStart(ctx context.Context) error { // initState -- and everything after it is actually starting the server, // using the listeners and init state. + initialStoreIDs, err := state.initialStoreIDs() + if err != nil { + return err + } + + // Inform the raft transport of these initial store IDs. + s.raftTransport.SetInitialStoreIDs(initialStoreIDs) + // Spawn a goroutine that will print a nice message when Gossip connects. // Note that we already know the clusterID, but we don't know that Gossip // has connected. The pertinent case is that of restarting an entire @@ -1810,6 +1830,14 @@ func (s *Server) PreStart(ctx context.Context) error { // stores) s.node.waitForAdditionalStoreInit() + additionalStoreIDs, err := state.additionalStoreIDs() + if err != nil { + return err + } + + // Inform the raft transport of these additional store IDs. + s.raftTransport.SetAdditionalStoreIDs(additionalStoreIDs) + // Connect the engines to the disk stats map constructor. This needs to // wait until after waitForAdditionalStoreInit returns since it realizes on // wholly initialized stores (it reads the StoreIdentKeys). It also needs @@ -1976,6 +2004,11 @@ func (s *Server) PreStart(ctx context.Context) error { // Start the closed timestamp loop. s.ctSender.Run(workersCtx, state.nodeID) + // Start dispatching extant flow tokens. + if err := s.raftTransport.Start(workersCtx); err != nil { + return err + } + // Attempt to upgrade cluster version now that the sql server has been // started. At this point we know that all startupmigrations and permanent // upgrades have successfully been run so it is safe to upgrade to the diff --git a/pkg/storage/engine.go b/pkg/storage/engine.go index 53a677e04359..ae214cbffaa2 100644 --- a/pkg/storage/engine.go +++ b/pkg/storage/engine.go @@ -984,6 +984,9 @@ type Engine interface { // Used to show the store ID in logs and to initialize the shared object // creator ID (if shared object storage is configured). SetStoreID(ctx context.Context, storeID int32) error + + // GetStoreID is used to retrieve the configured store ID. + GetStoreID() (int32, error) } // Batch is the interface for batch specific operations. diff --git a/pkg/storage/pebble.go b/pkg/storage/pebble.go index df1cde671daa..f696176823ad 100644 --- a/pkg/storage/pebble.go +++ b/pkg/storage/pebble.go @@ -859,6 +859,21 @@ func (p *Pebble) SetStoreID(ctx context.Context, storeID int32) error { return nil } +// GetStoreID returns to configured store ID. +func (p *Pebble) GetStoreID() (int32, error) { + if p == nil { + return 0, errors.AssertionFailedf("GetStoreID requires non-nil Pebble") + } + if p.storeIDPebbleLog == nil { + return 0, errors.AssertionFailedf("GetStoreID requires an initialized store ID container") + } + storeID := p.storeIDPebbleLog.Get() + if storeID == 0 { + return 0, errors.AssertionFailedf("GetStoreID must be called after calling SetStoreID") + } + return storeID, nil +} + // ResolveEncryptedEnvOptions creates the EncryptionEnv and associated file // registry if this store has encryption-at-rest enabled; otherwise returns a // nil EncryptionEnv. diff --git a/pkg/testutils/lint/passes/redactcheck/redactcheck.go b/pkg/testutils/lint/passes/redactcheck/redactcheck.go index 00ddddb11aa2..6effa04c3120 100644 --- a/pkg/testutils/lint/passes/redactcheck/redactcheck.go +++ b/pkg/testutils/lint/passes/redactcheck/redactcheck.go @@ -117,6 +117,7 @@ func runAnalyzer(pass *analysis.Pass) (interface{}, error) { "ReplicaID": {}, "ReplicaType": {}, "StoreID": {}, + "StoreIDSlice": {}, "TenantID": {}, "TransactionStatus": {}, }, diff --git a/pkg/util/admission/granter_test.go b/pkg/util/admission/granter_test.go index b12f8e1f195b..bae737636ac5 100644 --- a/pkg/util/admission/granter_test.go +++ b/pkg/util/admission/granter_test.go @@ -487,3 +487,17 @@ func (m *testMetricsProvider) setMetricsForStores(stores []int32, metrics pebble }) } } + +type noopOnLogEntryAdmitted struct{} + +func (n *noopOnLogEntryAdmitted) AdmittedLogEntry( + context.Context, + roachpb.NodeID, + admissionpb.WorkPriority, + roachpb.StoreID, + roachpb.RangeID, + LogPosition, +) { +} + +var _ OnLogEntryAdmitted = &noopOnLogEntryAdmitted{} diff --git a/pkg/util/admission/work_queue.go b/pkg/util/admission/work_queue.go index 44c5c0a8cb77..5d3ca55c968e 100644 --- a/pkg/util/admission/work_queue.go +++ b/pkg/util/admission/work_queue.go @@ -855,7 +855,16 @@ func (q *WorkQueue) granted(grantChainID grantChainID) int64 { } else { // NB: We don't use grant chains for store tokens, so they don't apply // to replicated writes. - + if log.V(1) { + log.Infof(q.ambientCtx, "async-path: len(waiting-work)=%d dequeued t%d pri=%s r%s origin=n%s log-position=%s ingested=%t", + tenant.waitingWorkHeap.Len(), + tenant.id, item.priority, + item.replicated.RangeID, + item.replicated.Origin, + item.replicated.LogPosition, + item.replicated.Ingested, + ) + } defer releaseWaitingWork(item) q.onAdmittedReplicatedWork.admittedReplicatedWork( roachpb.MustMakeTenantID(tenant.id), @@ -1968,6 +1977,7 @@ func (q *StoreWorkQueue) admittedReplicatedWork( // have a separate goroutine invoke these callbacks (without holding // coord.mu). We could directly invoke here too if not holding the lock. q.onLogEntryAdmitted.AdmittedLogEntry( + q.q[wc].ambientCtx, rwi.Origin, pri, q.storeID, @@ -1982,6 +1992,7 @@ func (q *StoreWorkQueue) admittedReplicatedWork( // post-admission bookkeeping. type OnLogEntryAdmitted interface { AdmittedLogEntry( + ctx context.Context, origin roachpb.NodeID, /* node where the entry originated */ pri admissionpb.WorkPriority, /* admission priority of the entry */ storeID roachpb.StoreID, /* store on which the entry was admitted */ @@ -1997,7 +2008,12 @@ type NoopOnLogEntryAdmitted struct{} var _ OnLogEntryAdmitted = &NoopOnLogEntryAdmitted{} func (n *NoopOnLogEntryAdmitted) AdmittedLogEntry( - roachpb.NodeID, admissionpb.WorkPriority, roachpb.StoreID, roachpb.RangeID, LogPosition, + context.Context, + roachpb.NodeID, + admissionpb.WorkPriority, + roachpb.StoreID, + roachpb.RangeID, + LogPosition, ) { } From 33c7754064f94a2a9bd81792882d4ed1340e72f3 Mon Sep 17 00:00:00 2001 From: irfan sharif Date: Thu, 27 Apr 2023 01:16:29 -0400 Subject: [PATCH 02/12] kvserver,kvflowcontrol: integrate flow control Part of #95563. This PR integrates various kvflowcontrol components into the critical path for replication traffic. It does so by introducing two "integration interfaces" in the kvserver package to intercept various points of a replica's lifecycle, using it to manage the underlying replication streams and flow tokens. The integration is mediated through two cluster settings: - kvadmission.flow_control.enabled This is a top-level kill-switch to revert to pre-kvflowcontrol behavior where follower writes unilaterally deducted IO tokens without blocking. - kvadmission.flow_control.mode It can take on one of two settings, each exercising the flow control machinery to varying degrees. - apply_to_elastic Only applies admission delays to elastic traffic. - apply_to_all Applies admission delays to {regular,elastic} traffic. When the mode is changed, we simply admit all waiting requests. This risks possibly over-admitting work, but that's ok -- we assume these mode changes are rare events and done under supervision. These settings are hooked into in the kvadmission and kvflowcontroller packages. As for the actual integration interfaces in kvserver, they are: - replicaFlowControlIntegration: used to integrate with replication flow control. It's intercepts various points in a replica's lifecycle, like it acquiring raft leadership or losing it, or its raft membership changing, etc. Accessing it requires Replica.mu to be held, exclusively (this is asserted on in the canonical implementation). type replicaFlowControlIntegration interface { handle() (kvflowcontrol.Handle, bool) onBecameLeader(context.Context) onBecameFollower(context.Context) onDescChanged(context.Context) onFollowersPaused(context.Context) onReplicaDestroyed(context.Context) onProposalQuotaUpdated(context.Context) } - replicaForFlowControl abstracts the interface of an individual Replica, as needed by replicaFlowControlIntegration. type replicaForFlowControl interface { assertLocked() annotateCtx(context.Context) context.Context getTenantID() roachpb.TenantID getReplicaID() roachpb.ReplicaID getRangeID() roachpb.RangeID getDescriptor() *roachpb.RangeDescriptor pausedFollowers() map[roachpb.ReplicaID]struct{} isFollowerActive(context.Context, roachpb.ReplicaID) bool appliedLogPosition() kvflowcontrolpb.RaftLogPosition withReplicaProgress(f func(roachpb.ReplicaID, rafttracker.Progress)) } Release note: None --- .../settings/settings-for-tenants.txt | 2 +- docs/generated/settings/settings.html | 2 +- pkg/clusterversion/cockroach_versions.go | 8 + pkg/kv/kvserver/BUILD.bazel | 3 + pkg/kv/kvserver/flow_control_replica.go | 107 +++++ .../flow_control_replica_integration.go | 432 ++++++++++++++++++ pkg/kv/kvserver/flow_control_stores.go | 43 +- pkg/kv/kvserver/kvadmission/BUILD.bazel | 3 + pkg/kv/kvserver/kvadmission/kvadmission.go | 92 +++- pkg/kv/kvserver/kvflowcontrol/BUILD.bazel | 1 + pkg/kv/kvserver/kvflowcontrol/doc.go | 15 + .../kvserver/kvflowcontrol/kvflowcontrol.go | 99 +++- .../kvflowcontroller/kvflowcontroller.go | 23 +- .../kvflowcontroller_metrics.go | 27 +- .../kvflowcontrol/kvflowcontrolpb/BUILD.bazel | 7 +- ...{raft_log_position.go => kvflowcontrol.go} | 15 +- .../kvflowcontrolpb/kvflowcontrol.proto | 2 + .../kvflowcontrol/kvflowhandle/BUILD.bazel | 2 + .../kvflowhandle/kvflowhandle.go | 91 +++- .../kvflowhandle/kvflowhandle_metrics.go | 30 +- .../kvflowhandle/kvflowhandle_test.go | 124 ++++- .../kvflowcontrol/kvflowhandle/noop.go | 63 +++ .../kvflowsimulator/simulation_test.go | 6 +- .../testdata/handle_stream_disconnection | 4 +- .../kvflowtokentracker/BUILD.bazel | 1 + .../kvflowtokentracker/tracker.go | 41 +- .../kvflowtokentracker/tracker_test.go | 4 +- pkg/kv/kvserver/raftlog/encoding_test.go | 2 +- pkg/kv/kvserver/replica.go | 10 + pkg/kv/kvserver/replica_app_batch.go | 7 +- pkg/kv/kvserver/replica_application_cmd.go | 6 + pkg/kv/kvserver/replica_application_result.go | 4 + pkg/kv/kvserver/replica_destroy.go | 3 +- pkg/kv/kvserver/replica_init.go | 5 + pkg/kv/kvserver/replica_init_test.go | 1 + pkg/kv/kvserver/replica_proposal.go | 11 + pkg/kv/kvserver/replica_proposal_buf.go | 105 ++++- pkg/kv/kvserver/replica_proposal_buf_test.go | 46 ++ pkg/kv/kvserver/replica_proposal_quota.go | 11 +- pkg/kv/kvserver/replica_raft.go | 79 +++- pkg/kv/kvserver/replica_raft_overload.go | 1 + pkg/kv/kvserver/store.go | 13 +- pkg/roachpb/metadata_replicas.go | 20 + pkg/roachpb/metadata_replicas_test.go | 40 ++ pkg/server/BUILD.bazel | 5 + pkg/server/admission.go | 55 +++ pkg/server/node.go | 8 +- pkg/server/server.go | 66 ++- pkg/util/admission/granter_test.go | 4 +- .../replicated_write_admission_test.go | 2 +- pkg/util/admission/work_queue.go | 38 +- pkg/util/admission/work_queue_test.go | 2 +- pkg/util/metric/registry.go | 4 + 53 files changed, 1667 insertions(+), 128 deletions(-) create mode 100644 pkg/kv/kvserver/flow_control_replica.go create mode 100644 pkg/kv/kvserver/flow_control_replica_integration.go rename pkg/kv/kvserver/kvflowcontrol/kvflowcontrolpb/{raft_log_position.go => kvflowcontrol.go} (72%) create mode 100644 pkg/kv/kvserver/kvflowcontrol/kvflowhandle/noop.go create mode 100644 pkg/server/admission.go diff --git a/docs/generated/settings/settings-for-tenants.txt b/docs/generated/settings/settings-for-tenants.txt index 177e2e491a57..e8ca9b68e80a 100644 --- a/docs/generated/settings/settings-for-tenants.txt +++ b/docs/generated/settings/settings-for-tenants.txt @@ -300,4 +300,4 @@ trace.opentelemetry.collector string address of an OpenTelemetry trace collecto trace.snapshot.rate duration 0s if non-zero, interval at which background trace snapshots are captured tenant-rw trace.span_registry.enabled boolean true if set, ongoing traces can be seen at https:///#/debug/tracez tenant-rw trace.zipkin.collector string the address of a Zipkin instance to receive traces, as :. If no port is specified, 9411 will be used. tenant-rw -version version 1000023.1-8 set the active cluster version in the format '.' tenant-rw +version version 1000023.1-10 set the active cluster version in the format '.' tenant-rw diff --git a/docs/generated/settings/settings.html b/docs/generated/settings/settings.html index 986931b1a29e..ed142364fa63 100644 --- a/docs/generated/settings/settings.html +++ b/docs/generated/settings/settings.html @@ -254,6 +254,6 @@
trace.snapshot.rate
duration0sif non-zero, interval at which background trace snapshots are capturedServerless/Dedicated/Self-Hosted
trace.span_registry.enabled
booleantrueif set, ongoing traces can be seen at https://<ui>/#/debug/tracezServerless/Dedicated/Self-Hosted
trace.zipkin.collector
stringthe address of a Zipkin instance to receive traces, as <host>:<port>. If no port is specified, 9411 will be used.Serverless/Dedicated/Self-Hosted -
version
version1000023.1-8set the active cluster version in the format '<major>.<minor>'Serverless/Dedicated/Self-Hosted +
version
version1000023.1-10set the active cluster version in the format '<major>.<minor>'Serverless/Dedicated/Self-Hosted diff --git a/pkg/clusterversion/cockroach_versions.go b/pkg/clusterversion/cockroach_versions.go index 89f698125593..6d0adb612248 100644 --- a/pkg/clusterversion/cockroach_versions.go +++ b/pkg/clusterversion/cockroach_versions.go @@ -538,6 +538,10 @@ const ( // the system tenant. V23_2_EnableRangeCoalescingForSystemTenant + // V23_2_UseACRaftEntryEntryEncodings gates the use of raft entry encodings + // that (optionally) embed below-raft admission data. + V23_2_UseACRaftEntryEntryEncodings + // ************************************************* // Step (1) Add new versions here. // Do not add new versions to a patch release. @@ -935,6 +939,10 @@ var rawVersionsSingleton = keyedVersions{ Key: V23_2_EnableRangeCoalescingForSystemTenant, Version: roachpb.Version{Major: 23, Minor: 1, Internal: 8}, }, + { + Key: V23_2_UseACRaftEntryEntryEncodings, + Version: roachpb.Version{Major: 23, Minor: 1, Internal: 10}, + }, // ************************************************* // Step (2): Add new versions here. diff --git a/pkg/kv/kvserver/BUILD.bazel b/pkg/kv/kvserver/BUILD.bazel index 2225938169b8..4bcb635527d1 100644 --- a/pkg/kv/kvserver/BUILD.bazel +++ b/pkg/kv/kvserver/BUILD.bazel @@ -13,6 +13,8 @@ go_library( "debug_print.go", "doc.go", "flow_control_raft_transport.go", + "flow_control_replica.go", + "flow_control_replica_integration.go", "flow_control_stores.go", "lease_history.go", "markers.go", @@ -141,6 +143,7 @@ go_library( "//pkg/kv/kvserver/kvflowcontrol", "//pkg/kv/kvserver/kvflowcontrol/kvflowcontrolpb", "//pkg/kv/kvserver/kvflowcontrol/kvflowdispatch", + "//pkg/kv/kvserver/kvflowcontrol/kvflowhandle", "//pkg/kv/kvserver/kvserverbase", "//pkg/kv/kvserver/kvserverpb", "//pkg/kv/kvserver/kvstorage", diff --git a/pkg/kv/kvserver/flow_control_replica.go b/pkg/kv/kvserver/flow_control_replica.go new file mode 100644 index 000000000000..0f73e3241995 --- /dev/null +++ b/pkg/kv/kvserver/flow_control_replica.go @@ -0,0 +1,107 @@ +// Copyright 2023 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package kvserver + +import ( + "context" + + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol/kvflowcontrolpb" + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/util/timeutil" + "go.etcd.io/raft/v3" + rafttracker "go.etcd.io/raft/v3/tracker" +) + +// replicaForFlowControl abstracts the interface of an individual Replica, as +// needed by replicaFlowControlIntegration. +type replicaForFlowControl interface { + annotateCtx(context.Context) context.Context + getTenantID() roachpb.TenantID + getReplicaID() roachpb.ReplicaID + getRangeID() roachpb.RangeID + getDescriptor() *roachpb.RangeDescriptor + getAppliedLogPosition() kvflowcontrolpb.RaftLogPosition + getPausedFollowers() map[roachpb.ReplicaID]struct{} + isFollowerLive(context.Context, roachpb.ReplicaID) bool + isRaftTransportConnectedTo(roachpb.StoreID) bool + withReplicaProgress(f func(roachpb.ReplicaID, rafttracker.Progress)) + + assertLocked() // only affects test builds +} + +// replicaFlowControl is a concrete implementation of the replicaForFlowControl +// interface. +type replicaFlowControl Replica + +var _ replicaForFlowControl = &replicaFlowControl{} + +func (rf *replicaFlowControl) assertLocked() { + rf.mu.AssertHeld() +} + +func (rf *replicaFlowControl) annotateCtx(ctx context.Context) context.Context { + return rf.AnnotateCtx(ctx) +} + +func (rf *replicaFlowControl) getTenantID() roachpb.TenantID { + rf.assertLocked() + return rf.mu.tenantID +} + +func (rf *replicaFlowControl) getReplicaID() roachpb.ReplicaID { + return rf.replicaID +} + +func (rf *replicaFlowControl) getRangeID() roachpb.RangeID { + return rf.RangeID +} + +func (rf *replicaFlowControl) getDescriptor() *roachpb.RangeDescriptor { + rf.assertLocked() + r := (*Replica)(rf) + return r.descRLocked() +} + +func (rf *replicaFlowControl) getPausedFollowers() map[roachpb.ReplicaID]struct{} { + rf.assertLocked() + return rf.mu.pausedFollowers +} + +func (rf *replicaFlowControl) isFollowerLive(ctx context.Context, replID roachpb.ReplicaID) bool { + rf.mu.AssertHeld() + return rf.mu.lastUpdateTimes.isFollowerActiveSince( + ctx, + replID, + timeutil.Now(), + rf.store.cfg.RangeLeaseDuration, + ) +} + +func (rf *replicaFlowControl) isRaftTransportConnectedTo(storeID roachpb.StoreID) bool { + rf.mu.AssertHeld() + return rf.store.cfg.Transport.isConnectedTo(storeID) +} + +func (rf *replicaFlowControl) getAppliedLogPosition() kvflowcontrolpb.RaftLogPosition { + rf.mu.AssertHeld() + status := rf.mu.internalRaftGroup.BasicStatus() + return kvflowcontrolpb.RaftLogPosition{ + Term: status.Term, + Index: status.Applied, + } +} + +func (rf *replicaFlowControl) withReplicaProgress(f func(roachpb.ReplicaID, rafttracker.Progress)) { + rf.mu.AssertHeld() + rf.mu.internalRaftGroup.WithProgress(func(id uint64, _ raft.ProgressType, progress rafttracker.Progress) { + f(roachpb.ReplicaID(id), progress) + }) +} diff --git a/pkg/kv/kvserver/flow_control_replica_integration.go b/pkg/kv/kvserver/flow_control_replica_integration.go new file mode 100644 index 000000000000..051fad507fcb --- /dev/null +++ b/pkg/kv/kvserver/flow_control_replica_integration.go @@ -0,0 +1,432 @@ +// Copyright 2023 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package kvserver + +import ( + "context" + "sort" + + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol" + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/util/log" + rafttracker "go.etcd.io/raft/v3/tracker" +) + +// TODO(irfansharif): Write integration tests, walking through +// kvflowcontrol/doc.go. Do this as part of #95563. Randomize an in-memory +// workload with various chaos events, like nodes dying, streams breaking, +// splits, merges, etc. and assert stable flow tokens. Leader replica removing +// itself from descriptor. Uninitialized replica below raft for which we've +// deducted flow tokens for (dealt with by looking at StateReplicate). Dropped +// proposals -- we should only be deducting tokens once submitting to raft. But +// if leader's raft messages to follower get dropped (and not vice versa), +// leader will still see follower as active and not disconnect streams. Has this +// changed with us upgrading asymmetric partitions to bidirectional ones? + +// TODO(irfansharif): Write data-driven unit tests for this interface. + +// replicaFlowControlIntegration is used to integrate with replication flow +// control. It's intercepts various points in a replica's lifecycle, like it +// acquiring raft leadership or losing it, or its raft membership changing, etc. +// +// Accessing it requires Replica.mu to be held, exclusively (this is asserted on +// in the canonical implementation). +type replicaFlowControlIntegration interface { + handle() (kvflowcontrol.Handle, bool) + onBecameLeader(context.Context) + onBecameFollower(context.Context) + onDescChanged(context.Context) + onFollowersPaused(context.Context) + onReplicaDestroyed(context.Context) + onProposalQuotaUpdated(context.Context) + onRaftTransportDisconnected(context.Context, ...roachpb.StoreID) +} + +var _ replicaFlowControlIntegration = &replicaFlowControlIntegrationImpl{} + +type replicaFlowControlIntegrationImpl struct { + replicaForFlowControl replicaForFlowControl + handleFactory kvflowcontrol.HandleFactory + + innerHandle kvflowcontrol.Handle + lastKnownReplicas roachpb.ReplicaSet + disconnectedStreams map[roachpb.ReplicaID]kvflowcontrol.Stream +} + +func newReplicaFlowControlIntegration( + replicaForFlowControl replicaForFlowControl, handleFactory kvflowcontrol.HandleFactory, +) *replicaFlowControlIntegrationImpl { + return &replicaFlowControlIntegrationImpl{ + replicaForFlowControl: replicaForFlowControl, + handleFactory: handleFactory, + } +} + +func (f *replicaFlowControlIntegrationImpl) handle() (kvflowcontrol.Handle, bool) { + f.replicaForFlowControl.assertLocked() + return f.innerHandle, f.innerHandle != nil +} + +func (f *replicaFlowControlIntegrationImpl) onBecameLeader(ctx context.Context) { + f.replicaForFlowControl.assertLocked() + if f.innerHandle != nil { + log.Fatal(ctx, "flow control handle was not nil before becoming the leader") + } + if !f.replicaForFlowControl.getTenantID().IsSet() { + log.Fatal(ctx, "unset tenant ID") + } + + // See I5 from kvflowcontrol/doc.go. The per-replica kvflowcontrol.Handle is + // tied to the lifetime of a leaseholder replica having raft leadership. + f.innerHandle = f.handleFactory.NewHandle( + f.replicaForFlowControl.getRangeID(), + f.replicaForFlowControl.getTenantID(), + ) + f.lastKnownReplicas = f.replicaForFlowControl.getDescriptor().Replicas() + f.disconnectedStreams = make(map[roachpb.ReplicaID]kvflowcontrol.Stream) + + appliedLogPosition := f.replicaForFlowControl.getAppliedLogPosition() + for _, desc := range f.replicaForFlowControl.getDescriptor().Replicas().Descriptors() { + // Start off every remote stream as disconnected. Later we'll try to + // reconnect them. + stream := kvflowcontrol.Stream{ + TenantID: f.replicaForFlowControl.getTenantID(), + StoreID: desc.StoreID, + } + if f.replicaForFlowControl.getReplicaID() != desc.ReplicaID { + f.disconnectedStreams[desc.ReplicaID] = stream + continue + } + // Connect to the local stream. + f.innerHandle.ConnectStream(ctx, appliedLogPosition, stream) + } + f.tryReconnect(ctx) + + if log.V(1) { + var disconnected []kvflowcontrol.Stream + for _, stream := range f.disconnectedStreams { + disconnected = append(disconnected, stream) + } + sort.Slice(disconnected, func(i, j int) bool { + return disconnected[i].StoreID < disconnected[j].StoreID + }) + log.Infof(ctx, "assumed raft leadership: initializing flow handle for %s starting at %s (disconnected streams: %s)", + f.replicaForFlowControl.getDescriptor(), appliedLogPosition, disconnected) + } +} + +func (f *replicaFlowControlIntegrationImpl) onBecameFollower(ctx context.Context) { + f.replicaForFlowControl.assertLocked() + if f.innerHandle == nil { + return + } + + // See I5 from kvflowcontrol/doc.go. The per-replica kvflowcontrol.Handle is + // tied to the lifetime of a leaseholder replica having raft leadership. + // When leadership is lost, or the lease changes hands, we release all held + // flow tokens. Here we're not dealing with prolonged leaseholder != leader + // scenarios. + log.VInfof(ctx, 1, "lost raft leadership: releasing flow tokens and closing handle for %s", + f.replicaForFlowControl.getDescriptor()) + f.innerHandle.Close(ctx) + f.innerHandle = nil + f.lastKnownReplicas = roachpb.MakeReplicaSet(nil) + f.disconnectedStreams = nil +} + +func (f *replicaFlowControlIntegrationImpl) onDescChanged(ctx context.Context) { + f.replicaForFlowControl.assertLocked() + if f.innerHandle == nil { + return // nothing to do + } + + addedReplicas, removedReplicas := f.lastKnownReplicas.Difference( + f.replicaForFlowControl.getDescriptor().Replicas(), + ) + for _, repl := range removedReplicas { + if repl.ReplicaID == f.replicaForFlowControl.getReplicaID() { + // We're observing ourselves get removed from the raft group, but + // are still retaining raft leadership. Close the underlying handle + // and bail. + // + // TODO(irfansharif): Is this even possible? + f.innerHandle.Close(ctx) + f.innerHandle = nil + f.lastKnownReplicas = roachpb.MakeReplicaSet(nil) + f.disconnectedStreams = nil + return + } + // See I10 from kvflowcontrol/doc.go. We stop deducting flow tokens for + // replicas that are no longer part of the raft group, free-ing up all + // held tokens. + f.innerHandle.DisconnectStream(ctx, kvflowcontrol.Stream{ + TenantID: f.replicaForFlowControl.getTenantID(), + StoreID: repl.StoreID, + }) + delete(f.disconnectedStreams, repl.ReplicaID) + } + + for _, repl := range addedReplicas { + // Start off new replicas as disconnected. We'll subsequently try to + // re-add them, once we know their log positions and consider them + // sufficiently caught up. See I3a from kvflowcontrol/doc.go. + if repl.ReplicaID == f.replicaForFlowControl.getReplicaID() { + log.Fatalf(ctx, "observed replica adding itself to the range descriptor") + } + if _, found := f.disconnectedStreams[repl.ReplicaID]; found { + continue // already disconnected, nothing to do + } + stream := kvflowcontrol.Stream{ + TenantID: f.replicaForFlowControl.getTenantID(), + StoreID: repl.StoreID, + } + f.disconnectedStreams[repl.ReplicaID] = stream + } + if len(addedReplicas) > 0 || len(removedReplicas) > 0 { + log.VInfof(ctx, 1, "desc changed from %s to %s: added=%s removed=%s", + f.lastKnownReplicas, f.replicaForFlowControl.getDescriptor(), addedReplicas, removedReplicas, + ) + } + f.lastKnownReplicas = f.replicaForFlowControl.getDescriptor().Replicas() +} + +func (f *replicaFlowControlIntegrationImpl) onFollowersPaused(ctx context.Context) { + f.replicaForFlowControl.assertLocked() + if f.innerHandle == nil { + return // nothing to do + } + + var toDisconnect []roachpb.ReplicaDescriptor + // See I3 from kvflowcontrol/doc.go. We don't deduct flow tokens for + // replication traffic that's not headed to paused replicas. + for replID := range f.replicaForFlowControl.getPausedFollowers() { + repl, ok := f.lastKnownReplicas.GetReplicaDescriptorByID(replID) + if !ok { + // As of 4/23, we don't make any strong guarantees around the set of + // paused followers we're tracking, nothing that ensures that what's + // tracked is guaranteed to be a member of the range descriptor. We + // treat the range descriptor derived state as authoritative. + continue + } + if repl.ReplicaID == f.replicaForFlowControl.getReplicaID() { + log.Fatalf(ctx, "observed replica pausing replication traffic to itself") + } + toDisconnect = append(toDisconnect, repl) + } + + f.disconnectStreams(ctx, toDisconnect, "paused followers") + f.tryReconnect(ctx) +} + +func (f *replicaFlowControlIntegrationImpl) onReplicaDestroyed(ctx context.Context) { + f.replicaForFlowControl.assertLocked() + if f.innerHandle == nil { + return // nothing to do + } + + // During merges, the context might have the subsuming range, so we + // explicitly (re-)annotate it here. + ctx = f.replicaForFlowControl.annotateCtx(ctx) + + // See I6, I9 from kvflowcontrol/doc.go. We want to free up all held flow + // tokens when a replica is being removed, for example when it's being + // rebalanced away, is no longer part of the raft group, is being GC-ed, + // destroyed as part of the EndTxn merge trigger, or subsumed if applying + // the merge as part of an incoming snapshot. + f.innerHandle.Close(ctx) + f.innerHandle = nil + f.lastKnownReplicas = roachpb.MakeReplicaSet(nil) + f.disconnectedStreams = nil +} + +func (f *replicaFlowControlIntegrationImpl) onProposalQuotaUpdated(ctx context.Context) { + f.replicaForFlowControl.assertLocked() + if f.innerHandle == nil { + return // nothing to do + } + + var toDisconnect []roachpb.ReplicaDescriptor + + // Disconnect any recently inactive followers. + // + // TODO(irfansharif): Experimentally this gets triggered quite often. It + // might be too sensitive and may result in ineffective flow control as + // a result. Fix as part of #95563. + for _, repl := range f.lastKnownReplicas.Descriptors() { + if f.replicaForFlowControl.isFollowerLive(ctx, repl.ReplicaID) { + continue // nothing to do + } + if repl.ReplicaID == f.replicaForFlowControl.getReplicaID() { + // NB: We ignore ourselves from this last-updated map. For followers + // we update the timestamps when we step a message from them into + // the local raft group, but for the leader we only update it + // whenever it ticks. So in workloads where the leader only sees + // occasional writes, it could see itself as non-live. This is + // likely unintentional, but we paper over it here. + continue // nothing to do + } + toDisconnect = append(toDisconnect, repl) + } + f.disconnectStreams(ctx, toDisconnect, "inactive followers") + + // Disconnect any streams we're not actively replicating to. + toDisconnect = nil + for _, replID := range f.notActivelyReplicatingTo() { + repl, ok := f.lastKnownReplicas.GetReplicaDescriptorByID(replID) + if !ok { + continue + } + if repl.ReplicaID == f.replicaForFlowControl.getReplicaID() { + log.Fatalf(ctx, "leader replica observed that it was not being actively replicated to") + } + toDisconnect = append(toDisconnect, repl) + } + f.disconnectStreams(ctx, toDisconnect, "not actively replicating") + + f.tryReconnect(ctx) +} + +// notActivelyReplicatingTo lists the replicas that aren't actively receiving +// log entries to append to its log, from raft's perspective (i.e. this is +// unrelated to CRDB-level follower pausing). This encompasses newly added +// replicas that we're still probing to figure out its last index, replicas +// that are pending raft snapshots because the leader has truncated away entries +// higher than its last position, and replicas we're not currently connected to +// via the raft transport. +func (f *replicaFlowControlIntegrationImpl) notActivelyReplicatingTo() []roachpb.ReplicaID { + var res []roachpb.ReplicaID + f.replicaForFlowControl.withReplicaProgress(func(replID roachpb.ReplicaID, progress rafttracker.Progress) { + if replID == f.replicaForFlowControl.getReplicaID() { + return + } + repl, ok := f.lastKnownReplicas.GetReplicaDescriptorByID(replID) + if !ok { + return + } + + if progress.State != rafttracker.StateReplicate { + res = append(res, replID) + // TODO(irfansharif): Integrating with these other progress fields + // from raft. For replicas exiting rafttracker.StateProbe, perhaps + // compare progress.Match against status.Commit to make sure it's + // sufficiently caught up with respect to its raft log before we + // start deducting tokens for it (lest we run into I3a from + // kvflowcontrol/doc.go). To play well with the replica-level + // proposal quota pool, maybe we also factor its base index? + // Replicas that crashed and came back could come back in + // StateReplicate but be behind on their logs. If we're deducting + // tokens right away for subsequent proposals, it would take some + // time for it to catch up and then later return those tokens to us. + // This is I3a again; do it as part of #95563. + _ = progress.RecentActive + _ = progress.MsgAppFlowPaused + _ = progress.Match + return + } + + if !f.replicaForFlowControl.isRaftTransportConnectedTo(repl.StoreID) { + res = append(res, replID) + } + }) + return res +} + +func (f *replicaFlowControlIntegrationImpl) disconnectStreams( + ctx context.Context, toDisconnect []roachpb.ReplicaDescriptor, reason string, +) { + for _, repl := range toDisconnect { + if _, found := f.disconnectedStreams[repl.ReplicaID]; found { + continue // already disconnected, nothing to do + } + stream := kvflowcontrol.Stream{ + TenantID: f.replicaForFlowControl.getTenantID(), + StoreID: repl.StoreID, + } + f.innerHandle.DisconnectStream(ctx, stream) + f.disconnectedStreams[repl.ReplicaID] = stream + log.VInfof(ctx, 1, "tracked disconnected stream: %s (reason: %s)", stream, reason) + } +} + +func (f *replicaFlowControlIntegrationImpl) onRaftTransportDisconnected( + ctx context.Context, storeIDs ...roachpb.StoreID, +) { + f.replicaForFlowControl.assertLocked() + if f.innerHandle == nil { + return // nothing to do + } + + disconnectedStores := make(map[roachpb.StoreID]struct{}) + for _, storeID := range storeIDs { + disconnectedStores[storeID] = struct{}{} + } + + var toDisconnect []roachpb.ReplicaDescriptor + for _, repl := range f.lastKnownReplicas.Descriptors() { + if _, found := disconnectedStores[repl.StoreID]; found { + toDisconnect = append(toDisconnect, repl) + } + } + f.disconnectStreams(ctx, toDisconnect, "raft transport disconnected") + f.tryReconnect(ctx) +} + +func (f *replicaFlowControlIntegrationImpl) tryReconnect(ctx context.Context) { + // Try reconnecting streams we disconnected. + pausedFollowers := f.replicaForFlowControl.getPausedFollowers() + notActivelyReplicatingTo := f.notActivelyReplicatingTo() + appliedLogPosition := f.replicaForFlowControl.getAppliedLogPosition() + + var disconnectedRepls []roachpb.ReplicaID + for replID := range f.disconnectedStreams { + disconnectedRepls = append(disconnectedRepls, replID) + } + sort.Slice(disconnectedRepls, func(i, j int) bool { // for determinism in tests + return disconnectedRepls[i] < disconnectedRepls[j] + }) + for _, replID := range disconnectedRepls { + stream := f.disconnectedStreams[replID] + if _, ok := pausedFollowers[replID]; ok { + continue // still paused, nothing to reconnect + } + + repl, ok := f.lastKnownReplicas.GetReplicaDescriptorByID(replID) + if !ok { + log.Fatalf(ctx, "%s: tracking %s in disconnected streams despite it not being in descriptor: %s", + f.replicaForFlowControl.getReplicaID(), replID, f.lastKnownReplicas) + } + if !f.replicaForFlowControl.isFollowerLive(ctx, replID) { + continue // still inactive, nothing to reconnect + } + + notReplicatedTo := false + for _, notReplicatedToRepl := range notActivelyReplicatingTo { + if replID == notReplicatedToRepl { + notReplicatedTo = true + break + } + } + if notReplicatedTo { + continue // not actively replicated to, yet; nothing to reconnect + } + + if !f.replicaForFlowControl.isRaftTransportConnectedTo(repl.StoreID) { + continue // not connected to via raft transport + } + + // See I1, I2, I3, I3a, I4 from kvflowcontrol/doc.go. Replica is + // connected to via the RaftTransport (I1), on a live node (I2), not + // paused (I3), and is being actively replicated to through log entries + // (I3a, I4). Re-connect so we can start deducting tokens for it. + f.innerHandle.ConnectStream(ctx, appliedLogPosition, stream) + delete(f.disconnectedStreams, replID) + } +} diff --git a/pkg/kv/kvserver/flow_control_stores.go b/pkg/kv/kvserver/flow_control_stores.go index fe0d14ad69ac..98ca2dcc4e93 100644 --- a/pkg/kv/kvserver/flow_control_stores.go +++ b/pkg/kv/kvserver/flow_control_stores.go @@ -14,6 +14,7 @@ import ( "context" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol/kvflowhandle" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/util/log" ) @@ -104,14 +105,22 @@ func (sh *storeForFlowControl) Lookup( if repl == nil { return nil, false } - return nil, false // TODO(irfansharif): Fill this in. + + repl.mu.Lock() + defer repl.mu.Unlock() + return repl.mu.replicaFlowControlIntegration.handle() } // ResetStreams is part of the StoresForFlowControl interface. func (sh *storeForFlowControl) ResetStreams(ctx context.Context) { s := (*Store)(sh) s.VisitReplicas(func(r *Replica) (wantMore bool) { - // TODO(irfansharif): Fill this in. + r.mu.Lock() + defer r.mu.Unlock() + handle, found := r.mu.replicaFlowControlIntegration.handle() + if found { + handle.ResetStreams(ctx) + } return true }) } @@ -123,10 +132,38 @@ func (sh *storeForFlowControl) OnRaftTransportDisconnected( ) { s := (*Store)(sh) s.mu.replicasByRangeID.Range(func(replica *Replica) { - // TODO(irfansharif): Fill this in. + replica.mu.Lock() + defer replica.mu.Unlock() + replica.mu.replicaFlowControlIntegration.onRaftTransportDisconnected(ctx, storeIDs...) }) } +// storeFlowControlHandleFactory is a concrete implementation of +// kvflowcontrol.HandleFactory. +type storeFlowControlHandleFactory Store + +var _ kvflowcontrol.HandleFactory = &storeFlowControlHandleFactory{} + +// makeStoreFlowControlHandleFactory returns a new storeFlowControlHandleFactory +// instance. +func makeStoreFlowControlHandleFactory(store *Store) *storeFlowControlHandleFactory { + return (*storeFlowControlHandleFactory)(store) +} + +// NewHandle is part of the kvflowcontrol.HandleFactory interface. +func (shf *storeFlowControlHandleFactory) NewHandle( + rangeID roachpb.RangeID, tenantID roachpb.TenantID, +) kvflowcontrol.Handle { + s := (*Store)(shf) + return kvflowhandle.New( + s.cfg.KVFlowController, + s.cfg.KVFlowHandleMetrics, + s.cfg.Clock, + rangeID, + tenantID, + ) +} + // NoopStoresFlowControlIntegration is a no-op implementation of the // StoresForFlowControl interface. type NoopStoresFlowControlIntegration struct{} diff --git a/pkg/kv/kvserver/kvadmission/BUILD.bazel b/pkg/kv/kvserver/kvadmission/BUILD.bazel index 175b864bfe01..fdf83611cc78 100644 --- a/pkg/kv/kvserver/kvadmission/BUILD.bazel +++ b/pkg/kv/kvserver/kvadmission/BUILD.bazel @@ -7,8 +7,11 @@ go_library( importpath = "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvadmission", visibility = ["//visibility:public"], deps = [ + "//pkg/base", + "//pkg/clusterversion", "//pkg/kv/kvpb", "//pkg/kv/kvserver/kvflowcontrol", + "//pkg/kv/kvserver/kvflowcontrol/kvflowcontrolpb", "//pkg/kv/kvserver/raftlog", "//pkg/roachpb", "//pkg/settings", diff --git a/pkg/kv/kvserver/kvadmission/kvadmission.go b/pkg/kv/kvserver/kvadmission/kvadmission.go index 0ebb62537f20..f0a61eea5aa9 100644 --- a/pkg/kv/kvserver/kvadmission/kvadmission.go +++ b/pkg/kv/kvserver/kvadmission/kvadmission.go @@ -18,8 +18,11 @@ import ( "sync" "time" + "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol/kvflowcontrolpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/raftlog" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings" @@ -182,13 +185,18 @@ type TenantWeightsForStore struct { // controllerImpl implements Controller interface. type controllerImpl struct { + nodeID *base.NodeIDContainer + // Admission control queues and coordinators. All three should be nil or // non-nil. kvAdmissionQ *admission.WorkQueue storeGrantCoords *admission.StoreGrantCoordinators elasticCPUGrantCoordinator *admission.ElasticCPUGrantCoordinator - settings *cluster.Settings - every log.EveryN + kvflowController kvflowcontrol.Controller + kvflowHandles kvflowcontrol.Handles + + settings *cluster.Settings + every log.EveryN } var _ Controller = &controllerImpl{} @@ -202,23 +210,42 @@ type Handle struct { tenantID roachpb.TenantID storeAdmissionQ *admission.StoreWorkQueue storeWorkHandle admission.StoreWorkHandle - ElasticCPUWorkHandle *admission.ElasticCPUWorkHandle + elasticCPUWorkHandle *admission.ElasticCPUWorkHandle + raftAdmissionMeta *kvflowcontrolpb.RaftAdmissionMeta callAdmittedWorkDoneOnKVAdmissionQ bool } +// AnnotateCtx annotates the given context with request-scoped admission +// data, plumbed through the KV stack using context.Contexts. +func (h *Handle) AnnotateCtx(ctx context.Context) context.Context { + if h.elasticCPUWorkHandle != nil { + ctx = admission.ContextWithElasticCPUWorkHandle(ctx, h.elasticCPUWorkHandle) + } + if h.raftAdmissionMeta != nil { + ctx = kvflowcontrol.ContextWithMeta(ctx, h.raftAdmissionMeta) + } + return ctx +} + // MakeController returns a Controller. All three parameters must together be // nil or non-nil. func MakeController( + nodeID *base.NodeIDContainer, kvAdmissionQ *admission.WorkQueue, elasticCPUGrantCoordinator *admission.ElasticCPUGrantCoordinator, storeGrantCoords *admission.StoreGrantCoordinators, + kvflowController kvflowcontrol.Controller, + kvflowHandles kvflowcontrol.Handles, settings *cluster.Settings, ) Controller { return &controllerImpl{ + nodeID: nodeID, kvAdmissionQ: kvAdmissionQ, storeGrantCoords: storeGrantCoords, elasticCPUGrantCoordinator: elasticCPUGrantCoordinator, + kvflowController: kvflowController, + kvflowHandles: kvflowHandles, settings: settings, every: log.Every(10 * time.Second), } @@ -273,22 +300,44 @@ func (n *controllerImpl) AdmitKVWork( // to continue even when throttling since there are often significant // number of tokens available. if ba.IsWrite() && !ba.IsSingleHeartbeatTxnRequest() { - storeAdmissionQ := n.storeGrantCoords.TryGetQueueForStore(int32(ba.Replica.StoreID)) - if storeAdmissionQ != nil { - storeWorkHandle, err := storeAdmissionQ.Admit( - ctx, admission.StoreWriteWorkInfo{WorkInfo: admissionInfo}) - if err != nil { + if !bypassAdmission && + kvflowcontrol.Enabled.Get(&n.settings.SV) && + n.settings.Version.IsActive(ctx, clusterversion.V23_2_UseACRaftEntryEntryEncodings) { + kvflowHandle, found := n.kvflowHandles.Lookup(ba.RangeID) + if !found { + return Handle{}, nil + } + if err := kvflowHandle.Admit(ctx, admissionInfo.Priority, timeutil.FromUnixNanos(createTime)); err != nil { return Handle{}, err } - admissionEnabled = storeWorkHandle.UseAdmittedWorkDone() - if admissionEnabled { - defer func() { - if retErr != nil { - // No bytes were written. - _ = storeAdmissionQ.AdmittedWorkDone(ah.storeWorkHandle, admission.StoreWorkDoneInfo{}) - } - }() - ah.storeAdmissionQ, ah.storeWorkHandle = storeAdmissionQ, storeWorkHandle + // NB: It's possible for us to be waiting for available flow tokens + // for a different set of streams that the ones we'll eventually + // deduct tokens from, if the range experiences a split between now + // and the point of deduction. That's ok, there's no strong + // synchronization needed between these two points. + ah.raftAdmissionMeta = &kvflowcontrolpb.RaftAdmissionMeta{ + AdmissionPriority: int32(admissionInfo.Priority), + AdmissionCreateTime: admissionInfo.CreateTime, + AdmissionOriginNode: n.nodeID.Get(), + } + } else { + storeAdmissionQ := n.storeGrantCoords.TryGetQueueForStore(int32(ba.Replica.StoreID)) + if storeAdmissionQ != nil { + storeWorkHandle, err := storeAdmissionQ.Admit( + ctx, admission.StoreWriteWorkInfo{WorkInfo: admissionInfo}) + if err != nil { + return Handle{}, err + } + admissionEnabled = storeWorkHandle.UseAdmittedWorkDone() + if admissionEnabled { + defer func() { + if retErr != nil { + // No bytes were written. + _ = storeAdmissionQ.AdmittedWorkDone(ah.storeWorkHandle, admission.StoreWorkDoneInfo{}) + } + }() + ah.storeAdmissionQ, ah.storeWorkHandle = storeAdmissionQ, storeWorkHandle + } } } } @@ -312,11 +361,11 @@ func (n *controllerImpl) AdmitKVWork( if err != nil { return Handle{}, err } - ah.ElasticCPUWorkHandle = elasticWorkHandle + ah.elasticCPUWorkHandle = elasticWorkHandle defer func() { if retErr != nil { // No elastic work was done. - n.elasticCPUGrantCoordinator.ElasticCPUWorkQueue.AdmittedWorkDone(ah.ElasticCPUWorkHandle) + n.elasticCPUGrantCoordinator.ElasticCPUWorkQueue.AdmittedWorkDone(ah.elasticCPUWorkHandle) } }() } else { @@ -332,7 +381,7 @@ func (n *controllerImpl) AdmitKVWork( // AdmittedKVWorkDone implements the Controller interface. func (n *controllerImpl) AdmittedKVWorkDone(ah Handle, writeBytes *StoreWriteBytes) { - n.elasticCPUGrantCoordinator.ElasticCPUWorkQueue.AdmittedWorkDone(ah.ElasticCPUWorkHandle) + n.elasticCPUGrantCoordinator.ElasticCPUWorkQueue.AdmittedWorkDone(ah.elasticCPUWorkHandle) if ah.callAdmittedWorkDoneOnKVAdmissionQ { n.kvAdmissionQ.AdmittedWorkDone(ah.tenantID) } @@ -466,10 +515,11 @@ func (n *controllerImpl) AdmitRaftEntry( } if log.V(1) { - log.Infof(ctx, "decoded raft admission meta below-raft: pri=%s create-time=%d proposer=n%s receiver=[n?,s%s] tenant=t%d tokensā‰ˆ%d sideloaded=%t raft-entry=%d/%d", + log.Infof(ctx, "decoded raft admission meta below-raft: pri=%s create-time=%d proposer=n%s receiver=[n%d,s%s] tenant=t%d tokensā‰ˆ%d sideloaded=%t raft-entry=%d/%d", admissionpb.WorkPriority(meta.AdmissionPriority), meta.AdmissionCreateTime, meta.AdmissionOriginNode, + n.nodeID.Get(), storeID, tenantID.ToUint64(), kvflowcontrol.Tokens(len(entry.Data)), diff --git a/pkg/kv/kvserver/kvflowcontrol/BUILD.bazel b/pkg/kv/kvserver/kvflowcontrol/BUILD.bazel index 23b2271aa510..2665a779d145 100644 --- a/pkg/kv/kvserver/kvflowcontrol/BUILD.bazel +++ b/pkg/kv/kvserver/kvflowcontrol/BUILD.bazel @@ -14,6 +14,7 @@ go_library( "//pkg/base", "//pkg/kv/kvserver/kvflowcontrol/kvflowcontrolpb", "//pkg/roachpb", + "//pkg/settings", "//pkg/util/admission/admissionpb", "@com_github_cockroachdb_redact//:redact", "@com_github_dustin_go_humanize//:go-humanize", diff --git a/pkg/kv/kvserver/kvflowcontrol/doc.go b/pkg/kv/kvserver/kvflowcontrol/doc.go index 49910b2790d5..28ccfa514c17 100644 --- a/pkg/kv/kvserver/kvflowcontrol/doc.go +++ b/pkg/kv/kvserver/kvflowcontrol/doc.go @@ -426,6 +426,21 @@ package kvflowcontrol // - Because the fan-in effects of epoch-LIFO are not well understood (by this // author at least), we just disable it below-raft. // +// I13. What happens when a range {un,}quiesces? +// - Quiescing a range only prevents its internal raft group from being ticked, +// which stops it from issuing MsgHeartbeats or calling elections. Quiesced +// ranges still have a raft leader and/or a leaseholder. Any raft operation +// (for example, proposals) on any replica ends up unquiescing the range, +// typically under stable raft leadership. As far as flow tokens are +// concerned: +// - Quiesced ranges have no steady stream of RaftTransport messages, which we +// use to piggyback flow token returns. But we guarantee timely delivery +// even without messages to piggyback on top of. See I8 above. +// - When returning flow tokens to a quiesced range's leaseholder, that's ok, +// we're able to look up the right kvflowcontrol.Handle since the replica is +// still around. When quiescing a range, we don't need to release all-held +// tokens, or wait until there are no held flow tokens. +// // --- // // [^1]: kvserverpb.RaftMessageRequest is the unit of what's sent diff --git a/pkg/kv/kvserver/kvflowcontrol/kvflowcontrol.go b/pkg/kv/kvserver/kvflowcontrol/kvflowcontrol.go index 1e95f1277c8a..05f8b481ff25 100644 --- a/pkg/kv/kvserver/kvflowcontrol/kvflowcontrol.go +++ b/pkg/kv/kvserver/kvflowcontrol/kvflowcontrol.go @@ -18,11 +18,72 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol/kvflowcontrolpb" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/util/admission/admissionpb" "github.com/cockroachdb/redact" "github.com/dustin/go-humanize" ) +// Enabled determines whether we use flow control for replication traffic in KV. +var Enabled = settings.RegisterBoolSetting( + settings.SystemOnly, + "kvadmission.flow_control.enabled", + "determines whether we use flow control for replication traffic in KV", + true, +) + +// Mode determines the 'mode' of flow control we use for replication traffic in +// KV, if enabled. +var Mode = settings.RegisterEnumSetting( + settings.SystemOnly, + "kvadmission.flow_control.mode", + "determines the 'mode' of flow control we use for replication traffic in KV, if enabled", + ApplyToAll.String(), + map[int64]string{ + int64(ApplyToElastic): modeDict[ApplyToElastic], + int64(ApplyToAll): modeDict[ApplyToAll], + }, +) + +var modeDict = map[ModeT]string{ + ApplyToElastic: "apply_to_elastic", + ApplyToAll: "apply_to_all", +} + +// ModeT represents the various modes of flow control for replication traffic. +type ModeT int64 + +const ( + // ApplyToElastic uses flow control for only elastic traffic, i.e. only + // elastic work will wait for flow tokens to be available. All work is + // virtually enqueued in below-raft admission queues and dequeued in + // priority order, but only empty elastic flow token buckets above-raft will + // block further elastic traffic from being admitted. + // + // TODO(irfansharif): We're potentially risking OOMs doing all this tracking + // for regular work, without coalescing state. With a bit of plumbing, for + // requests that bypass flow control we could fallback to using the non-AC + // raft encodings and avoid the potential OOMs. Address this as part of + // #95563. + ApplyToElastic ModeT = iota + // ApplyToAll uses flow control for both elastic and regular traffic, + // i.e. all work will wait for flow tokens to be available. + ApplyToAll +) + +func (m ModeT) String() string { + return redact.StringWithoutMarkers(m) +} + +// SafeFormat implements the redact.SafeFormatter interface. +func (m ModeT) SafeFormat(p redact.SafePrinter, verb rune) { + if s, ok := modeDict[m]; ok { + p.Print(s) + return + } + p.Print("unknown-mode") +} + // Stream models the stream over which we replicate data traffic, the // transmission for which we regulate using flow control. It's segmented by the // specific store the traffic is bound for and the tenant driving it. Despite @@ -127,7 +188,8 @@ type Handle interface { // DisconnectStream disconnects a stream from the handle. When disconnecting // a stream, (a) all previously held flow tokens are released and (b) we // unblock all requests waiting in Admit() for this stream's flow tokens in - // particular. + // particular. It's a no-op if disconnecting something we're not connected + // to. // // This is typically used when we're no longer replicating data to a member // of the raft group, because (a) it crashed, (b) it's no longer part of the @@ -137,6 +199,11 @@ type Handle interface { // AdmittedRaftLogEntries between what it admitted last and its latest // RaftLogPosition. DisconnectStream(context.Context, Stream) + // ResetStreams resets all connected streams, i.e. it disconnects and + // re-connects to each one. It effectively unblocks all requests waiting in + // Admit(). It's only used when cluster settings change, settings that + // affect all work waiting for flow tokens. + ResetStreams(ctx context.Context) // Close closes the handle and returns all held tokens back to the // underlying controller. Typically used when the replica loses its lease // and/or raft leadership, or ends up getting GC-ed (if it's being @@ -149,6 +216,12 @@ type Handle interface { // they're uniquely identified by their range ID. type Handles interface { Lookup(roachpb.RangeID) (Handle, bool) + ResetStreams(ctx context.Context) +} + +// HandleFactory is used to construct new Handles. +type HandleFactory interface { + NewHandle(roachpb.RangeID, roachpb.TenantID) Handle } // Dispatch is used (i) to dispatch information about admitted raft log entries @@ -205,3 +278,27 @@ func (s Stream) SafeFormat(p redact.SafePrinter, verb rune) { } p.Printf("t%s/s%s", tenantSt, s.StoreID.String()) } + +type raftAdmissionMetaKey struct{} + +// ContextWithMeta returns a Context wrapping the supplied raft admission meta, +// if any. +// +// TODO(irfansharif): This causes a heap allocation. Revisit as part of #95563. +func ContextWithMeta(ctx context.Context, meta *kvflowcontrolpb.RaftAdmissionMeta) context.Context { + if meta != nil { + ctx = context.WithValue(ctx, raftAdmissionMetaKey{}, meta) + } + return ctx +} + +// MetaFromContext returns the raft admission meta embedded in the Context, if +// any. +func MetaFromContext(ctx context.Context) *kvflowcontrolpb.RaftAdmissionMeta { + val := ctx.Value(raftAdmissionMetaKey{}) + h, ok := val.(*kvflowcontrolpb.RaftAdmissionMeta) + if !ok { + return nil + } + return h +} diff --git a/pkg/kv/kvserver/kvflowcontrol/kvflowcontroller/kvflowcontroller.go b/pkg/kv/kvserver/kvflowcontrol/kvflowcontroller/kvflowcontroller.go index a18472414715..073aa741e153 100644 --- a/pkg/kv/kvserver/kvflowcontrol/kvflowcontroller/kvflowcontroller.go +++ b/pkg/kv/kvserver/kvflowcontrol/kvflowcontroller/kvflowcontroller.go @@ -65,8 +65,9 @@ type Controller struct { // minutes), clear these out. buckets map[kvflowcontrol.Stream]bucket } - metrics *metrics - clock *hlc.Clock + metrics *metrics + clock *hlc.Clock + settings *cluster.Settings } var _ kvflowcontrol.Controller = &Controller{} @@ -74,7 +75,8 @@ var _ kvflowcontrol.Controller = &Controller{} // New constructs a new Controller. func New(registry *metric.Registry, settings *cluster.Settings, clock *hlc.Clock) *Controller { c := &Controller{ - clock: clock, + clock: clock, + settings: settings, } regularTokens := kvflowcontrol.Tokens(regularTokensPerStream.Get(&settings.SV)) @@ -115,6 +117,10 @@ func New(registry *metric.Registry, settings *cluster.Settings, clock *hlc.Clock return c } +func (c *Controller) mode() kvflowcontrol.ModeT { + return kvflowcontrol.ModeT(kvflowcontrol.Mode.Get(&c.settings.SV)) +} + // Admit is part of the kvflowcontrol.Controller interface. It blocks until // there are flow tokens available for replication over the given stream for // work of the given priority. @@ -135,10 +141,15 @@ func (c *Controller) Admit( tokens := b.tokens[class] c.mu.Unlock() - if tokens > 0 { + if tokens > 0 || + // In addition to letting requests through when there are tokens + // being available, we'll also let them through if we're not + // applying flow control to their specific work class. + c.mode() == kvflowcontrol.ApplyToElastic && class == admissionpb.RegularWorkClass { + if log.ExpensiveLogEnabled(ctx, 2) { - log.Infof(ctx, "flow tokens available (pri=%s stream=%s tokens=%s wait-duration=%s)", - pri, connection.Stream(), tokens, c.clock.PhysicalTime().Sub(tstart)) + log.Infof(ctx, "admitted request (pri=%s stream=%s tokens=%s wait-duration=%s mode=%s)", + pri, connection.Stream(), tokens, c.clock.PhysicalTime().Sub(tstart), c.mode()) } // TODO(irfansharif): Right now we continue forwarding admission diff --git a/pkg/kv/kvserver/kvflowcontrol/kvflowcontroller/kvflowcontroller_metrics.go b/pkg/kv/kvserver/kvflowcontrol/kvflowcontroller/kvflowcontroller_metrics.go index 41c2561575ed..2882ce56fbc2 100644 --- a/pkg/kv/kvserver/kvflowcontrol/kvflowcontroller/kvflowcontroller_metrics.go +++ b/pkg/kv/kvserver/kvflowcontrol/kvflowcontroller/kvflowcontroller_metrics.go @@ -11,11 +11,14 @@ package kvflowcontroller import ( + "context" "fmt" + "strings" "time" "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/util/admission/admissionpb" + "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/metric" ) @@ -197,17 +200,39 @@ func newMetrics(c *Controller) *metrics { return int64(len(c.mu.buckets)) }, ) + + var blockedStreamLogger = log.Every(30 * time.Second) + var buf strings.Builder m.BlockedStreamCount[wc] = metric.NewFunctionalGauge( annotateMetricTemplateWithWorkClass(wc, blockedStreamCount), func() int64 { + shouldLog := blockedStreamLogger.ShouldLog() + count := int64(0) c.mu.Lock() defer c.mu.Unlock() - for _, wbc := range c.mu.buckets { + + for s, wbc := range c.mu.buckets { if wbc.tokens[wc] <= 0 { count += 1 + + if shouldLog { + if count > 10 { + continue // cap output to 10 blocked streams + } + if count == 1 { + buf.Reset() + } + if count > 1 { + buf.WriteString(", ") + } + buf.WriteString(s.String()) + } } } + if shouldLog && count > 0 { + log.Warningf(context.Background(), "%d blocked %s replication stream(s): %s", count, wc, buf.String()) + } return count }, ) diff --git a/pkg/kv/kvserver/kvflowcontrol/kvflowcontrolpb/BUILD.bazel b/pkg/kv/kvserver/kvflowcontrol/kvflowcontrolpb/BUILD.bazel index 0a2fd87eda43..e73bf09982a5 100644 --- a/pkg/kv/kvserver/kvflowcontrol/kvflowcontrolpb/BUILD.bazel +++ b/pkg/kv/kvserver/kvflowcontrol/kvflowcontrolpb/BUILD.bazel @@ -25,11 +25,14 @@ go_proto_library( go_library( name = "kvflowcontrolpb", - srcs = ["raft_log_position.go"], + srcs = ["kvflowcontrol.go"], embed = [":kvflowcontrolpb_go_proto"], importpath = "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol/kvflowcontrolpb", visibility = ["//visibility:public"], - deps = ["@com_github_cockroachdb_redact//:redact"], + deps = [ + "//pkg/util/admission/admissionpb", + "@com_github_cockroachdb_redact//:redact", + ], ) get_x_data(name = "get_x_data") diff --git a/pkg/kv/kvserver/kvflowcontrol/kvflowcontrolpb/raft_log_position.go b/pkg/kv/kvserver/kvflowcontrol/kvflowcontrolpb/kvflowcontrol.go similarity index 72% rename from pkg/kv/kvserver/kvflowcontrol/kvflowcontrolpb/raft_log_position.go rename to pkg/kv/kvserver/kvflowcontrol/kvflowcontrolpb/kvflowcontrol.go index c363012cc24d..cb1c7f0a7621 100644 --- a/pkg/kv/kvserver/kvflowcontrol/kvflowcontrolpb/raft_log_position.go +++ b/pkg/kv/kvserver/kvflowcontrol/kvflowcontrolpb/kvflowcontrol.go @@ -10,7 +10,10 @@ package kvflowcontrolpb -import "github.com/cockroachdb/redact" +import ( + "github.com/cockroachdb/cockroach/pkg/util/admission/admissionpb" + "github.com/cockroachdb/redact" +) func (p RaftLogPosition) String() string { return redact.StringWithoutMarkers(p) @@ -40,3 +43,13 @@ func (p *RaftLogPosition) Less(o RaftLogPosition) bool { func (p *RaftLogPosition) LessEq(o RaftLogPosition) bool { return p.Less(o) || p.Equal(o) } + +func (a AdmittedRaftLogEntries) String() string { + return redact.StringWithoutMarkers(a) +} + +// SafeFormat implements the redact.SafeFormatter interface. +func (a AdmittedRaftLogEntries) SafeFormat(w redact.SafePrinter, _ rune) { + w.Printf("admitted-entries (r%s s%s pri=%s up-to-%s)", + a.RangeID, a.StoreID, admissionpb.WorkPriority(a.AdmissionPriority), a.UpToRaftLogPosition) +} diff --git a/pkg/kv/kvserver/kvflowcontrol/kvflowcontrolpb/kvflowcontrol.proto b/pkg/kv/kvserver/kvflowcontrol/kvflowcontrolpb/kvflowcontrol.proto index 890209973ab6..d0a0d6830bf8 100644 --- a/pkg/kv/kvserver/kvflowcontrol/kvflowcontrolpb/kvflowcontrol.proto +++ b/pkg/kv/kvserver/kvflowcontrol/kvflowcontrolpb/kvflowcontrol.proto @@ -82,6 +82,8 @@ message RaftAdmissionMeta { // this particular "replication stream" (i.e. flowing to a particular store, // remote or otherwise). message AdmittedRaftLogEntries { + option (gogoproto.goproto_stringer) = false; + // RangeID of the raft group these entries belong to. This is the range on // whose behalf work was admitted. int64 range_id = 1 [(gogoproto.customname) = "RangeID", diff --git a/pkg/kv/kvserver/kvflowcontrol/kvflowhandle/BUILD.bazel b/pkg/kv/kvserver/kvflowcontrol/kvflowhandle/BUILD.bazel index 0b4379c2a72b..269aaf135876 100644 --- a/pkg/kv/kvserver/kvflowcontrol/kvflowhandle/BUILD.bazel +++ b/pkg/kv/kvserver/kvflowcontrol/kvflowhandle/BUILD.bazel @@ -7,6 +7,7 @@ go_library( "connected_stream.go", "kvflowhandle.go", "kvflowhandle_metrics.go", + "noop.go", ], importpath = "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol/kvflowhandle", visibility = ["//visibility:public"], @@ -15,6 +16,7 @@ go_library( "//pkg/kv/kvserver/kvflowcontrol", "//pkg/kv/kvserver/kvflowcontrol/kvflowcontrolpb", "//pkg/kv/kvserver/kvflowcontrol/kvflowtokentracker", + "//pkg/roachpb", "//pkg/util/admission/admissionpb", "//pkg/util/hlc", "//pkg/util/log", diff --git a/pkg/kv/kvserver/kvflowcontrol/kvflowhandle/kvflowhandle.go b/pkg/kv/kvserver/kvflowcontrol/kvflowhandle/kvflowhandle.go index 0ff5ebf18a10..d9aea13641bd 100644 --- a/pkg/kv/kvserver/kvflowcontrol/kvflowhandle/kvflowhandle.go +++ b/pkg/kv/kvserver/kvflowcontrol/kvflowhandle/kvflowhandle.go @@ -18,6 +18,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol/kvflowcontrolpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol/kvflowtokentracker" + "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/util/admission/admissionpb" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -31,6 +32,8 @@ type Handle struct { controller kvflowcontrol.Controller metrics *Metrics clock *hlc.Clock + rangeID roachpb.RangeID + tenantID roachpb.TenantID mu struct { syncutil.Mutex @@ -45,11 +48,22 @@ type Handle struct { } // New constructs a new Handle. -func New(controller kvflowcontrol.Controller, metrics *Metrics, clock *hlc.Clock) *Handle { +func New( + controller kvflowcontrol.Controller, + metrics *Metrics, + clock *hlc.Clock, + rangeID roachpb.RangeID, + tenantID roachpb.TenantID, +) *Handle { + if metrics == nil { // only nil in tests + metrics = NewMetrics(nil) + } h := &Handle{ controller: controller, metrics: metrics, clock: clock, + rangeID: rangeID, + tenantID: tenantID, } h.mu.perStreamTokenTracker = map[kvflowcontrol.Stream]*kvflowtokentracker.Tracker{} return h @@ -126,9 +140,12 @@ func (h *Handle) deductTokensForInner( } for _, c := range h.mu.connections { - h.controller.DeductTokens(ctx, pri, tokens, c.Stream()) - h.mu.perStreamTokenTracker[c.Stream()].Track(ctx, pri, tokens, pos) - streams = append(streams, c.Stream()) + if h.mu.perStreamTokenTracker[c.Stream()].Track(ctx, pri, tokens, pos) { + // Only deduct tokens if we're able to track them for subsequent + // returns. We risk leaking flow tokens otherwise. + h.controller.DeductTokens(ctx, pri, tokens, c.Stream()) + streams = append(streams, c.Stream()) + } } return streams } @@ -160,6 +177,14 @@ func (h *Handle) ReturnTokensUpto( return } + if !stream.TenantID.IsSet() { + // NB: The tenant ID is set in the local fast path for token returns, + // through the kvflowcontrol.Dispatch. Tecnically we could set the + // tenant ID by looking up the local replica and reading it, but it's + // easier to do it this way having captured it when the handle was + // instantiated. + stream.TenantID = h.tenantID + } h.mu.Lock() defer h.mu.Unlock() if h.mu.closed { @@ -175,6 +200,13 @@ func (h *Handle) ReturnTokensUpto( func (h *Handle) ConnectStream( ctx context.Context, pos kvflowcontrolpb.RaftLogPosition, stream kvflowcontrol.Stream, ) { + if !stream.TenantID.IsSet() { + // See comment in (*Handle).ReturnTokensUpto above where this same check + // exists. The callers here do typically have this set, but it doesn't + // hurt to be defensive. + stream.TenantID = h.tenantID + } + h.mu.Lock() defer h.mu.Unlock() if h.mu.closed { @@ -182,6 +214,12 @@ func (h *Handle) ConnectStream( return } + h.connectStreamLocked(ctx, pos, stream) +} + +func (h *Handle) connectStreamLocked( + ctx context.Context, pos kvflowcontrolpb.RaftLogPosition, stream kvflowcontrol.Stream, +) { if _, ok := h.mu.perStreamTokenTracker[stream]; ok { log.Fatalf(ctx, "reconnecting already connected stream: %s", stream) } @@ -195,23 +233,54 @@ func (h *Handle) ConnectStream( // that case, this sorting will help avoid deadlocks. return h.mu.connections[i].Stream().StoreID < h.mu.connections[j].Stream().StoreID }) - h.mu.perStreamTokenTracker[stream] = kvflowtokentracker.New(pos, nil /* knobs */) + h.mu.perStreamTokenTracker[stream] = kvflowtokentracker.New(pos, stream, nil /* knobs */) + h.metrics.StreamsConnected.Inc(1) + log.VInfof(ctx, 1, "connected to stream: %s", stream) } // DisconnectStream is part of the kvflowcontrol.Handle interface. func (h *Handle) DisconnectStream(ctx context.Context, stream kvflowcontrol.Stream) { + if !stream.TenantID.IsSet() { + // See comment in (*Handle).ReturnTokensUpto above where this same check + // exists. The callers here do typically have this set, but it doesn't + // hurt to be defensive. + stream.TenantID = h.tenantID + } h.mu.Lock() defer h.mu.Unlock() h.disconnectStreamLocked(ctx, stream) } +// ResetStreams is part of the kvflowcontrol.Handle interface. +func (h *Handle) ResetStreams(ctx context.Context) { + h.mu.Lock() + defer h.mu.Unlock() + if h.mu.closed { + log.Errorf(ctx, "operating on a closed handle") + return + } + + var streams []kvflowcontrol.Stream + var lowerBounds []kvflowcontrolpb.RaftLogPosition + for stream, tracker := range h.mu.perStreamTokenTracker { + streams = append(streams, stream) + lowerBounds = append(lowerBounds, tracker.LowerBound()) + } + for i := range streams { + h.disconnectStreamLocked(ctx, streams[i]) + } + for i := range streams { + h.connectStreamLocked(ctx, lowerBounds[i], streams[i]) + } +} + func (h *Handle) disconnectStreamLocked(ctx context.Context, stream kvflowcontrol.Stream) { if h.mu.closed { log.Errorf(ctx, "operating on a closed handle") return } if _, ok := h.mu.perStreamTokenTracker[stream]; !ok { - log.Fatalf(ctx, "disconnecting non-existent stream: %s", stream) + return } h.mu.perStreamTokenTracker[stream].Iter(ctx, @@ -232,6 +301,8 @@ func (h *Handle) disconnectStreamLocked(ctx context.Context, stream kvflowcontro connection.Disconnect() h.mu.connections = append(h.mu.connections[:streamIdx], h.mu.connections[streamIdx+1:]...) + log.VInfof(ctx, 1, "disconnected stream: %s", stream) + h.metrics.StreamsDisconnected.Inc(1) // TODO(irfansharif): Optionally record lower bound raft log positions for // disconnected streams to guard against regressions when (re-)connecting -- // it must be done with higher positions. @@ -250,8 +321,12 @@ func (h *Handle) Close(ctx context.Context) { return } - for _, connection := range h.mu.connections { - h.disconnectStreamLocked(ctx, connection.Stream()) + var streams []kvflowcontrol.Stream + for stream := range h.mu.perStreamTokenTracker { + streams = append(streams, stream) + } + for _, stream := range streams { + h.disconnectStreamLocked(ctx, stream) } h.mu.closed = true } diff --git a/pkg/kv/kvserver/kvflowcontrol/kvflowhandle/kvflowhandle_metrics.go b/pkg/kv/kvserver/kvflowcontrol/kvflowhandle/kvflowhandle_metrics.go index 656805d39b66..5006781b53e8 100644 --- a/pkg/kv/kvserver/kvflowcontrol/kvflowhandle/kvflowhandle_metrics.go +++ b/pkg/kv/kvserver/kvflowcontrol/kvflowhandle/kvflowhandle_metrics.go @@ -20,6 +20,20 @@ import ( ) var ( + streamsConnected = metric.Metadata{ + Name: "kvadmission.flow_handle.streams_connected", + Help: "Number of times we've connected to a stream, at the handle level", + Measurement: "Streams", + Unit: metric.Unit_COUNT, + } + + streamsDisconnected = metric.Metadata{ + Name: "kvadmission.flow_handle.streams_disconnected", + Help: "Number of times we've disconnected from a stream, at the handle level", + Measurement: "Streams", + Unit: metric.Unit_COUNT, + } + requestsWaiting = metric.Metadata{ Name: "kvadmission.flow_handle.%s_requests_waiting", Help: "Number of %s requests waiting for flow tokens, at the handle level", @@ -62,17 +76,23 @@ func annotateMetricTemplateWithWorkClass( // Metrics is a metric.Struct for all kvflowcontrol.Handles. type Metrics struct { - RequestsWaiting [admissionpb.NumWorkClasses]*metric.Gauge - RequestsAdmitted [admissionpb.NumWorkClasses]*metric.Counter - RequestsErrored [admissionpb.NumWorkClasses]*metric.Counter - WaitDuration [admissionpb.NumWorkClasses]metric.IHistogram + StreamsConnected *metric.Counter + StreamsDisconnected *metric.Counter + RequestsWaiting [admissionpb.NumWorkClasses]*metric.Gauge + RequestsAdmitted [admissionpb.NumWorkClasses]*metric.Counter + RequestsErrored [admissionpb.NumWorkClasses]*metric.Counter + WaitDuration [admissionpb.NumWorkClasses]metric.IHistogram } var _ metric.Struct = &Metrics{} // NewMetrics returns a new instance of Metrics. func NewMetrics(registry *metric.Registry) *Metrics { - m := &Metrics{} + m := &Metrics{ + StreamsConnected: metric.NewCounter(streamsConnected), + StreamsDisconnected: metric.NewCounter(streamsDisconnected), + } + for _, wc := range []admissionpb.WorkClass{ admissionpb.RegularWorkClass, admissionpb.ElasticWorkClass, diff --git a/pkg/kv/kvserver/kvflowcontrol/kvflowhandle/kvflowhandle_test.go b/pkg/kv/kvserver/kvflowcontrol/kvflowhandle/kvflowhandle_test.go index a4f1182ac19f..f9a7dda82cc9 100644 --- a/pkg/kv/kvserver/kvflowcontrol/kvflowhandle/kvflowhandle_test.go +++ b/pkg/kv/kvserver/kvflowcontrol/kvflowhandle/kvflowhandle_test.go @@ -32,7 +32,8 @@ import ( // TestHandleAdmit tests the blocking behavior of Handle.Admit(): // - we block until there are flow tokens available; // - we unblock when streams without flow tokens are disconnected; -// - we unblock when the handle is closed. +// - we unblock when the handle is closed; +// - we unblock when the handle is reset. func TestHandleAdmit(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) @@ -69,12 +70,30 @@ func TestHandleAdmit(t *testing.T) { handle.Close(ctx) }, }, + { + name: "unblocked-when-reset", + unblockFn: func(ctx context.Context, handle kvflowcontrol.Handle) { + // Reset all streams on the handle; the call to .Admit() should + // unblock. + handle.ResetStreams(ctx) + }, + }, } { t.Run(tc.name, func(t *testing.T) { registry := metric.NewRegistry() clock := hlc.NewClockForTesting(nil) - controller := kvflowcontroller.New(registry, cluster.MakeTestingClusterSettings(), clock) - handle := kvflowhandle.New(controller, kvflowhandle.NewMetrics(registry), clock) + st := cluster.MakeTestingClusterSettings() + kvflowcontrol.Enabled.Override(ctx, &st.SV, true) + kvflowcontrol.Mode.Override(ctx, &st.SV, int64(kvflowcontrol.ApplyToAll)) + + controller := kvflowcontroller.New(registry, st, clock) + handle := kvflowhandle.New( + controller, + kvflowhandle.NewMetrics(registry), + clock, + roachpb.RangeID(1), + roachpb.SystemTenantID, + ) // Connect a single stream at pos=0 and deplete all 16MiB of regular // tokens at pos=1. @@ -105,3 +124,102 @@ func TestHandleAdmit(t *testing.T) { }) } } + +func TestFlowControlMode(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + ctx := context.Background() + stream := kvflowcontrol.Stream{ + TenantID: roachpb.MustMakeTenantID(42), + StoreID: roachpb.StoreID(42), + } + pos := func(d uint64) kvflowcontrolpb.RaftLogPosition { + return kvflowcontrolpb.RaftLogPosition{Term: 1, Index: d} + } + + for _, tc := range []struct { + mode kvflowcontrol.ModeT + blocks, ignores []admissionpb.WorkClass + }{ + { + mode: kvflowcontrol.ApplyToElastic, + blocks: []admissionpb.WorkClass{ + admissionpb.ElasticWorkClass, + }, + ignores: []admissionpb.WorkClass{ + admissionpb.RegularWorkClass, + }, + }, + { + mode: kvflowcontrol.ApplyToAll, + blocks: []admissionpb.WorkClass{ + admissionpb.ElasticWorkClass, admissionpb.RegularWorkClass, + }, + ignores: []admissionpb.WorkClass{}, + }, + } { + t.Run(tc.mode.String(), func(t *testing.T) { + registry := metric.NewRegistry() + clock := hlc.NewClockForTesting(nil) + st := cluster.MakeTestingClusterSettings() + kvflowcontrol.Enabled.Override(ctx, &st.SV, true) + kvflowcontrol.Mode.Override(ctx, &st.SV, int64(tc.mode)) + + controller := kvflowcontroller.New(registry, st, clock) + handle := kvflowhandle.New( + controller, + kvflowhandle.NewMetrics(registry), + clock, + roachpb.RangeID(1), + roachpb.SystemTenantID, + ) + defer handle.Close(ctx) + + // Connect a single stream at pos=0 and deplete all 16MiB of regular + // tokens at pos=1. It also puts elastic tokens in the -ve. + handle.ConnectStream(ctx, pos(0), stream) + handle.DeductTokensFor(ctx, admissionpb.NormalPri, pos(1), kvflowcontrol.Tokens(16<<20 /* 16MiB */)) + + // Invoke .Admit() for {regular,elastic} work in a separate + // goroutines, and test below whether the goroutines are blocked. + regularAdmitCh := make(chan struct{}) + elasticAdmitCh := make(chan struct{}) + go func() { + require.NoError(t, handle.Admit(ctx, admissionpb.NormalPri, time.Time{})) + close(regularAdmitCh) + }() + go func() { + require.NoError(t, handle.Admit(ctx, admissionpb.BulkNormalPri, time.Time{})) + close(elasticAdmitCh) + }() + + for _, ignoredClass := range tc.ignores { // work should not block + classAdmitCh := regularAdmitCh + if ignoredClass == admissionpb.ElasticWorkClass { + classAdmitCh = elasticAdmitCh + } + + select { + case <-classAdmitCh: + case <-time.After(5 * time.Second): + t.Fatalf("%s work didn't get admitted", ignoredClass) + } + } + + for _, blockedClass := range tc.blocks { // work should get blocked + classAdmitCh := regularAdmitCh + if blockedClass == admissionpb.ElasticWorkClass { + classAdmitCh = elasticAdmitCh + } + + select { + case <-classAdmitCh: + t.Fatalf("unexpectedly admitted %s work", blockedClass) + case <-time.After(10 * time.Millisecond): + } + } + }) + } + +} diff --git a/pkg/kv/kvserver/kvflowcontrol/kvflowhandle/noop.go b/pkg/kv/kvserver/kvflowcontrol/kvflowhandle/noop.go new file mode 100644 index 000000000000..5a311b265fa0 --- /dev/null +++ b/pkg/kv/kvserver/kvflowcontrol/kvflowhandle/noop.go @@ -0,0 +1,63 @@ +// Copyright 2023 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package kvflowhandle + +import ( + "context" + "time" + + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol/kvflowcontrolpb" + "github.com/cockroachdb/cockroach/pkg/util/admission/admissionpb" +) + +// Noop is a no-op implementation of the kvflowcontrol.Handle interface. +type Noop struct{} + +var _ kvflowcontrol.Handle = Noop{} + +// Admit is part of the kvflowcontrol.Handle interface. +func (n Noop) Admit(ctx context.Context, priority admissionpb.WorkPriority, time time.Time) error { + return nil +} + +// DeductTokensFor is part of the kvflowcontrol.Handle interface. +func (n Noop) DeductTokensFor( + ctx context.Context, + priority admissionpb.WorkPriority, + position kvflowcontrolpb.RaftLogPosition, + tokens kvflowcontrol.Tokens, +) { +} + +// ReturnTokensUpto is part of the kvflowcontrol.Handle interface. +func (n Noop) ReturnTokensUpto( + ctx context.Context, + priority admissionpb.WorkPriority, + position kvflowcontrolpb.RaftLogPosition, + stream kvflowcontrol.Stream, +) { +} + +// ConnectStream is part of the kvflowcontrol.Handle interface. +func (n Noop) ConnectStream( + ctx context.Context, position kvflowcontrolpb.RaftLogPosition, stream kvflowcontrol.Stream, +) { +} + +// DisconnectStream is part of the kvflowcontrol.Handle interface. +func (n Noop) DisconnectStream(ctx context.Context, stream kvflowcontrol.Stream) {} + +// ResetStreams is part of the kvflowcontrol.Handle interface. +func (n Noop) ResetStreams(ctx context.Context) {} + +// Close is part of the kvflowcontrol.Handle interface. +func (n Noop) Close(ctx context.Context) {} diff --git a/pkg/kv/kvserver/kvflowcontrol/kvflowsimulator/simulation_test.go b/pkg/kv/kvserver/kvflowcontrol/kvflowsimulator/simulation_test.go index d898c39ce43a..7bb66c32d45d 100644 --- a/pkg/kv/kvserver/kvflowcontrol/kvflowsimulator/simulation_test.go +++ b/pkg/kv/kvserver/kvflowcontrol/kvflowsimulator/simulation_test.go @@ -176,7 +176,7 @@ func TestUsingSimulation(t *testing.T) { for _, line := range strings.Split(d.Input, "\n") { name := strings.TrimPrefix(strings.TrimSpace(line), "handle=") replicaHandles[name] = &replicaHandle{ - handle: kvflowhandle.New(controller, hmetrics, clock), + handle: kvflowhandle.New(controller, hmetrics, clock, roachpb.RangeID(0), roachpb.TenantID{}), deductionTracker: make(map[kvflowcontrol.Stream]*kvflowtokentracker.Tracker), outstandingReturns: make(map[kvflowcontrol.Stream]kvflowcontrol.Tokens), snapshots: make([]snapshot, 0), @@ -632,7 +632,7 @@ func (h *replicaHandle) deductTokens( h.quorumLogPosition.Index += 1 streams := h.handle.TestingDeductTokensForInner(ctx, pri, h.quorumLogPosition, tokens) for _, stream := range streams { - h.deductionTracker[stream].Track(ctx, pri, tokens, h.quorumLogPosition) + _ = h.deductionTracker[stream].Track(ctx, pri, tokens, h.quorumLogPosition) } } @@ -988,7 +988,7 @@ func (ht *handleOpTicker) tick(ctx context.Context, t time.Time) { case "connect": ht.replicaHandle.quorumLogPosition = ht.position ht.replicaHandle.handle.ConnectStream(ctx, ht.position, ht.stream) - ht.replicaHandle.deductionTracker[ht.stream] = kvflowtokentracker.New(ht.position, nil /* knobs */) + ht.replicaHandle.deductionTracker[ht.stream] = kvflowtokentracker.New(ht.position, ht.stream, nil /* knobs */) case "snapshot": ht.replicaHandle.snapshots = append(ht.replicaHandle.snapshots, snapshot{ time: t, diff --git a/pkg/kv/kvserver/kvflowcontrol/kvflowsimulator/testdata/handle_stream_disconnection b/pkg/kv/kvserver/kvflowcontrol/kvflowsimulator/testdata/handle_stream_disconnection index 60bbf31d8d9f..290b409a57cf 100644 --- a/pkg/kv/kvserver/kvflowcontrol/kvflowsimulator/testdata/handle_stream_disconnection +++ b/pkg/kv/kvserver/kvflowcontrol/kvflowsimulator/testdata/handle_stream_disconnection @@ -31,13 +31,13 @@ simulate # # As soon as s3 is disconnected, we see a release of 16MiB of held tokens back # into the node-level controller (32MiB -> 48MiB). We see a burst in the number -# of stream-specific/controller-lvel requests bypassing Admit() due to the +# of stream-specific/controller-level requests bypassing Admit() due to the # stream disconnecting. At the handle-level this just appears as a burst in # admitted requests. After s3 disconnects, the handle-level admission rate goes # back to what it was before traffic was shaped by s3. # # TODO(irfansharif): The post-stream disconnection burst might lead to -# severe over-admission since it may have been long since we observed availble +# severe over-admission since it may have been long since we observed available # tokens for the still connected streams. In fact, many requests that started # waiting on the soon-to-be-disconnected-stream are in the same boat, all of # which will now get admitted. One thing we could do is to try and observe diff --git a/pkg/kv/kvserver/kvflowcontrol/kvflowtokentracker/BUILD.bazel b/pkg/kv/kvserver/kvflowcontrol/kvflowtokentracker/BUILD.bazel index bd3536c6fec0..3c1bb6a39379 100644 --- a/pkg/kv/kvserver/kvflowcontrol/kvflowtokentracker/BUILD.bazel +++ b/pkg/kv/kvserver/kvflowcontrol/kvflowtokentracker/BUILD.bazel @@ -10,6 +10,7 @@ go_library( "//pkg/kv/kvserver/kvflowcontrol", "//pkg/kv/kvserver/kvflowcontrol/kvflowcontrolpb", "//pkg/util/admission/admissionpb", + "//pkg/util/buildutil", "//pkg/util/log", ], ) diff --git a/pkg/kv/kvserver/kvflowcontrol/kvflowtokentracker/tracker.go b/pkg/kv/kvserver/kvflowcontrol/kvflowtokentracker/tracker.go index 5afc6aff7068..73578a1de596 100644 --- a/pkg/kv/kvserver/kvflowcontrol/kvflowtokentracker/tracker.go +++ b/pkg/kv/kvserver/kvflowcontrol/kvflowtokentracker/tracker.go @@ -18,6 +18,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol/kvflowcontrolpb" "github.com/cockroachdb/cockroach/pkg/util/admission/admissionpb" + "github.com/cockroachdb/cockroach/pkg/util/buildutil" "github.com/cockroachdb/cockroach/pkg/util/log" ) @@ -32,6 +33,8 @@ type Tracker struct { // we ignore token deductions. lowerBound kvflowcontrolpb.RaftLogPosition + stream kvflowcontrol.Stream // used for logging only + knobs *kvflowcontrol.TestingKnobs } @@ -45,7 +48,11 @@ type tracked struct { // New constructs a new Tracker with the given lower bound raft log position // (below which we're not allowed to deduct tokens). -func New(lb kvflowcontrolpb.RaftLogPosition, knobs *kvflowcontrol.TestingKnobs) *Tracker { +func New( + lb kvflowcontrolpb.RaftLogPosition, + stream kvflowcontrol.Stream, + knobs *kvflowcontrol.TestingKnobs, +) *Tracker { if knobs == nil { knobs = &kvflowcontrol.TestingKnobs{} } @@ -53,6 +60,7 @@ func New(lb kvflowcontrolpb.RaftLogPosition, knobs *kvflowcontrol.TestingKnobs) trackedM: make(map[admissionpb.WorkPriority][]tracked), lowerBound: lb, knobs: knobs, + stream: stream, } } @@ -63,7 +71,7 @@ func (dt *Tracker) Track( pri admissionpb.WorkPriority, tokens kvflowcontrol.Tokens, pos kvflowcontrolpb.RaftLogPosition, -) { +) bool { if !(dt.lowerBound.Less(pos)) { // We're trying to track a token deduction at a position less than the // stream's lower-bound. Shout loudly but ultimately no-op. This @@ -78,23 +86,37 @@ func (dt *Tracker) Track( // Handle.ConnectStream). // - token returns upto some log position don't precede deductions at // lower log positions (see Handle.ReturnTokensUpto); - log.Errorf(ctx, "observed raft log position less than per-stream lower bound (%s <= %s)", + logFn := log.Errorf + if buildutil.CrdbTestBuild { + logFn = log.Fatalf + } + logFn(ctx, "observed raft log position less than per-stream lower bound (%s <= %s)", pos, dt.lowerBound) - return + return false } dt.lowerBound = pos if len(dt.trackedM[pri]) >= 1 { last := dt.trackedM[pri][len(dt.trackedM[pri])-1] if !last.position.Less(pos) { - log.Fatalf(ctx, "expected in order tracked log positions (%s < %s)", + logFn := log.Errorf + if buildutil.CrdbTestBuild { + logFn = log.Fatalf + } + logFn(ctx, "expected in order tracked log positions (%s < %s)", last.position, pos) + return false } } dt.trackedM[pri] = append(dt.trackedM[pri], tracked{ tokens: tokens, position: pos, }) + if log.ExpensiveLogEnabled(ctx, 1) { + log.Infof(ctx, "tracking %s flow control tokens for pri=%s stream=%s pos=%s", + tokens, pri, dt.stream, pos) + } + return true } // Untrack all token deductions of the given priority that have log positions @@ -136,8 +158,8 @@ func (dt *Tracker) Untrack( if len(dt.trackedM[pri]) > 0 { remaining = fmt.Sprintf(" (%s, ...)", dt.trackedM[pri][0].tokens) } - log.VInfof(ctx, 1, "released flow control tokens for %d/%d pri=%s tracked deductions, upto %s; %d tracked deduction(s) remain%s", - untracked, trackedBefore, pri, upto, len(dt.trackedM[pri]), remaining) + log.Infof(ctx, "released %s flow control tokens for %d out of %d tracked deductions for pri=%s stream=%s, up to %s; %d tracked deduction(s) remain%s", + tokens, untracked, trackedBefore, pri, dt.stream, upto, len(dt.trackedM[pri]), remaining) } if len(dt.trackedM[pri]) == 0 { delete(dt.trackedM, pri) @@ -161,6 +183,11 @@ func (dt *Tracker) Iter(_ context.Context, f func(admissionpb.WorkPriority, kvfl } } +// LowerBound returns the log position below which we ignore token deductions. +func (dt *Tracker) LowerBound() kvflowcontrolpb.RaftLogPosition { + return dt.lowerBound +} + // TestingIter is a testing-only re-implementation of Iter. It iterates through // all tracked token deductions, invoking the provided callback with tracked // pri<->token<->position triples. diff --git a/pkg/kv/kvserver/kvflowcontrol/kvflowtokentracker/tracker_test.go b/pkg/kv/kvserver/kvflowcontrol/kvflowtokentracker/tracker_test.go index 7793501e0b75..58d4ed4063b5 100644 --- a/pkg/kv/kvserver/kvflowcontrol/kvflowtokentracker/tracker_test.go +++ b/pkg/kv/kvserver/kvflowcontrol/kvflowtokentracker/tracker_test.go @@ -39,7 +39,7 @@ func TestTracker(t *testing.T) { datadriven.RunTest(t, path, func(t *testing.T, d *datadriven.TestData) string { switch d.Cmd { case "init": - tracker = New(kvflowcontrolpb.RaftLogPosition{Term: 1, Index: 0}, knobs) + tracker = New(kvflowcontrolpb.RaftLogPosition{Term: 1, Index: 0}, kvflowcontrol.Stream{}, knobs) return "" case "track": @@ -81,7 +81,7 @@ func TestTracker(t *testing.T) { t.Fatalf("unrecognized prefix: %s", parts[i]) } } - tracker.Track(ctx, pri, tokens, logPosition) + require.True(t, tracker.Track(ctx, pri, tokens, logPosition)) } return "" diff --git a/pkg/kv/kvserver/raftlog/encoding_test.go b/pkg/kv/kvserver/raftlog/encoding_test.go index 0e5734d9d5bd..1d22660bf411 100644 --- a/pkg/kv/kvserver/raftlog/encoding_test.go +++ b/pkg/kv/kvserver/raftlog/encoding_test.go @@ -93,7 +93,7 @@ func BenchmarkRaftAdmissionMetaOverhead(b *testing.B) { // 4. If using below-raft admission, decode the raft // metadata. if withRaftAdmissionMeta { - _, err = DecodeRaftAdmissionMeta(encodingBuf) + _, err := DecodeRaftAdmissionMeta(encodingBuf) require.NoError(b, err) } diff --git a/pkg/kv/kvserver/replica.go b/pkg/kv/kvserver/replica.go index 97b5637832a5..0a3645cd89ba 100644 --- a/pkg/kv/kvserver/replica.go +++ b/pkg/kv/kvserver/replica.go @@ -801,6 +801,16 @@ type Replica struct { pausedFollowers map[roachpb.ReplicaID]struct{} slowProposalCount int64 // updated in refreshProposalsLocked + + // replicaFlowControlIntegration is used to interface with replication flow + // control. It's backed by the node-level kvflowcontrol.Controller that + // manages flow tokens for on a per basis, which it + // interfaces through a replica-level kvflowcontrol.Handle. It's + // actively used on replicas initiating replication traffic, i.e. are + // both the leaseholder and raft leader. + // + // Accessing it requires Replica.mu to be held, exclusively. + replicaFlowControlIntegration replicaFlowControlIntegration } // The raft log truncations that are pending. Access is protected by its own diff --git a/pkg/kv/kvserver/replica_app_batch.go b/pkg/kv/kvserver/replica_app_batch.go index 706d945e80b1..6e588aac1c12 100644 --- a/pkg/kv/kvserver/replica_app_batch.go +++ b/pkg/kv/kvserver/replica_app_batch.go @@ -250,7 +250,12 @@ func (b *replicaAppBatch) runPostAddTriggersReplicaOnly( // We don't track these stats in standalone log application since they depend // on whether the proposer is still waiting locally, and this concept does not // apply in a standalone context. - if !cmd.IsLocal() { + // + // TODO(irfansharif): This code block can be removed once below-raft + // admission control is the only form of IO admission control. It pre-dates + // it -- these stats were previously used to deduct IO tokens for follower + // writes/ingests without waiting. + if !cmd.IsLocal() && !cmd.ApplyAdmissionControl() { writeBytes, ingestedBytes := cmd.getStoreWriteByteSizes() b.followerStoreWriteBytes.NumEntries++ b.followerStoreWriteBytes.WriteBytes += writeBytes diff --git a/pkg/kv/kvserver/replica_application_cmd.go b/pkg/kv/kvserver/replica_application_cmd.go index d82eddffdc86..ae2cc0a228f3 100644 --- a/pkg/kv/kvserver/replica_application_cmd.go +++ b/pkg/kv/kvserver/replica_application_cmd.go @@ -71,6 +71,12 @@ func (c *replicatedCmd) IsLocal() bool { return c.proposal != nil } +// ApplyAdmissionControl indicates whether the command should be +// subject to replication admission control. +func (c *replicatedCmd) ApplyAdmissionControl() bool { + return c.Entry.ApplyAdmissionControl +} + // Ctx implements apply.Command. func (c *replicatedCmd) Ctx() context.Context { return c.ctx diff --git a/pkg/kv/kvserver/replica_application_result.go b/pkg/kv/kvserver/replica_application_result.go index aac8f3ae81b2..769e85bc6fde 100644 --- a/pkg/kv/kvserver/replica_application_result.go +++ b/pkg/kv/kvserver/replica_application_result.go @@ -281,6 +281,10 @@ func (r *Replica) tryReproposeWithNewLeaseIndex( // Some tests check for this log message in the trace. log.VEventf(ctx, 2, "retry: proposalIllegalLeaseIndex") + // See I7 from kvflowcontrol/doc.go: we don't re-deduct flow tokens on + // reproposals. + p.raftAdmissionMeta = nil + pErr := r.propose(ctx, p, tok.Move(ctx)) if pErr != nil { return pErr diff --git a/pkg/kv/kvserver/replica_destroy.go b/pkg/kv/kvserver/replica_destroy.go index 26a3ed2a6ed6..8a836067ac66 100644 --- a/pkg/kv/kvserver/replica_destroy.go +++ b/pkg/kv/kvserver/replica_destroy.go @@ -162,7 +162,7 @@ func (r *Replica) destroyRaftMuLocked(ctx context.Context, nextReplicaID roachpb // disconnectReplicationRaftMuLocked is called when a Replica is being removed. // It cancels all outstanding proposals, closes the proposalQuota if there -// is one, and removes the in-memory raft state. +// is one, releases all held flow tokens, and removes the in-memory raft state. func (r *Replica) disconnectReplicationRaftMuLocked(ctx context.Context) { r.raftMu.AssertHeld() r.mu.Lock() @@ -174,6 +174,7 @@ func (r *Replica) disconnectReplicationRaftMuLocked(ctx context.Context) { if pq := r.mu.proposalQuota; pq != nil { pq.Close("destroyed") } + r.mu.replicaFlowControlIntegration.onReplicaDestroyed(ctx) r.mu.proposalBuf.FlushLockedWithoutProposing(ctx) for _, p := range r.mu.proposals { r.cleanupFailedProposalLocked(p) diff --git a/pkg/kv/kvserver/replica_init.go b/pkg/kv/kvserver/replica_init.go index 734716ef32de..f152596728b0 100644 --- a/pkg/kv/kvserver/replica_init.go +++ b/pkg/kv/kvserver/replica_init.go @@ -161,6 +161,10 @@ func newUninitializedReplica( r.breaker = newReplicaCircuitBreaker( store.cfg.Settings, store.stopper, r.AmbientContext, r, onTrip, onReset, ) + r.mu.replicaFlowControlIntegration = newReplicaFlowControlIntegration( + (*replicaFlowControl)(r), + makeStoreFlowControlHandleFactory(r.store), + ) return r } @@ -367,6 +371,7 @@ func (r *Replica) setDescLockedRaftMuLocked(ctx context.Context, desc *roachpb.R r.connectionClass.set(rpc.ConnectionClassForKey(desc.StartKey)) r.concMgr.OnRangeDescUpdated(desc) r.mu.state.Desc = desc + r.mu.replicaFlowControlIntegration.onDescChanged(ctx) // Give the liveness and meta ranges high priority in the Raft scheduler, to // avoid head-of-line blocking and high scheduling latency. diff --git a/pkg/kv/kvserver/replica_init_test.go b/pkg/kv/kvserver/replica_init_test.go index cc2cae9389a6..c3109cfa39c3 100644 --- a/pkg/kv/kvserver/replica_init_test.go +++ b/pkg/kv/kvserver/replica_init_test.go @@ -64,6 +64,7 @@ func TestReplicaUpdateLastReplicaAdded(t *testing.T) { var r Replica r.mu.state.Desc = &c.oldDesc r.mu.lastReplicaAdded = c.lastReplicaAdded + r.mu.replicaFlowControlIntegration = newReplicaFlowControlIntegration((*replicaFlowControl)(&r), nil) r.store = tc.store r.concMgr = tc.repl.concMgr r.setDescRaftMuLocked(context.Background(), &c.newDesc) diff --git a/pkg/kv/kvserver/replica_proposal.go b/pkg/kv/kvserver/replica_proposal.go index ea03b019b5d7..e0e3e7aa5e22 100644 --- a/pkg/kv/kvserver/replica_proposal.go +++ b/pkg/kv/kvserver/replica_proposal.go @@ -20,6 +20,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol/kvflowcontrolpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/logstore" @@ -125,6 +126,16 @@ type ProposalData struct { // tok identifies the request to the propBuf. Once the proposal is made, the // token will be used to stop tracking this request. tok TrackedRequestToken + + // raftAdmissionMeta captures the metadata we encode as part of the command + // when first proposed for replication admission control. + raftAdmissionMeta *kvflowcontrolpb.RaftAdmissionMeta +} + +// useReplicationAdmissionControl indicates whether this raft command should +// be subject to replication admission control. +func (proposal *ProposalData) useReplicationAdmissionControl() bool { + return proposal.raftAdmissionMeta != nil } // finishApplication is called when a command application has finished. The diff --git a/pkg/kv/kvserver/replica_proposal_buf.go b/pkg/kv/kvserver/replica_proposal_buf.go index ee232f57ac93..d97220444277 100644 --- a/pkg/kv/kvserver/replica_proposal_buf.go +++ b/pkg/kv/kvserver/replica_proposal_buf.go @@ -12,16 +12,21 @@ package kvserver import ( "context" + "fmt" "sync" "sync/atomic" "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts/tracker" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol/kvflowcontrolpb" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol/kvflowhandle" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness/livenesspb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/raftutil" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/util/admission/admissionpb" "github.com/cockroachdb/cockroach/pkg/util/errorutil" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -132,10 +137,16 @@ type rangeLeaderInfo struct { leaderEligibleForLease bool } +type admitEntHandle struct { + handle *kvflowcontrolpb.RaftAdmissionMeta + pCtx context.Context +} + // A proposer is an object that uses a propBuf to coordinate Raft proposals. type proposer interface { locker() sync.Locker rlocker() sync.Locker + // The following require the proposer to hold (at least) a shared lock. getReplicaID() roachpb.ReplicaID destroyed() destroyStatus @@ -146,6 +157,7 @@ type proposer interface { leaderStatus(ctx context.Context, raftGroup proposerRaft) rangeLeaderInfo ownsValidLease(ctx context.Context, now hlc.ClockTimestamp) bool shouldCampaignOnRedirect(raftGroup proposerRaft) bool + flowControlHandle(ctx context.Context) kvflowcontrol.Handle // The following require the proposer to hold an exclusive lock. withGroupLocked(func(proposerRaft) error) error @@ -407,6 +419,11 @@ func (b *propBuf) FlushLockedWithRaftGroup( buf := b.arr.asSlice()[:used] ents := make([]raftpb.Entry, 0, used) + // Use this slice to track, for each entry that's proposed to raft, whether + // it's subject to replication admission control. Updated in tandem with + // slice above. + admitHandles := make([]admitEntHandle, 0, used) + // Compute the closed timestamp target, which will be used to assign a closed // timestamp to all proposals in this batch. closedTSTarget := b.p.closedTimestampTarget() @@ -490,11 +507,17 @@ func (b *propBuf) FlushLockedWithRaftGroup( // Flush any previously batched (non-conf change) proposals to // preserve the correct ordering or proposals. Later proposals // will start a new batch. - if err := proposeBatch(raftGroup, b.p.getReplicaID(), ents); err != nil { - firstErr = err + propErr, dropped := proposeBatch(raftGroup, b.p.getReplicaID(), ents) + if propErr != nil { + firstErr = propErr continue } + if !dropped { + b.maybeDeductFlowTokens(ctx, admitHandles, ents) + } + ents = ents[len(ents):] + admitHandles = admitHandles[len(admitHandles):] confChangeCtx := kvserverpb.ConfChangeContext{ CommandID: string(p.idKey), @@ -536,12 +559,38 @@ func (b *propBuf) FlushLockedWithRaftGroup( Data: p.encodedCommand, }) log.VEvent(p.ctx, 2, "flushing proposal to Raft") + + // We don't want deduct flow tokens for reproposed commands, and of + // course for proposals that didn't integrate with kvflowcontrol. + shouldAdmit := p.createdAtTicks == p.proposedAtTicks && !reproposal && p.raftAdmissionMeta != nil + if !shouldAdmit { + admitHandles = append(admitHandles, admitEntHandle{}) + } else { + admitHandles = append(admitHandles, admitEntHandle{ + handle: p.raftAdmissionMeta, + pCtx: p.ctx, + }) + } } } if firstErr != nil { return 0, firstErr } - return used, proposeBatch(raftGroup, b.p.getReplicaID(), ents) + + propErr, dropped := proposeBatch(raftGroup, b.p.getReplicaID(), ents) + if propErr == nil && !dropped { + // Now that we know what raft log position[1] this proposal is to end up + // in, deduct flow tokens for it. This is done without blocking (we've + // already waited for available flow tokens pre-evaluation). The tokens + // will later be returned once we're informed of the entry being + // admitted below raft. + // + // [1]: We're relying on an undocumented side effect of upstream raft + // API where it populates the index and term for the passed in + // slice of entries. See etcd-io/raft#57. + b.maybeDeductFlowTokens(ctx, admitHandles, ents) + } + return used, propErr } // maybeRejectUnsafeProposalLocked conditionally rejects proposals that are @@ -900,9 +949,11 @@ func (b *propBuf) forwardClosedTimestampLocked(closedTS hlc.Timestamp) bool { return b.assignedClosedTimestamp.Forward(closedTS) } -func proposeBatch(raftGroup proposerRaft, replID roachpb.ReplicaID, ents []raftpb.Entry) error { +func proposeBatch( + raftGroup proposerRaft, replID roachpb.ReplicaID, ents []raftpb.Entry, +) (_ error, dropped bool) { if len(ents) == 0 { - return nil + return nil, false } if err := raftGroup.Step(raftpb.Message{ Type: raftpb.MsgProp, @@ -913,11 +964,41 @@ func proposeBatch(raftGroup proposerRaft, replID roachpb.ReplicaID, ents []raftp // ignored prior to the introduction of ErrProposalDropped). // TODO(bdarnell): Handle ErrProposalDropped better. // https://github.com/cockroachdb/cockroach/issues/21849 - return nil + return nil, true } else if err != nil { - return err + return err, false + } + return nil, false +} + +func (b *propBuf) maybeDeductFlowTokens( + ctx context.Context, admitHandles []admitEntHandle, ents []raftpb.Entry, +) { + if len(admitHandles) != len(ents) || cap(admitHandles) != cap(ents) { + panic( + fmt.Sprintf("mismatched slice sizes: len(admit)=%d len(ents)=%d cap(admit)=%d cap(ents)=%d", + len(admitHandles), len(ents), cap(admitHandles), cap(ents)), + ) + } + for i, admitHandle := range admitHandles { + if admitHandle.handle == nil { + continue // nothing to do + } + log.VInfof(ctx, 1, "bound index/log terms for proposal entry: %s", + raft.DescribeEntry(ents[i], func(bytes []byte) string { + return "" + }), + ) + b.p.flowControlHandle(ctx).DeductTokensFor( + admitHandle.pCtx, + admissionpb.WorkPriority(admitHandle.handle.AdmissionPriority), + kvflowcontrolpb.RaftLogPosition{ + Term: ents[i].Term, + Index: ents[i].Index, + }, + kvflowcontrol.Tokens(int64(len(ents[i].Data))), + ) } - return nil } // FlushLockedWithoutProposing is like FlushLockedWithRaftGroup but it does not @@ -1259,6 +1340,14 @@ func (rp *replicaProposer) shouldCampaignOnRedirect(raftGroup proposerRaft) bool ) } +func (rp *replicaProposer) flowControlHandle(ctx context.Context) kvflowcontrol.Handle { + handle, found := rp.mu.replicaFlowControlIntegration.handle() + if !found { + return kvflowhandle.Noop{} + } + return handle +} + // rejectProposalWithRedirectLocked is part of the proposer interface. func (rp *replicaProposer) rejectProposalWithRedirectLocked( ctx context.Context, prop *ProposalData, redirectTo roachpb.ReplicaID, diff --git a/pkg/kv/kvserver/replica_proposal_buf_test.go b/pkg/kv/kvserver/replica_proposal_buf_test.go index 8cf5bb470540..161b0892d25d 100644 --- a/pkg/kv/kvserver/replica_proposal_buf_test.go +++ b/pkg/kv/kvserver/replica_proposal_buf_test.go @@ -21,12 +21,15 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts/tracker" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol/kvflowcontrolpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/raftlog" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/raftutil" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/util/admission/admissionpb" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -139,6 +142,9 @@ func (t *testProposer) locker() sync.Locker { func (t *testProposer) rlocker() sync.Locker { return t.RWMutex.RLocker() } +func (t *testProposer) flowControlHandle(ctx context.Context) kvflowcontrol.Handle { + return &testFlowTokenHandle{} +} func (t *testProposer) getReplicaID() roachpb.ReplicaID { return 1 @@ -1047,3 +1053,43 @@ func (t mockTracker) Count() int { } var _ tracker.Tracker = mockTracker{} + +type testFlowTokenHandle struct{} + +var _ kvflowcontrol.Handle = &testFlowTokenHandle{} + +func (t *testFlowTokenHandle) Admit( + ctx context.Context, priority admissionpb.WorkPriority, t2 time.Time, +) error { + return nil +} + +func (t *testFlowTokenHandle) DeductTokensFor( + ctx context.Context, + priority admissionpb.WorkPriority, + position kvflowcontrolpb.RaftLogPosition, + tokens kvflowcontrol.Tokens, +) { +} + +func (t *testFlowTokenHandle) ReturnTokensUpto( + ctx context.Context, + priority admissionpb.WorkPriority, + position kvflowcontrolpb.RaftLogPosition, + stream kvflowcontrol.Stream, +) { +} + +func (t *testFlowTokenHandle) ConnectStream( + ctx context.Context, position kvflowcontrolpb.RaftLogPosition, stream kvflowcontrol.Stream, +) { +} + +func (t *testFlowTokenHandle) DisconnectStream(ctx context.Context, stream kvflowcontrol.Stream) { +} + +func (t *testFlowTokenHandle) ResetStreams(ctx context.Context) { +} + +func (t *testFlowTokenHandle) Close(ctx context.Context) { +} diff --git a/pkg/kv/kvserver/replica_proposal_quota.go b/pkg/kv/kvserver/replica_proposal_quota.go index 33b1b5f9adc1..fb61391d8bf1 100644 --- a/pkg/kv/kvserver/replica_proposal_quota.go +++ b/pkg/kv/kvserver/replica_proposal_quota.go @@ -120,6 +120,7 @@ func (r *Replica) updateProposalQuotaRaftMuLocked( ) r.mu.lastUpdateTimes = make(map[roachpb.ReplicaID]time.Time) r.mu.lastUpdateTimes.updateOnBecomeLeader(r.mu.state.Desc.Replicas().Descriptors(), timeutil.Now()) + r.mu.replicaFlowControlIntegration.onBecameLeader(ctx) r.mu.lastProposalAtTicks = r.mu.ticks // delay imminent quiescence } else if r.mu.proposalQuota != nil { // We're becoming a follower. @@ -130,6 +131,7 @@ func (r *Replica) updateProposalQuotaRaftMuLocked( r.mu.quotaReleaseQueue = nil r.mu.proposalQuota = nil r.mu.lastUpdateTimes = nil + r.mu.replicaFlowControlIntegration.onBecameFollower(ctx) } return } else if r.mu.proposalQuota == nil { @@ -162,10 +164,10 @@ func (r *Replica) updateProposalQuotaRaftMuLocked( // Only consider followers that are active. Inactive ones don't decrease // minIndex - i.e. they don't hold up releasing quota. // - // The policy for determining who's active is more strict than the one used - // for purposes of quiescing. Failure to consider a dead/stuck node as such - // for the purposes of releasing quota can have bad consequences (writes - // will stall), whereas for quiescing the downside is lower. + // The policy for determining who's active is stricter than the one used + // for purposes of quiescing. Failure to consider a dead/stuck node as + // such for the purposes of releasing quota can have bad consequences + // (writes will stall), whereas for quiescing the downside is lower. if !r.mu.lastUpdateTimes.isFollowerActiveSince( ctx, rep.ReplicaID, now, r.store.cfg.RangeLeaseDuration, @@ -260,4 +262,5 @@ func (r *Replica) updateProposalQuotaRaftMuLocked( r.mu.proposalQuotaBaseIndex, len(r.mu.quotaReleaseQueue), releasableIndex, status.Applied) } + r.mu.replicaFlowControlIntegration.onProposalQuotaUpdated(ctx) } diff --git a/pkg/kv/kvserver/replica_raft.go b/pkg/kv/kvserver/replica_raft.go index 6714e4a493b7..22bbf796e7b4 100644 --- a/pkg/kv/kvserver/replica_raft.go +++ b/pkg/kv/kvserver/replica_raft.go @@ -23,6 +23,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/poison" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvadmission" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol/kvflowcontrolpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness/livenesspb" @@ -33,6 +35,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/util" + "github.com/cockroachdb/cockroach/pkg/util/admission/admissionpb" "github.com/cockroachdb/cockroach/pkg/util/buildutil" "github.com/cockroachdb/cockroach/pkg/util/encoding" "github.com/cockroachdb/cockroach/pkg/util/envutil" @@ -227,6 +230,10 @@ func (r *Replica) evalAndPropose( // Continue with proposal... } + if meta := kvflowcontrol.MetaFromContext(ctx); meta != nil { + proposal.raftAdmissionMeta = meta + } + // Attach information about the proposer's lease to the command, for // verification below raft. Lease requests are special since they are not // necessarily proposed under a valid lease (by necessity). Instead, they @@ -377,6 +384,9 @@ func (r *Replica) propose( // Determine the encoding style for the Raft command. prefix := true entryEncoding := raftlog.EntryEncodingStandardWithoutAC + if p.useReplicationAdmissionControl() { + entryEncoding = raftlog.EntryEncodingStandardWithAC + } if crt := p.command.ReplicatedEvalResult.ChangeReplicas; crt != nil { // EndTxnRequest with a ChangeReplicasTrigger is special because Raft // needs to understand it; it cannot simply be an opaque command. To @@ -451,6 +461,9 @@ func (r *Replica) propose( } else if p.command.ReplicatedEvalResult.AddSSTable != nil { log.VEvent(p.ctx, 4, "sideloadable proposal detected") entryEncoding = raftlog.EntryEncodingSideloadedWithoutAC + if p.useReplicationAdmissionControl() { + entryEncoding = raftlog.EntryEncodingSideloadedWithAC + } r.store.metrics.AddSSTableProposals.Inc(1) if p.command.ReplicatedEvalResult.AddSSTable.Data == nil { @@ -460,12 +473,25 @@ func (r *Replica) propose( log.Infof(p.ctx, "proposing command %x: %s", p.idKey, p.Request.Summary()) } + // NB: If (significantly) re-working how raft commands are encoded, make the + // equivalent change in raftlog.BenchmarkRaftAdmissionMetaOverhead. + // Create encoding buffer. preLen := 0 if prefix { preLen = raftlog.RaftCommandPrefixLen } - cmdLen := p.command.Size() + + raftAdmissionMeta := &kvflowcontrolpb.RaftAdmissionMeta{} + var admissionMetaLen int + if p.useReplicationAdmissionControl() { + // Encode admission metadata data at the start, right after the command + // prefix. + raftAdmissionMeta = p.raftAdmissionMeta + admissionMetaLen = raftAdmissionMeta.Size() + } + + cmdLen := p.command.Size() + admissionMetaLen // Allocate the data slice with enough capacity to eventually hold the two // "footers" that are filled later. needed := preLen + cmdLen + kvserverpb.MaxRaftCommandFooterSize() @@ -474,9 +500,40 @@ func (r *Replica) propose( if prefix { raftlog.EncodeRaftCommandPrefix(data, entryEncoding, p.idKey) } - // Encode body of command. + + // Encode the body of the command. data = data[:preLen+cmdLen] - if _, err := protoutil.MarshalTo(p.command, data[preLen:]); err != nil { + + // Encode below-raft admission data, if any. + if p.useReplicationAdmissionControl() { + if !prefix { + panic("expected to encode prefix for raft commands using replication admission control") + } + if buildutil.CrdbTestBuild { + if p.raftAdmissionMeta.AdmissionOriginNode == roachpb.NodeID(0) { + log.Fatalf(ctx, "missing origin node for flow token returns") + } + } + if _, err := protoutil.MarshalTo( + raftAdmissionMeta, + data[preLen:preLen+admissionMetaLen], + ); err != nil { + return kvpb.NewError(err) + } + log.VInfof(ctx, 1, "encoded raft admission meta: pri=%s create-time=%d proposer=n%s", + admissionpb.WorkPriority(raftAdmissionMeta.AdmissionPriority), + raftAdmissionMeta.AdmissionCreateTime, + raftAdmissionMeta.AdmissionOriginNode, + ) + // Zero out what we've already encoded and marshaled, out of an + // abundance of paranoia. + p.command.AdmissionPriority = 0 + p.command.AdmissionCreateTime = 0 + p.command.AdmissionOriginNode = 0 + } + + // Encode the rest of the command. + if _, err := protoutil.MarshalTo(p.command, data[preLen+admissionMetaLen:]); err != nil { return kvpb.NewError(err) } p.encodedCommand = data @@ -1001,6 +1058,20 @@ func (r *Replica) handleRaftReadyRaftMuLocked( } m := logstore.MakeMsgStorageAppend(msgStorageAppend) cb := (*replicaSyncCallback)(r) + if r.IsInitialized() && r.store.cfg.KVAdmissionController != nil { + // Enqueue raft log entries into admission queues. This is + // non-blocking; actual admission happens asynchronously. + tenantID, _ := r.TenantID() + for _, entry := range msgStorageAppend.Entries { + if len(entry.Data) == 0 { + continue // nothing to do + } + r.store.cfg.KVAdmissionController.AdmitRaftEntry( + ctx, tenantID, r.StoreID(), r.RangeID, entry, + ) + } + } + if state, err = s.StoreEntries(ctx, state, m, cb, &stats.append); err != nil { return stats, errors.Wrap(err, "while storing log entries") } @@ -1457,7 +1528,7 @@ func (r *Replica) refreshProposalsLocked( return } - log.VInfof(ctx, 1, + log.VInfof(ctx, 2, "pending commands: reproposing %d (at applied index %d, lease applied index %d) %s", len(reproposals), r.mu.state.RaftAppliedIndex, r.mu.state.LeaseAppliedIndex, reason) diff --git a/pkg/kv/kvserver/replica_raft_overload.go b/pkg/kv/kvserver/replica_raft_overload.go index a3ab9ee30757..04d829151d3b 100644 --- a/pkg/kv/kvserver/replica_raft_overload.go +++ b/pkg/kv/kvserver/replica_raft_overload.go @@ -374,4 +374,5 @@ func (r *Replica) updatePausedFollowersLocked(ctx context.Context, ioThresholdMa // with more wasted work. r.mu.internalRaftGroup.ReportUnreachable(uint64(replicaID)) } + r.mu.replicaFlowControlIntegration.onFollowersPaused(ctx) } diff --git a/pkg/kv/kvserver/store.go b/pkg/kv/kvserver/store.go index da9ad9a3a504..e5b7da4adf43 100644 --- a/pkg/kv/kvserver/store.go +++ b/pkg/kv/kvserver/store.go @@ -43,6 +43,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/idalloc" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/intentresolver" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvadmission" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol/kvflowhandle" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvstorage" @@ -1143,8 +1145,15 @@ type StoreConfig struct { // SpanConfigsDisabled is unset. SpanConfigSubscriber spanconfig.KVSubscriber - // KVAdmissionController is an optional field used for admission control. + // KVAdmissionController is used for admission control. KVAdmissionController kvadmission.Controller + // KVFlowController is used for replication admission control. + KVFlowController kvflowcontrol.Controller + // KVFlowHandles is used for replication admission control. + KVFlowHandles kvflowcontrol.Handles + // KVFlowHandleMetrics is a shared metrics struct for all + // kvflowcontrol.Handles. + KVFlowHandleMetrics *kvflowhandle.Metrics // SchedulerLatencyListener listens in on scheduling latencies, information // that's then used to adjust various admission control components (like how @@ -2572,7 +2581,7 @@ func ReadMaxHLCUpperBound(ctx context.Context, engines []storage.Engine) (int64, // GetReplica fetches a replica by Range ID. Returns an error if no replica is found. // -// See also GetReplicaIfExists for a more perfomant version. +// See also GetReplicaIfExists for a more performant version. func (s *Store) GetReplica(rangeID roachpb.RangeID) (*Replica, error) { if r := s.GetReplicaIfExists(rangeID); r != nil { return r, nil diff --git a/pkg/roachpb/metadata_replicas.go b/pkg/roachpb/metadata_replicas.go index 1217ed878167..9c764568c9e0 100644 --- a/pkg/roachpb/metadata_replicas.go +++ b/pkg/roachpb/metadata_replicas.go @@ -483,6 +483,26 @@ func (d ReplicaSet) ReplicationTargets() (out []ReplicationTarget) { return out } +// Difference compares two sets of replicas, returning the replica descriptors +// that were added and removed when going from one to the other. 'd' is the before +// state, 'o' is the one after. +func (d ReplicaSet) Difference(o ReplicaSet) (added, removed []ReplicaDescriptor) { + return o.Subtract(d), d.Subtract(o) +} + +// Subtract one sets of replicas from another. This returning the replica +// descriptors that were present in the original and not the other. 'd' is the +// original set of descriptors, 'o' is the other. +func (d ReplicaSet) Subtract(o ReplicaSet) []ReplicaDescriptor { + var repls []ReplicaDescriptor + for _, repl := range d.Descriptors() { + if _, found := o.GetReplicaDescriptorByID(repl.ReplicaID); !found { + repls = append(repls, repl) + } + } + return repls +} + // IsAddition returns true if `c` refers to a replica addition operation. func (c ReplicaChangeType) IsAddition() bool { switch c { diff --git a/pkg/roachpb/metadata_replicas_test.go b/pkg/roachpb/metadata_replicas_test.go index c63d877e0a98..90472940af9f 100644 --- a/pkg/roachpb/metadata_replicas_test.go +++ b/pkg/roachpb/metadata_replicas_test.go @@ -371,3 +371,43 @@ func TestReplicaDescriptorsCanMakeProgressRandom(t *testing.T) { log.Infof(ctx, "progress: %d cases. no progress: %d cases. skipped: %d cases.", progress, noProgress, skipped) } + +func TestReplicaSetOperations(t *testing.T) { + rs := func(ids ...uint64) ReplicaSet { + replicas := make([]ReplicaDescriptor, 0, len(ids)) + for _, id := range ids { + replicas = append(replicas, rd(VOTER_FULL, id)) + } + return MakeReplicaSet(replicas) + } + var empty []ReplicaDescriptor + t.Run("subtract", func(t *testing.T) { + require.Equal(t, rs(1, 2, 3).Subtract(rs(2, 3)), rs(1).Descriptors()) + require.Equal(t, rs(1, 2, 3).Subtract(rs()), rs(1, 2, 3).Descriptors()) + require.Equal(t, rs(1, 2, 3).Subtract(rs(4, 5, 6)), rs(1, 2, 3).Descriptors()) + require.Equal(t, rs(1, 2).Subtract(rs(6, 1)), rs(2).Descriptors()) + require.Equal(t, rs().Subtract(rs(6, 1)), empty) + }) + t.Run("difference", func(t *testing.T) { + { // {1,2,3}.difference({2,3,4}) + added, removed := rs(1, 2, 3).Difference(rs(2, 3, 4)) + require.Equal(t, added, rs(4).Descriptors()) + require.Equal(t, removed, rs(1).Descriptors()) + } + { // {1,2,3}.difference({1,2,3}) + added, removed := rs(1, 2, 3).Difference(rs(1, 2, 3)) + require.Equal(t, added, empty) + require.Equal(t, removed, empty) + } + { // {}.difference({1,2,3}) + added, removed := rs().Difference(rs(1, 2, 3)) + require.Equal(t, added, rs(1, 2, 3).Descriptors()) + require.Equal(t, removed, empty) + } + { // {1,2,3}.difference({}) + added, removed := rs(1, 2, 3).Difference(rs()) + require.Equal(t, added, empty) + require.Equal(t, removed, rs(1, 2, 3).Descriptors()) + } + }) +} diff --git a/pkg/server/BUILD.bazel b/pkg/server/BUILD.bazel index 219b1e095f71..1b2314975c67 100644 --- a/pkg/server/BUILD.bazel +++ b/pkg/server/BUILD.bazel @@ -7,6 +7,7 @@ go_library( "addjoin.go", "admin.go", "admin_test_utils.go", + "admission.go", "api_v2.go", "api_v2_auth.go", "api_v2_error.go", @@ -126,7 +127,11 @@ go_library( "//pkg/kv/kvserver/closedts/ctpb", "//pkg/kv/kvserver/closedts/sidetransport", "//pkg/kv/kvserver/kvadmission", + "//pkg/kv/kvserver/kvflowcontrol", + "//pkg/kv/kvserver/kvflowcontrol/kvflowcontroller", + "//pkg/kv/kvserver/kvflowcontrol/kvflowcontrolpb", "//pkg/kv/kvserver/kvflowcontrol/kvflowdispatch", + "//pkg/kv/kvserver/kvflowcontrol/kvflowhandle", "//pkg/kv/kvserver/kvserverbase", "//pkg/kv/kvserver/kvserverpb", "//pkg/kv/kvserver/kvstorage", diff --git a/pkg/server/admission.go b/pkg/server/admission.go new file mode 100644 index 000000000000..e738c6ad1eb4 --- /dev/null +++ b/pkg/server/admission.go @@ -0,0 +1,55 @@ +// Copyright 2023 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package server + +import ( + "context" + + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol/kvflowcontrolpb" + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/util/admission" + "github.com/cockroachdb/cockroach/pkg/util/admission/admissionpb" +) + +type admittedLogEntryAdaptor struct { + dispatchWriter kvflowcontrol.DispatchWriter +} + +var _ admission.OnLogEntryAdmitted = &admittedLogEntryAdaptor{} + +func newAdmittedLogEntryAdaptor( + dispatchWriter kvflowcontrol.DispatchWriter, +) *admittedLogEntryAdaptor { + return &admittedLogEntryAdaptor{ + dispatchWriter: dispatchWriter, + } +} + +// AdmittedLogEntry implements the admission.OnLogEntryAdmitted interface. +func (a *admittedLogEntryAdaptor) AdmittedLogEntry( + ctx context.Context, + origin roachpb.NodeID, + pri admissionpb.WorkPriority, + storeID roachpb.StoreID, + rangeID roachpb.RangeID, + pos admission.LogPosition, +) { + a.dispatchWriter.Dispatch(ctx, origin, kvflowcontrolpb.AdmittedRaftLogEntries{ + RangeID: rangeID, + AdmissionPriority: int32(pri), + UpToRaftLogPosition: kvflowcontrolpb.RaftLogPosition{ + Term: pos.Term, + Index: pos.Index, + }, + StoreID: storeID, + }) +} diff --git a/pkg/server/node.go b/pkg/server/node.go index 33a795f8d16f..bdd44d6c1a3e 100644 --- a/pkg/server/node.go +++ b/pkg/server/node.go @@ -433,10 +433,6 @@ func NewNode( testingErrorEvent: cfg.TestingKnobs.TestingResponseErrorEvent, spanStatsCollector: spanstatscollector.New(cfg.Settings), } - n.storeCfg.KVAdmissionController = kvadmission.MakeController( - kvAdmissionQ, elasticCPUGrantCoord, storeGrantCoords, cfg.Settings, - ) - n.storeCfg.SchedulerLatencyListener = elasticCPUGrantCoord.SchedulerLatencyListener n.perReplicaServer = kvserver.MakeServer(&n.Descriptor, n.stores) return n } @@ -1176,9 +1172,7 @@ func (n *Node) batchInternal( if err != nil { return nil, err } - if handle.ElasticCPUWorkHandle != nil { - ctx = admission.ContextWithElasticCPUWorkHandle(ctx, handle.ElasticCPUWorkHandle) - } + ctx = handle.AnnotateCtx(ctx) var writeBytes *kvadmission.StoreWriteBytes defer func() { diff --git a/pkg/server/server.go b/pkg/server/server.go index 9fd7b2dd7924..f6c4d714ad39 100644 --- a/pkg/server/server.go +++ b/pkg/server/server.go @@ -43,7 +43,11 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/allocator/storepool" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts/ctpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts/sidetransport" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvadmission" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol/kvflowcontroller" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol/kvflowdispatch" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol/kvflowhandle" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvstorage" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness" @@ -271,7 +275,6 @@ func NewServer(cfg Config, stopper *stop.Stopper) (*Server, error) { if opts, ok := cfg.TestingKnobs.AdmissionControl.(*admission.Options); ok { admissionOptions.Override(opts) } - gcoords := admission.NewGrantCoordinators(cfg.AmbientCtx, st, admissionOptions, registry, &admission.NoopOnLogEntryAdmitted{}) engines, err := cfg.CreateEngines(ctx) if err != nil { @@ -307,7 +310,7 @@ func NewServer(cfg Config, stopper *stop.Stopper) (*Server, error) { rpcCtxOpts := rpc.ContextOptions{ TenantID: roachpb.SystemTenantID, UseNodeAuth: true, - NodeID: cfg.IDContainer, + NodeID: nodeIDContainer, StorageClusterID: cfg.ClusterIDContainer, Config: cfg.Config, Clock: clock.WallClock(), @@ -455,17 +458,10 @@ func NewServer(cfg Config, stopper *stop.Stopper) (*Server, error) { } tcsFactory := kvcoord.NewTxnCoordSenderFactory(txnCoordSenderFactoryCfg, distSender) - cbID := goschedstats.RegisterRunnableCountCallback(gcoords.Regular.CPULoad) - stopper.AddCloser(stop.CloserFn(func() { - goschedstats.UnregisterRunnableCountCallback(cbID) - })) - stopper.AddCloser(gcoords) - dbCtx := kv.DefaultDBContext(stopper) dbCtx.NodeID = idContainer dbCtx.Stopper = stopper db := kv.NewDBWithContext(cfg.AmbientCtx, tcsFactory, clock, dbCtx) - db.SQLKVResponseAdmissionQ = gcoords.Regular.GetWorkQueue(admission.SQLKVResponseWork) nlActive, nlRenewal := cfg.NodeLivenessDurations() if knobs := cfg.TestingKnobs.NodeLiveness; knobs != nil { @@ -550,6 +546,47 @@ func NewServer(cfg Config, stopper *stop.Stopper) (*Server, error) { storesForFlowControl := kvserver.MakeStoresForFlowControl(stores) kvflowTokenDispatch := kvflowdispatch.New(registry, storesForFlowControl, nodeIDContainer) + admittedEntryAdaptor := newAdmittedLogEntryAdaptor(kvflowTokenDispatch) + gcoords := admission.NewGrantCoordinators( + cfg.AmbientCtx, + st, + admissionOptions, + registry, + admittedEntryAdaptor, + ) + db.SQLKVResponseAdmissionQ = gcoords.Regular.GetWorkQueue(admission.SQLKVResponseWork) + cbID := goschedstats.RegisterRunnableCountCallback(gcoords.Regular.CPULoad) + stopper.AddCloser(stop.CloserFn(func() { + goschedstats.UnregisterRunnableCountCallback(cbID) + })) + stopper.AddCloser(gcoords) + + var admissionControl struct { + schedulerLatencyListener admission.SchedulerLatencyListener + kvflowController kvflowcontrol.Controller + kvflowTokenDispatch kvflowcontrol.Dispatch + kvAdmissionController kvadmission.Controller + storesFlowControl kvserver.StoresForFlowControl + kvFlowHandleMetrics *kvflowhandle.Metrics + } + admissionControl.schedulerLatencyListener = gcoords.Elastic.SchedulerLatencyListener + admissionControl.kvflowController = kvflowcontroller.New(registry, st, clock) + admissionControl.kvflowTokenDispatch = kvflowTokenDispatch + admissionControl.storesFlowControl = storesForFlowControl + admissionControl.kvAdmissionController = kvadmission.MakeController( + nodeIDContainer, + gcoords.Regular.GetWorkQueue(admission.KVWork), + gcoords.Elastic, + gcoords.Stores, + admissionControl.kvflowController, + admissionControl.storesFlowControl, + cfg.Settings, + ) + admissionControl.kvFlowHandleMetrics = kvflowhandle.NewMetrics(registry) + kvflowcontrol.Mode.SetOnChange(&st.SV, func(ctx context.Context) { + admissionControl.storesFlowControl.ResetStreams(ctx) + }) + raftTransport := kvserver.NewRaftTransport( cfg.AmbientCtx, st, @@ -557,9 +594,9 @@ func NewServer(cfg Config, stopper *stop.Stopper) (*Server, error) { nodeDialer, grpcServer.Server, stopper, - kvflowTokenDispatch, - storesForFlowControl, - storesForFlowControl, + admissionControl.kvflowTokenDispatch, + admissionControl.storesFlowControl, + admissionControl.storesFlowControl, nil, /* knobs */ ) registry.AddMetricStruct(raftTransport.Metrics()) @@ -798,6 +835,11 @@ func NewServer(cfg Config, stopper *stop.Stopper) (*Server, error) { SnapshotApplyLimit: cfg.SnapshotApplyLimit, SnapshotSendLimit: cfg.SnapshotSendLimit, RangeLogWriter: rangeLogWriter, + KVAdmissionController: admissionControl.kvAdmissionController, + KVFlowController: admissionControl.kvflowController, + KVFlowHandles: admissionControl.storesFlowControl, + KVFlowHandleMetrics: admissionControl.kvFlowHandleMetrics, + SchedulerLatencyListener: admissionControl.schedulerLatencyListener, } if storeTestingKnobs := cfg.TestingKnobs.Store; storeTestingKnobs != nil { storeCfg.TestingKnobs = *storeTestingKnobs.(*kvserver.StoreTestingKnobs) diff --git a/pkg/util/admission/granter_test.go b/pkg/util/admission/granter_test.go index bae737636ac5..25fa20cd3bdf 100644 --- a/pkg/util/admission/granter_test.go +++ b/pkg/util/admission/granter_test.go @@ -98,7 +98,7 @@ func TestGranterBasic(t *testing.T) { return req } delayForGrantChainTermination = 0 - coords := NewGrantCoordinators(ambientCtx, settings, opts, registry, &NoopOnLogEntryAdmitted{}) + coords := NewGrantCoordinators(ambientCtx, settings, opts, registry, &noopOnLogEntryAdmitted{}) defer coords.Close() coord = coords.Regular return flushAndReset() @@ -323,7 +323,7 @@ func TestStoreCoordinators(t *testing.T) { return str }, } - coords := NewGrantCoordinators(ambientCtx, settings, opts, registry, &NoopOnLogEntryAdmitted{}) + coords := NewGrantCoordinators(ambientCtx, settings, opts, registry, &noopOnLogEntryAdmitted{}) // There is only 1 KVWork requester at this point in initialization, for the // Regular GrantCoordinator. require.Equal(t, 1, len(requesters)) diff --git a/pkg/util/admission/replicated_write_admission_test.go b/pkg/util/admission/replicated_write_admission_test.go index 4d1e368922dd..7fd37551e8fd 100644 --- a/pkg/util/admission/replicated_write_admission_test.go +++ b/pkg/util/admission/replicated_write_admission_test.go @@ -121,7 +121,7 @@ func TestReplicatedWriteAdmission(t *testing.T) { tg[admissionpb.RegularWorkClass], tg[admissionpb.ElasticWorkClass], }, - st, metrics, opts, knobs, &NoopOnLogEntryAdmitted{}, &mockCoordMu, + st, metrics, opts, knobs, &noopOnLogEntryAdmitted{}, &mockCoordMu, ).(*StoreWorkQueue) tg[admissionpb.RegularWorkClass].r = storeWorkQueue.getRequesters()[admissionpb.RegularWorkClass] tg[admissionpb.ElasticWorkClass].r = storeWorkQueue.getRequesters()[admissionpb.ElasticWorkClass] diff --git a/pkg/util/admission/work_queue.go b/pkg/util/admission/work_queue.go index 5d3ca55c968e..d7d2b344b731 100644 --- a/pkg/util/admission/work_queue.go +++ b/pkg/util/admission/work_queue.go @@ -627,8 +627,17 @@ func (q *WorkQueue) Admit(ctx context.Context, info WorkInfo) (enabled bool, err // by either adding more synchronization, getting rid of this // fast path, or swapping this entry from the top-most one in // the waiting heap (and fixing the heap). + if log.V(1) { + log.Infof(ctx, "fast-path: admitting t%d pri=%s r%s origin=n%s log-position=%s ingested=%t", + tenantID, info.Priority, + info.ReplicatedWorkInfo.RangeID, + info.ReplicatedWorkInfo.Origin, + info.ReplicatedWorkInfo.LogPosition.String(), + info.ReplicatedWorkInfo.Ingested, + ) + } q.onAdmittedReplicatedWork.admittedReplicatedWork( - roachpb.MustMakeTenantID(tenant.id), + roachpb.MustMakeTenantID(tenantID), info.Priority, info.ReplicatedWorkInfo, info.RequestedCount, @@ -722,6 +731,17 @@ func (q *WorkQueue) Admit(ctx context.Context, info WorkInfo) (enabled bool, err q.metrics.recordStartWait(info.Priority) if info.ReplicatedWorkInfo.Enabled { + if log.V(1) { + log.Infof(ctx, "async-path: len(waiting-work)=%d: enqueued t%d pri=%s r%s origin=n%s log-position=%s ingested=%t", + tenant.waitingWorkHeap.Len(), + tenant.id, info.Priority, + info.ReplicatedWorkInfo.RangeID, + info.ReplicatedWorkInfo.Origin, + info.ReplicatedWorkInfo.LogPosition, + info.ReplicatedWorkInfo.Ingested, + ) + } + return // return without waiting (admission is asynchronous) } @@ -2001,22 +2021,6 @@ type OnLogEntryAdmitted interface { ) } -// NoopOnLogEntryAdmitted is a no-op implementation of the OnLogEntryAdmitted -// interface. -type NoopOnLogEntryAdmitted struct{} - -var _ OnLogEntryAdmitted = &NoopOnLogEntryAdmitted{} - -func (n *NoopOnLogEntryAdmitted) AdmittedLogEntry( - context.Context, - roachpb.NodeID, - admissionpb.WorkPriority, - roachpb.StoreID, - roachpb.RangeID, - LogPosition, -) { -} - // AdmittedWorkDone indicates to the queue that the admitted work has completed. // It's used for the legacy above-raft admission control where we Admit() // upfront, with just an estimate of the write size, and after the write is diff --git a/pkg/util/admission/work_queue_test.go b/pkg/util/admission/work_queue_test.go index 66da1387e973..bdb287c81a6f 100644 --- a/pkg/util/admission/work_queue_test.go +++ b/pkg/util/admission/work_queue_test.go @@ -536,7 +536,7 @@ func TestStoreWorkQueueBasic(t *testing.T) { tg[admissionpb.RegularWorkClass], tg[admissionpb.ElasticWorkClass], }, - st, metrics, opts, nil /* testing knobs */, &NoopOnLogEntryAdmitted{}, &mockCoordMu).(*StoreWorkQueue) + st, metrics, opts, nil /* testing knobs */, &noopOnLogEntryAdmitted{}, &mockCoordMu).(*StoreWorkQueue) tg[admissionpb.RegularWorkClass].r = q.getRequesters()[admissionpb.RegularWorkClass] tg[admissionpb.ElasticWorkClass].r = q.getRequesters()[admissionpb.ElasticWorkClass] wrkMap.resetMap() diff --git a/pkg/util/metric/registry.go b/pkg/util/metric/registry.go index 8a9e071328c9..c665ab54a7e4 100644 --- a/pkg/util/metric/registry.go +++ b/pkg/util/metric/registry.go @@ -90,6 +90,10 @@ func (r *Registry) AddMetric(metric Iterable) { // AddMetricStruct examines all fields of metricStruct and adds // all Iterable or metric.Struct objects to the registry. func (r *Registry) AddMetricStruct(metricStruct interface{}) { + if r == nil { // for testing convenience + return + } + ctx := context.TODO() v := reflect.ValueOf(metricStruct) if v.Kind() == reflect.Ptr { From 44290bc2094591076d8d8e5f23bc8187c7e71260 Mon Sep 17 00:00:00 2001 From: irfan sharif Date: Thu, 6 Apr 2023 16:06:10 -0400 Subject: [PATCH 03/12] inspectz,kvflowinspectpb: add /inspectz pages This commit adds rudimentary /inspectz-style pages to CRDB. These hang off of /inspectz and support a few registered endpoints to inspect kvflowcontrol state in JSON form. - /inspectz/kvflowcontroller: marshals the state of registered flow control streams, and how many {regular,elastic} tokens are available for each one. - /inspectz/kvflowhandles: marshals the state of all in-memory kvflowcontrol.Handles held per leader+leaseholder replica, showing all deducted tokens and corresponding log positions, and what stream(s) each replica is connected to. It also supports querying for specific ranges through a ?ranges=[,] query parameter. To power these endpoints we introduced proto representations for various components under pkg/../kvflowcontrol/kvflowinspectpb. Select kvflowcontrol interfaces now expose an Inspect() method, returning the relevant proto. Other than just observing proto-state like the 90s, this commit also adds indexed crdb_internal vtables to do more sophisticated filtering of these inspectz-protos. It's easy to combine these tables to understand exactly which ranges are blocked on flow tokens, and for which streams in particular. They are: CREATE TABLE crdb_internal.kv_flow_controller ( tenant_id INT NOT NULL, store_id INT NOT NULL, available_regular_tokens INT NOT NULL, available_elastic_tokens INT NOT NULL ) CREATE TABLE crdb_internal.kv_flow_control_handles ( range_id INT NOT NULL, tenant_id INT NOT NULL, store_id INT NOT NULL, total_tracked_tokens INT NOT NULL, INDEX(range_id) ) CREATE TABLE crdb_internal.kv_flow_token_deductions ( range_id INT NOT NULL, tenant_id INT NOT NULL, store_id INT NOT NULL, priority STRING NOT NULL, log_term INT NOT NULL, log_index INT NOT NULL, tokens INT NOT NULL, INDEX(range_id) ) To see the set of ranges blocked on regular flow tokens, one can run something like: SELECT range_id, crdb_internal.humanize_bytes(available_regular_tokens) FROM crdb_internal.kv_flow_controller AS c INNER JOIN crdb_internal.kv_flow_control_handles AS hs ON c.tenant_id = hs.tenant_id AND c.store_id = hs.tenant_id WHERE available_regular_tokens <= 0; Or if looking to understand how many active replication streams each leaseholder+leader range is shaping write traffic through, something like: SELECT range_id, count(*) AS streams FROM crdb_internal.kv_flow_control_handles GROUP BY (range_id) ORDER BY streams DESC; Release note: None --- .github/CODEOWNERS | 2 + pkg/BUILD.bazel | 6 + .../testdata/logic_test/crdb_internal_tenant | 3 + pkg/cli/zip_test.go | 3 + pkg/gen/protobuf.bzl | 2 + pkg/inspectz/BUILD.bazel | 22 + pkg/inspectz/inspectz.go | 151 +++ pkg/inspectz/inspectzpb/BUILD.bazel | 36 + pkg/inspectz/inspectzpb/inspectz.proto | 42 + pkg/inspectz/unsupported.go | 39 + pkg/kv/kvserver/BUILD.bazel | 1 + pkg/kv/kvserver/flow_control_stores.go | 26 + pkg/kv/kvserver/kvflowcontrol/BUILD.bazel | 1 + .../kvserver/kvflowcontrol/kvflowcontrol.go | 11 + .../kvflowcontroller/BUILD.bazel | 1 + .../kvflowcontrol_token_adjustment_test.go | 2 +- .../kvflowcontroller/kvflowcontroller.go | 33 +- .../kvflowcontrol/kvflowhandle/BUILD.bazel | 1 + .../kvflowhandle/kvflowhandle.go | 19 + .../kvflowcontrol/kvflowhandle/noop.go | 6 + .../kvflowcontrol/kvflowinspectpb/BUILD.bazel | 38 + .../kvflowinspectpb/kvflowinspect.proto | 101 ++ .../kvflowtokentracker/BUILD.bazel | 1 + .../kvflowtokentracker/tracker.go | 16 + pkg/kv/kvserver/replica_proposal_buf_test.go | 5 + pkg/server/BUILD.bazel | 2 + pkg/server/server.go | 15 +- pkg/server/server_http.go | 7 + pkg/server/server_sql.go | 6 + pkg/server/tenant.go | 6 + pkg/sql/BUILD.bazel | 2 + pkg/sql/crdb_internal.go | 191 ++++ pkg/sql/exec_util.go | 5 + .../testdata/logic_test/crdb_internal | 3 + .../testdata/logic_test/crdb_internal_catalog | 884 +++++++++--------- .../logictest/testdata/logic_test/grant_table | 3 + .../testdata/logic_test/information_schema | 15 + .../logictest/testdata/logic_test/pg_builtins | 72 +- .../logictest/testdata/logic_test/pg_catalog | 246 ++--- pkg/sql/logictest/testdata/logic_test/table | 3 + pkg/sql/opt/exec/execbuilder/testdata/explain | 2 +- .../opt/xform/testdata/external/activerecord | 2 +- pkg/sql/planner.go | 1 + pkg/sql/sem/catconstants/constants.go | 3 + pkg/sql/sem/eval/BUILD.bazel | 1 + pkg/sql/sem/eval/context.go | 5 + 46 files changed, 1438 insertions(+), 604 deletions(-) create mode 100644 pkg/inspectz/BUILD.bazel create mode 100644 pkg/inspectz/inspectz.go create mode 100644 pkg/inspectz/inspectzpb/BUILD.bazel create mode 100644 pkg/inspectz/inspectzpb/inspectz.proto create mode 100644 pkg/inspectz/unsupported.go create mode 100644 pkg/kv/kvserver/kvflowcontrol/kvflowinspectpb/BUILD.bazel create mode 100644 pkg/kv/kvserver/kvflowcontrol/kvflowinspectpb/kvflowinspect.proto diff --git a/.github/CODEOWNERS b/.github/CODEOWNERS index e638c178b231..074f99d24e08 100644 --- a/.github/CODEOWNERS +++ b/.github/CODEOWNERS @@ -246,6 +246,8 @@ /pkg/geo/ @cockroachdb/spatial +/pkg/inspectz/ @cockroachdb/admission-control + # The KV team generally owns ./pkg/kv/... but not all of it. By convention, # inside of the /pkg/kv tree, we list out rules for each subdirectory, i.e. when # a new directory is created CODEOWNERS should mandate a new line below. This diff --git a/pkg/BUILD.bazel b/pkg/BUILD.bazel index b1c96905db16..47aa0fe55280 100644 --- a/pkg/BUILD.bazel +++ b/pkg/BUILD.bazel @@ -1184,6 +1184,8 @@ GO_TARGETS = [ "//pkg/gossip/simulation:simulation", "//pkg/gossip:gossip", "//pkg/gossip:gossip_test", + "//pkg/inspectz/inspectzpb:inspectzpb", + "//pkg/inspectz:inspectz", "//pkg/internal/client/requestbatcher:requestbatcher", "//pkg/internal/client/requestbatcher:requestbatcher_test", "//pkg/internal/codeowners:codeowners", @@ -1327,6 +1329,7 @@ GO_TARGETS = [ "//pkg/kv/kvserver/kvflowcontrol/kvflowdispatch:kvflowdispatch_test", "//pkg/kv/kvserver/kvflowcontrol/kvflowhandle:kvflowhandle", "//pkg/kv/kvserver/kvflowcontrol/kvflowhandle:kvflowhandle_test", + "//pkg/kv/kvserver/kvflowcontrol/kvflowinspectpb:kvflowinspectpb", "//pkg/kv/kvserver/kvflowcontrol/kvflowsimulator:kvflowsimulator_test", "//pkg/kv/kvserver/kvflowcontrol/kvflowtokentracker:kvflowtokentracker", "//pkg/kv/kvserver/kvflowcontrol/kvflowtokentracker:kvflowtokentracker_test", @@ -2722,6 +2725,8 @@ GET_X_DATA_TARGETS = [ "//pkg/geo/twkb:get_x_data", "//pkg/gossip:get_x_data", "//pkg/gossip/simulation:get_x_data", + "//pkg/inspectz:get_x_data", + "//pkg/inspectz/inspectzpb:get_x_data", "//pkg/internal/client/requestbatcher:get_x_data", "//pkg/internal/codeowners:get_x_data", "//pkg/internal/reporoot:get_x_data", @@ -2812,6 +2817,7 @@ GET_X_DATA_TARGETS = [ "//pkg/kv/kvserver/kvflowcontrol/kvflowcontrolpb:get_x_data", "//pkg/kv/kvserver/kvflowcontrol/kvflowdispatch:get_x_data", "//pkg/kv/kvserver/kvflowcontrol/kvflowhandle:get_x_data", + "//pkg/kv/kvserver/kvflowcontrol/kvflowinspectpb:get_x_data", "//pkg/kv/kvserver/kvflowcontrol/kvflowsimulator:get_x_data", "//pkg/kv/kvserver/kvflowcontrol/kvflowtokentracker:get_x_data", "//pkg/kv/kvserver/kvserverbase:get_x_data", diff --git a/pkg/ccl/logictestccl/testdata/logic_test/crdb_internal_tenant b/pkg/ccl/logictestccl/testdata/logic_test/crdb_internal_tenant index 472a62d33b45..c8c8c41839d7 100644 --- a/pkg/ccl/logictestccl/testdata/logic_test/crdb_internal_tenant +++ b/pkg/ccl/logictestccl/testdata/logic_test/crdb_internal_tenant @@ -64,6 +64,9 @@ crdb_internal kv_catalog_descriptor table admin NULL NULL crdb_internal kv_catalog_namespace table admin NULL NULL crdb_internal kv_catalog_zones table admin NULL NULL crdb_internal kv_dropped_relations view admin NULL NULL +crdb_internal kv_flow_control_handles table admin NULL NULL +crdb_internal kv_flow_controller table admin NULL NULL +crdb_internal kv_flow_token_deductions table admin NULL NULL crdb_internal kv_inherited_role_members table admin NULL NULL crdb_internal kv_node_liveness table admin NULL NULL crdb_internal kv_node_status table admin NULL NULL diff --git a/pkg/cli/zip_test.go b/pkg/cli/zip_test.go index 899d21f948a0..37533e83fa47 100644 --- a/pkg/cli/zip_test.go +++ b/pkg/cli/zip_test.go @@ -86,6 +86,9 @@ table_name NOT IN ( 'kv_catalog_zones', 'kv_dropped_relations', 'kv_inherited_role_members', + 'kv_flow_control_handles', + 'kv_flow_controller', + 'kv_flow_token_deductions', 'lost_descriptors_with_data', 'table_columns', 'table_row_statistics', diff --git a/pkg/gen/protobuf.bzl b/pkg/gen/protobuf.bzl index ce1acd15449f..83b96208c1b2 100644 --- a/pkg/gen/protobuf.bzl +++ b/pkg/gen/protobuf.bzl @@ -24,6 +24,7 @@ PROTOBUF_SRCS = [ "//pkg/geo/geoindex:geoindex_go_proto", "//pkg/geo/geopb:geopb_go_proto", "//pkg/gossip:gossip_go_proto", + "//pkg/inspectz/inspectzpb:inspectzpb_go_proto", "//pkg/jobs/jobspb:jobspb_go_proto", "//pkg/keyvisualizer/keyvispb:keyvispb_go_proto", "//pkg/kv/bulk/bulkpb:bulkpb_go_proto", @@ -34,6 +35,7 @@ PROTOBUF_SRCS = [ "//pkg/kv/kvserver/concurrency/lock:lock_go_proto", "//pkg/kv/kvserver/concurrency/poison:poison_go_proto", "//pkg/kv/kvserver/kvflowcontrol/kvflowcontrolpb:kvflowcontrolpb_go_proto", + "//pkg/kv/kvserver/kvflowcontrol/kvflowinspectpb:kvflowinspectpb_go_proto", "//pkg/kv/kvserver/kvserverpb:kvserverpb_go_proto", "//pkg/kv/kvserver/liveness/livenesspb:livenesspb_go_proto", "//pkg/kv/kvserver/loqrecovery/loqrecoverypb:loqrecoverypb_go_proto", diff --git a/pkg/inspectz/BUILD.bazel b/pkg/inspectz/BUILD.bazel new file mode 100644 index 000000000000..ac3392bb77a5 --- /dev/null +++ b/pkg/inspectz/BUILD.bazel @@ -0,0 +1,22 @@ +load("//build/bazelutil/unused_checker:unused.bzl", "get_x_data") +load("@io_bazel_rules_go//go:def.bzl", "go_library") + +go_library( + name = "inspectz", + srcs = [ + "inspectz.go", + "unsupported.go", + ], + importpath = "github.com/cockroachdb/cockroach/pkg/inspectz", + visibility = ["//visibility:public"], + deps = [ + "//pkg/inspectz/inspectzpb", + "//pkg/kv/kvserver/kvflowcontrol", + "//pkg/kv/kvserver/kvflowcontrol/kvflowinspectpb", + "//pkg/roachpb", + "//pkg/util/errorutil", + "//pkg/util/log", + ], +) + +get_x_data(name = "get_x_data") diff --git a/pkg/inspectz/inspectz.go b/pkg/inspectz/inspectz.go new file mode 100644 index 000000000000..c0cf4f952d38 --- /dev/null +++ b/pkg/inspectz/inspectz.go @@ -0,0 +1,151 @@ +// Copyright 2023 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package inspectz + +import ( + "context" + "encoding/json" + "net/http" + "strconv" + "strings" + + "github.com/cockroachdb/cockroach/pkg/inspectz/inspectzpb" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol/kvflowinspectpb" + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/util/log" +) + +// URLPrefix is the prefix for all inspectz endpoints hosted by the server. +const URLPrefix = "/inspectz/" + +// Server is a concrete implementation of the InspectzServer interface, +// organizing everything under /inspectz/*. It's the top-level component that +// houses parsing logic for common inspectz URL parameters and maintains routing +// logic. +type Server struct { + log.AmbientContext + + mux *http.ServeMux + handles kvflowcontrol.Handles + kvflowController kvflowcontrol.Controller +} + +var _ inspectzpb.InspectzServer = &Server{} + +// NewServer sets up an inspectz server. +func NewServer( + ambient log.AmbientContext, + handles kvflowcontrol.Handles, + kvflowController kvflowcontrol.Controller, +) *Server { + mux := http.NewServeMux() + server := &Server{ + AmbientContext: ambient, + + mux: mux, + handles: handles, + kvflowController: kvflowController, + } + mux.Handle("/inspectz/kvflowhandles", http.HandlerFunc( + func(w http.ResponseWriter, r *http.Request) { + ctx := server.AnnotateCtx(context.Background()) + + req := &kvflowinspectpb.HandlesRequest{} + if rangeIDs, ok := parseRangeIDs(r.URL.Query().Get("ranges"), w); ok { + req.RangeIDs = rangeIDs + } + resp, err := server.KVFlowHandles(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) + }, + )) + mux.Handle("/inspectz/kvflowcontroller", http.HandlerFunc( + func(w http.ResponseWriter, r *http.Request) { + ctx := server.AnnotateCtx(context.Background()) + + req := &kvflowinspectpb.ControllerRequest{} + resp, err := server.KVFlowController(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) + }, + )) + + return server +} + +// KVFlowController implements the InspectzServer interface. +func (s *Server) KVFlowController( + ctx context.Context, request *kvflowinspectpb.ControllerRequest, +) (*kvflowinspectpb.ControllerResponse, error) { + return &kvflowinspectpb.ControllerResponse{ + Streams: s.kvflowController.Inspect(ctx), + }, nil +} + +// KVFlowHandles implements the InspectzServer interface. +func (s *Server) KVFlowHandles( + ctx context.Context, request *kvflowinspectpb.HandlesRequest, +) (*kvflowinspectpb.HandlesResponse, error) { + resp := &kvflowinspectpb.HandlesResponse{} + if len(request.RangeIDs) == 0 { + request.RangeIDs = s.handles.Inspect() + } + for _, rangeID := range request.RangeIDs { + handle, found := s.handles.Lookup(rangeID) + if !found { + continue // nothing to do + } + resp.Handles = append(resp.Handles, handle.Inspect(ctx)) + } + return resp, nil +} + +// ServeHTTP serves various tools under the /debug endpoint. +func (s *Server) ServeHTTP(w http.ResponseWriter, r *http.Request) { + s.mux.ServeHTTP(w, r) +} + +func respond(ctx context.Context, w http.ResponseWriter, code int, payload interface{}) { + res, err := json.Marshal(payload) + if err != nil { + log.ErrorfDepth(ctx, 1, "%s", err) + http.Error(w, "internal error: check logs for details", http.StatusInternalServerError) + return + } + w.Header().Set("Content-Type", "application/json") + w.WriteHeader(code) + _, _ = w.Write(res) +} + +func parseRangeIDs(input string, w http.ResponseWriter) (ranges []roachpb.RangeID, ok bool) { + if len(input) == 0 { + return nil, true + } + for _, part := range strings.Split(input, ",") { + rangeID, err := strconv.ParseInt(part, 10, 64) + if err != nil { + http.Error(w, "invalid range id", http.StatusBadRequest) + return nil, false + } + + ranges = append(ranges, roachpb.RangeID(rangeID)) + } + return ranges, true +} diff --git a/pkg/inspectz/inspectzpb/BUILD.bazel b/pkg/inspectz/inspectzpb/BUILD.bazel new file mode 100644 index 000000000000..0d2d9372bfe7 --- /dev/null +++ b/pkg/inspectz/inspectzpb/BUILD.bazel @@ -0,0 +1,36 @@ +load("//build/bazelutil/unused_checker:unused.bzl", "get_x_data") +load("@rules_proto//proto:defs.bzl", "proto_library") +load("@io_bazel_rules_go//go:def.bzl", "go_library") +load("@io_bazel_rules_go//proto:def.bzl", "go_proto_library") + +proto_library( + name = "inspectzpb_proto", + srcs = ["inspectz.proto"], + strip_import_prefix = "/pkg", + visibility = ["//visibility:public"], + deps = [ + "//pkg/kv/kvserver/kvflowcontrol/kvflowinspectpb:kvflowinspectpb_proto", + "@go_googleapis//google/api:annotations_proto", + ], +) + +go_proto_library( + name = "inspectzpb_go_proto", + compilers = ["//pkg/cmd/protoc-gen-gogoroach:protoc-gen-gogoroach_grpc_compiler"], + importpath = "github.com/cockroachdb/cockroach/pkg/inspectz/inspectzpb", + proto = ":inspectzpb_proto", + visibility = ["//visibility:public"], + deps = [ + "//pkg/kv/kvserver/kvflowcontrol/kvflowinspectpb", + "@org_golang_google_genproto//googleapis/api/annotations:go_default_library", + ], +) + +go_library( + name = "inspectzpb", + embed = [":inspectzpb_go_proto"], + importpath = "github.com/cockroachdb/cockroach/pkg/inspectz/inspectzpb", + visibility = ["//visibility:public"], +) + +get_x_data(name = "get_x_data") diff --git a/pkg/inspectz/inspectzpb/inspectz.proto b/pkg/inspectz/inspectzpb/inspectz.proto new file mode 100644 index 000000000000..6c216a42c37b --- /dev/null +++ b/pkg/inspectz/inspectzpb/inspectz.proto @@ -0,0 +1,42 @@ +// Copyright 2023 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +syntax = "proto3"; +package cockroach.inspectz.inspectzpb; +option go_package = "github.com/cockroachdb/cockroach/pkg/inspectz/inspectzpb"; + +import "google/api/annotations.proto"; +import "kv/kvserver/kvflowcontrol/kvflowinspectpb/kvflowinspect.proto"; + +// Inspectz exposes in-memory state of various CRDB components. +// +// TODO(irfansharif): Expose endpoints for the admission package, for example +// what each work queue's occupancy is (with respect to tenants, "held" flow +// tokens, etc). Also expose queues in the latch/lock manager, including who is +// waiting and for how long. +service Inspectz { + // KVFlowController exposes in-memory state of the node-level + // kvflowcontrol.Controller. It's housed under /inspectz/kvflowcontroller. + rpc KVFlowController(kv.kvserver.kvflowcontrol.kvflowinspectpb.ControllerRequest) + returns (kv.kvserver.kvflowcontrol.kvflowinspectpb.ControllerResponse) {} + + // KVFlowHandles exposes in-memory state of kvflowcontrol.Handles. It's housed + // under /inspectz/kvflowhandles. + rpc KVFlowHandles(kv.kvserver.kvflowcontrol.kvflowinspectpb.HandlesRequest) + returns (kv.kvserver.kvflowcontrol.kvflowinspectpb.HandlesResponse) {} + +} + +// As of 04/23, we're not invoking these RPC interfaces as RPCs. But they're +// kept as such for semi-opinionated code organization reasons. We want all +// inspectz APIs and protos inspectable at glance in this one file even if +// actual type definitions appear elsewhere. It's possible that other +// subsystems that use /inspectz make use of the RPC interfaces (to fetch +// remote inspectable state, for example). diff --git a/pkg/inspectz/unsupported.go b/pkg/inspectz/unsupported.go new file mode 100644 index 000000000000..0fdc5adecfee --- /dev/null +++ b/pkg/inspectz/unsupported.go @@ -0,0 +1,39 @@ +// Copyright 2023 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package inspectz + +import ( + "context" + + "github.com/cockroachdb/cockroach/pkg/inspectz/inspectzpb" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol/kvflowinspectpb" + "github.com/cockroachdb/cockroach/pkg/util/errorutil" +) + +// Unsupported is an inspectzpb.InspectzServer that only returns "unsupported" +// errors. +type Unsupported struct{} + +var _ inspectzpb.InspectzServer = Unsupported{} + +// KVFlowController is part of the inspectzpb.InspectzServer interface. +func (u Unsupported) KVFlowController( + ctx context.Context, request *kvflowinspectpb.ControllerRequest, +) (*kvflowinspectpb.ControllerResponse, error) { + return nil, errorutil.UnsupportedWithMultiTenancy(errorutil.FeatureNotAvailableToNonSystemTenantsIssue) +} + +// KVFlowHandles is part of the inspectzpb.InspectzServer interface. +func (u Unsupported) KVFlowHandles( + ctx context.Context, request *kvflowinspectpb.HandlesRequest, +) (*kvflowinspectpb.HandlesResponse, error) { + return nil, errorutil.UnsupportedWithMultiTenancy(errorutil.FeatureNotAvailableToNonSystemTenantsIssue) +} diff --git a/pkg/kv/kvserver/BUILD.bazel b/pkg/kv/kvserver/BUILD.bazel index 4bcb635527d1..a5bdff1be81c 100644 --- a/pkg/kv/kvserver/BUILD.bazel +++ b/pkg/kv/kvserver/BUILD.bazel @@ -391,6 +391,7 @@ go_test( "//pkg/kv/kvserver/kvflowcontrol", "//pkg/kv/kvserver/kvflowcontrol/kvflowcontrolpb", "//pkg/kv/kvserver/kvflowcontrol/kvflowdispatch", + "//pkg/kv/kvserver/kvflowcontrol/kvflowinspectpb", "//pkg/kv/kvserver/kvserverbase", "//pkg/kv/kvserver/kvserverpb", "//pkg/kv/kvserver/kvstorage", diff --git a/pkg/kv/kvserver/flow_control_stores.go b/pkg/kv/kvserver/flow_control_stores.go index 98ca2dcc4e93..e1f58c832d85 100644 --- a/pkg/kv/kvserver/flow_control_stores.go +++ b/pkg/kv/kvserver/flow_control_stores.go @@ -58,6 +58,21 @@ func (sh *storesForFlowControl) Lookup( return handle, found } +// Inspect is part of the StoresForFlowControl interface. +func (sh *storesForFlowControl) Inspect() []roachpb.RangeID { + ls := (*Stores)(sh) + var rangeIDs []roachpb.RangeID + if err := ls.VisitStores(func(s *Store) error { + rangeIDs = append(rangeIDs, makeStoreForFlowControl(s).Inspect()...) + return nil + }); err != nil { + ctx := ls.AnnotateCtx(context.Background()) + log.Errorf(ctx, "unexpected error: %s", err) + return nil + } + return rangeIDs +} + // ResetStreams is part of the StoresForFlowControl interface. func (sh *storesForFlowControl) ResetStreams(ctx context.Context) { ls := (*Stores)(sh) @@ -125,6 +140,17 @@ func (sh *storeForFlowControl) ResetStreams(ctx context.Context) { }) } +// Inspect is part of the StoresForFlowControl interface. +func (sh *storeForFlowControl) Inspect() []roachpb.RangeID { + s := (*Store)(sh) + var rangeIDs []roachpb.RangeID + s.VisitReplicas(func(replica *Replica) (wantMore bool) { + rangeIDs = append(rangeIDs, replica.RangeID) + return true + }) + return rangeIDs +} + // OnRaftTransportDisconnected is part of the StoresForFlowControl // interface. func (sh *storeForFlowControl) OnRaftTransportDisconnected( diff --git a/pkg/kv/kvserver/kvflowcontrol/BUILD.bazel b/pkg/kv/kvserver/kvflowcontrol/BUILD.bazel index 2665a779d145..bd45f4fc91d4 100644 --- a/pkg/kv/kvserver/kvflowcontrol/BUILD.bazel +++ b/pkg/kv/kvserver/kvflowcontrol/BUILD.bazel @@ -13,6 +13,7 @@ go_library( deps = [ "//pkg/base", "//pkg/kv/kvserver/kvflowcontrol/kvflowcontrolpb", + "//pkg/kv/kvserver/kvflowcontrol/kvflowinspectpb", "//pkg/roachpb", "//pkg/settings", "//pkg/util/admission/admissionpb", diff --git a/pkg/kv/kvserver/kvflowcontrol/kvflowcontrol.go b/pkg/kv/kvserver/kvflowcontrol/kvflowcontrol.go index 05f8b481ff25..b46fbf1a95e2 100644 --- a/pkg/kv/kvserver/kvflowcontrol/kvflowcontrol.go +++ b/pkg/kv/kvserver/kvflowcontrol/kvflowcontrol.go @@ -17,6 +17,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol/kvflowcontrolpb" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol/kvflowinspectpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/util/admission/admissionpb" @@ -129,6 +130,12 @@ type Controller interface { // expected to have been deducted earlier with the same priority provided // here. ReturnTokens(context.Context, admissionpb.WorkPriority, Tokens, Stream) + // Inspect returns a snapshot of all underlying streams and their available + // {regular,elastic} tokens. It's used to power /inspectz. + Inspect(context.Context) []kvflowinspectpb.Stream + // InspectStream returns a snapshot of a specific underlying stream and its + // available {regular,elastic} tokens. It's used to power /inspectz. + InspectStream(context.Context, Stream) kvflowinspectpb.Stream // TODO(irfansharif): We might need the ability to "disable" specific // streams/corresponding token buckets when there are failures or @@ -204,6 +211,9 @@ type Handle interface { // Admit(). It's only used when cluster settings change, settings that // affect all work waiting for flow tokens. ResetStreams(ctx context.Context) + // Inspect returns a serialized form of the underlying handle. It's used to + // power /inspectz. + Inspect(context.Context) kvflowinspectpb.Handle // Close closes the handle and returns all held tokens back to the // underlying controller. Typically used when the replica loses its lease // and/or raft leadership, or ends up getting GC-ed (if it's being @@ -217,6 +227,7 @@ type Handle interface { type Handles interface { Lookup(roachpb.RangeID) (Handle, bool) ResetStreams(ctx context.Context) + Inspect() []roachpb.RangeID } // HandleFactory is used to construct new Handles. diff --git a/pkg/kv/kvserver/kvflowcontrol/kvflowcontroller/BUILD.bazel b/pkg/kv/kvserver/kvflowcontrol/kvflowcontroller/BUILD.bazel index 1c2ac6ecb8d0..7b8a8b4ca957 100644 --- a/pkg/kv/kvserver/kvflowcontrol/kvflowcontroller/BUILD.bazel +++ b/pkg/kv/kvserver/kvflowcontrol/kvflowcontroller/BUILD.bazel @@ -12,6 +12,7 @@ go_library( deps = [ "//pkg/base", "//pkg/kv/kvserver/kvflowcontrol", + "//pkg/kv/kvserver/kvflowcontrol/kvflowinspectpb", "//pkg/settings", "//pkg/settings/cluster", "//pkg/util/admission/admissionpb", diff --git a/pkg/kv/kvserver/kvflowcontrol/kvflowcontroller/kvflowcontrol_token_adjustment_test.go b/pkg/kv/kvserver/kvflowcontrol/kvflowcontroller/kvflowcontrol_token_adjustment_test.go index e8f4ad69ae16..ce00d7019b0d 100644 --- a/pkg/kv/kvserver/kvflowcontrol/kvflowcontroller/kvflowcontrol_token_adjustment_test.go +++ b/pkg/kv/kvserver/kvflowcontrol/kvflowcontroller/kvflowcontrol_token_adjustment_test.go @@ -95,7 +95,7 @@ func TestFlowTokenAdjustment(t *testing.T) { adjustments = append(adjustments, adjustment{ pri: pri, delta: delta, - post: controller.testingGetTokensForStream(stream), + post: controller.getTokensForStream(stream), }) } return "" diff --git a/pkg/kv/kvserver/kvflowcontrol/kvflowcontroller/kvflowcontroller.go b/pkg/kv/kvserver/kvflowcontrol/kvflowcontroller/kvflowcontroller.go index 073aa741e153..664ce48da991 100644 --- a/pkg/kv/kvserver/kvflowcontrol/kvflowcontroller/kvflowcontroller.go +++ b/pkg/kv/kvserver/kvflowcontrol/kvflowcontroller/kvflowcontroller.go @@ -16,6 +16,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol/kvflowinspectpb" "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/util/admission/admissionpb" @@ -230,6 +231,36 @@ func (c *Controller) ReturnTokens( c.adjustTokens(ctx, pri, +tokens, stream) } +// Inspect is part of the kvflowcontrol.Controller interface. +func (c *Controller) Inspect(ctx context.Context) []kvflowinspectpb.Stream { + c.mu.Lock() + defer c.mu.Unlock() + + var streams []kvflowinspectpb.Stream + for stream, b := range c.mu.buckets { + streams = append(streams, kvflowinspectpb.Stream{ + TenantID: stream.TenantID, + StoreID: stream.StoreID, + AvailableRegularTokens: int64(b.tokens[regular]), + AvailableElasticTokens: int64(b.tokens[elastic]), + }) + } + return streams +} + +// InspectStream is part of the kvflowcontrol.Controller interface. +func (c *Controller) InspectStream( + _ context.Context, stream kvflowcontrol.Stream, +) kvflowinspectpb.Stream { + tokens := c.getTokensForStream(stream) + return kvflowinspectpb.Stream{ + TenantID: stream.TenantID, + StoreID: stream.StoreID, + AvailableRegularTokens: int64(tokens[regular]), + AvailableElasticTokens: int64(tokens[elastic]), + } +} + func (c *Controller) adjustTokens( ctx context.Context, pri admissionpb.WorkPriority, @@ -373,7 +404,7 @@ func validateTokenRange(b int64) error { return nil } -func (c *Controller) testingGetTokensForStream(stream kvflowcontrol.Stream) tokensPerWorkClass { +func (c *Controller) getTokensForStream(stream kvflowcontrol.Stream) tokensPerWorkClass { c.mu.Lock() defer c.mu.Unlock() ret := make(map[admissionpb.WorkClass]kvflowcontrol.Tokens) diff --git a/pkg/kv/kvserver/kvflowcontrol/kvflowhandle/BUILD.bazel b/pkg/kv/kvserver/kvflowcontrol/kvflowhandle/BUILD.bazel index 269aaf135876..d54387329f89 100644 --- a/pkg/kv/kvserver/kvflowcontrol/kvflowhandle/BUILD.bazel +++ b/pkg/kv/kvserver/kvflowcontrol/kvflowhandle/BUILD.bazel @@ -15,6 +15,7 @@ go_library( "//pkg/base", "//pkg/kv/kvserver/kvflowcontrol", "//pkg/kv/kvserver/kvflowcontrol/kvflowcontrolpb", + "//pkg/kv/kvserver/kvflowcontrol/kvflowinspectpb", "//pkg/kv/kvserver/kvflowcontrol/kvflowtokentracker", "//pkg/roachpb", "//pkg/util/admission/admissionpb", diff --git a/pkg/kv/kvserver/kvflowcontrol/kvflowhandle/kvflowhandle.go b/pkg/kv/kvserver/kvflowcontrol/kvflowhandle/kvflowhandle.go index d9aea13641bd..778768130ef5 100644 --- a/pkg/kv/kvserver/kvflowcontrol/kvflowhandle/kvflowhandle.go +++ b/pkg/kv/kvserver/kvflowcontrol/kvflowhandle/kvflowhandle.go @@ -17,6 +17,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol/kvflowcontrolpb" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol/kvflowinspectpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol/kvflowtokentracker" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/util/admission/admissionpb" @@ -274,6 +275,24 @@ func (h *Handle) ResetStreams(ctx context.Context) { } } +// Inspect is part of the kvflowcontrol.Handle interface. +func (h *Handle) Inspect(ctx context.Context) kvflowinspectpb.Handle { + h.mu.Lock() + defer h.mu.Unlock() + + handle := kvflowinspectpb.Handle{ + RangeID: h.rangeID, + } + for _, c := range h.mu.connections { + connected := kvflowinspectpb.ConnectedStream{ + Stream: h.controller.InspectStream(ctx, c.Stream()), + TrackedDeductions: h.mu.perStreamTokenTracker[c.Stream()].Inspect(ctx), + } + handle.ConnectedStreams = append(handle.ConnectedStreams, connected) + } + return handle +} + func (h *Handle) disconnectStreamLocked(ctx context.Context, stream kvflowcontrol.Stream) { if h.mu.closed { log.Errorf(ctx, "operating on a closed handle") diff --git a/pkg/kv/kvserver/kvflowcontrol/kvflowhandle/noop.go b/pkg/kv/kvserver/kvflowcontrol/kvflowhandle/noop.go index 5a311b265fa0..69878f5adb15 100644 --- a/pkg/kv/kvserver/kvflowcontrol/kvflowhandle/noop.go +++ b/pkg/kv/kvserver/kvflowcontrol/kvflowhandle/noop.go @@ -16,6 +16,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol/kvflowcontrolpb" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol/kvflowinspectpb" "github.com/cockroachdb/cockroach/pkg/util/admission/admissionpb" ) @@ -59,5 +60,10 @@ func (n Noop) DisconnectStream(ctx context.Context, stream kvflowcontrol.Stream) // ResetStreams is part of the kvflowcontrol.Handle interface. func (n Noop) ResetStreams(ctx context.Context) {} +// Inspect is part of the kvflowcontrol.Handle interface. +func (n Noop) Inspect(ctx context.Context) kvflowinspectpb.Handle { + return kvflowinspectpb.Handle{} +} + // Close is part of the kvflowcontrol.Handle interface. func (n Noop) Close(ctx context.Context) {} diff --git a/pkg/kv/kvserver/kvflowcontrol/kvflowinspectpb/BUILD.bazel b/pkg/kv/kvserver/kvflowcontrol/kvflowinspectpb/BUILD.bazel new file mode 100644 index 000000000000..2735591f2fdd --- /dev/null +++ b/pkg/kv/kvserver/kvflowcontrol/kvflowinspectpb/BUILD.bazel @@ -0,0 +1,38 @@ +load("//build/bazelutil/unused_checker:unused.bzl", "get_x_data") +load("@rules_proto//proto:defs.bzl", "proto_library") +load("@io_bazel_rules_go//go:def.bzl", "go_library") +load("@io_bazel_rules_go//proto:def.bzl", "go_proto_library") + +go_proto_library( + name = "kvflowinspectpb_go_proto", + compilers = ["//pkg/cmd/protoc-gen-gogoroach:protoc-gen-gogoroach_compiler"], + importpath = "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol/kvflowinspectpb", + proto = ":kvflowinspectpb_proto", + visibility = ["//visibility:public"], + deps = [ + "//pkg/kv/kvserver/kvflowcontrol/kvflowcontrolpb", + "//pkg/roachpb", + "@com_github_gogo_protobuf//gogoproto", + ], +) + +go_library( + name = "kvflowinspectpb", + embed = [":kvflowinspectpb_go_proto"], + importpath = "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol/kvflowinspectpb", + visibility = ["//visibility:public"], +) + +proto_library( + name = "kvflowinspectpb_proto", + srcs = ["kvflowinspect.proto"], + strip_import_prefix = "/pkg", + visibility = ["//visibility:public"], + deps = [ + "//pkg/kv/kvserver/kvflowcontrol/kvflowcontrolpb:kvflowcontrolpb_proto", + "//pkg/roachpb:roachpb_proto", + "@com_github_gogo_protobuf//gogoproto:gogo_proto", + ], +) + +get_x_data(name = "get_x_data") diff --git a/pkg/kv/kvserver/kvflowcontrol/kvflowinspectpb/kvflowinspect.proto b/pkg/kv/kvserver/kvflowcontrol/kvflowinspectpb/kvflowinspect.proto new file mode 100644 index 000000000000..fbc808de3cbc --- /dev/null +++ b/pkg/kv/kvserver/kvflowcontrol/kvflowinspectpb/kvflowinspect.proto @@ -0,0 +1,101 @@ +// Copyright 2023 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +syntax = "proto3"; +package cockroach.kv.kvserver.kvflowcontrol.kvflowinspectpb; +option go_package = "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol/kvflowinspectpb"; + +import "gogoproto/gogo.proto"; +import "roachpb/data.proto"; +import "kv/kvserver/kvflowcontrol/kvflowcontrolpb/kvflowcontrol.proto"; + +// ControllerRequest is used to inspect the state of the node-level +// kvflowcontrol.Controller. +message ControllerRequest { + // TODO(irfansharif): Allow inspection of a specific stream. We already have + // the underlying APIs for it. +} + +// ControllerResponse is the response structure for inspecting the node-level +// kvflowcontrol.Controller, a snapshot of its in-memory state. Data is +// organized in the following structure: +// +// - [stream] +// - tenant id +// - store id +// - available regular tokens +// - available elastic tokens +message ControllerResponse { + repeated Stream streams = 1 [(gogoproto.nullable) = false]; +} + +// HandlesRequest is used to inspect the state of named kvflowcontrol.Handles, +// identified by owning replica's range ID. +message HandlesRequest { + repeated int32 range_ids = 1 [ + (gogoproto.customname) = "RangeIDs", + (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.RangeID"]; +} + +// HandlesResponse is the response structure for inspecting named +// kvflowcontrol.Handles, a snapshot their in-memory state. If no handles are +// specified, we'll sift through all available ones. For each handle, data is +// organized in the following structure: +// +// - range id +// - [connected streams] +// - stream: +// - tenant id +// - store id +// - available regular tokens +// - available elastic tokens +// - [tracked token deductions] +// - raft log position +// - priority +// - tokens +message HandlesResponse { + repeated Handle handles = 1 [(gogoproto.nullable) = false]; +} + +// Handle represents the in-memory state of a given kvflowcontrol.Handle. +message Handle { + int32 range_id = 1 [ + (gogoproto.customname) = "RangeID", + (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.RangeID"]; + repeated ConnectedStream connected_streams = 2 [(gogoproto.nullable) = false]; +} + +// ConnectedStream represents the in-memory state of a connected stream, +// typically hanging off a kvflowcontrol.Handle. +message ConnectedStream { + Stream stream = 1 [(gogoproto.nullable) = false]; + repeated TrackedDeduction tracked_deductions = 2 [(gogoproto.nullable) = false]; +} + +// Stream represents a given kvflowcontrol.Stream and the number of tokens +// available for it (as maintained by the node-level kvflowcontrol.Controller). +message Stream { + roachpb.TenantID tenant_id = 1 [ + (gogoproto.nullable) = false, + (gogoproto.customname) = "TenantID"]; + uint64 store_id = 2 [ + (gogoproto.customname) = "StoreID", + (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.StoreID"]; + int64 available_regular_tokens = 3; + int64 available_elastic_tokens = 4; +} + +// TrackedDeductions represents a tracked token deduction, as typically done by +// a kvflowcontrol.Handle. +message TrackedDeduction { + int32 priority = 1; + int64 tokens = 2; + kv.kvserver.kvflowcontrol.kvflowcontrolpb.RaftLogPosition raft_log_position = 3 [(gogoproto.nullable) = false]; +} diff --git a/pkg/kv/kvserver/kvflowcontrol/kvflowtokentracker/BUILD.bazel b/pkg/kv/kvserver/kvflowcontrol/kvflowtokentracker/BUILD.bazel index 3c1bb6a39379..7e2de5d684e0 100644 --- a/pkg/kv/kvserver/kvflowcontrol/kvflowtokentracker/BUILD.bazel +++ b/pkg/kv/kvserver/kvflowcontrol/kvflowtokentracker/BUILD.bazel @@ -9,6 +9,7 @@ go_library( deps = [ "//pkg/kv/kvserver/kvflowcontrol", "//pkg/kv/kvserver/kvflowcontrol/kvflowcontrolpb", + "//pkg/kv/kvserver/kvflowcontrol/kvflowinspectpb", "//pkg/util/admission/admissionpb", "//pkg/util/buildutil", "//pkg/util/log", diff --git a/pkg/kv/kvserver/kvflowcontrol/kvflowtokentracker/tracker.go b/pkg/kv/kvserver/kvflowcontrol/kvflowtokentracker/tracker.go index 73578a1de596..3b5a528bd59b 100644 --- a/pkg/kv/kvserver/kvflowcontrol/kvflowtokentracker/tracker.go +++ b/pkg/kv/kvserver/kvflowcontrol/kvflowtokentracker/tracker.go @@ -17,6 +17,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol/kvflowcontrolpb" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol/kvflowinspectpb" "github.com/cockroachdb/cockroach/pkg/util/admission/admissionpb" "github.com/cockroachdb/cockroach/pkg/util/buildutil" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -188,6 +189,21 @@ func (dt *Tracker) LowerBound() kvflowcontrolpb.RaftLogPosition { return dt.lowerBound } +// Inspect returns a snapshot of all tracked token deductions. It's used to +// power /inspectz-style debugging pages. +func (dt *Tracker) Inspect(ctx context.Context) []kvflowinspectpb.TrackedDeduction { + var deductions []kvflowinspectpb.TrackedDeduction + dt.TestingIter(func(pri admissionpb.WorkPriority, tokens kvflowcontrol.Tokens, pos kvflowcontrolpb.RaftLogPosition) bool { + deductions = append(deductions, kvflowinspectpb.TrackedDeduction{ + Priority: int32(pri), + Tokens: int64(tokens), + RaftLogPosition: pos, + }) + return true + }) + return deductions +} + // TestingIter is a testing-only re-implementation of Iter. It iterates through // all tracked token deductions, invoking the provided callback with tracked // pri<->token<->position triples. diff --git a/pkg/kv/kvserver/replica_proposal_buf_test.go b/pkg/kv/kvserver/replica_proposal_buf_test.go index 161b0892d25d..c0ba04eeaa26 100644 --- a/pkg/kv/kvserver/replica_proposal_buf_test.go +++ b/pkg/kv/kvserver/replica_proposal_buf_test.go @@ -23,6 +23,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts/tracker" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol/kvflowcontrolpb" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol/kvflowinspectpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/raftlog" @@ -1091,5 +1092,9 @@ func (t *testFlowTokenHandle) DisconnectStream(ctx context.Context, stream kvflo func (t *testFlowTokenHandle) ResetStreams(ctx context.Context) { } +func (t *testFlowTokenHandle) Inspect(context.Context) kvflowinspectpb.Handle { + return kvflowinspectpb.Handle{} +} + func (t *testFlowTokenHandle) Close(ctx context.Context) { } diff --git a/pkg/server/BUILD.bazel b/pkg/server/BUILD.bazel index 1b2314975c67..882b99a4c6b1 100644 --- a/pkg/server/BUILD.bazel +++ b/pkg/server/BUILD.bazel @@ -100,6 +100,8 @@ go_library( "//pkg/docs", "//pkg/featureflag", "//pkg/gossip", + "//pkg/inspectz", + "//pkg/inspectz/inspectzpb", "//pkg/jobs", "//pkg/jobs/jobspb", "//pkg/jobs/jobsprotectedts", diff --git a/pkg/server/server.go b/pkg/server/server.go index f6c4d714ad39..b488079804f4 100644 --- a/pkg/server/server.go +++ b/pkg/server/server.go @@ -27,6 +27,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/build" "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/gossip" + "github.com/cockroachdb/cockroach/pkg/inspectz" "github.com/cockroachdb/cockroach/pkg/jobs" "github.com/cockroachdb/cockroach/pkg/jobs/jobsprotectedts" "github.com/cockroachdb/cockroach/pkg/keys" @@ -177,8 +178,9 @@ type Server struct { stopper *stop.Stopper stopTrigger *stopTrigger - debug *debug.Server - kvProber *kvprober.Prober + debug *debug.Server + kvProber *kvprober.Prober + inspectzServer *inspectz.Server replicationReporter *reports.Reporter protectedtsProvider protectedts.Provider @@ -1077,6 +1079,12 @@ func NewServer(cfg Config, stopper *stop.Stopper) (*Server, error) { registry.AddMetricStruct(m) } + inspectzServer := inspectz.NewServer( + cfg.BaseConfig.AmbientCtx, + node.storeCfg.KVFlowHandles, + node.storeCfg.KVFlowController, + ) + // Instantiate the SQL server proper. sqlServer, err := newSQLServer(ctx, sqlServerArgs{ sqlServerOptionalKVArgs: sqlServerOptionalKVArgs{ @@ -1091,6 +1099,7 @@ func NewServer(cfg Config, stopper *stop.Stopper) (*Server, error) { sqlSQLResponseAdmissionQ: gcoords.Regular.GetWorkQueue(admission.SQLSQLResponseWork), spanConfigKVAccessor: spanConfig.kvAccessorForTenantRecords, kvStoresIterator: kvserver.MakeStoresIterator(node.stores), + inspectzServer: inspectzServer, }, SQLConfig: &cfg.SQLConfig, BaseConfig: &cfg.BaseConfig, @@ -1282,6 +1291,7 @@ func NewServer(cfg Config, stopper *stop.Stopper) (*Server, error) { storeGrantCoords: gcoords.Stores, kvMemoryMonitor: kvMemoryMonitor, keyVisualizerServer: keyVisualizerServer, + inspectzServer: inspectzServer, } return lateBoundServer, err @@ -1949,6 +1959,7 @@ func (s *Server) PreStart(ctx context.Context) error { s.runtime, /* runtimeStatsSampler */ gwMux, /* handleRequestsUnauthenticated */ s.debug, /* handleDebugUnauthenticated */ + s.inspectzServer, /* handleInspectzUnauthenticated */ newAPIV2Server(ctx, &apiV2ServerOpts{ admin: s.admin, status: s.status, diff --git a/pkg/server/server_http.go b/pkg/server/server_http.go index b92171c063fc..897bccfb02d5 100644 --- a/pkg/server/server_http.go +++ b/pkg/server/server_http.go @@ -17,6 +17,7 @@ import ( "github.com/NYTimes/gziphandler" "github.com/cockroachdb/cmux" + "github.com/cockroachdb/cockroach/pkg/inspectz" "github.com/cockroachdb/cockroach/pkg/rpc" "github.com/cockroachdb/cockroach/pkg/server/debug" "github.com/cockroachdb/cockroach/pkg/server/serverpb" @@ -94,6 +95,7 @@ func (s *httpServer) setupRoutes( runtimeStatSampler *status.RuntimeStatSampler, handleRequestsUnauthenticated http.Handler, handleDebugUnauthenticated http.Handler, + handleInspectzUnauthenticated http.Handler, apiServer http.Handler, flags serverpb.FeatureFlags, ) error { @@ -173,12 +175,17 @@ func (s *httpServer) setupRoutes( // Register debugging endpoints. handleDebugAuthenticated := handleDebugUnauthenticated + handleInspectzAuthenticated := handleInspectzUnauthenticated if !s.cfg.InsecureWebAccess() { // Mandate both authentication and admin authorization. handleDebugAuthenticated = makeAdminAuthzCheckHandler(adminAuthzCheck, handleDebugAuthenticated) handleDebugAuthenticated = newAuthenticationMux(authnServer, handleDebugAuthenticated) + + handleInspectzAuthenticated = makeAdminAuthzCheckHandler(adminAuthzCheck, handleInspectzAuthenticated) + handleInspectzAuthenticated = newAuthenticationMux(authnServer, handleInspectzAuthenticated) } s.mux.Handle(debug.Endpoint, handleDebugAuthenticated) + s.mux.Handle(inspectz.URLPrefix, handleInspectzAuthenticated) log.Event(ctx, "added http endpoints") diff --git a/pkg/server/server_sql.go b/pkg/server/server_sql.go index b127ea0111be..196bb1482af2 100644 --- a/pkg/server/server_sql.go +++ b/pkg/server/server_sql.go @@ -26,6 +26,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/featureflag" "github.com/cockroachdb/cockroach/pkg/gossip" + "github.com/cockroachdb/cockroach/pkg/inspectz/inspectzpb" "github.com/cockroachdb/cockroach/pkg/jobs" "github.com/cockroachdb/cockroach/pkg/jobs/jobsprotectedts" "github.com/cockroachdb/cockroach/pkg/keys" @@ -244,6 +245,10 @@ type sqlServerOptionalKVArgs struct { // kvStores is used by crdb_internal builtins to access the stores on this // node. kvStoresIterator kvserverbase.StoresIterator + + // inspectzServer is used to power various crdb_internal vtables, exposing + // the equivalent of /inspectz but through SQL. + inspectzServer inspectzpb.InspectzServer } // sqlServerOptionalTenantArgs are the arguments supplied to newSQLServer which @@ -990,6 +995,7 @@ func newSQLServer(ctx context.Context, cfg sqlServerArgs) (*SQLServer, error) { TraceCollector: traceCollector, TenantUsageServer: cfg.tenantUsageServer, KVStoresIterator: cfg.kvStoresIterator, + InspectzServer: cfg.inspectzServer, RangeDescIteratorFactory: cfg.rangeDescIteratorFactory, SyntheticPrivilegeCache: syntheticprivilegecache.New( cfg.Settings, cfg.stopper, cfg.db, diff --git a/pkg/server/tenant.go b/pkg/server/tenant.go index e3f3f56bbe59..2eff7dd1a8da 100644 --- a/pkg/server/tenant.go +++ b/pkg/server/tenant.go @@ -14,6 +14,7 @@ import ( "context" "fmt" "net" + "net/http" "os" "path/filepath" "strconv" @@ -23,6 +24,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/build" "github.com/cockroachdb/cockroach/pkg/gossip" + "github.com/cockroachdb/cockroach/pkg/inspectz" "github.com/cockroachdb/cockroach/pkg/jobs" "github.com/cockroachdb/cockroach/pkg/jobs/jobsprotectedts" "github.com/cockroachdb/cockroach/pkg/keys" @@ -715,6 +717,9 @@ func (s *SQLServerWrapper) PreStart(ctx context.Context) error { s.runtime, /* runtimeStatsSampler */ gwMux, /* handleRequestsUnauthenticated */ s.debug, /* handleDebugUnauthenticated */ + http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) { + writeJSONResponse(r.Context(), w, http.StatusNotImplemented, nil) + }), newAPIV2Server(workersCtx, &apiV2ServerOpts{ admin: s.tenantAdmin, status: s.tenantStatus, @@ -1202,6 +1207,7 @@ func makeTenantSQLServerArgs( nodeIDContainer: deps.instanceIDContainer, spanConfigKVAccessor: tenantConnect, kvStoresIterator: kvserverbase.UnsupportedStoresIterator{}, + inspectzServer: inspectz.Unsupported{}, }, sqlServerOptionalTenantArgs: sqlServerOptionalTenantArgs{ spanLimiterFactory: deps.spanLimiterFactory, diff --git a/pkg/sql/BUILD.bazel b/pkg/sql/BUILD.bazel index 00f2ed584618..2e2c405d9f65 100644 --- a/pkg/sql/BUILD.bazel +++ b/pkg/sql/BUILD.bazel @@ -308,6 +308,7 @@ go_library( "//pkg/geo/geopb", "//pkg/geo/geoprojbase", "//pkg/gossip", + "//pkg/inspectz/inspectzpb", "//pkg/jobs", "//pkg/jobs/jobsauth", "//pkg/jobs/jobspb", @@ -322,6 +323,7 @@ go_library( "//pkg/kv/kvpb", "//pkg/kv/kvserver/concurrency/isolation", "//pkg/kv/kvserver/concurrency/lock", + "//pkg/kv/kvserver/kvflowcontrol/kvflowinspectpb", "//pkg/kv/kvserver/kvserverbase", "//pkg/kv/kvserver/liveness/livenesspb", "//pkg/kv/kvserver/protectedts", diff --git a/pkg/sql/crdb_internal.go b/pkg/sql/crdb_internal.go index 58422db8d34f..806eb4d75263 100644 --- a/pkg/sql/crdb_internal.go +++ b/pkg/sql/crdb_internal.go @@ -35,6 +35,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvcoord" "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/lock" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol/kvflowinspectpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness/livenesspb" "github.com/cockroachdb/cockroach/pkg/multitenant/tenantcapabilities" "github.com/cockroachdb/cockroach/pkg/multitenant/tenantcapabilities/tenantcapabilitiespb" @@ -86,6 +87,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sqlstats/sslocal" "github.com/cockroachdb/cockroach/pkg/sql/syntheticprivilege" "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/cockroach/pkg/util/admission/admissionpb" "github.com/cockroachdb/cockroach/pkg/util/duration" "github.com/cockroachdb/cockroach/pkg/util/errorutil" "github.com/cockroachdb/cockroach/pkg/util/json" @@ -209,6 +211,9 @@ var crdbInternal = virtualSchema{ catconstants.CrdbInternalShowTenantCapabilitiesCacheTableID: crdbInternalShowTenantCapabilitiesCache, catconstants.CrdbInternalInheritedRoleMembersTableID: crdbInternalInheritedRoleMembers, catconstants.CrdbInternalKVSystemPrivilegesViewID: crdbInternalKVSystemPrivileges, + catconstants.CrdbInternalKVFlowHandlesID: crdbInternalKVFlowHandles, + catconstants.CrdbInternalKVFlowControllerID: crdbInternalKVFlowController, + catconstants.CrdbInternalKVFlowTokenDeductions: crdbInternalKVFlowTokenDeductions, }, validWithNoDatabaseContext: true, } @@ -8124,3 +8129,189 @@ SELECT * FROM system.privileges;`, resultColumns: resultColsFromColDescs(systemschema.SystemPrivilegeTable.TableDesc().Columns), } + +var crdbInternalKVFlowController = virtualSchemaTable{ + comment: `node-level view of the kv flow controller, its active streams and available tokens state`, + schema: ` +CREATE TABLE crdb_internal.kv_flow_controller ( + tenant_id INT NOT NULL, + store_id INT NOT NULL, + available_regular_tokens INT NOT NULL, + available_elastic_tokens INT 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.KVFlowController(ctx, &kvflowinspectpb.ControllerRequest{}) + if err != nil { + return err + } + for _, stream := range resp.Streams { + if err := addRow( + tree.NewDInt(tree.DInt(stream.TenantID.ToUint64())), + tree.NewDInt(tree.DInt(stream.StoreID)), + tree.NewDInt(tree.DInt(stream.AvailableRegularTokens)), + tree.NewDInt(tree.DInt(stream.AvailableElasticTokens)), + ); err != nil { + return err + } + } + return nil + }, +} + +var crdbInternalKVFlowHandles = virtualSchemaTable{ + comment: `node-level view of active kv flow control handles, their underlying streams, and tracked state`, + schema: ` +CREATE TABLE crdb_internal.kv_flow_control_handles ( + range_id INT NOT NULL, + tenant_id INT NOT NULL, + store_id INT NOT NULL, + total_tracked_tokens INT NOT NULL, + INDEX(range_id) +);`, + + indexes: []virtualIndex{ + { + populate: func(ctx context.Context, constraint tree.Datum, p *planner, _ catalog.DatabaseDescriptor, addRow func(...tree.Datum) error) (matched bool, err error) { + hasRoleOption, err := p.HasViewActivityOrViewActivityRedactedRole(ctx) + if err != nil { + return false, err + } + if !hasRoleOption { + return false, noViewActivityOrViewActivityRedactedRoleError(p.User()) + } + + rangeID := roachpb.RangeID(tree.MustBeDInt(constraint)) + resp, err := p.extendedEvalCtx.ExecCfg.InspectzServer.KVFlowHandles( + ctx, &kvflowinspectpb.HandlesRequest{ + RangeIDs: []roachpb.RangeID{rangeID}, + }) + if err != nil { + return false, err + } + return true, populateFlowHandlesResponse(resp, addRow) + }, + }, + }, + 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.KVFlowHandles(ctx, &kvflowinspectpb.HandlesRequest{}) + if err != nil { + return err + } + return populateFlowHandlesResponse(resp, addRow) + }, +} + +func populateFlowHandlesResponse( + resp *kvflowinspectpb.HandlesResponse, addRow func(...tree.Datum) error, +) error { + for _, handle := range resp.Handles { + for _, connected := range handle.ConnectedStreams { + totalTrackedTokens := int64(0) + for _, tracked := range connected.TrackedDeductions { + totalTrackedTokens += tracked.Tokens + } + if err := addRow( + tree.NewDInt(tree.DInt(handle.RangeID)), + tree.NewDInt(tree.DInt(connected.Stream.TenantID.ToUint64())), + tree.NewDInt(tree.DInt(connected.Stream.StoreID)), + tree.NewDInt(tree.DInt(totalTrackedTokens)), + ); err != nil { + return err + } + } + } + return nil +} + +var crdbInternalKVFlowTokenDeductions = virtualSchemaTable{ + comment: `node-level view of tracked kv flow tokens`, + schema: ` +CREATE TABLE crdb_internal.kv_flow_token_deductions ( + range_id INT NOT NULL, + tenant_id INT NOT NULL, + store_id INT NOT NULL, + priority STRING NOT NULL, + log_term INT NOT NULL, + log_index INT NOT NULL, + tokens INT NOT NULL, + INDEX(range_id) +);`, + + indexes: []virtualIndex{ + { + populate: func(ctx context.Context, constraint tree.Datum, p *planner, _ catalog.DatabaseDescriptor, addRow func(...tree.Datum) error) (matched bool, err error) { + hasRoleOption, err := p.HasViewActivityOrViewActivityRedactedRole(ctx) + if err != nil { + return false, err + } + if !hasRoleOption { + return false, noViewActivityOrViewActivityRedactedRoleError(p.User()) + } + + rangeID := roachpb.RangeID(tree.MustBeDInt(constraint)) + resp, err := p.extendedEvalCtx.ExecCfg.InspectzServer.KVFlowHandles( + ctx, &kvflowinspectpb.HandlesRequest{ + RangeIDs: []roachpb.RangeID{rangeID}, + }) + if err != nil { + return false, err + } + return true, populateFlowTokensResponse(resp, addRow) + }, + }, + }, + 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.KVFlowHandles(ctx, &kvflowinspectpb.HandlesRequest{}) + if err != nil { + return err + } + return populateFlowTokensResponse(resp, addRow) + }, +} + +func populateFlowTokensResponse( + resp *kvflowinspectpb.HandlesResponse, addRow func(...tree.Datum) error, +) error { + for _, handle := range resp.Handles { + for _, connected := range handle.ConnectedStreams { + for _, deduction := range connected.TrackedDeductions { + if err := addRow( + tree.NewDInt(tree.DInt(handle.RangeID)), + tree.NewDInt(tree.DInt(connected.Stream.TenantID.ToUint64())), + tree.NewDInt(tree.DInt(connected.Stream.StoreID)), + tree.NewDString(admissionpb.WorkPriority(deduction.Priority).String()), + tree.NewDInt(tree.DInt(deduction.RaftLogPosition.Term)), + tree.NewDInt(tree.DInt(deduction.RaftLogPosition.Index)), + tree.NewDInt(tree.DInt(deduction.Tokens)), + ); err != nil { + return err + } + } + } + } + return nil +} diff --git a/pkg/sql/exec_util.go b/pkg/sql/exec_util.go index f8c97dd307b0..fee6be10dbc8 100644 --- a/pkg/sql/exec_util.go +++ b/pkg/sql/exec_util.go @@ -33,6 +33,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/config/zonepb" "github.com/cockroachdb/cockroach/pkg/featureflag" "github.com/cockroachdb/cockroach/pkg/gossip" + "github.com/cockroachdb/cockroach/pkg/inspectz/inspectzpb" "github.com/cockroachdb/cockroach/pkg/jobs" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/keyvisualizer" @@ -1384,6 +1385,10 @@ type ExecutorConfig struct { // access stores on this node. KVStoresIterator kvserverbase.StoresIterator + // InspectzServer is used to power various crdb_internal vtables, exposing + // the equivalent of /inspectz but through SQL. + InspectzServer inspectzpb.InspectzServer + // RangeDescIteratorFactory is used to construct Iterators over range // descriptors. RangeDescIteratorFactory rangedesc.IteratorFactory diff --git a/pkg/sql/logictest/testdata/logic_test/crdb_internal b/pkg/sql/logictest/testdata/logic_test/crdb_internal index 00eff1b6afff..bf12b4e78705 100644 --- a/pkg/sql/logictest/testdata/logic_test/crdb_internal +++ b/pkg/sql/logictest/testdata/logic_test/crdb_internal @@ -56,6 +56,9 @@ crdb_internal kv_catalog_descriptor table admin NULL NULL crdb_internal kv_catalog_namespace table admin NULL NULL crdb_internal kv_catalog_zones table admin NULL NULL crdb_internal kv_dropped_relations view admin NULL NULL +crdb_internal kv_flow_control_handles table admin NULL NULL +crdb_internal kv_flow_controller table admin NULL NULL +crdb_internal kv_flow_token_deductions table admin NULL NULL crdb_internal kv_inherited_role_members table admin NULL NULL crdb_internal kv_node_liveness table admin NULL NULL crdb_internal kv_node_status table admin 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 453da103c018..2fdc40eea4e8 100644 --- a/pkg/sql/logictest/testdata/logic_test/crdb_internal_catalog +++ b/pkg/sql/logictest/testdata/logic_test/crdb_internal_catalog @@ -171,227 +171,230 @@ SELECT id, strip_volatile(descriptor) FROM crdb_internal.kv_catalog_descriptor 111 {"table": {"checks": [{"columnIds": [1], "constraintId": 2, "expr": "k > 0:::INT8", "name": "ck"}], "columns": [{"id": 1, "name": "k", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "v", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "dependedOnBy": [{"columnIds": [1, 2], "id": 112}], "formatVersion": 3, "id": 111, "name": "kv", "nextColumnId": 3, "nextConstraintId": 3, "nextIndexId": 2, "nextMutationId": 1, "parentId": 106, "primaryIndex": {"constraintId": 1, "encodingType": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [1], "keyColumnNames": ["k"], "name": "kv_pkey", "partitioning": {}, "sharded": {}, "storeColumnIds": [2], "storeColumnNames": ["v"], "unique": true, "version": 4}, "privileges": {"ownerProto": "root", "users": [{"privileges": "2", "userProto": "admin", "withGrantOption": "2"}, {"privileges": "2", "userProto": "root", "withGrantOption": "2"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 107, "version": "4"}} 112 {"table": {"columns": [{"id": 1, "name": "k", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "v", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"defaultExpr": "unique_rowid()", "hidden": true, "id": 3, "name": "rowid", "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "dependsOn": [111], "formatVersion": 3, "id": 112, "indexes": [{"createdExplicitly": true, "foreignKey": {}, "geoConfig": {}, "id": 2, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [2], "keyColumnNames": ["v"], "keySuffixColumnIds": [3], "name": "idx", "partitioning": {}, "sharded": {}, "version": 4}], "isMaterializedView": true, "name": "mv", "nextColumnId": 4, "nextConstraintId": 2, "nextIndexId": 4, "nextMutationId": 1, "parentId": 106, "primaryIndex": {"constraintId": 1, "encodingType": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [3], "keyColumnNames": ["rowid"], "name": "mv_pkey", "partitioning": {}, "sharded": {}, "storeColumnIds": [1, 2], "storeColumnNames": ["k", "v"], "unique": true, "version": 4}, "privileges": {"ownerProto": "root", "users": [{"privileges": "2", "userProto": "admin", "withGrantOption": "2"}, {"privileges": "2", "userProto": "root", "withGrantOption": "2"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 107, "version": "9", "viewQuery": "SELECT k, v FROM db.public.kv"}} 113 {"function": {"functionBody": "SELECT json_remove_path(json_remove_path(json_remove_path(json_remove_path(json_remove_path(json_remove_path(json_remove_path(json_remove_path(json_remove_path(json_remove_path(json_remove_path(json_remove_path(d, ARRAY['table', 'families']), ARRAY['table', 'nextFamilyId']), ARRAY['table', 'indexes', '0', 'createdAtNanos']), ARRAY['table', 'indexes', '1', 'createdAtNanos']), ARRAY['table', 'indexes', '2', 'createdAtNanos']), ARRAY['table', 'primaryIndex', 'createdAtNanos']), ARRAY['table', 'createAsOfTime']), ARRAY['table', 'modificationTime']), ARRAY['function', 'modificationTime']), ARRAY['type', 'modificationTime']), ARRAY['schema', 'modificationTime']), ARRAY['database', 'modificationTime']);", "id": 113, "lang": "SQL", "name": "strip_volatile", "nullInputBehavior": "CALLED_ON_NULL_INPUT", "params": [{"class": "IN", "name": "d", "type": {"family": "JsonFamily", "oid": 3802}}], "parentId": 104, "parentSchemaId": 105, "privileges": {"ownerProto": "root", "users": [{"privileges": "2", "userProto": "admin", "withGrantOption": "2"}, {"privileges": "2", "userProto": "root", "withGrantOption": "2"}], "version": 2}, "returnType": {"type": {"family": "JsonFamily", "oid": 3802}}, "version": "1", "volatility": "STABLE"}} -4294966983 {"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": 4294966983, "name": "spatial_ref_sys", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294966986, "version": "1"}} -4294966984 {"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": 4294966984, "name": "geometry_columns", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294966986, "version": "1"}} -4294966985 {"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": 4294966985, "name": "geography_columns", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294966986, "version": "1"}} -4294966986 {"schema": {"defaultPrivileges": {"type": "SCHEMA"}, "id": 4294966986, "name": "pg_extension", "privileges": {"ownerProto": "node", "users": [{"privileges": "512", "userProto": "public"}], "version": 2}, "version": "1"}} -4294966987 {"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": 4294966987, "name": "pg_views", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294966988 {"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": 4294966988, "name": "pg_user", "nextColumnId": 10, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294966989 {"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": 4294966989, "name": "pg_user_mappings", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294966990 {"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": 4294966990, "name": "pg_user_mapping", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294966991 {"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": 4294966991, "indexes": [{"foreignKey": {}, "geoConfig": {}, "id": 2, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [1], "keyColumnNames": ["oid"], "name": "pg_type_oid_idx", "partitioning": {}, "sharded": {}, "storeColumnIds": [2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31], "storeColumnNames": ["typname", "typnamespace", "typowner", "typlen", "typbyval", "typtype", "typcategory", "typispreferred", "typisdefined", "typdelim", "typrelid", "typelem", "typarray", "typinput", "typoutput", "typreceive", "typsend", "typmodin", "typmodout", "typanalyze", "typalign", "typstorage", "typnotnull", "typbasetype", "typtypmod", "typndims", "typcollation", "typdefaultbin", "typdefault", "typacl"], "version": 3}], "name": "pg_type", "nextColumnId": 32, "nextConstraintId": 2, "nextIndexId": 3, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294966992 {"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": 4294966992, "name": "pg_ts_template", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294966993 {"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": 4294966993, "name": "pg_ts_parser", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294966994 {"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": 4294966994, "name": "pg_ts_dict", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294966995 {"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": 4294966995, "name": "pg_ts_config", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294966996 {"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": 4294966996, "name": "pg_ts_config_map", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294966997 {"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": 4294966997, "name": "pg_trigger", "nextColumnId": 20, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294966998 {"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": 4294966998, "name": "pg_transform", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294966999 {"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": 4294966999, "indexes": [{"foreignKey": {}, "geoConfig": {}, "id": 2, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [1], "keyColumnNames": ["name"], "name": "pg_timezone_names_name_idx", "partitioning": {}, "sharded": {}, "storeColumnIds": [2, 3, 4], "storeColumnNames": ["abbrev", "utc_offset", "is_dst"], "version": 3}], "name": "pg_timezone_names", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 3, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294967000 {"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": 4294967000, "name": "pg_timezone_abbrevs", "nextColumnId": 4, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294967001 {"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": 4294967001, "name": "pg_tablespace", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294967002 {"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": 4294967002, "name": "pg_tables", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294967003 {"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": 4294967003, "name": "pg_subscription", "nextColumnId": 10, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294967004 {"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": 4294967004, "name": "pg_subscription_rel", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294967005 {"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": 4294967005, "name": "pg_stats", "nextColumnId": 15, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294967006 {"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": 4294967006, "name": "pg_stats_ext", "nextColumnId": 14, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294967007 {"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": 4294967007, "name": "pg_statistic", "nextColumnId": 32, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294967008 {"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": 4294967008, "name": "pg_statistic_ext", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294967009 {"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": 4294967009, "name": "pg_statistic_ext_data", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "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": "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": 4294967010, "name": "pg_statio_user_tables", "nextColumnId": 12, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294967011 {"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": 4294967011, "name": "pg_statio_user_sequences", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294967012 {"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": 4294967012, "name": "pg_statio_user_indexes", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294967013 {"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": 4294967013, "name": "pg_statio_sys_tables", "nextColumnId": 12, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294967014 {"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": 4294967014, "name": "pg_statio_sys_sequences", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294967015 {"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": 4294967015, "name": "pg_statio_sys_indexes", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294967016 {"table": {"columns": [{"id": 1, "name": "relid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "relname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "heap_blks_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "heap_blks_hit", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "idx_blks_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "idx_blks_hit", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "toast_blks_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 9, "name": "toast_blks_hit", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 10, "name": "tidx_blks_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 11, "name": "tidx_blks_hit", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967016, "name": "pg_statio_all_tables", "nextColumnId": 12, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294967017 {"table": {"columns": [{"id": 1, "name": "relid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "relname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "blks_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "blks_hit", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967017, "name": "pg_statio_all_sequences", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294967018 {"table": {"columns": [{"id": 1, "name": "relid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "indexrelid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "relname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 5, "name": "indexrelname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 6, "name": "idx_blks_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "idx_blks_hit", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967018, "name": "pg_statio_all_indexes", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294967019 {"table": {"columns": [{"id": 1, "name": "relid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "relname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "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": 4294967019, "name": "pg_stat_xact_user_tables", "nextColumnId": 12, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294967020 {"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": 4294967020, "name": "pg_stat_xact_user_functions", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294967021 {"table": {"columns": [{"id": 1, "name": "relid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "relname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "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": 4294967021, "name": "pg_stat_xact_sys_tables", "nextColumnId": 12, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294967022 {"table": {"columns": [{"id": 1, "name": "relid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "relname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "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": 4294967022, "name": "pg_stat_xact_all_tables", "nextColumnId": 12, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294967023 {"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": 4294967023, "name": "pg_stat_wal_receiver", "nextColumnId": 16, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "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_user_tables", "nextColumnId": 24, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "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_user_indexes", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294967026 {"table": {"columns": [{"id": 1, "name": "funcid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "funcname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "calls", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "total_time", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 6, "name": "self_time", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}], "formatVersion": 3, "id": 4294967026, "name": "pg_stat_user_functions", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294967027 {"table": {"columns": [{"id": 1, "name": "relid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "relname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "seq_scan", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "seq_tup_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "idx_scan", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "idx_tup_fetch", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "n_tup_ins", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 9, "name": "n_tup_upd", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 10, "name": "n_tup_del", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 11, "name": "n_tup_hot_upd", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"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": 4294967027, "name": "pg_stat_sys_tables", "nextColumnId": 24, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294967028 {"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": 4294967028, "name": "pg_stat_sys_indexes", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294967029 {"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": 4294967029, "name": "pg_stat_subscription", "nextColumnId": 10, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294967030 {"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": 4294967030, "name": "pg_stat_ssl", "nextColumnId": 10, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294967031 {"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": 4294967031, "name": "pg_stat_slru", "nextColumnId": 10, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294967032 {"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": 4294967032, "name": "pg_stat_replication", "nextColumnId": 21, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294967033 {"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": 4294967033, "name": "pg_stat_progress_vacuum", "nextColumnId": 12, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294967034 {"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": 4294967034, "name": "pg_stat_progress_create_index", "nextColumnId": 17, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294967035 {"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": 4294967035, "name": "pg_stat_progress_cluster", "nextColumnId": 13, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294967036 {"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": 4294967036, "name": "pg_stat_progress_basebackup", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294967037 {"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": 4294967037, "name": "pg_stat_progress_analyze", "nextColumnId": 13, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294967038 {"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": 4294967038, "name": "pg_stat_gssapi", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294967039 {"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": 4294967039, "name": "pg_stat_database", "nextColumnId": 22, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294967040 {"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": 4294967040, "name": "pg_stat_database_conflicts", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294967041 {"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": 4294967041, "name": "pg_stat_bgwriter", "nextColumnId": 12, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294967042 {"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": 4294967042, "name": "pg_stat_archiver", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294967043 {"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": 4294967043, "name": "pg_stat_all_tables", "nextColumnId": 24, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294967044 {"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": 4294967044, "name": "pg_stat_all_indexes", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294967045 {"table": {"columns": [{"id": 1, "name": "datid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "datname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "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": 4294967045, "name": "pg_stat_activity", "nextColumnId": 22, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294967046 {"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": 4294967046, "name": "pg_shmem_allocations", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294967047 {"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": 4294967047, "name": "pg_shdepend", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294967048 {"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": 4294967048, "name": "pg_shseclabel", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294967049 {"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": 4294967049, "name": "pg_shdescription", "nextColumnId": 4, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294967050 {"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": 4294967050, "name": "pg_shadow", "nextColumnId": 10, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294967051 {"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": 4294967051, "name": "pg_settings", "nextColumnId": 18, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294967052 {"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": 4294967052, "name": "pg_sequences", "nextColumnId": 12, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294967053 {"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": 4294967053, "name": "pg_sequence", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294967054 {"table": {"columns": [{"id": 1, "name": "objoid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "classoid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "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": 4294967054, "name": "pg_seclabel", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294967055 {"table": {"columns": [{"id": 1, "name": "objoid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "classoid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "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": 4294967055, "name": "pg_seclabels", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294967056 {"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": 4294967056, "name": "pg_rules", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294967057 {"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": 4294967057, "name": "pg_roles", "nextColumnId": 15, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294967058 {"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": 4294967058, "name": "pg_rewrite", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294967059 {"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": 4294967059, "name": "pg_replication_slots", "nextColumnId": 15, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294967060 {"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": 4294967060, "name": "pg_replication_origin", "nextColumnId": 3, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294967061 {"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": 4294967061, "name": "pg_replication_origin_status", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294967062 {"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": 4294967062, "name": "pg_range", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294967063 {"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": 4294967063, "name": "pg_publication_tables", "nextColumnId": 4, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294967064 {"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": 4294967064, "name": "pg_publication", "nextColumnId": 10, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294967065 {"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": 4294967065, "name": "pg_publication_rel", "nextColumnId": 4, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294967066 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "proname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "pronamespace", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "proowner", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "prolang", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 6, "name": "procost", "nullable": true, "type": {"family": "FloatFamily", "oid": 700, "visibleType": 5, "width": 32}}, {"id": 7, "name": "prorows", "nullable": true, "type": {"family": "FloatFamily", "oid": 700, "visibleType": 5, "width": 32}}, {"id": 8, "name": "provariadic", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 9, "name": "protransform", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "proisagg", "nullable": true, "type": {"oid": 16}}, {"id": 11, "name": "proiswindow", "nullable": true, "type": {"oid": 16}}, {"id": 12, "name": "prosecdef", "nullable": true, "type": {"oid": 16}}, {"id": 13, "name": "proleakproof", "nullable": true, "type": {"oid": 16}}, {"id": 14, "name": "proisstrict", "nullable": true, "type": {"oid": 16}}, {"id": 15, "name": "proretset", "nullable": true, "type": {"oid": 16}}, {"id": 16, "name": "provolatile", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 17, "name": "proparallel", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 18, "name": "pronargs", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}, {"id": 19, "name": "pronargdefaults", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}, {"id": 20, "name": "prorettype", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 21, "name": "proargtypes", "nullable": true, "type": {"arrayContents": {"family": "OidFamily", "oid": 26}, "arrayElemType": "OidFamily", "family": 201, "oid": 30}}, {"id": 22, "name": "proallargtypes", "nullable": true, "type": {"arrayContents": {"family": "OidFamily", "oid": 26}, "arrayElemType": "OidFamily", "family": "ArrayFamily", "oid": 1028}}, {"id": 23, "name": "proargmodes", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1002, "visibleType": 9, "width": 1}}, {"id": 24, "name": "proargnames", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 25, "name": "proargdefaults", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 26, "name": "protrftypes", "nullable": true, "type": {"arrayContents": {"family": "OidFamily", "oid": 26}, "arrayElemType": "OidFamily", "family": "ArrayFamily", "oid": 1028}}, {"id": 27, "name": "prosrc", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 28, "name": "probin", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 29, "name": "proconfig", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 30, "name": "proacl", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 31, "name": "prokind", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 32, "name": "prosupport", "nullable": true, "type": {"family": "OidFamily", "oid": 24}}], "formatVersion": 3, "id": 4294967066, "indexes": [{"foreignKey": {}, "geoConfig": {}, "id": 2, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [1], "keyColumnNames": ["oid"], "name": "pg_proc_oid_idx", "partitioning": {}, "sharded": {}, "storeColumnIds": [2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32], "storeColumnNames": ["proname", "pronamespace", "proowner", "prolang", "procost", "prorows", "provariadic", "protransform", "proisagg", "proiswindow", "prosecdef", "proleakproof", "proisstrict", "proretset", "provolatile", "proparallel", "pronargs", "pronargdefaults", "prorettype", "proargtypes", "proallargtypes", "proargmodes", "proargnames", "proargdefaults", "protrftypes", "prosrc", "probin", "proconfig", "proacl", "prokind", "prosupport"], "version": 3}], "name": "pg_proc", "nextColumnId": 33, "nextConstraintId": 2, "nextIndexId": 3, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294967067 {"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": 4294967067, "name": "pg_prepared_xacts", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294967068 {"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": 4294967068, "name": "pg_prepared_statements", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294967069 {"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": 4294967069, "name": "pg_policy", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294967070 {"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": 4294967070, "name": "pg_policies", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294967071 {"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": 4294967071, "name": "pg_partitioned_table", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294967072 {"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": 4294967072, "name": "pg_opfamily", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294967073 {"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": 4294967073, "name": "pg_operator", "nextColumnId": 16, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294967074 {"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": 4294967074, "name": "pg_opclass", "nextColumnId": 10, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294967075 {"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": 4294967075, "indexes": [{"foreignKey": {}, "geoConfig": {}, "id": 2, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [1], "keyColumnNames": ["oid"], "name": "pg_namespace_oid_idx", "partitioning": {}, "sharded": {}, "storeColumnIds": [2, 3, 4], "storeColumnNames": ["nspname", "nspowner", "nspacl"], "version": 3}], "name": "pg_namespace", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 3, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294967076 {"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": 4294967076, "name": "pg_matviews", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294967077 {"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": 4294967077, "name": "pg_locks", "nextColumnId": 16, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294967078 {"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": 4294967078, "name": "pg_largeobject", "nextColumnId": 4, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294967079 {"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": 4294967079, "name": "pg_largeobject_metadata", "nextColumnId": 4, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294967080 {"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": 4294967080, "name": "pg_language", "nextColumnId": 10, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294967081 {"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": 4294967081, "name": "pg_init_privs", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294967082 {"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": 4294967082, "name": "pg_inherits", "nextColumnId": 4, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294967083 {"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": 4294967083, "name": "pg_indexes", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294967084 {"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": 4294967084, "name": "pg_index", "nextColumnId": 22, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294967085 {"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": 4294967085, "name": "pg_hba_file_rules", "nextColumnId": 10, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294967086 {"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": 4294967086, "name": "pg_group", "nextColumnId": 4, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294967087 {"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": 4294967087, "name": "pg_foreign_table", "nextColumnId": 4, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294967088 {"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": 4294967088, "name": "pg_foreign_server", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294967089 {"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": 4294967089, "name": "pg_foreign_data_wrapper", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294967090 {"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": 4294967090, "name": "pg_file_settings", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294967091 {"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": 4294967091, "name": "pg_extension", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294967092 {"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": 4294967092, "name": "pg_event_trigger", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294967093 {"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": 4294967093, "name": "pg_enum", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294967094 {"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": 4294967094, "indexes": [{"foreignKey": {}, "geoConfig": {}, "id": 2, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [1], "keyColumnNames": ["objoid"], "name": "pg_description_objoid_idx", "partitioning": {}, "sharded": {}, "storeColumnIds": [2, 3, 4], "storeColumnNames": ["classoid", "objsubid", "description"], "version": 3}], "name": "pg_description", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 3, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294967095 {"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": 4294967095, "name": "pg_depend", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294967096 {"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": 4294967096, "name": "pg_default_acl", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294967097 {"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": 4294967097, "name": "pg_db_role_setting", "nextColumnId": 4, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294967098 {"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": 4294967098, "name": "pg_database", "nextColumnId": 15, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294967099 {"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": 4294967099, "name": "pg_cursors", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294967100 {"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": 4294967100, "name": "pg_conversion", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294967101 {"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": 4294967101, "indexes": [{"foreignKey": {}, "geoConfig": {}, "id": 2, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [8], "keyColumnNames": ["conrelid"], "name": "pg_constraint_conrelid_idx", "partitioning": {}, "sharded": {}, "storeColumnIds": [1, 2, 3, 4, 5, 6, 7, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27], "storeColumnNames": ["oid", "conname", "connamespace", "contype", "condeferrable", "condeferred", "convalidated", "contypid", "conindid", "confrelid", "confupdtype", "confdeltype", "confmatchtype", "conislocal", "coninhcount", "connoinherit", "conkey", "confkey", "conpfeqop", "conppeqop", "conffeqop", "conexclop", "conbin", "consrc", "condef", "conparentid"], "version": 3}], "name": "pg_constraint", "nextColumnId": 28, "nextConstraintId": 2, "nextIndexId": 3, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294967102 {"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": 4294967102, "name": "pg_config", "nextColumnId": 3, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294967103 {"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": 4294967103, "name": "pg_collation", "nextColumnId": 11, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294967104 {"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": 4294967104, "indexes": [{"foreignKey": {}, "geoConfig": {}, "id": 2, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [1], "keyColumnNames": ["oid"], "name": "pg_class_oid_idx", "partitioning": {}, "sharded": {}, "storeColumnIds": [2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36], "storeColumnNames": ["relname", "relnamespace", "reltype", "reloftype", "relowner", "relam", "relfilenode", "reltablespace", "relpages", "reltuples", "relallvisible", "reltoastrelid", "relhasindex", "relisshared", "relpersistence", "relistemp", "relkind", "relnatts", "relchecks", "relhasoids", "relhaspkey", "relhasrules", "relhastriggers", "relhassubclass", "relfrozenxid", "relacl", "reloptions", "relforcerowsecurity", "relispartition", "relispopulated", "relreplident", "relrewrite", "relrowsecurity", "relpartbound", "relminmxid"], "version": 3}], "name": "pg_class", "nextColumnId": 37, "nextConstraintId": 2, "nextIndexId": 3, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294967105 {"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": 4294967105, "name": "pg_cast", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294967106 {"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": 4294967106, "name": "pg_available_extensions", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294967107 {"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": 4294967107, "name": "pg_available_extension_versions", "nextColumnId": 10, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294967108 {"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": 4294967108, "name": "pg_auth_members", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294967109 {"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": 4294967109, "name": "pg_authid", "nextColumnId": 13, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294967110 {"table": {"columns": [{"id": 1, "name": "attrelid", "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "attname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "atttypid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "attstattarget", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 5, "name": "attlen", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}, {"id": 6, "name": "attnum", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}, {"id": 7, "name": "attndims", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 8, "name": "attcacheoff", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 9, "name": "atttypmod", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 10, "name": "attbyval", "nullable": true, "type": {"oid": 16}}, {"id": 11, "name": "attstorage", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 12, "name": "attalign", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 13, "name": "attnotnull", "nullable": true, "type": {"oid": 16}}, {"id": 14, "name": "atthasdef", "nullable": true, "type": {"oid": 16}}, {"id": 15, "name": "attidentity", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 16, "name": "attgenerated", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 17, "name": "attisdropped", "nullable": true, "type": {"oid": 16}}, {"id": 18, "name": "attislocal", "nullable": true, "type": {"oid": 16}}, {"id": 19, "name": "attinhcount", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 20, "name": "attcollation", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 21, "name": "attacl", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 22, "name": "attoptions", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 23, "name": "attfdwoptions", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 24, "name": "atthasmissing", "nullable": true, "type": {"oid": 16}}, {"id": 25, "name": "attmissingval", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}], "formatVersion": 3, "id": 4294967110, "indexes": [{"foreignKey": {}, "geoConfig": {}, "id": 2, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [1], "keyColumnNames": ["attrelid"], "name": "pg_attribute_attrelid_idx", "partitioning": {}, "sharded": {}, "storeColumnIds": [2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25], "storeColumnNames": ["attname", "atttypid", "attstattarget", "attlen", "attnum", "attndims", "attcacheoff", "atttypmod", "attbyval", "attstorage", "attalign", "attnotnull", "atthasdef", "attidentity", "attgenerated", "attisdropped", "attislocal", "attinhcount", "attcollation", "attacl", "attoptions", "attfdwoptions", "atthasmissing", "attmissingval"], "version": 3}], "name": "pg_attribute", "nextColumnId": 26, "nextConstraintId": 2, "nextIndexId": 3, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294967111 {"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": 4294967111, "indexes": [{"foreignKey": {}, "geoConfig": {}, "id": 2, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [2], "keyColumnNames": ["adrelid"], "name": "pg_attrdef_adrelid_idx", "partitioning": {}, "sharded": {}, "storeColumnIds": [1, 3, 4, 5], "storeColumnNames": ["oid", "adnum", "adbin", "adsrc"], "version": 3}], "name": "pg_attrdef", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 3, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294967112 {"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": 4294967112, "name": "pg_amproc", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294967113 {"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": 4294967113, "name": "pg_amop", "nextColumnId": 10, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294967114 {"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": 4294967114, "name": "pg_am", "nextColumnId": 34, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294967115 {"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": 4294967115, "name": "pg_aggregate", "nextColumnId": 23, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967116, "version": "1"}} -4294967116 {"schema": {"defaultPrivileges": {"type": "SCHEMA"}, "id": 4294967116, "name": "pg_catalog", "privileges": {"ownerProto": "node", "users": [{"privileges": "512", "userProto": "public"}], "version": 2}, "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": "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": 4294967117, "name": "views", "nextColumnId": 11, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967203, "version": "1"}} -4294967118 {"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": 4294967118, "name": "view_table_usage", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967203, "version": "1"}} -4294967119 {"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": 4294967119, "name": "view_routine_usage", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967203, "version": "1"}} -4294967120 {"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": 4294967120, "name": "view_column_usage", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967203, "version": "1"}} -4294967121 {"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": 4294967121, "name": "user_privileges", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967203, "version": "1"}} -4294967122 {"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": 4294967122, "name": "user_mappings", "nextColumnId": 4, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967203, "version": "1"}} -4294967123 {"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": 4294967123, "name": "user_mapping_options", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967203, "version": "1"}} -4294967124 {"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": 4294967124, "name": "user_defined_types", "nextColumnId": 30, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967203, "version": "1"}} -4294967125 {"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": 4294967125, "name": "user_attributes", "nextColumnId": 4, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967203, "version": "1"}} -4294967126 {"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": 4294967126, "name": "usage_privileges", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967203, "version": "1"}} -4294967127 {"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": 4294967127, "name": "udt_privileges", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967203, "version": "1"}} -4294967128 {"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": 4294967128, "name": "type_privileges", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967203, "version": "1"}} -4294967129 {"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": 4294967129, "name": "triggers", "nextColumnId": 18, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967203, "version": "1"}} -4294967130 {"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": 4294967130, "name": "triggered_update_columns", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967203, "version": "1"}} -4294967131 {"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": 4294967131, "name": "transforms", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967203, "version": "1"}} -4294967132 {"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": 4294967132, "name": "tablespaces", "nextColumnId": 10, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967203, "version": "1"}} -4294967133 {"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": 4294967133, "name": "tablespaces_extensions", "nextColumnId": 3, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967203, "version": "1"}} -4294967134 {"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": 4294967134, "name": "tables", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967203, "version": "1"}} -4294967135 {"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": 4294967135, "name": "tables_extensions", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967203, "version": "1"}} -4294967136 {"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": 4294967136, "name": "table_privileges", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967203, "version": "1"}} -4294967137 {"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": 4294967137, "name": "table_constraints_extensions", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967203, "version": "1"}} -4294967138 {"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": 4294967138, "name": "table_constraints", "nextColumnId": 10, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967203, "version": "1"}} -4294967139 {"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": 4294967139, "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": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967203, "version": "1"}} -4294967140 {"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": 4294967140, "name": "st_units_of_measure", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967203, "version": "1"}} -4294967141 {"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": 4294967141, "name": "st_spatial_reference_systems", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967203, "version": "1"}} -4294967142 {"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": 4294967142, "name": "st_geometry_columns", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967203, "version": "1"}} -4294967143 {"table": {"columns": [{"id": 1, "name": "variable", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "value", "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967143, "name": "session_variables", "nextColumnId": 3, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967203, "version": "1"}} -4294967144 {"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": 4294967144, "name": "sequences", "nextColumnId": 13, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967203, "version": "1"}} -4294967145 {"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": 4294967145, "name": "schema_privileges", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967203, "version": "1"}} -4294967146 {"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": 4294967146, "name": "schemata", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967203, "version": "1"}} -4294967147 {"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": 4294967147, "name": "schemata_extensions", "nextColumnId": 4, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967203, "version": "1"}} -4294967148 {"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": 4294967148, "name": "sql_sizing", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967203, "version": "1"}} -4294967149 {"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": 4294967149, "name": "sql_parts", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967203, "version": "1"}} -4294967150 {"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": 4294967150, "name": "sql_implementation_info", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967203, "version": "1"}} -4294967151 {"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": 4294967151, "name": "sql_features", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967203, "version": "1"}} -4294967152 {"table": {"columns": [{"id": 1, "name": "specific_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "specific_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "specific_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "routine_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "routine_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "routine_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "routine_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "module_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "module_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "module_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 11, "name": "udt_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 12, "name": "udt_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 13, "name": "udt_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 14, "name": "data_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 15, "name": "character_maximum_length", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 16, "name": "character_octet_length", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 17, "name": "character_set_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 18, "name": "character_set_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 19, "name": "character_set_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 20, "name": "collation_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 21, "name": "collation_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 22, "name": "collation_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 23, "name": "numeric_precision", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 24, "name": "numeric_precision_radix", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 25, "name": "numeric_scale", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 26, "name": "datetime_precision", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 27, "name": "interval_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 28, "name": "interval_precision", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 29, "name": "type_udt_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 30, "name": "type_udt_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 31, "name": "type_udt_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 32, "name": "scope_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 33, "name": "scope_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 34, "name": "maximum_cardinality", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 35, "name": "dtd_identifier", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 36, "name": "routine_body", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 37, "name": "routine_definition", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 38, "name": "external_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 39, "name": "external_language", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 40, "name": "parameter_style", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 41, "name": "is_deterministic", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 42, "name": "sql_data_access", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 43, "name": "is_null_call", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 44, "name": "sql_path", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 45, "name": "schema_level_routine", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 46, "name": "max_dynamic_result_sets", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 47, "name": "is_user_defined_cast", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 48, "name": "is_implicitly_invocable", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 49, "name": "security_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 50, "name": "to_sql_specific_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 51, "name": "to_sql_specific_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 52, "name": "to_sql_specific_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 53, "name": "as_locator", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 54, "name": "created", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 55, "name": "last_altered", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 56, "name": "new_savepoint_level", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 57, "name": "is_udt_dependent", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 58, "name": "result_cast_from_data_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 59, "name": "result_cast_as_locator", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 60, "name": "result_cast_char_max_length", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 61, "name": "result_cast_char_octet_length", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 62, "name": "result_cast_char_set_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 63, "name": "result_cast_char_set_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 64, "name": "result_cast_char_set_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 65, "name": "result_cast_collation_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 66, "name": "result_cast_collation_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 67, "name": "result_cast_collation_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 68, "name": "result_cast_numeric_precision", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 69, "name": "result_cast_numeric_precision_radix", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 70, "name": "result_cast_numeric_scale", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 71, "name": "result_cast_datetime_precision", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 72, "name": "result_cast_interval_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 73, "name": "result_cast_interval_precision", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 74, "name": "result_cast_type_udt_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 75, "name": "result_cast_type_udt_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 76, "name": "result_cast_type_udt_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 77, "name": "result_cast_scope_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 78, "name": "result_cast_scope_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 79, "name": "result_cast_scope_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 80, "name": "result_cast_maximum_cardinality", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 81, "name": "result_cast_dtd_identifier", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967152, "name": "routines", "nextColumnId": 82, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967203, "version": "1"}} -4294967153 {"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": 4294967153, "name": "routine_privileges", "nextColumnId": 11, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967203, "version": "1"}} -4294967154 {"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": 4294967154, "name": "role_usage_grants", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967203, "version": "1"}} -4294967155 {"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": 4294967155, "name": "role_udt_grants", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967203, "version": "1"}} -4294967156 {"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": 4294967156, "name": "role_table_grants", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967203, "version": "1"}} -4294967157 {"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": 4294967157, "name": "role_routine_grants", "nextColumnId": 11, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967203, "version": "1"}} -4294967158 {"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": 4294967158, "name": "role_column_grants", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967203, "version": "1"}} -4294967159 {"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": 4294967159, "name": "resource_groups", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967203, "version": "1"}} -4294967160 {"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": 4294967160, "name": "referential_constraints", "nextColumnId": 12, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967203, "version": "1"}} -4294967161 {"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": 4294967161, "name": "profiling", "nextColumnId": 19, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967203, "version": "1"}} -4294967162 {"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": 4294967162, "name": "processlist", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967203, "version": "1"}} -4294967163 {"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": 4294967163, "name": "plugins", "nextColumnId": 12, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967203, "version": "1"}} -4294967164 {"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": 4294967164, "name": "partitions", "nextColumnId": 26, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967203, "version": "1"}} -4294967165 {"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": 4294967165, "name": "parameters", "nextColumnId": 33, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967203, "version": "1"}} -4294967166 {"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": 4294967166, "name": "optimizer_trace", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967203, "version": "1"}} -4294967167 {"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": 4294967167, "name": "keywords", "nextColumnId": 3, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967203, "version": "1"}} -4294967168 {"table": {"columns": [{"id": 1, "name": "constraint_catalog", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "constraint_schema", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "constraint_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "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": 4294967168, "name": "key_column_usage", "nextColumnId": 10, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967203, "version": "1"}} -4294967169 {"table": {"columns": [{"id": 1, "name": "catalog_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967169, "name": "information_schema_catalog_name", "nextColumnId": 2, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967203, "version": "1"}} -4294967170 {"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": 4294967170, "name": "foreign_tables", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967203, "version": "1"}} -4294967171 {"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": 4294967171, "name": "foreign_table_options", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967203, "version": "1"}} -4294967172 {"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": 4294967172, "name": "foreign_servers", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967203, "version": "1"}} -4294967173 {"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": 4294967173, "name": "foreign_server_options", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967203, "version": "1"}} -4294967174 {"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": 4294967174, "name": "foreign_data_wrappers", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967203, "version": "1"}} -4294967175 {"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": 4294967175, "name": "foreign_data_wrapper_options", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967203, "version": "1"}} -4294967176 {"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": 4294967176, "name": "files", "nextColumnId": 39, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967203, "version": "1"}} -4294967177 {"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": 4294967177, "name": "events", "nextColumnId": 25, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967203, "version": "1"}} -4294967178 {"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": 4294967178, "name": "engines", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967203, "version": "1"}} -4294967179 {"table": {"columns": [{"id": 1, "name": "role_name", "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967179, "name": "enabled_roles", "nextColumnId": 2, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967203, "version": "1"}} -4294967180 {"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": 4294967180, "name": "element_types", "nextColumnId": 30, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967203, "version": "1"}} -4294967181 {"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": 4294967181, "name": "domains", "nextColumnId": 28, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967203, "version": "1"}} -4294967182 {"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": 4294967182, "name": "domain_udt_usage", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967203, "version": "1"}} -4294967183 {"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": 4294967183, "name": "domain_constraints", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967203, "version": "1"}} -4294967184 {"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": 4294967184, "name": "data_type_privileges", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967203, "version": "1"}} -4294967185 {"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": 4294967185, "name": "constraint_table_usage", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967203, "version": "1"}} -4294967186 {"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": 4294967186, "name": "constraint_column_usage", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967203, "version": "1"}} -4294967187 {"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": 4294967187, "name": "columns", "nextColumnId": 48, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967203, "version": "1"}} -4294967188 {"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": 4294967188, "name": "columns_extensions", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967203, "version": "1"}} -4294967189 {"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": 4294967189, "name": "column_udt_usage", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967203, "version": "1"}} -4294967190 {"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": 4294967190, "name": "column_statistics", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967203, "version": "1"}} -4294967191 {"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": 4294967191, "name": "column_privileges", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967203, "version": "1"}} -4294967192 {"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": 4294967192, "name": "column_options", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967203, "version": "1"}} -4294967193 {"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": 4294967193, "name": "column_domain_usage", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967203, "version": "1"}} -4294967194 {"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": 4294967194, "name": "column_column_usage", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967203, "version": "1"}} -4294967195 {"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": 4294967195, "name": "collations", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967203, "version": "1"}} -4294967196 {"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": 4294967196, "name": "collation_character_set_applicability", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967203, "version": "1"}} -4294967197 {"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": 4294967197, "name": "check_constraints", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967203, "version": "1"}} -4294967198 {"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": 4294967198, "name": "check_constraint_routine_usage", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967203, "version": "1"}} -4294967199 {"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": 4294967199, "name": "character_sets", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967203, "version": "1"}} -4294967200 {"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": 4294967200, "name": "attributes", "nextColumnId": 32, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967203, "version": "1"}} -4294967201 {"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": 4294967201, "name": "applicable_roles", "nextColumnId": 4, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967203, "version": "1"}} -4294967202 {"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": 4294967202, "name": "administrable_role_authorizations", "nextColumnId": 4, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967203, "version": "1"}} -4294967203 {"schema": {"defaultPrivileges": {"type": "SCHEMA"}, "id": 4294967203, "name": "information_schema", "privileges": {"ownerProto": "node", "users": [{"privileges": "512", "userProto": "public"}], "version": 2}, "version": "1"}} +4294966980 {"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": 4294966980, "name": "spatial_ref_sys", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294966983, "version": "1"}} +4294966981 {"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": 4294966981, "name": "geometry_columns", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294966983, "version": "1"}} +4294966982 {"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": 4294966982, "name": "geography_columns", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294966983, "version": "1"}} +4294966983 {"schema": {"defaultPrivileges": {"type": "SCHEMA"}, "id": 4294966983, "name": "pg_extension", "privileges": {"ownerProto": "node", "users": [{"privileges": "512", "userProto": "public"}], "version": 2}, "version": "1"}} +4294966984 {"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": 4294966984, "name": "pg_views", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294966985 {"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": 4294966985, "name": "pg_user", "nextColumnId": 10, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294966986 {"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": 4294966986, "name": "pg_user_mappings", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294966987 {"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": 4294966987, "name": "pg_user_mapping", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294966988 {"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": 4294966988, "indexes": [{"foreignKey": {}, "geoConfig": {}, "id": 2, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [1], "keyColumnNames": ["oid"], "name": "pg_type_oid_idx", "partitioning": {}, "sharded": {}, "storeColumnIds": [2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31], "storeColumnNames": ["typname", "typnamespace", "typowner", "typlen", "typbyval", "typtype", "typcategory", "typispreferred", "typisdefined", "typdelim", "typrelid", "typelem", "typarray", "typinput", "typoutput", "typreceive", "typsend", "typmodin", "typmodout", "typanalyze", "typalign", "typstorage", "typnotnull", "typbasetype", "typtypmod", "typndims", "typcollation", "typdefaultbin", "typdefault", "typacl"], "version": 3}], "name": "pg_type", "nextColumnId": 32, "nextConstraintId": 2, "nextIndexId": 3, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294966989 {"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": 4294966989, "name": "pg_ts_template", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294966990 {"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": 4294966990, "name": "pg_ts_parser", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294966991 {"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": 4294966991, "name": "pg_ts_dict", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294966992 {"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": 4294966992, "name": "pg_ts_config", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294966993 {"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": 4294966993, "name": "pg_ts_config_map", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294966994 {"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": 4294966994, "name": "pg_trigger", "nextColumnId": 20, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294966995 {"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": 4294966995, "name": "pg_transform", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294966996 {"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": 4294966996, "indexes": [{"foreignKey": {}, "geoConfig": {}, "id": 2, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [1], "keyColumnNames": ["name"], "name": "pg_timezone_names_name_idx", "partitioning": {}, "sharded": {}, "storeColumnIds": [2, 3, 4], "storeColumnNames": ["abbrev", "utc_offset", "is_dst"], "version": 3}], "name": "pg_timezone_names", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 3, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294966997 {"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": 4294966997, "name": "pg_timezone_abbrevs", "nextColumnId": 4, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294966998 {"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": 4294966998, "name": "pg_tablespace", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294966999 {"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": 4294966999, "name": "pg_tables", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294967000 {"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": 4294967000, "name": "pg_subscription", "nextColumnId": 10, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294967001 {"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": 4294967001, "name": "pg_subscription_rel", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294967002 {"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": 4294967002, "name": "pg_stats", "nextColumnId": 15, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294967003 {"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": 4294967003, "name": "pg_stats_ext", "nextColumnId": 14, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294967004 {"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": 4294967004, "name": "pg_statistic", "nextColumnId": 32, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294967005 {"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": 4294967005, "name": "pg_statistic_ext", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294967006 {"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": 4294967006, "name": "pg_statistic_ext_data", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "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": "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": 4294967007, "name": "pg_statio_user_tables", "nextColumnId": 12, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "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": "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": 4294967008, "name": "pg_statio_user_sequences", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "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_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": 4294967009, "name": "pg_statio_user_indexes", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "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": "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": 4294967010, "name": "pg_statio_sys_tables", "nextColumnId": 12, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294967011 {"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": 4294967011, "name": "pg_statio_sys_sequences", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294967012 {"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": 4294967012, "name": "pg_statio_sys_indexes", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294967013 {"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": 4294967013, "name": "pg_statio_all_tables", "nextColumnId": 12, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294967014 {"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": 4294967014, "name": "pg_statio_all_sequences", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294967015 {"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": 4294967015, "name": "pg_statio_all_indexes", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294967016 {"table": {"columns": [{"id": 1, "name": "relid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "relname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "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": 4294967016, "name": "pg_stat_xact_user_tables", "nextColumnId": 12, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294967017 {"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": 4294967017, "name": "pg_stat_xact_user_functions", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294967018 {"table": {"columns": [{"id": 1, "name": "relid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "relname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "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": 4294967018, "name": "pg_stat_xact_sys_tables", "nextColumnId": 12, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294967019 {"table": {"columns": [{"id": 1, "name": "relid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "relname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "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": 4294967019, "name": "pg_stat_xact_all_tables", "nextColumnId": 12, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294967020 {"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": 4294967020, "name": "pg_stat_wal_receiver", "nextColumnId": 16, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294967021 {"table": {"columns": [{"id": 1, "name": "relid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "relname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "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": 4294967021, "name": "pg_stat_user_tables", "nextColumnId": 24, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294967022 {"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": 4294967022, "name": "pg_stat_user_indexes", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294967023 {"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": 4294967023, "name": "pg_stat_user_functions", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "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_sys_tables", "nextColumnId": 24, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "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_sys_indexes", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294967026 {"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": 4294967026, "name": "pg_stat_subscription", "nextColumnId": 10, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294967027 {"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": 4294967027, "name": "pg_stat_ssl", "nextColumnId": 10, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294967028 {"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": 4294967028, "name": "pg_stat_slru", "nextColumnId": 10, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294967029 {"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": 4294967029, "name": "pg_stat_replication", "nextColumnId": 21, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294967030 {"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": 4294967030, "name": "pg_stat_progress_vacuum", "nextColumnId": 12, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294967031 {"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": 4294967031, "name": "pg_stat_progress_create_index", "nextColumnId": 17, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294967032 {"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": 4294967032, "name": "pg_stat_progress_cluster", "nextColumnId": 13, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294967033 {"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": 4294967033, "name": "pg_stat_progress_basebackup", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294967034 {"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": 4294967034, "name": "pg_stat_progress_analyze", "nextColumnId": 13, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294967035 {"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": 4294967035, "name": "pg_stat_gssapi", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294967036 {"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": 4294967036, "name": "pg_stat_database", "nextColumnId": 22, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294967037 {"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": 4294967037, "name": "pg_stat_database_conflicts", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294967038 {"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": 4294967038, "name": "pg_stat_bgwriter", "nextColumnId": 12, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294967039 {"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": 4294967039, "name": "pg_stat_archiver", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294967040 {"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": 4294967040, "name": "pg_stat_all_tables", "nextColumnId": 24, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294967041 {"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": 4294967041, "name": "pg_stat_all_indexes", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294967042 {"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": 4294967042, "name": "pg_stat_activity", "nextColumnId": 22, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294967043 {"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": 4294967043, "name": "pg_shmem_allocations", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294967044 {"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": 4294967044, "name": "pg_shdepend", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294967045 {"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": 4294967045, "name": "pg_shseclabel", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294967046 {"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": 4294967046, "name": "pg_shdescription", "nextColumnId": 4, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294967047 {"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": 4294967047, "name": "pg_shadow", "nextColumnId": 10, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294967048 {"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": 4294967048, "name": "pg_settings", "nextColumnId": 18, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294967049 {"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": 4294967049, "name": "pg_sequences", "nextColumnId": 12, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294967050 {"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": 4294967050, "name": "pg_sequence", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294967051 {"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": 4294967051, "name": "pg_seclabel", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294967052 {"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": 4294967052, "name": "pg_seclabels", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "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": "rulename", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "definition", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967053, "name": "pg_rules", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294967054 {"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": 4294967054, "name": "pg_roles", "nextColumnId": 15, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294967055 {"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": 4294967055, "name": "pg_rewrite", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294967056 {"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": 4294967056, "name": "pg_replication_slots", "nextColumnId": 15, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294967057 {"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": 4294967057, "name": "pg_replication_origin", "nextColumnId": 3, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294967058 {"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": 4294967058, "name": "pg_replication_origin_status", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294967059 {"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": 4294967059, "name": "pg_range", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294967060 {"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": 4294967060, "name": "pg_publication_tables", "nextColumnId": 4, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294967061 {"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": 4294967061, "name": "pg_publication", "nextColumnId": 10, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294967062 {"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": 4294967062, "name": "pg_publication_rel", "nextColumnId": 4, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294967063 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "proname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "pronamespace", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "proowner", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "prolang", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 6, "name": "procost", "nullable": true, "type": {"family": "FloatFamily", "oid": 700, "visibleType": 5, "width": 32}}, {"id": 7, "name": "prorows", "nullable": true, "type": {"family": "FloatFamily", "oid": 700, "visibleType": 5, "width": 32}}, {"id": 8, "name": "provariadic", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 9, "name": "protransform", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "proisagg", "nullable": true, "type": {"oid": 16}}, {"id": 11, "name": "proiswindow", "nullable": true, "type": {"oid": 16}}, {"id": 12, "name": "prosecdef", "nullable": true, "type": {"oid": 16}}, {"id": 13, "name": "proleakproof", "nullable": true, "type": {"oid": 16}}, {"id": 14, "name": "proisstrict", "nullable": true, "type": {"oid": 16}}, {"id": 15, "name": "proretset", "nullable": true, "type": {"oid": 16}}, {"id": 16, "name": "provolatile", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 17, "name": "proparallel", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 18, "name": "pronargs", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}, {"id": 19, "name": "pronargdefaults", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}, {"id": 20, "name": "prorettype", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 21, "name": "proargtypes", "nullable": true, "type": {"arrayContents": {"family": "OidFamily", "oid": 26}, "arrayElemType": "OidFamily", "family": 201, "oid": 30}}, {"id": 22, "name": "proallargtypes", "nullable": true, "type": {"arrayContents": {"family": "OidFamily", "oid": 26}, "arrayElemType": "OidFamily", "family": "ArrayFamily", "oid": 1028}}, {"id": 23, "name": "proargmodes", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1002, "visibleType": 9, "width": 1}}, {"id": 24, "name": "proargnames", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 25, "name": "proargdefaults", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 26, "name": "protrftypes", "nullable": true, "type": {"arrayContents": {"family": "OidFamily", "oid": 26}, "arrayElemType": "OidFamily", "family": "ArrayFamily", "oid": 1028}}, {"id": 27, "name": "prosrc", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 28, "name": "probin", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 29, "name": "proconfig", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 30, "name": "proacl", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 31, "name": "prokind", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 32, "name": "prosupport", "nullable": true, "type": {"family": "OidFamily", "oid": 24}}], "formatVersion": 3, "id": 4294967063, "indexes": [{"foreignKey": {}, "geoConfig": {}, "id": 2, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [1], "keyColumnNames": ["oid"], "name": "pg_proc_oid_idx", "partitioning": {}, "sharded": {}, "storeColumnIds": [2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32], "storeColumnNames": ["proname", "pronamespace", "proowner", "prolang", "procost", "prorows", "provariadic", "protransform", "proisagg", "proiswindow", "prosecdef", "proleakproof", "proisstrict", "proretset", "provolatile", "proparallel", "pronargs", "pronargdefaults", "prorettype", "proargtypes", "proallargtypes", "proargmodes", "proargnames", "proargdefaults", "protrftypes", "prosrc", "probin", "proconfig", "proacl", "prokind", "prosupport"], "version": 3}], "name": "pg_proc", "nextColumnId": 33, "nextConstraintId": 2, "nextIndexId": 3, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294967064 {"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": 4294967064, "name": "pg_prepared_xacts", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294967065 {"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": 4294967065, "name": "pg_prepared_statements", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294967066 {"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": 4294967066, "name": "pg_policy", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294967067 {"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": 4294967067, "name": "pg_policies", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294967068 {"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": 4294967068, "name": "pg_partitioned_table", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294967069 {"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": 4294967069, "name": "pg_opfamily", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294967070 {"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": 4294967070, "name": "pg_operator", "nextColumnId": 16, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294967071 {"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": 4294967071, "name": "pg_opclass", "nextColumnId": 10, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294967072 {"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": 4294967072, "indexes": [{"foreignKey": {}, "geoConfig": {}, "id": 2, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [1], "keyColumnNames": ["oid"], "name": "pg_namespace_oid_idx", "partitioning": {}, "sharded": {}, "storeColumnIds": [2, 3, 4], "storeColumnNames": ["nspname", "nspowner", "nspacl"], "version": 3}], "name": "pg_namespace", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 3, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294967073 {"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": 4294967073, "name": "pg_matviews", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294967074 {"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": 4294967074, "name": "pg_locks", "nextColumnId": 16, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294967075 {"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": 4294967075, "name": "pg_largeobject", "nextColumnId": 4, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294967076 {"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": 4294967076, "name": "pg_largeobject_metadata", "nextColumnId": 4, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294967077 {"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": 4294967077, "name": "pg_language", "nextColumnId": 10, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294967078 {"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": 4294967078, "name": "pg_init_privs", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294967079 {"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": 4294967079, "name": "pg_inherits", "nextColumnId": 4, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294967080 {"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": 4294967080, "name": "pg_indexes", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294967081 {"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": 4294967081, "name": "pg_index", "nextColumnId": 22, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294967082 {"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": 4294967082, "name": "pg_hba_file_rules", "nextColumnId": 10, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294967083 {"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": 4294967083, "name": "pg_group", "nextColumnId": 4, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294967084 {"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": 4294967084, "name": "pg_foreign_table", "nextColumnId": 4, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294967085 {"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": 4294967085, "name": "pg_foreign_server", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294967086 {"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": 4294967086, "name": "pg_foreign_data_wrapper", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294967087 {"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": 4294967087, "name": "pg_file_settings", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294967088 {"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": 4294967088, "name": "pg_extension", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294967089 {"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": 4294967089, "name": "pg_event_trigger", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294967090 {"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": 4294967090, "name": "pg_enum", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294967091 {"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": 4294967091, "indexes": [{"foreignKey": {}, "geoConfig": {}, "id": 2, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [1], "keyColumnNames": ["objoid"], "name": "pg_description_objoid_idx", "partitioning": {}, "sharded": {}, "storeColumnIds": [2, 3, 4], "storeColumnNames": ["classoid", "objsubid", "description"], "version": 3}], "name": "pg_description", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 3, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294967092 {"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": 4294967092, "name": "pg_depend", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294967093 {"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": 4294967093, "name": "pg_default_acl", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294967094 {"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": 4294967094, "name": "pg_db_role_setting", "nextColumnId": 4, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294967095 {"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": 4294967095, "name": "pg_database", "nextColumnId": 15, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294967096 {"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": 4294967096, "name": "pg_cursors", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294967097 {"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": 4294967097, "name": "pg_conversion", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294967098 {"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": 4294967098, "indexes": [{"foreignKey": {}, "geoConfig": {}, "id": 2, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [8], "keyColumnNames": ["conrelid"], "name": "pg_constraint_conrelid_idx", "partitioning": {}, "sharded": {}, "storeColumnIds": [1, 2, 3, 4, 5, 6, 7, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27], "storeColumnNames": ["oid", "conname", "connamespace", "contype", "condeferrable", "condeferred", "convalidated", "contypid", "conindid", "confrelid", "confupdtype", "confdeltype", "confmatchtype", "conislocal", "coninhcount", "connoinherit", "conkey", "confkey", "conpfeqop", "conppeqop", "conffeqop", "conexclop", "conbin", "consrc", "condef", "conparentid"], "version": 3}], "name": "pg_constraint", "nextColumnId": 28, "nextConstraintId": 2, "nextIndexId": 3, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294967099 {"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": 4294967099, "name": "pg_config", "nextColumnId": 3, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294967100 {"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": 4294967100, "name": "pg_collation", "nextColumnId": 11, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294967101 {"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": 4294967101, "indexes": [{"foreignKey": {}, "geoConfig": {}, "id": 2, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [1], "keyColumnNames": ["oid"], "name": "pg_class_oid_idx", "partitioning": {}, "sharded": {}, "storeColumnIds": [2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36], "storeColumnNames": ["relname", "relnamespace", "reltype", "reloftype", "relowner", "relam", "relfilenode", "reltablespace", "relpages", "reltuples", "relallvisible", "reltoastrelid", "relhasindex", "relisshared", "relpersistence", "relistemp", "relkind", "relnatts", "relchecks", "relhasoids", "relhaspkey", "relhasrules", "relhastriggers", "relhassubclass", "relfrozenxid", "relacl", "reloptions", "relforcerowsecurity", "relispartition", "relispopulated", "relreplident", "relrewrite", "relrowsecurity", "relpartbound", "relminmxid"], "version": 3}], "name": "pg_class", "nextColumnId": 37, "nextConstraintId": 2, "nextIndexId": 3, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294967102 {"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": 4294967102, "name": "pg_cast", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294967103 {"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": 4294967103, "name": "pg_available_extensions", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294967104 {"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": 4294967104, "name": "pg_available_extension_versions", "nextColumnId": 10, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294967105 {"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": 4294967105, "name": "pg_auth_members", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294967106 {"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": 4294967106, "name": "pg_authid", "nextColumnId": 13, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294967107 {"table": {"columns": [{"id": 1, "name": "attrelid", "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "attname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "atttypid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "attstattarget", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 5, "name": "attlen", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}, {"id": 6, "name": "attnum", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}, {"id": 7, "name": "attndims", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 8, "name": "attcacheoff", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 9, "name": "atttypmod", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 10, "name": "attbyval", "nullable": true, "type": {"oid": 16}}, {"id": 11, "name": "attstorage", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 12, "name": "attalign", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 13, "name": "attnotnull", "nullable": true, "type": {"oid": 16}}, {"id": 14, "name": "atthasdef", "nullable": true, "type": {"oid": 16}}, {"id": 15, "name": "attidentity", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 16, "name": "attgenerated", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 17, "name": "attisdropped", "nullable": true, "type": {"oid": 16}}, {"id": 18, "name": "attislocal", "nullable": true, "type": {"oid": 16}}, {"id": 19, "name": "attinhcount", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 20, "name": "attcollation", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 21, "name": "attacl", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 22, "name": "attoptions", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 23, "name": "attfdwoptions", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 24, "name": "atthasmissing", "nullable": true, "type": {"oid": 16}}, {"id": 25, "name": "attmissingval", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}], "formatVersion": 3, "id": 4294967107, "indexes": [{"foreignKey": {}, "geoConfig": {}, "id": 2, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [1], "keyColumnNames": ["attrelid"], "name": "pg_attribute_attrelid_idx", "partitioning": {}, "sharded": {}, "storeColumnIds": [2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25], "storeColumnNames": ["attname", "atttypid", "attstattarget", "attlen", "attnum", "attndims", "attcacheoff", "atttypmod", "attbyval", "attstorage", "attalign", "attnotnull", "atthasdef", "attidentity", "attgenerated", "attisdropped", "attislocal", "attinhcount", "attcollation", "attacl", "attoptions", "attfdwoptions", "atthasmissing", "attmissingval"], "version": 3}], "name": "pg_attribute", "nextColumnId": 26, "nextConstraintId": 2, "nextIndexId": 3, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294967108 {"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": 4294967108, "indexes": [{"foreignKey": {}, "geoConfig": {}, "id": 2, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [2], "keyColumnNames": ["adrelid"], "name": "pg_attrdef_adrelid_idx", "partitioning": {}, "sharded": {}, "storeColumnIds": [1, 3, 4, 5], "storeColumnNames": ["oid", "adnum", "adbin", "adsrc"], "version": 3}], "name": "pg_attrdef", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 3, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294967109 {"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": 4294967109, "name": "pg_amproc", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294967110 {"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": 4294967110, "name": "pg_amop", "nextColumnId": 10, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294967111 {"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": 4294967111, "name": "pg_am", "nextColumnId": 34, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294967112 {"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": 4294967112, "name": "pg_aggregate", "nextColumnId": 23, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967113, "version": "1"}} +4294967113 {"schema": {"defaultPrivileges": {"type": "SCHEMA"}, "id": 4294967113, "name": "pg_catalog", "privileges": {"ownerProto": "node", "users": [{"privileges": "512", "userProto": "public"}], "version": 2}, "version": "1"}} +4294967114 {"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": 4294967114, "name": "views", "nextColumnId": 11, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967200, "version": "1"}} +4294967115 {"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": 4294967115, "name": "view_table_usage", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967200, "version": "1"}} +4294967116 {"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": 4294967116, "name": "view_routine_usage", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967200, "version": "1"}} +4294967117 {"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": 4294967117, "name": "view_column_usage", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967200, "version": "1"}} +4294967118 {"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": 4294967118, "name": "user_privileges", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967200, "version": "1"}} +4294967119 {"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": 4294967119, "name": "user_mappings", "nextColumnId": 4, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967200, "version": "1"}} +4294967120 {"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": 4294967120, "name": "user_mapping_options", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967200, "version": "1"}} +4294967121 {"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": 4294967121, "name": "user_defined_types", "nextColumnId": 30, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967200, "version": "1"}} +4294967122 {"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": 4294967122, "name": "user_attributes", "nextColumnId": 4, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967200, "version": "1"}} +4294967123 {"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": 4294967123, "name": "usage_privileges", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967200, "version": "1"}} +4294967124 {"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": 4294967124, "name": "udt_privileges", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967200, "version": "1"}} +4294967125 {"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": 4294967125, "name": "type_privileges", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967200, "version": "1"}} +4294967126 {"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": 4294967126, "name": "triggers", "nextColumnId": 18, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967200, "version": "1"}} +4294967127 {"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": 4294967127, "name": "triggered_update_columns", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967200, "version": "1"}} +4294967128 {"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": 4294967128, "name": "transforms", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967200, "version": "1"}} +4294967129 {"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": 4294967129, "name": "tablespaces", "nextColumnId": 10, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967200, "version": "1"}} +4294967130 {"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": 4294967130, "name": "tablespaces_extensions", "nextColumnId": 3, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967200, "version": "1"}} +4294967131 {"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": 4294967131, "name": "tables", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967200, "version": "1"}} +4294967132 {"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": 4294967132, "name": "tables_extensions", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967200, "version": "1"}} +4294967133 {"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": 4294967133, "name": "table_privileges", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967200, "version": "1"}} +4294967134 {"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": 4294967134, "name": "table_constraints_extensions", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967200, "version": "1"}} +4294967135 {"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": 4294967135, "name": "table_constraints", "nextColumnId": 10, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967200, "version": "1"}} +4294967136 {"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": 4294967136, "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": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967200, "version": "1"}} +4294967137 {"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": 4294967137, "name": "st_units_of_measure", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967200, "version": "1"}} +4294967138 {"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": 4294967138, "name": "st_spatial_reference_systems", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967200, "version": "1"}} +4294967139 {"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": 4294967139, "name": "st_geometry_columns", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967200, "version": "1"}} +4294967140 {"table": {"columns": [{"id": 1, "name": "variable", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "value", "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967140, "name": "session_variables", "nextColumnId": 3, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967200, "version": "1"}} +4294967141 {"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": 4294967141, "name": "sequences", "nextColumnId": 13, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967200, "version": "1"}} +4294967142 {"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": 4294967142, "name": "schema_privileges", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967200, "version": "1"}} +4294967143 {"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": 4294967143, "name": "schemata", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967200, "version": "1"}} +4294967144 {"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": 4294967144, "name": "schemata_extensions", "nextColumnId": 4, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967200, "version": "1"}} +4294967145 {"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": 4294967145, "name": "sql_sizing", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967200, "version": "1"}} +4294967146 {"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": 4294967146, "name": "sql_parts", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967200, "version": "1"}} +4294967147 {"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": 4294967147, "name": "sql_implementation_info", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967200, "version": "1"}} +4294967148 {"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": 4294967148, "name": "sql_features", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967200, "version": "1"}} +4294967149 {"table": {"columns": [{"id": 1, "name": "specific_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "specific_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "specific_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "routine_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "routine_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "routine_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "routine_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "module_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "module_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "module_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 11, "name": "udt_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 12, "name": "udt_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 13, "name": "udt_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 14, "name": "data_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 15, "name": "character_maximum_length", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 16, "name": "character_octet_length", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 17, "name": "character_set_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 18, "name": "character_set_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 19, "name": "character_set_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 20, "name": "collation_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 21, "name": "collation_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 22, "name": "collation_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 23, "name": "numeric_precision", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 24, "name": "numeric_precision_radix", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 25, "name": "numeric_scale", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 26, "name": "datetime_precision", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 27, "name": "interval_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 28, "name": "interval_precision", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 29, "name": "type_udt_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 30, "name": "type_udt_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 31, "name": "type_udt_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 32, "name": "scope_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 33, "name": "scope_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 34, "name": "maximum_cardinality", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 35, "name": "dtd_identifier", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 36, "name": "routine_body", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 37, "name": "routine_definition", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 38, "name": "external_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 39, "name": "external_language", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 40, "name": "parameter_style", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 41, "name": "is_deterministic", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 42, "name": "sql_data_access", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 43, "name": "is_null_call", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 44, "name": "sql_path", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 45, "name": "schema_level_routine", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 46, "name": "max_dynamic_result_sets", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 47, "name": "is_user_defined_cast", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 48, "name": "is_implicitly_invocable", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 49, "name": "security_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 50, "name": "to_sql_specific_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 51, "name": "to_sql_specific_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 52, "name": "to_sql_specific_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 53, "name": "as_locator", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 54, "name": "created", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 55, "name": "last_altered", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 56, "name": "new_savepoint_level", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 57, "name": "is_udt_dependent", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 58, "name": "result_cast_from_data_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 59, "name": "result_cast_as_locator", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 60, "name": "result_cast_char_max_length", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 61, "name": "result_cast_char_octet_length", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 62, "name": "result_cast_char_set_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 63, "name": "result_cast_char_set_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 64, "name": "result_cast_char_set_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 65, "name": "result_cast_collation_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 66, "name": "result_cast_collation_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 67, "name": "result_cast_collation_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 68, "name": "result_cast_numeric_precision", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 69, "name": "result_cast_numeric_precision_radix", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 70, "name": "result_cast_numeric_scale", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 71, "name": "result_cast_datetime_precision", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 72, "name": "result_cast_interval_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 73, "name": "result_cast_interval_precision", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 74, "name": "result_cast_type_udt_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 75, "name": "result_cast_type_udt_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 76, "name": "result_cast_type_udt_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 77, "name": "result_cast_scope_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 78, "name": "result_cast_scope_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 79, "name": "result_cast_scope_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 80, "name": "result_cast_maximum_cardinality", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 81, "name": "result_cast_dtd_identifier", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967149, "name": "routines", "nextColumnId": 82, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967200, "version": "1"}} +4294967150 {"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": 4294967150, "name": "routine_privileges", "nextColumnId": 11, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967200, "version": "1"}} +4294967151 {"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": 4294967151, "name": "role_usage_grants", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967200, "version": "1"}} +4294967152 {"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": 4294967152, "name": "role_udt_grants", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967200, "version": "1"}} +4294967153 {"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": 4294967153, "name": "role_table_grants", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967200, "version": "1"}} +4294967154 {"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": 4294967154, "name": "role_routine_grants", "nextColumnId": 11, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967200, "version": "1"}} +4294967155 {"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": 4294967155, "name": "role_column_grants", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967200, "version": "1"}} +4294967156 {"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": 4294967156, "name": "resource_groups", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967200, "version": "1"}} +4294967157 {"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": 4294967157, "name": "referential_constraints", "nextColumnId": 12, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967200, "version": "1"}} +4294967158 {"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": 4294967158, "name": "profiling", "nextColumnId": 19, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967200, "version": "1"}} +4294967159 {"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": 4294967159, "name": "processlist", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967200, "version": "1"}} +4294967160 {"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": 4294967160, "name": "plugins", "nextColumnId": 12, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967200, "version": "1"}} +4294967161 {"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": 4294967161, "name": "partitions", "nextColumnId": 26, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967200, "version": "1"}} +4294967162 {"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": 4294967162, "name": "parameters", "nextColumnId": 33, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967200, "version": "1"}} +4294967163 {"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": 4294967163, "name": "optimizer_trace", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967200, "version": "1"}} +4294967164 {"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": 4294967164, "name": "keywords", "nextColumnId": 3, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967200, "version": "1"}} +4294967165 {"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": 4294967165, "name": "key_column_usage", "nextColumnId": 10, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967200, "version": "1"}} +4294967166 {"table": {"columns": [{"id": 1, "name": "catalog_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967166, "name": "information_schema_catalog_name", "nextColumnId": 2, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967200, "version": "1"}} +4294967167 {"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": 4294967167, "name": "foreign_tables", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967200, "version": "1"}} +4294967168 {"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": 4294967168, "name": "foreign_table_options", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967200, "version": "1"}} +4294967169 {"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": 4294967169, "name": "foreign_servers", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967200, "version": "1"}} +4294967170 {"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": 4294967170, "name": "foreign_server_options", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967200, "version": "1"}} +4294967171 {"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": 4294967171, "name": "foreign_data_wrappers", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967200, "version": "1"}} +4294967172 {"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": 4294967172, "name": "foreign_data_wrapper_options", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967200, "version": "1"}} +4294967173 {"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": 4294967173, "name": "files", "nextColumnId": 39, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967200, "version": "1"}} +4294967174 {"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": 4294967174, "name": "events", "nextColumnId": 25, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967200, "version": "1"}} +4294967175 {"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": 4294967175, "name": "engines", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967200, "version": "1"}} +4294967176 {"table": {"columns": [{"id": 1, "name": "role_name", "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967176, "name": "enabled_roles", "nextColumnId": 2, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967200, "version": "1"}} +4294967177 {"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": 4294967177, "name": "element_types", "nextColumnId": 30, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967200, "version": "1"}} +4294967178 {"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": 4294967178, "name": "domains", "nextColumnId": 28, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967200, "version": "1"}} +4294967179 {"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": 4294967179, "name": "domain_udt_usage", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967200, "version": "1"}} +4294967180 {"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": 4294967180, "name": "domain_constraints", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967200, "version": "1"}} +4294967181 {"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": 4294967181, "name": "data_type_privileges", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967200, "version": "1"}} +4294967182 {"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": 4294967182, "name": "constraint_table_usage", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967200, "version": "1"}} +4294967183 {"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": 4294967183, "name": "constraint_column_usage", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967200, "version": "1"}} +4294967184 {"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": 4294967184, "name": "columns", "nextColumnId": 48, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967200, "version": "1"}} +4294967185 {"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": 4294967185, "name": "columns_extensions", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967200, "version": "1"}} +4294967186 {"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": 4294967186, "name": "column_udt_usage", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967200, "version": "1"}} +4294967187 {"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": 4294967187, "name": "column_statistics", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967200, "version": "1"}} +4294967188 {"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": 4294967188, "name": "column_privileges", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967200, "version": "1"}} +4294967189 {"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": 4294967189, "name": "column_options", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967200, "version": "1"}} +4294967190 {"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": 4294967190, "name": "column_domain_usage", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967200, "version": "1"}} +4294967191 {"table": {"columns": [{"id": 1, "name": "table_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "table_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "table_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "column_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "dependent_column", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967191, "name": "column_column_usage", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967200, "version": "1"}} +4294967192 {"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": 4294967192, "name": "collations", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967200, "version": "1"}} +4294967193 {"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": 4294967193, "name": "collation_character_set_applicability", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967200, "version": "1"}} +4294967194 {"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": 4294967194, "name": "check_constraints", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967200, "version": "1"}} +4294967195 {"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": 4294967195, "name": "check_constraint_routine_usage", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967200, "version": "1"}} +4294967196 {"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": 4294967196, "name": "character_sets", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967200, "version": "1"}} +4294967197 {"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": 4294967197, "name": "attributes", "nextColumnId": 32, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967200, "version": "1"}} +4294967198 {"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": 4294967198, "name": "applicable_roles", "nextColumnId": 4, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967200, "version": "1"}} +4294967199 {"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": 4294967199, "name": "administrable_role_authorizations", "nextColumnId": 4, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967200, "version": "1"}} +4294967200 {"schema": {"defaultPrivileges": {"type": "SCHEMA"}, "id": 4294967200, "name": "information_schema", "privileges": {"ownerProto": "node", "users": [{"privileges": "512", "userProto": "public"}], "version": 2}, "version": "1"}} +4294967201 {"table": {"columns": [{"id": 1, "name": "range_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "tenant_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "store_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 4, "name": "priority", "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "log_term", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "log_index", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "tokens", "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967201, "indexes": [{"foreignKey": {}, "geoConfig": {}, "id": 2, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [1], "keyColumnNames": ["range_id"], "name": "kv_flow_token_deductions_range_id_idx", "partitioning": {}, "sharded": {}, "storeColumnIds": [2, 3, 4, 5, 6, 7], "storeColumnNames": ["tenant_id", "store_id", "priority", "log_term", "log_index", "tokens"], "version": 3}], "name": "kv_flow_token_deductions", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 3, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} +4294967202 {"table": {"columns": [{"id": 1, "name": "range_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "tenant_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "store_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 4, "name": "total_tracked_tokens", "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967202, "indexes": [{"foreignKey": {}, "geoConfig": {}, "id": 2, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [1], "keyColumnNames": ["range_id"], "name": "kv_flow_control_handles_range_id_idx", "partitioning": {}, "sharded": {}, "storeColumnIds": [2, 3, 4], "storeColumnNames": ["tenant_id", "store_id", "total_tracked_tokens"], "version": 3}], "name": "kv_flow_control_handles", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 3, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} +4294967203 {"table": {"columns": [{"id": 1, "name": "tenant_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "store_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "available_regular_tokens", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 4, "name": "available_elastic_tokens", "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967203, "name": "kv_flow_controller", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} 4294967204 {"table": {"columns": [{"id": 1, "name": "username", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "path", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "privileges", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 4, "name": "grant_options", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 5, "name": "user_id", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}], "formatVersion": 3, "id": 4294967204, "name": "kv_system_privileges", "nextColumnId": 6, "nextConstraintId": 1, "nextMutationId": 1, "primaryIndex": {"foreignKey": {}, "geoConfig": {}, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1", "viewQuery": "SELECT * FROM system.privileges"}} 4294967205 {"table": {"columns": [{"id": 1, "name": "role", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "inheriting_member", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "member_is_explicit", "nullable": true, "type": {"oid": 16}}, {"id": 4, "name": "member_is_admin", "nullable": true, "type": {"oid": 16}}], "formatVersion": 3, "id": 4294967205, "name": "kv_inherited_role_members", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} 4294967206 {"table": {"columns": [{"id": 1, "name": "tenant_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "capability_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "capability_value", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967206, "name": "node_tenant_capabilities_cache", "nextColumnId": 4, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} @@ -587,224 +590,227 @@ WHERE type != 'FunctionCommentType'; -- exclude builtin comments since there are ---- DatabaseCommentType 104 0 "this is the test database" TableCommentType 111 0 "this is a table" -TableCommentType 4294966983 0 "Shows all defined Spatial Reference Identifiers (SRIDs). Matches PostGIS' spatial_ref_sys table." -TableCommentType 4294966984 0 "Shows all defined geometry columns. Matches PostGIS' geometry_columns functionality." -TableCommentType 4294966985 0 "Shows all defined geography columns. Matches PostGIS' geography_columns functionality." -TableCommentType 4294966987 0 "view definitions (incomplete - see also information_schema.views)\nhttps://www.postgresql.org/docs/9.5/view-pg-views.html" -TableCommentType 4294966988 0 "database users\nhttps://www.postgresql.org/docs/9.5/view-pg-user.html" -TableCommentType 4294966989 0 "pg_user_mappings was created for compatibility and is currently unimplemented" -TableCommentType 4294966990 0 "local to remote user mapping (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-user-mapping.html" -TableCommentType 4294966991 0 "scalar types (incomplete)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-type.html" -TableCommentType 4294966992 0 "pg_ts_template was created for compatibility and is currently unimplemented" -TableCommentType 4294966993 0 "pg_ts_parser was created for compatibility and is currently unimplemented" -TableCommentType 4294966994 0 "pg_ts_dict was created for compatibility and is currently unimplemented" -TableCommentType 4294966995 0 "pg_ts_config was created for compatibility and is currently unimplemented" -TableCommentType 4294966996 0 "pg_ts_config_map was created for compatibility and is currently unimplemented" -TableCommentType 4294966997 0 "triggers (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-trigger.html" -TableCommentType 4294966998 0 "pg_transform was created for compatibility and is currently unimplemented" -TableCommentType 4294966999 0 "pg_timezone_names lists all the timezones that are supported by SET timezone" -TableCommentType 4294967000 0 "pg_timezone_abbrevs was created for compatibility and is currently unimplemented" -TableCommentType 4294967001 0 "available tablespaces (incomplete; concept inapplicable to CockroachDB)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-tablespace.html" -TableCommentType 4294967002 0 "tables summary (see also information_schema.tables, pg_catalog.pg_class)\nhttps://www.postgresql.org/docs/9.5/view-pg-tables.html" -TableCommentType 4294967003 0 "pg_subscription was created for compatibility and is currently unimplemented" -TableCommentType 4294967004 0 "pg_subscription_rel was created for compatibility and is currently unimplemented" -TableCommentType 4294967005 0 "pg_stats was created for compatibility and is currently unimplemented" -TableCommentType 4294967006 0 "pg_stats_ext was created for compatibility and is currently unimplemented" -TableCommentType 4294967007 0 "pg_statistic was created for compatibility and is currently unimplemented" -TableCommentType 4294967008 0 "pg_statistic_ext has the statistics objects created with CREATE STATISTICS\nhttps://www.postgresql.org/docs/13/catalog-pg-statistic-ext.html" -TableCommentType 4294967009 0 "pg_statistic_ext_data was created for compatibility and is currently unimplemented" -TableCommentType 4294967010 0 "pg_statio_user_tables was created for compatibility and is currently unimplemented" -TableCommentType 4294967011 0 "pg_statio_user_sequences was created for compatibility and is currently unimplemented" -TableCommentType 4294967012 0 "pg_statio_user_indexes was created for compatibility and is currently unimplemented" -TableCommentType 4294967013 0 "pg_statio_sys_tables was created for compatibility and is currently unimplemented" -TableCommentType 4294967014 0 "pg_statio_sys_sequences was created for compatibility and is currently unimplemented" -TableCommentType 4294967015 0 "pg_statio_sys_indexes was created for compatibility and is currently unimplemented" -TableCommentType 4294967016 0 "pg_statio_all_tables was created for compatibility and is currently unimplemented" -TableCommentType 4294967017 0 "pg_statio_all_sequences was created for compatibility and is currently unimplemented" -TableCommentType 4294967018 0 "pg_statio_all_indexes was created for compatibility and is currently unimplemented" -TableCommentType 4294967019 0 "pg_stat_xact_user_tables was created for compatibility and is currently unimplemented" -TableCommentType 4294967020 0 "pg_stat_xact_user_functions was created for compatibility and is currently unimplemented" -TableCommentType 4294967021 0 "pg_stat_xact_sys_tables was created for compatibility and is currently unimplemented" -TableCommentType 4294967022 0 "pg_stat_xact_all_tables was created for compatibility and is currently unimplemented" -TableCommentType 4294967023 0 "pg_stat_wal_receiver was created for compatibility and is currently unimplemented" -TableCommentType 4294967024 0 "pg_stat_user_tables was created for compatibility and is currently unimplemented" -TableCommentType 4294967025 0 "pg_stat_user_indexes was created for compatibility and is currently unimplemented" -TableCommentType 4294967026 0 "pg_stat_user_functions was created for compatibility and is currently unimplemented" -TableCommentType 4294967027 0 "pg_stat_sys_tables was created for compatibility and is currently unimplemented" -TableCommentType 4294967028 0 "pg_stat_sys_indexes was created for compatibility and is currently unimplemented" -TableCommentType 4294967029 0 "pg_stat_subscription was created for compatibility and is currently unimplemented" -TableCommentType 4294967030 0 "pg_stat_ssl was created for compatibility and is currently unimplemented" -TableCommentType 4294967031 0 "pg_stat_slru was created for compatibility and is currently unimplemented" -TableCommentType 4294967032 0 "pg_stat_replication was created for compatibility and is currently unimplemented" -TableCommentType 4294967033 0 "pg_stat_progress_vacuum was created for compatibility and is currently unimplemented" -TableCommentType 4294967034 0 "pg_stat_progress_create_index was created for compatibility and is currently unimplemented" -TableCommentType 4294967035 0 "pg_stat_progress_cluster was created for compatibility and is currently unimplemented" -TableCommentType 4294967036 0 "pg_stat_progress_basebackup was created for compatibility and is currently unimplemented" -TableCommentType 4294967037 0 "pg_stat_progress_analyze was created for compatibility and is currently unimplemented" -TableCommentType 4294967038 0 "pg_stat_gssapi was created for compatibility and is currently unimplemented" -TableCommentType 4294967039 0 "pg_stat_database was created for compatibility and is currently unimplemented" -TableCommentType 4294967040 0 "pg_stat_database_conflicts was created for compatibility and is currently unimplemented" -TableCommentType 4294967041 0 "pg_stat_bgwriter was created for compatibility and is currently unimplemented" -TableCommentType 4294967042 0 "pg_stat_archiver was created for compatibility and is currently unimplemented" -TableCommentType 4294967043 0 "pg_stat_all_tables was created for compatibility and is currently unimplemented" -TableCommentType 4294967044 0 "pg_stat_all_indexes was created for compatibility and is currently unimplemented" -TableCommentType 4294967045 0 "backend access statistics (empty - monitoring works differently in CockroachDB)\nhttps://www.postgresql.org/docs/9.6/monitoring-stats.html#PG-STAT-ACTIVITY-VIEW" -TableCommentType 4294967046 0 "pg_shmem_allocations was created for compatibility and is currently unimplemented" -TableCommentType 4294967047 0 "Shared Dependencies (Roles depending on objects). \nhttps://www.postgresql.org/docs/9.6/catalog-pg-shdepend.html" -TableCommentType 4294967048 0 "shared security labels (empty - feature not supported)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-shseclabel.html" -TableCommentType 4294967049 0 "shared object comments\nhttps://www.postgresql.org/docs/9.5/catalog-pg-shdescription.html" -TableCommentType 4294967050 0 "pg_shadow lists properties for roles that are marked as rolcanlogin in pg_authid\nhttps://www.postgresql.org/docs/13/view-pg-shadow.html" -TableCommentType 4294967051 0 "session variables (incomplete)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-settings.html" -TableCommentType 4294967052 0 "pg_sequences is very similar as pg_sequence.\nhttps://www.postgresql.org/docs/13/view-pg-sequences.html" -TableCommentType 4294967053 0 "sequences (see also information_schema.sequences)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-sequence.html" -TableCommentType 4294967054 0 "security labels (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-seclabel.html" -TableCommentType 4294967055 0 "security labels (empty)\nhttps://www.postgresql.org/docs/9.6/view-pg-seclabels.html" -TableCommentType 4294967056 0 "pg_rules was created for compatibility and is currently unimplemented" -TableCommentType 4294967057 0 "database roles\nhttps://www.postgresql.org/docs/9.5/view-pg-roles.html" -TableCommentType 4294967058 0 "rewrite rules (only for referencing on pg_depend for table-view dependencies)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-rewrite.html" -TableCommentType 4294967059 0 "pg_replication_slots was created for compatibility and is currently unimplemented" -TableCommentType 4294967060 0 "pg_replication_origin was created for compatibility and is currently unimplemented" -TableCommentType 4294967061 0 "pg_replication_origin_status was created for compatibility and is currently unimplemented" -TableCommentType 4294967062 0 "range types (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-range.html" -TableCommentType 4294967063 0 "pg_publication_tables was created for compatibility and is currently unimplemented" -TableCommentType 4294967064 0 "pg_publication was created for compatibility and is currently unimplemented" -TableCommentType 4294967065 0 "pg_publication_rel was created for compatibility and is currently unimplemented" -TableCommentType 4294967066 0 "built-in functions (incomplete)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-proc.html" -TableCommentType 4294967067 0 "prepared transactions (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.6/view-pg-prepared-xacts.html" -TableCommentType 4294967068 0 "prepared statements\nhttps://www.postgresql.org/docs/9.6/view-pg-prepared-statements.html" -TableCommentType 4294967069 0 "pg_policy was created for compatibility and is currently unimplemented" -TableCommentType 4294967070 0 "pg_policies was created for compatibility and is currently unimplemented" -TableCommentType 4294967071 0 "pg_partitioned_table was created for compatibility and is currently unimplemented" -TableCommentType 4294967072 0 "pg_opfamily was created for compatibility and is currently unimplemented" -TableCommentType 4294967073 0 "operators (incomplete)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-operator.html" -TableCommentType 4294967074 0 "opclass (empty - Operator classes not supported yet)\nhttps://www.postgresql.org/docs/12/catalog-pg-opclass.html" -TableCommentType 4294967075 0 "available namespaces\nhttps://www.postgresql.org/docs/9.5/catalog-pg-namespace.html" -TableCommentType 4294967076 0 "available materialized views (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.6/view-pg-matviews.html" -TableCommentType 4294967077 0 "locks held by active processes (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.6/view-pg-locks.html" -TableCommentType 4294967078 0 "pg_largeobject was created for compatibility and is currently unimplemented" -TableCommentType 4294967079 0 "pg_largeobject_metadata was created for compatibility and is currently unimplemented" -TableCommentType 4294967080 0 "available languages (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-language.html" -TableCommentType 4294967081 0 "pg_init_privs was created for compatibility and is currently unimplemented" -TableCommentType 4294967082 0 "table inheritance hierarchy (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-inherits.html" -TableCommentType 4294967083 0 "index creation statements\nhttps://www.postgresql.org/docs/9.5/view-pg-indexes.html" -TableCommentType 4294967084 0 "indexes (incomplete)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-index.html" -TableCommentType 4294967085 0 "pg_hba_file_rules was created for compatibility and is currently unimplemented" -TableCommentType 4294967086 0 "pg_group was created for compatibility and is currently unimplemented" -TableCommentType 4294967087 0 "foreign tables (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-foreign-table.html" -TableCommentType 4294967088 0 "foreign servers (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-foreign-server.html" -TableCommentType 4294967089 0 "foreign data wrappers (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-foreign-data-wrapper.html" -TableCommentType 4294967090 0 "pg_file_settings was created for compatibility and is currently unimplemented" -TableCommentType 4294967091 0 "installed extensions (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-extension.html" -TableCommentType 4294967092 0 "event triggers (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.6/catalog-pg-event-trigger.html" -TableCommentType 4294967093 0 "enum types and labels (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-enum.html" -TableCommentType 4294967094 0 "object comments\nhttps://www.postgresql.org/docs/9.5/catalog-pg-description.html" -TableCommentType 4294967095 0 "dependency relationships (incomplete)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-depend.html" -TableCommentType 4294967096 0 "default ACLs; these are the privileges that will be assigned to newly created objects\nhttps://www.postgresql.org/docs/13/catalog-pg-default-acl.html" -TableCommentType 4294967097 0 "contains the default values that have been configured for session variables\nhttps://www.postgresql.org/docs/13/catalog-pg-db-role-setting.html" -TableCommentType 4294967098 0 "available databases (incomplete)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-database.html" -TableCommentType 4294967099 0 "contains currently active SQL cursors created with DECLARE\nhttps://www.postgresql.org/docs/14/view-pg-cursors.html" -TableCommentType 4294967100 0 "encoding conversions (empty - unimplemented)\nhttps://www.postgresql.org/docs/9.6/catalog-pg-conversion.html" -TableCommentType 4294967101 0 "table constraints (incomplete - see also information_schema.table_constraints)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-constraint.html" -TableCommentType 4294967102 0 "pg_config was created for compatibility and is currently unimplemented" -TableCommentType 4294967103 0 "available collations (incomplete)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-collation.html" -TableCommentType 4294967104 0 "tables and relation-like objects (incomplete - see also information_schema.tables/sequences/views)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-class.html" -TableCommentType 4294967105 0 "casts (empty - needs filling out)\nhttps://www.postgresql.org/docs/9.6/catalog-pg-cast.html" -TableCommentType 4294967106 0 "available extensions\nhttps://www.postgresql.org/docs/9.6/view-pg-available-extensions.html" -TableCommentType 4294967107 0 "pg_available_extension_versions was created for compatibility and is currently unimplemented" -TableCommentType 4294967108 0 "role membership\nhttps://www.postgresql.org/docs/9.5/catalog-pg-auth-members.html" -TableCommentType 4294967109 0 "authorization identifiers - differs from postgres as we do not display passwords, \nand thus do not require admin privileges for access. \nhttps://www.postgresql.org/docs/9.5/catalog-pg-authid.html" -TableCommentType 4294967110 0 "table columns (incomplete - see also information_schema.columns)\nhttps://www.postgresql.org/docs/12/catalog-pg-attribute.html" -TableCommentType 4294967111 0 "column default values\nhttps://www.postgresql.org/docs/9.5/catalog-pg-attrdef.html" -TableCommentType 4294967112 0 "pg_amproc was created for compatibility and is currently unimplemented" -TableCommentType 4294967113 0 "pg_amop was created for compatibility and is currently unimplemented" -TableCommentType 4294967114 0 "index access methods (incomplete)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-am.html" -TableCommentType 4294967115 0 "aggregated built-in functions (incomplete)\nhttps://www.postgresql.org/docs/9.6/catalog-pg-aggregate.html" -TableCommentType 4294967117 0 "views (incomplete)\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#views\nhttps://www.postgresql.org/docs/9.5/infoschema-views.html" -TableCommentType 4294967118 0 "view_table_usage was created for compatibility and is currently unimplemented" -TableCommentType 4294967119 0 "view_routine_usage was created for compatibility and is currently unimplemented" -TableCommentType 4294967120 0 "view_column_usage was created for compatibility and is currently unimplemented" -TableCommentType 4294967121 0 "grantable privileges (incomplete)" -TableCommentType 4294967122 0 "user_mappings was created for compatibility and is currently unimplemented" -TableCommentType 4294967123 0 "user_mapping_options was created for compatibility and is currently unimplemented" -TableCommentType 4294967124 0 "user_defined_types was created for compatibility and is currently unimplemented" -TableCommentType 4294967125 0 "user_attributes was created for compatibility and is currently unimplemented" -TableCommentType 4294967126 0 "usage_privileges was created for compatibility and is currently unimplemented" -TableCommentType 4294967127 0 "udt_privileges was created for compatibility and is currently unimplemented" -TableCommentType 4294967128 0 "type privileges (incomplete; may contain excess users or roles)\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#type_privileges" -TableCommentType 4294967129 0 "triggers was created for compatibility and is currently unimplemented" -TableCommentType 4294967130 0 "triggered_update_columns was created for compatibility and is currently unimplemented" -TableCommentType 4294967131 0 "transforms was created for compatibility and is currently unimplemented" -TableCommentType 4294967132 0 "tablespaces was created for compatibility and is currently unimplemented" -TableCommentType 4294967133 0 "tablespaces_extensions was created for compatibility and is currently unimplemented" -TableCommentType 4294967134 0 "tables and views\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#tables\nhttps://www.postgresql.org/docs/9.5/infoschema-tables.html" -TableCommentType 4294967135 0 "tables_extensions was created for compatibility and is currently unimplemented" -TableCommentType 4294967136 0 "privileges granted on table or views (incomplete; may contain excess users or roles)\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#table_privileges\nhttps://www.postgresql.org/docs/9.5/infoschema-table-privileges.html" -TableCommentType 4294967137 0 "table_constraints_extensions was created for compatibility and is currently unimplemented" -TableCommentType 4294967138 0 "table constraints\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#table_constraints\nhttps://www.postgresql.org/docs/9.5/infoschema-table-constraints.html" -TableCommentType 4294967139 0 "index metadata and statistics (incomplete)\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#statistics" -TableCommentType 4294967140 0 "st_units_of_measure was created for compatibility and is currently unimplemented" -TableCommentType 4294967141 0 "st_spatial_reference_systems was created for compatibility and is currently unimplemented" -TableCommentType 4294967142 0 "st_geometry_columns was created for compatibility and is currently unimplemented" -TableCommentType 4294967143 0 "exposes the session variables." -TableCommentType 4294967144 0 "sequences\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#sequences\nhttps://www.postgresql.org/docs/9.5/infoschema-sequences.html" -TableCommentType 4294967145 0 "schema privileges (incomplete; may contain excess users or roles)\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#schema_privileges" -TableCommentType 4294967146 0 "database schemas (may contain schemata without permission)\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#schemata\nhttps://www.postgresql.org/docs/9.5/infoschema-schemata.html" -TableCommentType 4294967147 0 "schemata_extensions was created for compatibility and is currently unimplemented" -TableCommentType 4294967148 0 "sql_sizing was created for compatibility and is currently unimplemented" -TableCommentType 4294967149 0 "sql_parts was created for compatibility and is currently unimplemented" -TableCommentType 4294967150 0 "sql_implementation_info was created for compatibility and is currently unimplemented" -TableCommentType 4294967151 0 "sql_features was created for compatibility and is currently unimplemented" -TableCommentType 4294967152 0 "built-in functions (empty - introspection not yet supported)\nhttps://www.postgresql.org/docs/9.5/infoschema-routines.html" -TableCommentType 4294967153 0 "routine_privileges was created for compatibility and is currently unimplemented" -TableCommentType 4294967154 0 "role_usage_grants was created for compatibility and is currently unimplemented" -TableCommentType 4294967155 0 "role_udt_grants was created for compatibility and is currently unimplemented" -TableCommentType 4294967156 0 "privileges granted on table or views (incomplete; see also information_schema.table_privileges; may contain excess users or roles)\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#role_table_grants\nhttps://www.postgresql.org/docs/9.5/infoschema-role-table-grants.html" -TableCommentType 4294967157 0 "privileges granted on functions (incomplete; only contains privileges of user-defined functions)" -TableCommentType 4294967158 0 "role_column_grants was created for compatibility and is currently unimplemented" -TableCommentType 4294967159 0 "resource_groups was created for compatibility and is currently unimplemented" -TableCommentType 4294967160 0 "foreign key constraints\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#referential_constraints\nhttps://www.postgresql.org/docs/9.5/infoschema-referential-constraints.html" -TableCommentType 4294967161 0 "profiling was created for compatibility and is currently unimplemented" -TableCommentType 4294967162 0 "processlist was created for compatibility and is currently unimplemented" -TableCommentType 4294967163 0 "plugins was created for compatibility and is currently unimplemented" -TableCommentType 4294967164 0 "partitions was created for compatibility and is currently unimplemented" -TableCommentType 4294967165 0 "built-in function parameters (empty - introspection not yet supported)\nhttps://www.postgresql.org/docs/9.5/infoschema-parameters.html" -TableCommentType 4294967166 0 "optimizer_trace was created for compatibility and is currently unimplemented" -TableCommentType 4294967167 0 "keywords was created for compatibility and is currently unimplemented" -TableCommentType 4294967168 0 "column usage by indexes and key constraints\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#key_column_usage\nhttps://www.postgresql.org/docs/9.5/infoschema-key-column-usage.html" -TableCommentType 4294967169 0 "information_schema_catalog_name was created for compatibility and is currently unimplemented" -TableCommentType 4294967170 0 "foreign_tables was created for compatibility and is currently unimplemented" -TableCommentType 4294967171 0 "foreign_table_options was created for compatibility and is currently unimplemented" -TableCommentType 4294967172 0 "foreign_servers was created for compatibility and is currently unimplemented" -TableCommentType 4294967173 0 "foreign_server_options was created for compatibility and is currently unimplemented" -TableCommentType 4294967174 0 "foreign_data_wrappers was created for compatibility and is currently unimplemented" -TableCommentType 4294967175 0 "foreign_data_wrapper_options was created for compatibility and is currently unimplemented" -TableCommentType 4294967176 0 "files was created for compatibility and is currently unimplemented" -TableCommentType 4294967177 0 "events was created for compatibility and is currently unimplemented" -TableCommentType 4294967178 0 "engines was created for compatibility and is currently unimplemented" -TableCommentType 4294967179 0 "roles for the current user\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#enabled_roles\nhttps://www.postgresql.org/docs/9.5/infoschema-enabled-roles.html" -TableCommentType 4294967180 0 "element_types was created for compatibility and is currently unimplemented" -TableCommentType 4294967181 0 "domains was created for compatibility and is currently unimplemented" -TableCommentType 4294967182 0 "domain_udt_usage was created for compatibility and is currently unimplemented" -TableCommentType 4294967183 0 "domain_constraints was created for compatibility and is currently unimplemented" -TableCommentType 4294967184 0 "data_type_privileges was created for compatibility and is currently unimplemented" -TableCommentType 4294967185 0 "constraint_table_usage was created for compatibility and is currently unimplemented" -TableCommentType 4294967186 0 "columns usage by constraints\nhttps://www.postgresql.org/docs/9.5/infoschema-constraint-column-usage.html" -TableCommentType 4294967187 0 "table and view columns (incomplete)\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#columns\nhttps://www.postgresql.org/docs/9.5/infoschema-columns.html" -TableCommentType 4294967188 0 "columns_extensions was created for compatibility and is currently unimplemented" -TableCommentType 4294967189 0 "columns with user defined types\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#column_udt_usage\nhttps://www.postgresql.org/docs/current/infoschema-column-udt-usage.html" -TableCommentType 4294967190 0 "column_statistics was created for compatibility and is currently unimplemented" -TableCommentType 4294967191 0 "column privilege grants (incomplete)\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#column_privileges\nhttps://www.postgresql.org/docs/9.5/infoschema-column-privileges.html" -TableCommentType 4294967192 0 "column_options was created for compatibility and is currently unimplemented" -TableCommentType 4294967193 0 "column_domain_usage was created for compatibility and is currently unimplemented" -TableCommentType 4294967194 0 "column_column_usage was created for compatibility and is currently unimplemented" -TableCommentType 4294967195 0 "shows the collations available in the current database\nhttps://www.postgresql.org/docs/current/infoschema-collations.html" -TableCommentType 4294967196 0 "identifies which character set the available collations are \napplicable to. As UTF-8 is the only available encoding this table does not\nprovide much useful information.\nhttps://www.postgresql.org/docs/current/infoschema-collation-character-set-applicab.html" -TableCommentType 4294967197 0 "check constraints\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#check_constraints\nhttps://www.postgresql.org/docs/9.5/infoschema-check-constraints.html" -TableCommentType 4294967198 0 "check_constraint_routine_usage was created for compatibility and is currently unimplemented" -TableCommentType 4294967199 0 "character sets available in the current database\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#character_sets\nhttps://www.postgresql.org/docs/9.5/infoschema-character-sets.html" -TableCommentType 4294967200 0 "attributes was created for compatibility and is currently unimplemented" -TableCommentType 4294967201 0 "roles available to the current user\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#applicable_roles\nhttps://www.postgresql.org/docs/9.5/infoschema-applicable-roles.html" -TableCommentType 4294967202 0 "roles for which the current user has admin option\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#administrable_role_authorizations\nhttps://www.postgresql.org/docs/9.5/infoschema-administrable-role-authorizations.html" +TableCommentType 4294966980 0 "Shows all defined Spatial Reference Identifiers (SRIDs). Matches PostGIS' spatial_ref_sys table." +TableCommentType 4294966981 0 "Shows all defined geometry columns. Matches PostGIS' geometry_columns functionality." +TableCommentType 4294966982 0 "Shows all defined geography columns. Matches PostGIS' geography_columns functionality." +TableCommentType 4294966984 0 "view definitions (incomplete - see also information_schema.views)\nhttps://www.postgresql.org/docs/9.5/view-pg-views.html" +TableCommentType 4294966985 0 "database users\nhttps://www.postgresql.org/docs/9.5/view-pg-user.html" +TableCommentType 4294966986 0 "pg_user_mappings was created for compatibility and is currently unimplemented" +TableCommentType 4294966987 0 "local to remote user mapping (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-user-mapping.html" +TableCommentType 4294966988 0 "scalar types (incomplete)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-type.html" +TableCommentType 4294966989 0 "pg_ts_template was created for compatibility and is currently unimplemented" +TableCommentType 4294966990 0 "pg_ts_parser was created for compatibility and is currently unimplemented" +TableCommentType 4294966991 0 "pg_ts_dict was created for compatibility and is currently unimplemented" +TableCommentType 4294966992 0 "pg_ts_config was created for compatibility and is currently unimplemented" +TableCommentType 4294966993 0 "pg_ts_config_map was created for compatibility and is currently unimplemented" +TableCommentType 4294966994 0 "triggers (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-trigger.html" +TableCommentType 4294966995 0 "pg_transform was created for compatibility and is currently unimplemented" +TableCommentType 4294966996 0 "pg_timezone_names lists all the timezones that are supported by SET timezone" +TableCommentType 4294966997 0 "pg_timezone_abbrevs was created for compatibility and is currently unimplemented" +TableCommentType 4294966998 0 "available tablespaces (incomplete; concept inapplicable to CockroachDB)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-tablespace.html" +TableCommentType 4294966999 0 "tables summary (see also information_schema.tables, pg_catalog.pg_class)\nhttps://www.postgresql.org/docs/9.5/view-pg-tables.html" +TableCommentType 4294967000 0 "pg_subscription was created for compatibility and is currently unimplemented" +TableCommentType 4294967001 0 "pg_subscription_rel was created for compatibility and is currently unimplemented" +TableCommentType 4294967002 0 "pg_stats was created for compatibility and is currently unimplemented" +TableCommentType 4294967003 0 "pg_stats_ext was created for compatibility and is currently unimplemented" +TableCommentType 4294967004 0 "pg_statistic was created for compatibility and is currently unimplemented" +TableCommentType 4294967005 0 "pg_statistic_ext has the statistics objects created with CREATE STATISTICS\nhttps://www.postgresql.org/docs/13/catalog-pg-statistic-ext.html" +TableCommentType 4294967006 0 "pg_statistic_ext_data was created for compatibility and is currently unimplemented" +TableCommentType 4294967007 0 "pg_statio_user_tables was created for compatibility and is currently unimplemented" +TableCommentType 4294967008 0 "pg_statio_user_sequences was created for compatibility and is currently unimplemented" +TableCommentType 4294967009 0 "pg_statio_user_indexes was created for compatibility and is currently unimplemented" +TableCommentType 4294967010 0 "pg_statio_sys_tables was created for compatibility and is currently unimplemented" +TableCommentType 4294967011 0 "pg_statio_sys_sequences was created for compatibility and is currently unimplemented" +TableCommentType 4294967012 0 "pg_statio_sys_indexes was created for compatibility and is currently unimplemented" +TableCommentType 4294967013 0 "pg_statio_all_tables was created for compatibility and is currently unimplemented" +TableCommentType 4294967014 0 "pg_statio_all_sequences was created for compatibility and is currently unimplemented" +TableCommentType 4294967015 0 "pg_statio_all_indexes was created for compatibility and is currently unimplemented" +TableCommentType 4294967016 0 "pg_stat_xact_user_tables was created for compatibility and is currently unimplemented" +TableCommentType 4294967017 0 "pg_stat_xact_user_functions was created for compatibility and is currently unimplemented" +TableCommentType 4294967018 0 "pg_stat_xact_sys_tables was created for compatibility and is currently unimplemented" +TableCommentType 4294967019 0 "pg_stat_xact_all_tables was created for compatibility and is currently unimplemented" +TableCommentType 4294967020 0 "pg_stat_wal_receiver was created for compatibility and is currently unimplemented" +TableCommentType 4294967021 0 "pg_stat_user_tables was created for compatibility and is currently unimplemented" +TableCommentType 4294967022 0 "pg_stat_user_indexes was created for compatibility and is currently unimplemented" +TableCommentType 4294967023 0 "pg_stat_user_functions was created for compatibility and is currently unimplemented" +TableCommentType 4294967024 0 "pg_stat_sys_tables was created for compatibility and is currently unimplemented" +TableCommentType 4294967025 0 "pg_stat_sys_indexes was created for compatibility and is currently unimplemented" +TableCommentType 4294967026 0 "pg_stat_subscription was created for compatibility and is currently unimplemented" +TableCommentType 4294967027 0 "pg_stat_ssl was created for compatibility and is currently unimplemented" +TableCommentType 4294967028 0 "pg_stat_slru was created for compatibility and is currently unimplemented" +TableCommentType 4294967029 0 "pg_stat_replication was created for compatibility and is currently unimplemented" +TableCommentType 4294967030 0 "pg_stat_progress_vacuum was created for compatibility and is currently unimplemented" +TableCommentType 4294967031 0 "pg_stat_progress_create_index was created for compatibility and is currently unimplemented" +TableCommentType 4294967032 0 "pg_stat_progress_cluster was created for compatibility and is currently unimplemented" +TableCommentType 4294967033 0 "pg_stat_progress_basebackup was created for compatibility and is currently unimplemented" +TableCommentType 4294967034 0 "pg_stat_progress_analyze was created for compatibility and is currently unimplemented" +TableCommentType 4294967035 0 "pg_stat_gssapi was created for compatibility and is currently unimplemented" +TableCommentType 4294967036 0 "pg_stat_database was created for compatibility and is currently unimplemented" +TableCommentType 4294967037 0 "pg_stat_database_conflicts was created for compatibility and is currently unimplemented" +TableCommentType 4294967038 0 "pg_stat_bgwriter was created for compatibility and is currently unimplemented" +TableCommentType 4294967039 0 "pg_stat_archiver was created for compatibility and is currently unimplemented" +TableCommentType 4294967040 0 "pg_stat_all_tables was created for compatibility and is currently unimplemented" +TableCommentType 4294967041 0 "pg_stat_all_indexes was created for compatibility and is currently unimplemented" +TableCommentType 4294967042 0 "backend access statistics (empty - monitoring works differently in CockroachDB)\nhttps://www.postgresql.org/docs/9.6/monitoring-stats.html#PG-STAT-ACTIVITY-VIEW" +TableCommentType 4294967043 0 "pg_shmem_allocations was created for compatibility and is currently unimplemented" +TableCommentType 4294967044 0 "Shared Dependencies (Roles depending on objects). \nhttps://www.postgresql.org/docs/9.6/catalog-pg-shdepend.html" +TableCommentType 4294967045 0 "shared security labels (empty - feature not supported)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-shseclabel.html" +TableCommentType 4294967046 0 "shared object comments\nhttps://www.postgresql.org/docs/9.5/catalog-pg-shdescription.html" +TableCommentType 4294967047 0 "pg_shadow lists properties for roles that are marked as rolcanlogin in pg_authid\nhttps://www.postgresql.org/docs/13/view-pg-shadow.html" +TableCommentType 4294967048 0 "session variables (incomplete)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-settings.html" +TableCommentType 4294967049 0 "pg_sequences is very similar as pg_sequence.\nhttps://www.postgresql.org/docs/13/view-pg-sequences.html" +TableCommentType 4294967050 0 "sequences (see also information_schema.sequences)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-sequence.html" +TableCommentType 4294967051 0 "security labels (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-seclabel.html" +TableCommentType 4294967052 0 "security labels (empty)\nhttps://www.postgresql.org/docs/9.6/view-pg-seclabels.html" +TableCommentType 4294967053 0 "pg_rules was created for compatibility and is currently unimplemented" +TableCommentType 4294967054 0 "database roles\nhttps://www.postgresql.org/docs/9.5/view-pg-roles.html" +TableCommentType 4294967055 0 "rewrite rules (only for referencing on pg_depend for table-view dependencies)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-rewrite.html" +TableCommentType 4294967056 0 "pg_replication_slots was created for compatibility and is currently unimplemented" +TableCommentType 4294967057 0 "pg_replication_origin was created for compatibility and is currently unimplemented" +TableCommentType 4294967058 0 "pg_replication_origin_status was created for compatibility and is currently unimplemented" +TableCommentType 4294967059 0 "range types (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-range.html" +TableCommentType 4294967060 0 "pg_publication_tables was created for compatibility and is currently unimplemented" +TableCommentType 4294967061 0 "pg_publication was created for compatibility and is currently unimplemented" +TableCommentType 4294967062 0 "pg_publication_rel was created for compatibility and is currently unimplemented" +TableCommentType 4294967063 0 "built-in functions (incomplete)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-proc.html" +TableCommentType 4294967064 0 "prepared transactions (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.6/view-pg-prepared-xacts.html" +TableCommentType 4294967065 0 "prepared statements\nhttps://www.postgresql.org/docs/9.6/view-pg-prepared-statements.html" +TableCommentType 4294967066 0 "pg_policy was created for compatibility and is currently unimplemented" +TableCommentType 4294967067 0 "pg_policies was created for compatibility and is currently unimplemented" +TableCommentType 4294967068 0 "pg_partitioned_table was created for compatibility and is currently unimplemented" +TableCommentType 4294967069 0 "pg_opfamily was created for compatibility and is currently unimplemented" +TableCommentType 4294967070 0 "operators (incomplete)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-operator.html" +TableCommentType 4294967071 0 "opclass (empty - Operator classes not supported yet)\nhttps://www.postgresql.org/docs/12/catalog-pg-opclass.html" +TableCommentType 4294967072 0 "available namespaces\nhttps://www.postgresql.org/docs/9.5/catalog-pg-namespace.html" +TableCommentType 4294967073 0 "available materialized views (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.6/view-pg-matviews.html" +TableCommentType 4294967074 0 "locks held by active processes (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.6/view-pg-locks.html" +TableCommentType 4294967075 0 "pg_largeobject was created for compatibility and is currently unimplemented" +TableCommentType 4294967076 0 "pg_largeobject_metadata was created for compatibility and is currently unimplemented" +TableCommentType 4294967077 0 "available languages (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-language.html" +TableCommentType 4294967078 0 "pg_init_privs was created for compatibility and is currently unimplemented" +TableCommentType 4294967079 0 "table inheritance hierarchy (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-inherits.html" +TableCommentType 4294967080 0 "index creation statements\nhttps://www.postgresql.org/docs/9.5/view-pg-indexes.html" +TableCommentType 4294967081 0 "indexes (incomplete)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-index.html" +TableCommentType 4294967082 0 "pg_hba_file_rules was created for compatibility and is currently unimplemented" +TableCommentType 4294967083 0 "pg_group was created for compatibility and is currently unimplemented" +TableCommentType 4294967084 0 "foreign tables (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-foreign-table.html" +TableCommentType 4294967085 0 "foreign servers (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-foreign-server.html" +TableCommentType 4294967086 0 "foreign data wrappers (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-foreign-data-wrapper.html" +TableCommentType 4294967087 0 "pg_file_settings was created for compatibility and is currently unimplemented" +TableCommentType 4294967088 0 "installed extensions (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-extension.html" +TableCommentType 4294967089 0 "event triggers (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.6/catalog-pg-event-trigger.html" +TableCommentType 4294967090 0 "enum types and labels (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-enum.html" +TableCommentType 4294967091 0 "object comments\nhttps://www.postgresql.org/docs/9.5/catalog-pg-description.html" +TableCommentType 4294967092 0 "dependency relationships (incomplete)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-depend.html" +TableCommentType 4294967093 0 "default ACLs; these are the privileges that will be assigned to newly created objects\nhttps://www.postgresql.org/docs/13/catalog-pg-default-acl.html" +TableCommentType 4294967094 0 "contains the default values that have been configured for session variables\nhttps://www.postgresql.org/docs/13/catalog-pg-db-role-setting.html" +TableCommentType 4294967095 0 "available databases (incomplete)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-database.html" +TableCommentType 4294967096 0 "contains currently active SQL cursors created with DECLARE\nhttps://www.postgresql.org/docs/14/view-pg-cursors.html" +TableCommentType 4294967097 0 "encoding conversions (empty - unimplemented)\nhttps://www.postgresql.org/docs/9.6/catalog-pg-conversion.html" +TableCommentType 4294967098 0 "table constraints (incomplete - see also information_schema.table_constraints)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-constraint.html" +TableCommentType 4294967099 0 "pg_config was created for compatibility and is currently unimplemented" +TableCommentType 4294967100 0 "available collations (incomplete)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-collation.html" +TableCommentType 4294967101 0 "tables and relation-like objects (incomplete - see also information_schema.tables/sequences/views)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-class.html" +TableCommentType 4294967102 0 "casts (empty - needs filling out)\nhttps://www.postgresql.org/docs/9.6/catalog-pg-cast.html" +TableCommentType 4294967103 0 "available extensions\nhttps://www.postgresql.org/docs/9.6/view-pg-available-extensions.html" +TableCommentType 4294967104 0 "pg_available_extension_versions was created for compatibility and is currently unimplemented" +TableCommentType 4294967105 0 "role membership\nhttps://www.postgresql.org/docs/9.5/catalog-pg-auth-members.html" +TableCommentType 4294967106 0 "authorization identifiers - differs from postgres as we do not display passwords, \nand thus do not require admin privileges for access. \nhttps://www.postgresql.org/docs/9.5/catalog-pg-authid.html" +TableCommentType 4294967107 0 "table columns (incomplete - see also information_schema.columns)\nhttps://www.postgresql.org/docs/12/catalog-pg-attribute.html" +TableCommentType 4294967108 0 "column default values\nhttps://www.postgresql.org/docs/9.5/catalog-pg-attrdef.html" +TableCommentType 4294967109 0 "pg_amproc was created for compatibility and is currently unimplemented" +TableCommentType 4294967110 0 "pg_amop was created for compatibility and is currently unimplemented" +TableCommentType 4294967111 0 "index access methods (incomplete)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-am.html" +TableCommentType 4294967112 0 "aggregated built-in functions (incomplete)\nhttps://www.postgresql.org/docs/9.6/catalog-pg-aggregate.html" +TableCommentType 4294967114 0 "views (incomplete)\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#views\nhttps://www.postgresql.org/docs/9.5/infoschema-views.html" +TableCommentType 4294967115 0 "view_table_usage was created for compatibility and is currently unimplemented" +TableCommentType 4294967116 0 "view_routine_usage was created for compatibility and is currently unimplemented" +TableCommentType 4294967117 0 "view_column_usage was created for compatibility and is currently unimplemented" +TableCommentType 4294967118 0 "grantable privileges (incomplete)" +TableCommentType 4294967119 0 "user_mappings was created for compatibility and is currently unimplemented" +TableCommentType 4294967120 0 "user_mapping_options was created for compatibility and is currently unimplemented" +TableCommentType 4294967121 0 "user_defined_types was created for compatibility and is currently unimplemented" +TableCommentType 4294967122 0 "user_attributes was created for compatibility and is currently unimplemented" +TableCommentType 4294967123 0 "usage_privileges was created for compatibility and is currently unimplemented" +TableCommentType 4294967124 0 "udt_privileges was created for compatibility and is currently unimplemented" +TableCommentType 4294967125 0 "type privileges (incomplete; may contain excess users or roles)\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#type_privileges" +TableCommentType 4294967126 0 "triggers was created for compatibility and is currently unimplemented" +TableCommentType 4294967127 0 "triggered_update_columns was created for compatibility and is currently unimplemented" +TableCommentType 4294967128 0 "transforms was created for compatibility and is currently unimplemented" +TableCommentType 4294967129 0 "tablespaces was created for compatibility and is currently unimplemented" +TableCommentType 4294967130 0 "tablespaces_extensions was created for compatibility and is currently unimplemented" +TableCommentType 4294967131 0 "tables and views\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#tables\nhttps://www.postgresql.org/docs/9.5/infoschema-tables.html" +TableCommentType 4294967132 0 "tables_extensions was created for compatibility and is currently unimplemented" +TableCommentType 4294967133 0 "privileges granted on table or views (incomplete; may contain excess users or roles)\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#table_privileges\nhttps://www.postgresql.org/docs/9.5/infoschema-table-privileges.html" +TableCommentType 4294967134 0 "table_constraints_extensions was created for compatibility and is currently unimplemented" +TableCommentType 4294967135 0 "table constraints\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#table_constraints\nhttps://www.postgresql.org/docs/9.5/infoschema-table-constraints.html" +TableCommentType 4294967136 0 "index metadata and statistics (incomplete)\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#statistics" +TableCommentType 4294967137 0 "st_units_of_measure was created for compatibility and is currently unimplemented" +TableCommentType 4294967138 0 "st_spatial_reference_systems was created for compatibility and is currently unimplemented" +TableCommentType 4294967139 0 "st_geometry_columns was created for compatibility and is currently unimplemented" +TableCommentType 4294967140 0 "exposes the session variables." +TableCommentType 4294967141 0 "sequences\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#sequences\nhttps://www.postgresql.org/docs/9.5/infoschema-sequences.html" +TableCommentType 4294967142 0 "schema privileges (incomplete; may contain excess users or roles)\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#schema_privileges" +TableCommentType 4294967143 0 "database schemas (may contain schemata without permission)\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#schemata\nhttps://www.postgresql.org/docs/9.5/infoschema-schemata.html" +TableCommentType 4294967144 0 "schemata_extensions was created for compatibility and is currently unimplemented" +TableCommentType 4294967145 0 "sql_sizing was created for compatibility and is currently unimplemented" +TableCommentType 4294967146 0 "sql_parts was created for compatibility and is currently unimplemented" +TableCommentType 4294967147 0 "sql_implementation_info was created for compatibility and is currently unimplemented" +TableCommentType 4294967148 0 "sql_features was created for compatibility and is currently unimplemented" +TableCommentType 4294967149 0 "built-in functions (empty - introspection not yet supported)\nhttps://www.postgresql.org/docs/9.5/infoschema-routines.html" +TableCommentType 4294967150 0 "routine_privileges was created for compatibility and is currently unimplemented" +TableCommentType 4294967151 0 "role_usage_grants was created for compatibility and is currently unimplemented" +TableCommentType 4294967152 0 "role_udt_grants was created for compatibility and is currently unimplemented" +TableCommentType 4294967153 0 "privileges granted on table or views (incomplete; see also information_schema.table_privileges; may contain excess users or roles)\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#role_table_grants\nhttps://www.postgresql.org/docs/9.5/infoschema-role-table-grants.html" +TableCommentType 4294967154 0 "privileges granted on functions (incomplete; only contains privileges of user-defined functions)" +TableCommentType 4294967155 0 "role_column_grants was created for compatibility and is currently unimplemented" +TableCommentType 4294967156 0 "resource_groups was created for compatibility and is currently unimplemented" +TableCommentType 4294967157 0 "foreign key constraints\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#referential_constraints\nhttps://www.postgresql.org/docs/9.5/infoschema-referential-constraints.html" +TableCommentType 4294967158 0 "profiling was created for compatibility and is currently unimplemented" +TableCommentType 4294967159 0 "processlist was created for compatibility and is currently unimplemented" +TableCommentType 4294967160 0 "plugins was created for compatibility and is currently unimplemented" +TableCommentType 4294967161 0 "partitions was created for compatibility and is currently unimplemented" +TableCommentType 4294967162 0 "built-in function parameters (empty - introspection not yet supported)\nhttps://www.postgresql.org/docs/9.5/infoschema-parameters.html" +TableCommentType 4294967163 0 "optimizer_trace was created for compatibility and is currently unimplemented" +TableCommentType 4294967164 0 "keywords was created for compatibility and is currently unimplemented" +TableCommentType 4294967165 0 "column usage by indexes and key constraints\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#key_column_usage\nhttps://www.postgresql.org/docs/9.5/infoschema-key-column-usage.html" +TableCommentType 4294967166 0 "information_schema_catalog_name was created for compatibility and is currently unimplemented" +TableCommentType 4294967167 0 "foreign_tables was created for compatibility and is currently unimplemented" +TableCommentType 4294967168 0 "foreign_table_options was created for compatibility and is currently unimplemented" +TableCommentType 4294967169 0 "foreign_servers was created for compatibility and is currently unimplemented" +TableCommentType 4294967170 0 "foreign_server_options was created for compatibility and is currently unimplemented" +TableCommentType 4294967171 0 "foreign_data_wrappers was created for compatibility and is currently unimplemented" +TableCommentType 4294967172 0 "foreign_data_wrapper_options was created for compatibility and is currently unimplemented" +TableCommentType 4294967173 0 "files was created for compatibility and is currently unimplemented" +TableCommentType 4294967174 0 "events was created for compatibility and is currently unimplemented" +TableCommentType 4294967175 0 "engines was created for compatibility and is currently unimplemented" +TableCommentType 4294967176 0 "roles for the current user\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#enabled_roles\nhttps://www.postgresql.org/docs/9.5/infoschema-enabled-roles.html" +TableCommentType 4294967177 0 "element_types was created for compatibility and is currently unimplemented" +TableCommentType 4294967178 0 "domains was created for compatibility and is currently unimplemented" +TableCommentType 4294967179 0 "domain_udt_usage was created for compatibility and is currently unimplemented" +TableCommentType 4294967180 0 "domain_constraints was created for compatibility and is currently unimplemented" +TableCommentType 4294967181 0 "data_type_privileges was created for compatibility and is currently unimplemented" +TableCommentType 4294967182 0 "constraint_table_usage was created for compatibility and is currently unimplemented" +TableCommentType 4294967183 0 "columns usage by constraints\nhttps://www.postgresql.org/docs/9.5/infoschema-constraint-column-usage.html" +TableCommentType 4294967184 0 "table and view columns (incomplete)\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#columns\nhttps://www.postgresql.org/docs/9.5/infoschema-columns.html" +TableCommentType 4294967185 0 "columns_extensions was created for compatibility and is currently unimplemented" +TableCommentType 4294967186 0 "columns with user defined types\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#column_udt_usage\nhttps://www.postgresql.org/docs/current/infoschema-column-udt-usage.html" +TableCommentType 4294967187 0 "column_statistics was created for compatibility and is currently unimplemented" +TableCommentType 4294967188 0 "column privilege grants (incomplete)\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#column_privileges\nhttps://www.postgresql.org/docs/9.5/infoschema-column-privileges.html" +TableCommentType 4294967189 0 "column_options was created for compatibility and is currently unimplemented" +TableCommentType 4294967190 0 "column_domain_usage was created for compatibility and is currently unimplemented" +TableCommentType 4294967191 0 "column_column_usage was created for compatibility and is currently unimplemented" +TableCommentType 4294967192 0 "shows the collations available in the current database\nhttps://www.postgresql.org/docs/current/infoschema-collations.html" +TableCommentType 4294967193 0 "identifies which character set the available collations are \napplicable to. As UTF-8 is the only available encoding this table does not\nprovide much useful information.\nhttps://www.postgresql.org/docs/current/infoschema-collation-character-set-applicab.html" +TableCommentType 4294967194 0 "check constraints\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#check_constraints\nhttps://www.postgresql.org/docs/9.5/infoschema-check-constraints.html" +TableCommentType 4294967195 0 "check_constraint_routine_usage was created for compatibility and is currently unimplemented" +TableCommentType 4294967196 0 "character sets available in the current database\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#character_sets\nhttps://www.postgresql.org/docs/9.5/infoschema-character-sets.html" +TableCommentType 4294967197 0 "attributes was created for compatibility and is currently unimplemented" +TableCommentType 4294967198 0 "roles available to the current user\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#applicable_roles\nhttps://www.postgresql.org/docs/9.5/infoschema-applicable-roles.html" +TableCommentType 4294967199 0 "roles for which the current user has admin option\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#administrable_role_authorizations\nhttps://www.postgresql.org/docs/9.5/infoschema-administrable-role-authorizations.html" +TableCommentType 4294967201 0 "node-level view of tracked kv flow tokens" +TableCommentType 4294967202 0 "node-level view of active kv flow control handles, their underlying streams, and tracked state" +TableCommentType 4294967203 0 "node-level view of the kv flow controller, its active streams and available tokens state" TableCommentType 4294967204 0 "this vtable is a view on system.privileges with the root user and is used to back SHOW SYSTEM GRANTS" TableCommentType 4294967205 0 "table listing transitive closure of system.role_members" TableCommentType 4294967206 0 "eventually consistent in-memory tenant capability cache for this node" diff --git a/pkg/sql/logictest/testdata/logic_test/grant_table b/pkg/sql/logictest/testdata/logic_test/grant_table index 4bc5b9e2622a..71c544195261 100644 --- a/pkg/sql/logictest/testdata/logic_test/grant_table +++ b/pkg/sql/logictest/testdata/logic_test/grant_table @@ -69,6 +69,9 @@ test crdb_internal kv_catalog_descriptor publi test crdb_internal kv_catalog_namespace public SELECT false test crdb_internal kv_catalog_zones public SELECT false test crdb_internal kv_dropped_relations public SELECT false +test crdb_internal kv_flow_control_handles public SELECT false +test crdb_internal kv_flow_controller public SELECT false +test crdb_internal kv_flow_token_deductions public SELECT false test crdb_internal kv_inherited_role_members public SELECT false test crdb_internal kv_node_liveness public SELECT false test crdb_internal kv_node_status public SELECT false diff --git a/pkg/sql/logictest/testdata/logic_test/information_schema b/pkg/sql/logictest/testdata/logic_test/information_schema index bd566c63fc79..0f48032839dc 100644 --- a/pkg/sql/logictest/testdata/logic_test/information_schema +++ b/pkg/sql/logictest/testdata/logic_test/information_schema @@ -434,6 +434,9 @@ crdb_internal kv_catalog_descriptor crdb_internal kv_catalog_namespace crdb_internal kv_catalog_zones crdb_internal kv_dropped_relations +crdb_internal kv_flow_control_handles +crdb_internal kv_flow_controller +crdb_internal kv_flow_token_deductions crdb_internal kv_inherited_role_members crdb_internal kv_node_liveness crdb_internal kv_node_status @@ -779,6 +782,9 @@ kv_catalog_descriptor kv_catalog_namespace kv_catalog_zones kv_dropped_relations +kv_flow_control_handles +kv_flow_controller +kv_flow_token_deductions kv_inherited_role_members kv_node_liveness kv_node_status @@ -1168,6 +1174,9 @@ system crdb_internal kv_catalog_descriptor SYSTE system crdb_internal kv_catalog_namespace SYSTEM VIEW NO 1 system crdb_internal kv_catalog_zones SYSTEM VIEW NO 1 system crdb_internal kv_dropped_relations SYSTEM VIEW NO 1 +system crdb_internal kv_flow_control_handles SYSTEM VIEW NO 1 +system crdb_internal kv_flow_controller SYSTEM VIEW NO 1 +system crdb_internal kv_flow_token_deductions SYSTEM VIEW NO 1 system crdb_internal kv_inherited_role_members SYSTEM VIEW NO 1 system crdb_internal kv_node_liveness SYSTEM VIEW NO 1 system crdb_internal kv_node_status SYSTEM VIEW NO 1 @@ -3153,6 +3162,9 @@ NULL public system crdb_internal kv_catalog_descriptor NULL public system crdb_internal kv_catalog_namespace SELECT NO YES NULL public system crdb_internal kv_catalog_zones SELECT NO YES NULL public system crdb_internal kv_dropped_relations SELECT NO YES +NULL public system crdb_internal kv_flow_control_handles SELECT NO YES +NULL public system crdb_internal kv_flow_controller SELECT NO YES +NULL public system crdb_internal kv_flow_token_deductions SELECT NO YES NULL public system crdb_internal kv_inherited_role_members SELECT NO YES NULL public system crdb_internal kv_node_liveness SELECT NO YES NULL public system crdb_internal kv_node_status SELECT NO YES @@ -3809,6 +3821,9 @@ NULL public system crdb_internal kv_catalog_descriptor NULL public system crdb_internal kv_catalog_namespace SELECT NO YES NULL public system crdb_internal kv_catalog_zones SELECT NO YES NULL public system crdb_internal kv_dropped_relations SELECT NO YES +NULL public system crdb_internal kv_flow_control_handles SELECT NO YES +NULL public system crdb_internal kv_flow_controller SELECT NO YES +NULL public system crdb_internal kv_flow_token_deductions SELECT NO YES NULL public system crdb_internal kv_inherited_role_members SELECT NO YES NULL public system crdb_internal kv_node_liveness SELECT NO YES NULL public system crdb_internal kv_node_status SELECT NO YES diff --git a/pkg/sql/logictest/testdata/logic_test/pg_builtins b/pkg/sql/logictest/testdata/logic_test/pg_builtins index 2a27d5871278..94ef61107095 100644 --- a/pkg/sql/logictest/testdata/logic_test/pg_builtins +++ b/pkg/sql/logictest/testdata/logic_test/pg_builtins @@ -210,42 +210,42 @@ is_updatable b 123 2 28 is_updatable c 123 3 28 false is_updatable_view a 124 1 0 false is_updatable_view b 124 2 0 false -pg_class oid 4294967104 1 0 false -pg_class relname 4294967104 2 0 false -pg_class relnamespace 4294967104 3 0 false -pg_class reltype 4294967104 4 0 false -pg_class reloftype 4294967104 5 0 false -pg_class relowner 4294967104 6 0 false -pg_class relam 4294967104 7 0 false -pg_class relfilenode 4294967104 8 0 false -pg_class reltablespace 4294967104 9 0 false -pg_class relpages 4294967104 10 0 false -pg_class reltuples 4294967104 11 0 false -pg_class relallvisible 4294967104 12 0 false -pg_class reltoastrelid 4294967104 13 0 false -pg_class relhasindex 4294967104 14 0 false -pg_class relisshared 4294967104 15 0 false -pg_class relpersistence 4294967104 16 0 false -pg_class relistemp 4294967104 17 0 false -pg_class relkind 4294967104 18 0 false -pg_class relnatts 4294967104 19 0 false -pg_class relchecks 4294967104 20 0 false -pg_class relhasoids 4294967104 21 0 false -pg_class relhaspkey 4294967104 22 0 false -pg_class relhasrules 4294967104 23 0 false -pg_class relhastriggers 4294967104 24 0 false -pg_class relhassubclass 4294967104 25 0 false -pg_class relfrozenxid 4294967104 26 0 false -pg_class relacl 4294967104 27 0 false -pg_class reloptions 4294967104 28 0 false -pg_class relforcerowsecurity 4294967104 29 0 false -pg_class relispartition 4294967104 30 0 false -pg_class relispopulated 4294967104 31 0 false -pg_class relreplident 4294967104 32 0 false -pg_class relrewrite 4294967104 33 0 false -pg_class relrowsecurity 4294967104 34 0 false -pg_class relpartbound 4294967104 35 0 false -pg_class relminmxid 4294967104 36 0 false +pg_class oid 4294967101 1 0 false +pg_class relname 4294967101 2 0 false +pg_class relnamespace 4294967101 3 0 false +pg_class reltype 4294967101 4 0 false +pg_class reloftype 4294967101 5 0 false +pg_class relowner 4294967101 6 0 false +pg_class relam 4294967101 7 0 false +pg_class relfilenode 4294967101 8 0 false +pg_class reltablespace 4294967101 9 0 false +pg_class relpages 4294967101 10 0 false +pg_class reltuples 4294967101 11 0 false +pg_class relallvisible 4294967101 12 0 false +pg_class reltoastrelid 4294967101 13 0 false +pg_class relhasindex 4294967101 14 0 false +pg_class relisshared 4294967101 15 0 false +pg_class relpersistence 4294967101 16 0 false +pg_class relistemp 4294967101 17 0 false +pg_class relkind 4294967101 18 0 false +pg_class relnatts 4294967101 19 0 false +pg_class relchecks 4294967101 20 0 false +pg_class relhasoids 4294967101 21 0 false +pg_class relhaspkey 4294967101 22 0 false +pg_class relhasrules 4294967101 23 0 false +pg_class relhastriggers 4294967101 24 0 false +pg_class relhassubclass 4294967101 25 0 false +pg_class relfrozenxid 4294967101 26 0 false +pg_class relacl 4294967101 27 0 false +pg_class reloptions 4294967101 28 0 false +pg_class relforcerowsecurity 4294967101 29 0 false +pg_class relispartition 4294967101 30 0 false +pg_class relispopulated 4294967101 31 0 false +pg_class relreplident 4294967101 32 0 false +pg_class relrewrite 4294967101 33 0 false +pg_class relrowsecurity 4294967101 34 0 false +pg_class relpartbound 4294967101 35 0 false +pg_class relminmxid 4294967101 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 117fb78e014b..d64407b621b6 100644 --- a/pkg/sql/logictest/testdata/logic_test/pg_catalog +++ b/pkg/sql/logictest/testdata/logic_test/pg_catalog @@ -428,9 +428,9 @@ SELECT * FROM pg_catalog.pg_namespace ---- oid nspname nspowner nspacl 4294967295 crdb_internal NULL NULL -4294967203 information_schema NULL NULL -4294967116 pg_catalog NULL NULL -4294966986 pg_extension NULL NULL +4294967200 information_schema NULL NULL +4294967113 pg_catalog NULL NULL +4294966983 pg_extension NULL NULL 105 public 2310524507 NULL # Verify that we can still see the schemas even if we don't have any privilege @@ -447,9 +447,9 @@ SELECT * FROM pg_catalog.pg_namespace ---- oid nspname nspowner nspacl 4294967295 crdb_internal NULL NULL -4294967203 information_schema NULL NULL -4294967116 pg_catalog NULL NULL -4294966986 pg_extension NULL NULL +4294967200 information_schema NULL NULL +4294967113 pg_catalog NULL NULL +4294966983 pg_extension NULL NULL 105 public 2310524507 NULL user root @@ -1437,7 +1437,7 @@ SELECT * FROM pg_collation WHERE collname='en-US' ---- oid collname collnamespace collowner collencoding collcollate collctype collprovider collversion collisdeterministic -3903121477 en-US 4294967116 NULL 6 NULL NULL NULL NULL NULL +3903121477 en-US 4294967113 NULL 6 NULL NULL NULL NULL NULL user testuser @@ -1636,16 +1636,16 @@ FROM pg_catalog.pg_depend ORDER BY objid, refobjid, refobjsubid ---- classid objid objsubid refclassid refobjid refobjsubid deptype -4294967101 111 0 4294967104 110 14 a -4294967101 112 0 4294967104 110 15 a -4294967058 842401391 0 4294967104 110 1 n -4294967058 842401391 0 4294967104 110 2 n -4294967058 842401391 0 4294967104 110 3 n -4294967058 842401391 0 4294967104 110 4 n -4294967101 1179276562 0 4294967104 3687884464 0 n -4294967101 3935750373 0 4294967104 3687884465 0 n -4294967101 4072017905 0 4294967104 0 0 n -4294967101 4170826110 0 4294967104 0 0 n +4294967098 111 0 4294967101 110 14 a +4294967098 112 0 4294967101 110 15 a +4294967055 842401391 0 4294967101 110 1 n +4294967055 842401391 0 4294967101 110 2 n +4294967055 842401391 0 4294967101 110 3 n +4294967055 842401391 0 4294967101 110 4 n +4294967098 1179276562 0 4294967101 3687884464 0 n +4294967098 3935750373 0 4294967101 3687884465 0 n +4294967098 4072017905 0 4294967101 0 0 n +4294967098 4170826110 0 4294967101 0 0 n # Some entries in pg_depend are dependency links from the pg_constraint system # table to the pg_class system table. Other entries are links to pg_class when it is @@ -1658,8 +1658,8 @@ JOIN pg_class cla ON classid=cla.oid JOIN pg_class refcla ON refclassid=refcla.oid ---- classid refclassid tablename reftablename -4294967058 4294967104 pg_rewrite pg_class -4294967101 4294967104 pg_constraint pg_class +4294967055 4294967101 pg_rewrite pg_class +4294967098 4294967101 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 @@ -1758,88 +1758,88 @@ WHERE oid < 4194967002 -- exclude implicit types for virtual tables ORDER BY oid ---- oid typname typnamespace typowner typlen typbyval typtype -16 bool 4294967116 NULL 1 true b -17 bytea 4294967116 NULL -1 false b -18 char 4294967116 NULL 1 true b -19 name 4294967116 NULL -1 false b -20 int8 4294967116 NULL 8 true b -21 int2 4294967116 NULL 2 true b -22 int2vector 4294967116 NULL -1 false b -23 int4 4294967116 NULL 4 true b -24 regproc 4294967116 NULL 4 true b -25 text 4294967116 NULL -1 false b -26 oid 4294967116 NULL 4 true b -30 oidvector 4294967116 NULL -1 false b -700 float4 4294967116 NULL 4 true b -701 float8 4294967116 NULL 8 true b -705 unknown 4294967116 NULL 0 true b -869 inet 4294967116 NULL 24 true b -1000 _bool 4294967116 NULL -1 false b -1001 _bytea 4294967116 NULL -1 false b -1002 _char 4294967116 NULL -1 false b -1003 _name 4294967116 NULL -1 false b -1005 _int2 4294967116 NULL -1 false b -1006 _int2vector 4294967116 NULL -1 false b -1007 _int4 4294967116 NULL -1 false b -1008 _regproc 4294967116 NULL -1 false b -1009 _text 4294967116 NULL -1 false b -1013 _oidvector 4294967116 NULL -1 false b -1014 _bpchar 4294967116 NULL -1 false b -1015 _varchar 4294967116 NULL -1 false b -1016 _int8 4294967116 NULL -1 false b -1021 _float4 4294967116 NULL -1 false b -1022 _float8 4294967116 NULL -1 false b -1028 _oid 4294967116 NULL -1 false b -1041 _inet 4294967116 NULL -1 false b -1042 bpchar 4294967116 NULL -1 false b -1043 varchar 4294967116 NULL -1 false b -1082 date 4294967116 NULL 4 true b -1083 time 4294967116 NULL 8 true b -1114 timestamp 4294967116 NULL 8 true b -1115 _timestamp 4294967116 NULL -1 false b -1182 _date 4294967116 NULL -1 false b -1183 _time 4294967116 NULL -1 false b -1184 timestamptz 4294967116 NULL 8 true b -1185 _timestamptz 4294967116 NULL -1 false b -1186 interval 4294967116 NULL 24 true b -1187 _interval 4294967116 NULL -1 false b -1231 _numeric 4294967116 NULL -1 false b -1266 timetz 4294967116 NULL 12 true b -1270 _timetz 4294967116 NULL -1 false b -1560 bit 4294967116 NULL -1 false b -1561 _bit 4294967116 NULL -1 false b -1562 varbit 4294967116 NULL -1 false b -1563 _varbit 4294967116 NULL -1 false b -1700 numeric 4294967116 NULL -1 false b -2202 regprocedure 4294967116 NULL 4 true b -2205 regclass 4294967116 NULL 4 true b -2206 regtype 4294967116 NULL 4 true b -2207 _regprocedure 4294967116 NULL -1 false b -2210 _regclass 4294967116 NULL -1 false b -2211 _regtype 4294967116 NULL -1 false b -2249 record 4294967116 NULL 0 true p -2277 anyarray 4294967116 NULL -1 false p -2278 void 4294967116 NULL 0 true p -2283 anyelement 4294967116 NULL -1 false p -2287 _record 4294967116 NULL -1 false b -2950 uuid 4294967116 NULL 16 true b -2951 _uuid 4294967116 NULL -1 false b -3614 tsvector 4294967116 NULL -1 false b -3615 tsquery 4294967116 NULL -1 false b -3643 _tsvector 4294967116 NULL -1 false b -3645 _tsquery 4294967116 NULL -1 false b -3802 jsonb 4294967116 NULL -1 false b -3807 _jsonb 4294967116 NULL -1 false b -4089 regnamespace 4294967116 NULL 4 true b -4090 _regnamespace 4294967116 NULL -1 false b -4096 regrole 4294967116 NULL 4 true b -4097 _regrole 4294967116 NULL -1 false b -90000 geometry 4294967116 NULL -1 false b -90001 _geometry 4294967116 NULL -1 false b -90002 geography 4294967116 NULL -1 false b -90003 _geography 4294967116 NULL -1 false b -90004 box2d 4294967116 NULL 32 true b -90005 _box2d 4294967116 NULL -1 false b +16 bool 4294967113 NULL 1 true b +17 bytea 4294967113 NULL -1 false b +18 char 4294967113 NULL 1 true b +19 name 4294967113 NULL -1 false b +20 int8 4294967113 NULL 8 true b +21 int2 4294967113 NULL 2 true b +22 int2vector 4294967113 NULL -1 false b +23 int4 4294967113 NULL 4 true b +24 regproc 4294967113 NULL 4 true b +25 text 4294967113 NULL -1 false b +26 oid 4294967113 NULL 4 true b +30 oidvector 4294967113 NULL -1 false b +700 float4 4294967113 NULL 4 true b +701 float8 4294967113 NULL 8 true b +705 unknown 4294967113 NULL 0 true b +869 inet 4294967113 NULL 24 true b +1000 _bool 4294967113 NULL -1 false b +1001 _bytea 4294967113 NULL -1 false b +1002 _char 4294967113 NULL -1 false b +1003 _name 4294967113 NULL -1 false b +1005 _int2 4294967113 NULL -1 false b +1006 _int2vector 4294967113 NULL -1 false b +1007 _int4 4294967113 NULL -1 false b +1008 _regproc 4294967113 NULL -1 false b +1009 _text 4294967113 NULL -1 false b +1013 _oidvector 4294967113 NULL -1 false b +1014 _bpchar 4294967113 NULL -1 false b +1015 _varchar 4294967113 NULL -1 false b +1016 _int8 4294967113 NULL -1 false b +1021 _float4 4294967113 NULL -1 false b +1022 _float8 4294967113 NULL -1 false b +1028 _oid 4294967113 NULL -1 false b +1041 _inet 4294967113 NULL -1 false b +1042 bpchar 4294967113 NULL -1 false b +1043 varchar 4294967113 NULL -1 false b +1082 date 4294967113 NULL 4 true b +1083 time 4294967113 NULL 8 true b +1114 timestamp 4294967113 NULL 8 true b +1115 _timestamp 4294967113 NULL -1 false b +1182 _date 4294967113 NULL -1 false b +1183 _time 4294967113 NULL -1 false b +1184 timestamptz 4294967113 NULL 8 true b +1185 _timestamptz 4294967113 NULL -1 false b +1186 interval 4294967113 NULL 24 true b +1187 _interval 4294967113 NULL -1 false b +1231 _numeric 4294967113 NULL -1 false b +1266 timetz 4294967113 NULL 12 true b +1270 _timetz 4294967113 NULL -1 false b +1560 bit 4294967113 NULL -1 false b +1561 _bit 4294967113 NULL -1 false b +1562 varbit 4294967113 NULL -1 false b +1563 _varbit 4294967113 NULL -1 false b +1700 numeric 4294967113 NULL -1 false b +2202 regprocedure 4294967113 NULL 4 true b +2205 regclass 4294967113 NULL 4 true b +2206 regtype 4294967113 NULL 4 true b +2207 _regprocedure 4294967113 NULL -1 false b +2210 _regclass 4294967113 NULL -1 false b +2211 _regtype 4294967113 NULL -1 false b +2249 record 4294967113 NULL 0 true p +2277 anyarray 4294967113 NULL -1 false p +2278 void 4294967113 NULL 0 true p +2283 anyelement 4294967113 NULL -1 false p +2287 _record 4294967113 NULL -1 false b +2950 uuid 4294967113 NULL 16 true b +2951 _uuid 4294967113 NULL -1 false b +3614 tsvector 4294967113 NULL -1 false b +3615 tsquery 4294967113 NULL -1 false b +3643 _tsvector 4294967113 NULL -1 false b +3645 _tsquery 4294967113 NULL -1 false b +3802 jsonb 4294967113 NULL -1 false b +3807 _jsonb 4294967113 NULL -1 false b +4089 regnamespace 4294967113 NULL 4 true b +4090 _regnamespace 4294967113 NULL -1 false b +4096 regrole 4294967113 NULL 4 true b +4097 _regrole 4294967113 NULL -1 false b +90000 geometry 4294967113 NULL -1 false b +90001 _geometry 4294967113 NULL -1 false b +90002 geography 4294967113 NULL -1 false b +90003 _geography 4294967113 NULL -1 false b +90004 box2d 4294967113 NULL 32 true b +90005 _box2d 4294967113 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 @@ -2315,7 +2315,7 @@ FROM pg_catalog.pg_type WHERE oid = 1000 ---- oid typname typnamespace typowner typlen typbyval typtype -1000 _bool 4294967116 NULL -1 false b +1000 _bool 4294967113 NULL -1 false b query OTOOIBT colnames SELECT oid, typname, typnamespace, typowner, typlen, typbyval, typtype @@ -2373,7 +2373,7 @@ FROM pg_catalog.pg_type WHERE oid = $vtableSourceId ---- oid typname typnamespace typowner typlen typbyval typtype -4294967066 pg_proc 4294967116 2310524507 -1 false c +4294967063 pg_proc 4294967113 2310524507 -1 false c ## pg_catalog.pg_proc @@ -2384,14 +2384,14 @@ JOIN pg_catalog.pg_namespace n ON n.oid = p.pronamespace WHERE proname='substring' ---- proname pronamespace nspname proowner prolang procost prorows provariadic -substring 4294967116 pg_catalog NULL 0 NULL NULL 0 -substring 4294967116 pg_catalog NULL 0 NULL NULL 0 -substring 4294967116 pg_catalog NULL 0 NULL NULL 0 -substring 4294967116 pg_catalog NULL 0 NULL NULL 0 -substring 4294967116 pg_catalog NULL 0 NULL NULL 0 -substring 4294967116 pg_catalog NULL 0 NULL NULL 0 -substring 4294967116 pg_catalog NULL 0 NULL NULL 0 -substring 4294967116 pg_catalog NULL 0 NULL NULL 0 +substring 4294967113 pg_catalog NULL 0 NULL NULL 0 +substring 4294967113 pg_catalog NULL 0 NULL NULL 0 +substring 4294967113 pg_catalog NULL 0 NULL NULL 0 +substring 4294967113 pg_catalog NULL 0 NULL NULL 0 +substring 4294967113 pg_catalog NULL 0 NULL NULL 0 +substring 4294967113 pg_catalog NULL 0 NULL NULL 0 +substring 4294967113 pg_catalog NULL 0 NULL NULL 0 +substring 4294967113 pg_catalog NULL 0 NULL NULL 0 query TTBBBB colnames SELECT proname, protransform, proisagg, proiswindow, prosecdef, proleakproof @@ -2473,7 +2473,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 4294967203 information_schema 20 26 23 +_pg_char_max_length _pg_char_max_length 4294967200 information_schema 20 26 23 query TOIOTTB colnames SELECT proname, provariadic, pronargs, prorettype, proargtypes, proargmodes, proisstrict @@ -2593,14 +2593,14 @@ OR (c.relname = 'pg_class' AND d.objoid IN (SELECT oid FROM pg_catalog.pg_class OR (c.relname = 'pg_namespace' AND d.objoid IN (SELECT oid FROM pg_catalog.pg_namespace WHERE nspname = 'testschema')) ---- relname objoid classoid objsubid description -pg_class 135 4294967104 0 mycomment1 -pg_class 4294966988 4294967104 0 database users -pg_class 135 4294967104 1 mycomment2 -pg_class 125730312 4294967104 0 mycomment3 -pg_namespace 136 4294967075 0 mycomment4 -pg_proc 738 4294967066 0 Calculates the absolute value of `val`. -pg_proc 739 4294967066 0 Calculates the absolute value of `val`. -pg_proc 740 4294967066 0 Calculates the absolute value of `val`. +pg_class 135 4294967101 0 mycomment1 +pg_class 4294966985 4294967101 0 database users +pg_class 135 4294967101 1 mycomment2 +pg_class 125730312 4294967101 0 mycomment3 +pg_namespace 136 4294967072 0 mycomment4 +pg_proc 738 4294967063 0 Calculates the absolute value of `val`. +pg_proc 739 4294967063 0 Calculates the absolute value of `val`. +pg_proc 740 4294967063 0 Calculates the absolute value of `val`. ## pg_catalog.pg_shdescription @@ -2611,7 +2611,7 @@ query OOT colnames SELECT objoid, classoid, description FROM pg_catalog.pg_shdescription ---- objoid classoid description -100 4294967098 mydbcomment +100 4294967095 mydbcomment ## pg_catalog.pg_event_trigger @@ -3180,7 +3180,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 + 4294967116 NULL b false false 701 701 701 NULL NULL NULL NULL NULL +74817020 + 4294967113 NULL b false false 701 701 701 NULL NULL NULL NULL NULL # Verify proper functionality of system information functions. @@ -4029,7 +4029,7 @@ indoption query TTI SELECT database_name, descriptor_name, descriptor_id from test.crdb_internal.create_statements where descriptor_name = 'pg_views' ---- -test pg_views 4294966987 +test pg_views 4294966984 # Verify INCLUDED columns appear in pg_index. See issue #59563 statement ok diff --git a/pkg/sql/logictest/testdata/logic_test/table b/pkg/sql/logictest/testdata/logic_test/table index 2820f05fc5ec..7e7da3e9a92c 100644 --- a/pkg/sql/logictest/testdata/logic_test/table +++ b/pkg/sql/logictest/testdata/logic_test/table @@ -594,6 +594,9 @@ kv_catalog_descriptor NULL kv_catalog_namespace NULL kv_catalog_zones NULL kv_dropped_relations NULL +kv_flow_control_handles NULL +kv_flow_controller NULL +kv_flow_token_deductions NULL kv_inherited_role_members NULL kv_node_liveness NULL kv_node_status NULL diff --git a/pkg/sql/opt/exec/execbuilder/testdata/explain b/pkg/sql/opt/exec/execbuilder/testdata/explain index 57728b9bd6ef..3806a7ed11a8 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/explain +++ b/pkg/sql/opt/exec/execbuilder/testdata/explain @@ -437,7 +437,7 @@ vectorized: true ā”‚ ā”‚ ā”‚ table: pg_class@pg_class_oid_idx ā”‚ ā”‚ ā”‚ ā”‚ ā”‚ ā””ā”€ā”€ ā€¢ filter - ā”‚ ā”‚ ā”‚ filter: (objsubid = 0) AND (classoid = 4294967104) + ā”‚ ā”‚ ā”‚ filter: (objsubid = 0) AND (classoid = 4294967101) ā”‚ ā”‚ ā”‚ ā”‚ ā”‚ ā””ā”€ā”€ ā€¢ virtual table ā”‚ ā”‚ table: pg_description@pg_description_objoid_idx diff --git a/pkg/sql/opt/xform/testdata/external/activerecord b/pkg/sql/opt/xform/testdata/external/activerecord index 767834b649be..f357c8761e13 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 < 4294966983 [outer=(1), constraints=(/1: (/NULL - /4294966982]; tight)] + ā”‚ ā”‚ ā””ā”€ā”€ attrelid:1 < 4294966980 [outer=(1), constraints=(/1: (/NULL - /4294966979]; tight)] ā”‚ ā””ā”€ā”€ aggregations ā”‚ ā”œā”€ā”€ const-agg [as=attname:2, outer=(2)] ā”‚ ā”‚ ā””ā”€ā”€ attname:2 diff --git a/pkg/sql/planner.go b/pkg/sql/planner.go index af6dd3f212bc..5bd90982d058 100644 --- a/pkg/sql/planner.go +++ b/pkg/sql/planner.go @@ -138,6 +138,7 @@ func (evalCtx *extendedEvalContext) copyFromExecCfg(execCfg *ExecutorConfig) { evalCtx.DistSQLPlanner = execCfg.DistSQLPlanner evalCtx.VirtualSchemas = execCfg.VirtualSchemas evalCtx.KVStoresIterator = execCfg.KVStoresIterator + evalCtx.InspectzServer = execCfg.InspectzServer } // copy returns a deep copy of ctx. diff --git a/pkg/sql/sem/catconstants/constants.go b/pkg/sql/sem/catconstants/constants.go index 5f7c6887d832..aa6f743aab96 100644 --- a/pkg/sql/sem/catconstants/constants.go +++ b/pkg/sql/sem/catconstants/constants.go @@ -193,6 +193,9 @@ const ( CrdbInternalShowTenantCapabilitiesCacheTableID CrdbInternalInheritedRoleMembersTableID CrdbInternalKVSystemPrivilegesViewID + CrdbInternalKVFlowControllerID + CrdbInternalKVFlowHandlesID + CrdbInternalKVFlowTokenDeductions InformationSchemaID InformationSchemaAdministrableRoleAuthorizationsID InformationSchemaApplicableRolesID diff --git a/pkg/sql/sem/eval/BUILD.bazel b/pkg/sql/sem/eval/BUILD.bazel index c94b0d4d1c84..ad60fb5bdcfb 100644 --- a/pkg/sql/sem/eval/BUILD.bazel +++ b/pkg/sql/sem/eval/BUILD.bazel @@ -40,6 +40,7 @@ go_library( "//pkg/clusterversion", "//pkg/geo", "//pkg/geo/geopb", + "//pkg/inspectz/inspectzpb", "//pkg/jobs/jobspb", "//pkg/keys", "//pkg/kv", diff --git a/pkg/sql/sem/eval/context.go b/pkg/sql/sem/eval/context.go index f72025adb269..fb6270f308cf 100644 --- a/pkg/sql/sem/eval/context.go +++ b/pkg/sql/sem/eval/context.go @@ -17,6 +17,7 @@ import ( apd "github.com/cockroachdb/apd/v3" "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/inspectz/inspectzpb" "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" @@ -231,6 +232,10 @@ type Context struct { // access stores on this node. KVStoresIterator kvserverbase.StoresIterator + // InspectzServer is used to power various crdb_internal vtables, exposing + // the equivalent of /inspectz but through SQL. + InspectzServer inspectzpb.InspectzServer + // ConsistencyChecker is to generate the results in calls to // crdb_internal.check_consistency. ConsistencyChecker ConsistencyCheckRunner From 3ae5b7d41c6d9665fcbf04e2f00b9500a6c7643d Mon Sep 17 00:00:00 2001 From: irfan sharif Date: Wed, 26 Apr 2023 03:45:46 -0400 Subject: [PATCH 04/12] kvserver: add datadriven test for kvflowcontrol integration We add TestFlowControlIntegration, exercising the kvflowcontrol integration interface (replicaFlowControlIntegration) introduced in an earlier commit. It tests how the underlying kvflowcontrol.Handle is constructed/destroyed, and its stream-management APIs invoked, as replicas acquire raft leadership, lost it, observe paused and/or inactive followers, change range descriptors, and react to raft progress updates. We can write tests of the following form: # Observe how the integration layer deals with paused followers. # Start off with a triply replicated range r1/t1, with replicas on # n1/s1, n2/s2, and n3/s3 (with replica IDs 1-3 respectively). init tenant=t1 range=r1 replid=1 ---- state descriptor=(1,2,3) applied=1/10 ---- # Set up replid=1 (declared in init above) to be the raft leader. It # should connect to all three replication streams. integration op=became-leader ---- initialized flow control handle for r1/t1 connected to replication stream t1/s1 starting at log-position=1/10 connected to replication stream t1/s2 starting at log-position=1/10 connected to replication stream t1/s3 starting at log-position=1/10 # Pause replid=2. Observe that we disconnect the stream to t1/s2. state descriptor=(1,2,3) paused=(2) ---- integration op=followers-paused ---- disconnected from replication stream t1/s2 These are still unit tests, testing the interface at just the interface-level. We'll introduce more end-to-end integration testing of an actual replica's lifecycle in subsequent commits. Release note: None --- pkg/kv/kvserver/BUILD.bazel | 1 + .../flow_control_replica_integration.go | 2 - .../flow_control_replica_integration_test.go | 492 ++++++++++++++++++ pkg/kv/kvserver/kvflowcontrol/doc.go | 4 + .../kvserver/kvflowcontrol/kvflowcontrol.go | 16 + .../desc_changed | 63 +++ .../inactive_followers | 43 ++ .../paused_followers | 43 ++ .../replica_destroyed | 31 ++ .../replica_raft_leadership | 75 +++ .../replica_removed_self | 29 ++ 11 files changed, 797 insertions(+), 2 deletions(-) create mode 100644 pkg/kv/kvserver/flow_control_replica_integration_test.go create mode 100644 pkg/kv/kvserver/testdata/flow_control_replica_integration/desc_changed create mode 100644 pkg/kv/kvserver/testdata/flow_control_replica_integration/inactive_followers create mode 100644 pkg/kv/kvserver/testdata/flow_control_replica_integration/paused_followers create mode 100644 pkg/kv/kvserver/testdata/flow_control_replica_integration/replica_destroyed create mode 100644 pkg/kv/kvserver/testdata/flow_control_replica_integration/replica_raft_leadership create mode 100644 pkg/kv/kvserver/testdata/flow_control_replica_integration/replica_removed_self diff --git a/pkg/kv/kvserver/BUILD.bazel b/pkg/kv/kvserver/BUILD.bazel index a5bdff1be81c..8d9bea30c1ad 100644 --- a/pkg/kv/kvserver/BUILD.bazel +++ b/pkg/kv/kvserver/BUILD.bazel @@ -273,6 +273,7 @@ go_test( "debug_print_test.go", "errors_test.go", "flow_control_raft_transport_test.go", + "flow_control_replica_integration_test.go", "gossip_test.go", "helpers_test.go", "intent_resolver_integration_test.go", diff --git a/pkg/kv/kvserver/flow_control_replica_integration.go b/pkg/kv/kvserver/flow_control_replica_integration.go index 051fad507fcb..fe658e86f8ba 100644 --- a/pkg/kv/kvserver/flow_control_replica_integration.go +++ b/pkg/kv/kvserver/flow_control_replica_integration.go @@ -31,8 +31,6 @@ import ( // leader will still see follower as active and not disconnect streams. Has this // changed with us upgrading asymmetric partitions to bidirectional ones? -// TODO(irfansharif): Write data-driven unit tests for this interface. - // replicaFlowControlIntegration is used to integrate with replication flow // control. It's intercepts various points in a replica's lifecycle, like it // acquiring raft leadership or losing it, or its raft membership changing, etc. diff --git a/pkg/kv/kvserver/flow_control_replica_integration_test.go b/pkg/kv/kvserver/flow_control_replica_integration_test.go new file mode 100644 index 000000000000..4d501292531d --- /dev/null +++ b/pkg/kv/kvserver/flow_control_replica_integration_test.go @@ -0,0 +1,492 @@ +// Copyright 2023 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package kvserver + +import ( + "context" + "fmt" + "strconv" + "strings" + "testing" + "time" + + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol/kvflowcontrolpb" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol/kvflowinspectpb" + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/testutils/datapathutils" + "github.com/cockroachdb/cockroach/pkg/util/admission/admissionpb" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/datadriven" + "github.com/stretchr/testify/require" + "go.etcd.io/raft/v3/tracker" +) + +// TestFlowControlReplicaIntegration tests the kvflowcontrol replica integration +// interfaces. It offers the following commands: +// +// - "init" tenant=t range=r replid= +// ---- +// Initializes the flow control integration interface, using a replica for +// the given range+tenant and the given replica ID. +// +// - "state" [applied=/] [descriptor=([,][,]*) [inactive=([][,]*) \ +// [progress=([@:[probe | replicate | snapshot]:[!,]active:[!,]paused]*] \ +// [connected=([][,]*)] +// ---- +// Set up relevant state of the underlying replica and/or raft transport. +// +// A. For replicas, we can control the applied state (term/index), +// descriptor (set of replica IDs), paused and/or inactive replicas, and +// per-replica raft progress. The raft progress syntax is structured as +// follows: +// => progress=(replid@match:::,...) +// Where is one of {probe,replicate,snapshot}, is +// {active,!inactive}, and is {paused,!paused}. +// +// B. For the raft transport, we can specify the set of store IDs we're +// connected to. +// +// - "integration" op=[became-leader | became-follower | desc-changed | +// followers-paused |replica-destroyed | +// proposal-quota-updated] +// ---- +// Invoke the specific APIs integration interface, informing it of the +// underlying replica acquire raft leadership, losing it, its range +// descriptor changing, a change in the set of paused followers, it being +// destroyed, and its proposal quota being updated +func TestFlowControlReplicaIntegration(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + ctx := context.Background() + datadriven.Walk(t, datapathutils.TestDataPath(t, "flow_control_replica_integration"), + func(t *testing.T, path string) { + var mockReplica *mockReplicaForFlowControl + var mockHandleFactory *mockFlowHandleFactory + var integration replicaFlowControlIntegration + var logger *testLogger + datadriven.RunTest(t, path, + func(t *testing.T, d *datadriven.TestData) string { + if d.Cmd == "init" { + require.Nil(t, mockReplica) + require.Nil(t, mockHandleFactory) + require.Nil(t, integration) + require.Nil(t, logger) + } else { + require.NotNil(t, mockReplica) + require.NotNil(t, mockHandleFactory) + require.NotNil(t, integration) + require.NotNil(t, logger) + } + + switch d.Cmd { + case "init": + var arg string + + // Parse range=r. + d.ScanArgs(t, "range", &arg) + ri, err := strconv.Atoi(strings.TrimPrefix(arg, "r")) + require.NoError(t, err) + rangeID := roachpb.RangeID(ri) + + // Parse tenant=t. + d.ScanArgs(t, "tenant", &arg) + ti, err := strconv.Atoi(strings.TrimPrefix(arg, "t")) + require.NoError(t, err) + tenantID := roachpb.MustMakeTenantID(uint64(ti)) + + // Parse replid=. + d.ScanArgs(t, "replid", &arg) + repli, err := strconv.Atoi(arg) + require.NoError(t, err) + replID := roachpb.ReplicaID(repli) + + logger = newTestLogger() + + mockHandleFactory = newMockFlowHandleFactory(t, logger) + mockReplica = newMockReplicaForFlowControl(t, rangeID, tenantID, replID) + integration = newReplicaFlowControlIntegration(mockReplica, mockHandleFactory) + return "" + + case "state": + for _, arg := range d.CmdArgs { + replicas := roachpb.MakeReplicaSet(nil) + progress := make(map[roachpb.ReplicaID]tracker.Progress) + connected := make(map[roachpb.StoreID]struct{}) + for i := range arg.Vals { + if arg.Vals[i] == "" { + continue // we support syntax like inactive=(); there's nothing to do + } + switch arg.Key { + case "progress": + // Parse progress=(repl@match:::,...). + // = one of probe, replicate, or snapshot + // = one of active or !inactive + // = one of paused of !paused + parts := strings.Split(arg.Vals[i], ":") + require.Len(t, parts, 4) + + // Parse repl@match. + match := strings.Split(parts[0], "@") + require.Len(t, match, 2) + repli, err := strconv.Atoi(match[0]) + require.NoError(t, err) + replID := roachpb.ReplicaID(repli) + index, err := strconv.Atoi(match[1]) + require.NoError(t, err) + + // Parse (one of probe, replicate, or snapshot). + var state tracker.StateType + switch parts[1] { + case "probe": + state = tracker.StateProbe + case "replicate": + state = tracker.StateReplicate + case "snapshot": + state = tracker.StateSnapshot + default: + t.Fatalf("unknown : %s", parts[1]) + } + + // Parse (one of active or !inactive). + require.True(t, parts[2] == "active" || parts[2] == "!active") + active := parts[2] == "active" + + // Parse (one of paused or !paused). + require.True(t, parts[3] == "paused" || parts[3] == "!paused") + paused := parts[3] == "paused" + + progress[replID] = tracker.Progress{ + Match: uint64(index), + State: state, + RecentActive: active, + MsgAppFlowPaused: paused, + Inflights: tracker.NewInflights(1, 0), // avoid NPE + IsLearner: false, + } + + case "descriptor", "paused", "inactive": + // Parse key=(,,...). + var id uint64 + arg.Scan(t, i, &id) + replicas.AddReplica( + roachpb.ReplicaDescriptor{ + NodeID: roachpb.NodeID(id), + StoreID: roachpb.StoreID(id), + ReplicaID: roachpb.ReplicaID(id), + Type: roachpb.VOTER_FULL, + }, + ) + + case "applied": + // Fall through. + + case "connected": + // Parse key=(,,...). + var id uint64 + arg.Scan(t, i, &id) + connected[roachpb.StoreID(id)] = struct{}{} + + default: + t.Fatalf("unknown: %s", arg.Key) + } + } + + switch arg.Key { + case "descriptor": + mockReplica.descriptor.SetReplicas(replicas) + + case "paused": + mockReplica.paused = make(map[roachpb.ReplicaID]struct{}) + for _, repl := range replicas.Descriptors() { + mockReplica.paused[repl.ReplicaID] = struct{}{} + } + + case "inactive": + mockReplica.inactive = make(map[roachpb.ReplicaID]struct{}) + for _, repl := range replicas.Descriptors() { + mockReplica.inactive[repl.ReplicaID] = struct{}{} + } + + case "progress": + mockReplica.progress = progress + + case "applied": + // Parse applied=/. + mockReplica.applied = parseLogPosition(t, arg.Vals[0]) + + case "connected": + mockReplica.connected = connected + + default: + t.Fatalf("unknown: %s", arg.Key) + } + } + return "" + + case "integration": + var op string + d.ScanArgs(t, "op", &op) + switch op { + case "became-leader": + integration.onBecameLeader(ctx) + case "became-follower": + integration.onBecameFollower(ctx) + case "desc-changed": + integration.onDescChanged(ctx) + case "followers-paused": + integration.onFollowersPaused(ctx) + case "replica-destroyed": + integration.onReplicaDestroyed(ctx) + case "proposal-quota-updated": + integration.onProposalQuotaUpdated(ctx) + default: + t.Fatalf("unknown op: %s", op) + } + return logger.output() + + default: + return "unknown command" + } + }) + }, + ) +} + +type mockReplicaForFlowControl struct { + t *testing.T + rangeID roachpb.RangeID + tenantID roachpb.TenantID + replicaID roachpb.ReplicaID + + paused map[roachpb.ReplicaID]struct{} + inactive map[roachpb.ReplicaID]struct{} + progress map[roachpb.ReplicaID]tracker.Progress + connected map[roachpb.StoreID]struct{} + applied kvflowcontrolpb.RaftLogPosition + descriptor *roachpb.RangeDescriptor +} + +var _ replicaForFlowControl = &mockReplicaForFlowControl{} + +func newMockReplicaForFlowControl( + t *testing.T, rangeID roachpb.RangeID, tenantID roachpb.TenantID, replicaID roachpb.ReplicaID, +) *mockReplicaForFlowControl { + repl := &mockReplicaForFlowControl{ + t: t, + rangeID: rangeID, + tenantID: tenantID, + replicaID: replicaID, + + paused: make(map[roachpb.ReplicaID]struct{}), + inactive: make(map[roachpb.ReplicaID]struct{}), + progress: make(map[roachpb.ReplicaID]tracker.Progress), + } + repl.descriptor = roachpb.NewRangeDescriptor( + rangeID, roachpb.RKeyMin, roachpb.RKeyMax, + roachpb.MakeReplicaSet([]roachpb.ReplicaDescriptor{ + repl.getReplicaDescriptor(), + }), + ) + return repl +} + +func (m *mockReplicaForFlowControl) assertLocked() {} + +func (m *mockReplicaForFlowControl) annotateCtx(ctx context.Context) context.Context { + return ctx +} + +func (m *mockReplicaForFlowControl) getTenantID() roachpb.TenantID { + return m.tenantID +} + +func (m *mockReplicaForFlowControl) getReplicaID() roachpb.ReplicaID { + return m.replicaID +} + +func (m *mockReplicaForFlowControl) getRangeID() roachpb.RangeID { + return m.rangeID +} + +func (m *mockReplicaForFlowControl) getDescriptor() *roachpb.RangeDescriptor { + return m.descriptor +} + +func (m *mockReplicaForFlowControl) getPausedFollowers() map[roachpb.ReplicaID]struct{} { + return m.paused +} + +func (m *mockReplicaForFlowControl) isFollowerLive( + ctx context.Context, replID roachpb.ReplicaID, +) bool { + _, inactive := m.inactive[replID] + return !inactive +} + +func (m *mockReplicaForFlowControl) isRaftTransportConnectedTo(storeID roachpb.StoreID) bool { + _, found := m.connected[storeID] + return found +} + +func (m *mockReplicaForFlowControl) getAppliedLogPosition() kvflowcontrolpb.RaftLogPosition { + return m.applied +} + +func (m *mockReplicaForFlowControl) withReplicaProgress( + f func(roachpb.ReplicaID, tracker.Progress), +) { + for replID, progress := range m.progress { + f(replID, progress) + } +} + +func (m *mockReplicaForFlowControl) isScratchRange() bool { + return false +} + +func (m *mockReplicaForFlowControl) getReplicaDescriptor() roachpb.ReplicaDescriptor { + return roachpb.ReplicaDescriptor{ + ReplicaID: m.replicaID, + NodeID: roachpb.NodeID(m.replicaID), + StoreID: roachpb.StoreID(m.replicaID), + Type: roachpb.VOTER_FULL, + } +} + +type mockFlowHandleFactory struct { + t *testing.T + logger *testLogger +} + +var _ kvflowcontrol.HandleFactory = &mockFlowHandleFactory{} + +func newMockFlowHandleFactory(t *testing.T, logger *testLogger) *mockFlowHandleFactory { + return &mockFlowHandleFactory{ + t: t, + logger: logger, + } +} + +// NewHandle implements the kvflowcontrol.HandleFactory interface. +func (m *mockFlowHandleFactory) NewHandle( + rangeID roachpb.RangeID, tenantID roachpb.TenantID, +) kvflowcontrol.Handle { + return newMockFlowHandle(m.t, rangeID, tenantID, m.logger) +} + +type mockFlowHandle struct { + t *testing.T + logger *testLogger + rangeID roachpb.RangeID + tenantID roachpb.TenantID +} + +var _ kvflowcontrol.Handle = &mockFlowHandle{} + +func newMockFlowHandle( + t *testing.T, rangeID roachpb.RangeID, tenantID roachpb.TenantID, logger *testLogger, +) *mockFlowHandle { + m := &mockFlowHandle{ + t: t, + logger: logger, + rangeID: rangeID, + tenantID: tenantID, + } + m.logger.log(fmt.Sprintf("initialized flow control handle for r%s/t%d", + m.rangeID, m.tenantID.ToUint64())) + return m +} + +func (m *mockFlowHandle) Admit( + ctx context.Context, pri admissionpb.WorkPriority, ct time.Time, +) error { + m.t.Fatal("unimplemented") + return nil +} + +func (m *mockFlowHandle) DeductTokensFor( + ctx context.Context, + pri admissionpb.WorkPriority, + pos kvflowcontrolpb.RaftLogPosition, + tokens kvflowcontrol.Tokens, +) { + m.t.Fatal("unimplemented") +} + +func (m *mockFlowHandle) ReturnTokensUpto( + ctx context.Context, + pri admissionpb.WorkPriority, + pos kvflowcontrolpb.RaftLogPosition, + stream kvflowcontrol.Stream, +) { + m.t.Fatal("unimplemented") +} + +func (m *mockFlowHandle) ConnectStream( + ctx context.Context, pos kvflowcontrolpb.RaftLogPosition, stream kvflowcontrol.Stream, +) { + m.logger.log(fmt.Sprintf("connected to replication stream %s starting at %s", stream, pos)) +} + +func (m *mockFlowHandle) DisconnectStream(ctx context.Context, stream kvflowcontrol.Stream) { + m.logger.log(fmt.Sprintf("disconnected from replication stream %s", stream)) +} + +func (m *mockFlowHandle) ResetStreams(ctx context.Context) { + m.logger.log("reset all replication streams") +} + +func (m *mockFlowHandle) Inspect(ctx context.Context) kvflowinspectpb.Handle { + m.t.Fatal("unimplemented") + return kvflowinspectpb.Handle{} +} + +func (m *mockFlowHandle) Close(ctx context.Context) { + m.logger.log(fmt.Sprintf("closed flow control handle for r%s/t%d", + m.rangeID, m.tenantID.ToUint64())) +} + +func parseLogPosition(t *testing.T, input string) kvflowcontrolpb.RaftLogPosition { + inner := strings.Split(input, "/") + require.Len(t, inner, 2) + term, err := strconv.Atoi(inner[0]) + require.NoError(t, err) + index, err := strconv.Atoi(inner[1]) + require.NoError(t, err) + return kvflowcontrolpb.RaftLogPosition{ + Term: uint64(term), + Index: uint64(index), + } +} + +type testLogger struct { + buffer *strings.Builder +} + +func newTestLogger() *testLogger { + return &testLogger{ + buffer: &strings.Builder{}, + } +} + +func (l *testLogger) log(s string) { + l.buffer.WriteString(fmt.Sprintf("%s\n", s)) +} + +func (l *testLogger) output() string { + output := l.buffer.String() + l.buffer.Reset() + return output +} diff --git a/pkg/kv/kvserver/kvflowcontrol/doc.go b/pkg/kv/kvserver/kvflowcontrol/doc.go index 28ccfa514c17..c48e23e7ebfb 100644 --- a/pkg/kv/kvserver/kvflowcontrol/doc.go +++ b/pkg/kv/kvserver/kvflowcontrol/doc.go @@ -280,6 +280,10 @@ package kvflowcontrol // re-acquire on another attempt), or that it simply free up all tokens for // this replication stream. This could also apply to dropped messages on the // sender side queue, where we just free up all held tokens. +// - In addition to relying on higher log positions getting admitted or +// proposals getting abandoned to return deducted tokens when proposals are +// dropped from full send queues, we could also intercept every dropped +// proposal and return whatever tokens were deducted for it specifically. // - If messages containing the entry gets dropped from the raft transport // receive queue, we rely on raft re-transmitting said entries. Similar to // above, we're relying on the logical admission of some entry with log diff --git a/pkg/kv/kvserver/kvflowcontrol/kvflowcontrol.go b/pkg/kv/kvserver/kvflowcontrol/kvflowcontrol.go index b46fbf1a95e2..7e8c5d9fdb32 100644 --- a/pkg/kv/kvserver/kvflowcontrol/kvflowcontrol.go +++ b/pkg/kv/kvserver/kvflowcontrol/kvflowcontrol.go @@ -225,9 +225,25 @@ type Handle interface { // typically held on replicas initiating replication traffic, so on a given node // they're uniquely identified by their range ID. type Handles interface { + // Lookup the kvflowcontrol.Handle for the specific range (or rather, the + // replica of the specific range that's locally held). Lookup(roachpb.RangeID) (Handle, bool) + // ResetStreams resets all underlying streams for all underlying + // kvflowcontrol.Handles, i.e. disconnect and reconnect each one. It + // effectively unblocks all requests waiting in Admit(). ResetStreams(ctx context.Context) + // Inspect returns the set of ranges that have an embedded + // kvflowcontrol.Handle. It's used to power /inspectz. Inspect() []roachpb.RangeID + // TODO(irfansharif): When fixing I1 and I2 from kvflowcontrol/node.go, + // we'll want to disconnect all streams for a specific node. Expose + // something like the following to disconnect all replication streams bound + // to the specific node. Back it by a reverse-lookup dictionary, keyed by + // StoreID (or NodeID, if we also maintain a mapping between NodeID -> + // []StoreID) and the set of Handles currently connected to it. Do it as + // part of #95563. + // + // Iterate(roachpb.StoreID, func(context.Context, Handle, Stream)) } // HandleFactory is used to construct new Handles. diff --git a/pkg/kv/kvserver/testdata/flow_control_replica_integration/desc_changed b/pkg/kv/kvserver/testdata/flow_control_replica_integration/desc_changed new file mode 100644 index 000000000000..a8c16289dba3 --- /dev/null +++ b/pkg/kv/kvserver/testdata/flow_control_replica_integration/desc_changed @@ -0,0 +1,63 @@ +# Observe how the integration layer deals with change raft group members. Start +# off with a double replicated range r1/t1, with replicas on n1/s1 and n2/s2 +# (with replica IDs 1-2 respectively). +init tenant=t1 range=r1 replid=1 +---- + +state applied=1/10 descriptor=(1,2) connected=(1,2,3,4) +---- + +# Set up replid=1 (declared in init above) to be the raft leader. It should +# connect to both replication streams. +integration op=became-leader +---- +initialized flow control handle for r1/t1 +connected to replication stream t1/s1 starting at log-position=1/10 +connected to replication stream t1/s2 starting at log-position=1/10 + +# Add replid=3. New replicas start off without a match index, in StateProbe, as +# inactive and paused according to raft (this different from the +# last-updated-derived activity and CRDB-level follower pausing). This is +# represented using '3@0:probe:!active:paused' below. As a result, we don't +# immediately connect to the t1/s3 stream. +state applied=1/11 descriptor=(1,2,3) progress=(1@11:replicate:active:!paused, 2@11:replicate:active:!paused, 3@0:probe:!active:paused) +---- + +integration op=desc-changed +---- + +integration op=proposal-quota-updated +---- + +# replid=3 soon enters StateProbe with an up-to-date match index. It's also +# considered RecentlyActive according to raft, and !MsgAppFlowPaused. This is +# represented using '3@11:replicate:active:!paused' below. As a result, we +# connect to the t1/s3 stream. +state applied=1/11 descriptor=(1,2,3) progress=(1@11:replicate:active:!paused, 2@11:replicate:active:!paused, 3@11:replicate:active:!paused) +---- + +integration op=proposal-quota-updated +---- +connected to replication stream t1/s3 starting at log-position=1/11 + +# The integration layer maintains internal state about the set of raft group +# members we're already connected to. Informing it of a descriptor change when +# there is none simply simply no-ops. +integration op=desc-changed +---- + +# Remove replid=2 and add replid=4 (do so immediately using +# state==StateReplicate && RecentlyActive && !MsgAppFlowPaused). We should +# disconnect from t1/s2 and connect out to t1/s4 at the current applied index. +state applied=1/12 descriptor=(1,3,4) progress=(1@12:replicate:active:!paused, 3@12:replicate:active:!paused, 4@12:replicate:active:!paused) +---- + +integration op=desc-changed +---- +disconnected from replication stream t1/s2 + +integration op=proposal-quota-updated +---- +connected to replication stream t1/s4 starting at log-position=1/12 + +# vim:ft=sh diff --git a/pkg/kv/kvserver/testdata/flow_control_replica_integration/inactive_followers b/pkg/kv/kvserver/testdata/flow_control_replica_integration/inactive_followers new file mode 100644 index 000000000000..be95ae1b6911 --- /dev/null +++ b/pkg/kv/kvserver/testdata/flow_control_replica_integration/inactive_followers @@ -0,0 +1,43 @@ +# Observe how the integration layer deals with inactive followers. Start off +# with a triply replicated range r1/t1, with replicas on n1/s1, n2/s2, and +# n3/s3 (with replica IDs 1-3 respectively). +init tenant=t1 range=r1 replid=1 +---- + +state descriptor=(1,2,3) applied=1/10 connected=(1,2,3) +---- + +# Set up replid=1 (declared in init above) to be the raft leader. It should +# connect to all three replication streams. +integration op=became-leader +---- +initialized flow control handle for r1/t1 +connected to replication stream t1/s1 starting at log-position=1/10 +connected to replication stream t1/s2 starting at log-position=1/10 +connected to replication stream t1/s3 starting at log-position=1/10 + +# Mark replid=2 as inactive. Observe that we disconnect the stream to t1/s2. +state descriptor=(1,2,3) inactive=(2) +---- + +integration op=proposal-quota-updated +---- +disconnected from replication stream t1/s2 + +# The integration layer maintains internal state about the set of inactive +# followers we've already disconnected from. Informing it of inactive followers +# again simply no-ops. +integration op=proposal-quota-updated +---- + +# Mark replid=2 as active and replid=3 as inactive. We should see us disconnect +# from t1/s3 and reconnect to t1/s2 using our current applied state. +state descriptor=(1,2,3) inactive=(3) applied=1/11 +---- + +integration op=proposal-quota-updated +---- +disconnected from replication stream t1/s3 +connected to replication stream t1/s2 starting at log-position=1/11 + +# vim:ft=sh diff --git a/pkg/kv/kvserver/testdata/flow_control_replica_integration/paused_followers b/pkg/kv/kvserver/testdata/flow_control_replica_integration/paused_followers new file mode 100644 index 000000000000..21f90b8d29f6 --- /dev/null +++ b/pkg/kv/kvserver/testdata/flow_control_replica_integration/paused_followers @@ -0,0 +1,43 @@ +# Observe how the integration layer deals with paused followers. Start off with +# a triply replicated range r1/t1, with replicas on n1/s1, n2/s2, and n3/s3 +# (with replica IDs 1-3 respectively). +init tenant=t1 range=r1 replid=1 +---- + +state descriptor=(1,2,3) applied=1/10 connected=(1,2,3) +---- + +# Set up replid=1 (declared in init above) to be the raft leader. It should +# connect to all three replication streams. +integration op=became-leader +---- +initialized flow control handle for r1/t1 +connected to replication stream t1/s1 starting at log-position=1/10 +connected to replication stream t1/s2 starting at log-position=1/10 +connected to replication stream t1/s3 starting at log-position=1/10 + +# Pause replid=2. Observe that we disconnect the stream to t1/s2. +state descriptor=(1,2,3) paused=(2) +---- + +integration op=followers-paused +---- +disconnected from replication stream t1/s2 + +# The integration layer maintains internal state about the set of paused +# followers we've already disconnected from. Informing it of paused followers +# again simply no-ops. +integration op=followers-paused +---- + +# Unpause replid=2 and pause replid=3. We should see us disconnect from t1/s3 +# and reconnect to t1/s2 using our current applied state. +state descriptor=(1,2,3) paused=(3) applied=1/11 +---- + +integration op=followers-paused +---- +disconnected from replication stream t1/s3 +connected to replication stream t1/s2 starting at log-position=1/11 + +# vim:ft=sh diff --git a/pkg/kv/kvserver/testdata/flow_control_replica_integration/replica_destroyed b/pkg/kv/kvserver/testdata/flow_control_replica_integration/replica_destroyed new file mode 100644 index 000000000000..4e835803cac5 --- /dev/null +++ b/pkg/kv/kvserver/testdata/flow_control_replica_integration/replica_destroyed @@ -0,0 +1,31 @@ +# Observe how the integration layer deals with replicas being destroyed. +# followers. Typically replicas being destroyed first lose raft leadership +# and/or observe themselves being removed from the range descriptor. For those +# variants, see replica_raft_leadership and replica_removed_self. +# +# Start off with a triply replicated range r1/t1, with replicas on n1/s1, +# n2/s2, and n3/s3 (with replica IDs 1-3 respectively). +init tenant=t1 range=r1 replid=1 +---- + +state descriptor=(1,2,3) applied=1/10 connected=(1,2,3) +---- + +# Set up replid=1 (declared in init above) to be the raft leader. It should +# connect to all three replication streams. +integration op=became-leader +---- +initialized flow control handle for r1/t1 +connected to replication stream t1/s1 starting at log-position=1/10 +connected to replication stream t1/s2 starting at log-position=1/10 +connected to replication stream t1/s3 starting at log-position=1/10 + +# Destroy the replica without first losing raft leadership (we didn't use +# 'integration op=became-follower') or observing itself being removed from the +# range ('integration op=desc-changed'). We should just close the handle, +# disconnecting all underlying streams. +integration op=replica-destroyed +---- +closed flow control handle for r1/t1 + +# vim:ft=sh diff --git a/pkg/kv/kvserver/testdata/flow_control_replica_integration/replica_raft_leadership b/pkg/kv/kvserver/testdata/flow_control_replica_integration/replica_raft_leadership new file mode 100644 index 000000000000..ab7a5dfc6a0f --- /dev/null +++ b/pkg/kv/kvserver/testdata/flow_control_replica_integration/replica_raft_leadership @@ -0,0 +1,75 @@ +# Observe how the integration layer deals with losing/regaining raft +# leadership. Start off with a triply replicated range r1/t1, with replicas on +# n1/s1, n2/s2, and n3/s3 (with replica IDs 1-3 respectively). +init tenant=t1 range=r1 replid=1 +---- + +state descriptor=(1,2,3) applied=1/10 connected=(1,2,3,4,5) +---- + +# Set up replid=1 (declared in init above) to be the raft leader. It should +# connect to all three replication streams. None of them are paused or +# inactive. We haven't set up raft progress for any of them here, so we don't +# filter out streams depending on it, but we'll test that further below. +integration op=became-leader +---- +initialized flow control handle for r1/t1 +connected to replication stream t1/s1 starting at log-position=1/10 +connected to replication stream t1/s2 starting at log-position=1/10 +connected to replication stream t1/s3 starting at log-position=1/10 + +# If it loses raft leadership, it should close the underlying handle and all +# connected replication streams. +integration op=became-follower +---- +closed flow control handle for r1/t1 + +# state applied=1/12 descriptor=(1,2,3,4,5) paused=(2) inactive=(3) + +# Modify the descriptor. When re-acquiring leadership, we should connect to all +# current members of the raft group. +state applied=1/12 descriptor=(1,2,4) +---- + +integration op=became-leader +---- +initialized flow control handle for r1/t1 +connected to replication stream t1/s1 starting at log-position=1/12 +connected to replication stream t1/s2 starting at log-position=1/12 +connected to replication stream t1/s4 starting at log-position=1/12 + +integration op=became-follower +---- +closed flow control handle for r1/t1 + +# Try again, but this time with a paused and another inactive follower. We +# shouldn't connect to those streams. +state applied=1/14 descriptor=(1,2,4,5) paused=(4) inactive=(5) +---- + +integration op=became-leader +---- +initialized flow control handle for r1/t1 +connected to replication stream t1/s1 starting at log-position=1/14 +connected to replication stream t1/s2 starting at log-position=1/14 + +integration op=became-follower +---- +closed flow control handle for r1/t1 + +# Try again, but this time with: +# - replid=4 waiting for a snapshot (it's stuck at index=14, we could've +# truncated our log ahead of it), and +# - replid=5 just being recently added to the raft group, where we don't know +# its match index just yet. +# When acquiring raft leadership, we shouldn't connect to t1/s4 and t1/s5. +state applied=1/16 descriptor=(1,2,4,5) progress=(1@16:replicate:active:!paused, 2@16:replicate:active:!paused, 4@14:snapshot:!active:paused, 5@0:probe:!active:paused) +---- + +integration op=became-leader +---- +initialized flow control handle for r1/t1 +connected to replication stream t1/s1 starting at log-position=1/16 +connected to replication stream t1/s2 starting at log-position=1/16 + +# vim:ft=sh diff --git a/pkg/kv/kvserver/testdata/flow_control_replica_integration/replica_removed_self b/pkg/kv/kvserver/testdata/flow_control_replica_integration/replica_removed_self new file mode 100644 index 000000000000..7ccfe9ffd8de --- /dev/null +++ b/pkg/kv/kvserver/testdata/flow_control_replica_integration/replica_removed_self @@ -0,0 +1,29 @@ +# Observe how the integration layer deals with replicas observing themselves +# being removed from the raft group. +# +# Start off with a triply replicated range r1/t1, with replicas on n1/s1, +# n2/s2, and n3/s3 (with replica IDs 1-3 respectively). +init tenant=t1 range=r1 replid=1 +---- + +state descriptor=(1,2,3) applied=1/10 connected=(1,2,3) +---- + +# Set up replid=1 (declared in init above) to be the raft leader. It should +# connect to all three replication streams. +integration op=became-leader +---- +initialized flow control handle for r1/t1 +connected to replication stream t1/s1 starting at log-position=1/10 +connected to replication stream t1/s2 starting at log-position=1/10 +connected to replication stream t1/s3 starting at log-position=1/10 + +# Remove replid=1 from the descriptor. Observe that the handle just gets closed. +state descriptor=(2,3,4) +---- + +integration op=desc-changed +---- +closed flow control handle for r1/t1 + +# vim:ft=sh From 433e20be95a1370fd16a9e383b8d012a96e197c9 Mon Sep 17 00:00:00 2001 From: irfan sharif Date: Mon, 10 Apr 2023 12:33:59 -0400 Subject: [PATCH 05/12] kvflowcontrol: add tests for Inspect() APIs Release note: None --- .../kvflowcontroller/BUILD.bazel | 3 +- .../kvflowcontroller/kvflowcontroller.go | 11 + ...tment_test.go => kvflowcontroller_test.go} | 74 ++++++ .../kvflowcontrol/kvflowhandle/BUILD.bazel | 4 + .../kvflowhandle/kvflowhandle_test.go | 76 ++++++ .../kvflowhandle/testdata/handle_inspect | 228 ++++++++++++++++++ .../kvflowtokentracker/testdata/overview | 21 ++ .../kvflowtokentracker/tracker.go | 10 + .../kvflowtokentracker/tracker_test.go | 11 + 9 files changed, 437 insertions(+), 1 deletion(-) rename pkg/kv/kvserver/kvflowcontrol/kvflowcontroller/{kvflowcontrol_token_adjustment_test.go => kvflowcontroller_test.go} (62%) create mode 100644 pkg/kv/kvserver/kvflowcontrol/kvflowhandle/testdata/handle_inspect diff --git a/pkg/kv/kvserver/kvflowcontrol/kvflowcontroller/BUILD.bazel b/pkg/kv/kvserver/kvflowcontrol/kvflowcontroller/BUILD.bazel index 7b8a8b4ca957..8e98b3cb3c31 100644 --- a/pkg/kv/kvserver/kvflowcontrol/kvflowcontroller/BUILD.bazel +++ b/pkg/kv/kvserver/kvflowcontrol/kvflowcontroller/BUILD.bazel @@ -26,12 +26,13 @@ go_library( go_test( name = "kvflowcontroller_test", - srcs = ["kvflowcontrol_token_adjustment_test.go"], + srcs = ["kvflowcontroller_test.go"], args = ["-test.timeout=295s"], data = glob(["testdata/**"]), embed = [":kvflowcontroller"], deps = [ "//pkg/kv/kvserver/kvflowcontrol", + "//pkg/kv/kvserver/kvflowcontrol/kvflowinspectpb", "//pkg/roachpb", "//pkg/settings/cluster", "//pkg/util/admission/admissionpb", diff --git a/pkg/kv/kvserver/kvflowcontrol/kvflowcontroller/kvflowcontroller.go b/pkg/kv/kvserver/kvflowcontrol/kvflowcontroller/kvflowcontroller.go index 664ce48da991..9ded7387c1e2 100644 --- a/pkg/kv/kvserver/kvflowcontrol/kvflowcontroller/kvflowcontroller.go +++ b/pkg/kv/kvserver/kvflowcontrol/kvflowcontroller/kvflowcontroller.go @@ -13,6 +13,7 @@ package kvflowcontroller import ( "context" "fmt" + "sort" "time" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol" @@ -27,6 +28,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/syncutil" ) +// regularTokensPerStream determines the flow tokens available for regular work +// on a per-stream basis. var regularTokensPerStream = settings.RegisterByteSizeSetting( settings.SystemOnly, "kvadmission.flow_controller.regular_tokens_per_stream", @@ -35,6 +38,8 @@ var regularTokensPerStream = settings.RegisterByteSizeSetting( validateTokenRange, ) +// elasticTokensPerStream determines the flow tokens available for elastic work +// on a per-stream basis. var elasticTokensPerStream = settings.RegisterByteSizeSetting( settings.SystemOnly, "kvadmission.flow_controller.elastic_tokens_per_stream", @@ -245,6 +250,12 @@ func (c *Controller) Inspect(ctx context.Context) []kvflowinspectpb.Stream { AvailableElasticTokens: int64(b.tokens[elastic]), }) } + sort.Slice(streams, func(i, j int) bool { // for determinism + if streams[i].TenantID != streams[j].TenantID { + return streams[i].TenantID.ToUint64() < streams[j].TenantID.ToUint64() + } + return streams[i].StoreID < streams[j].StoreID + }) return streams } diff --git a/pkg/kv/kvserver/kvflowcontrol/kvflowcontroller/kvflowcontrol_token_adjustment_test.go b/pkg/kv/kvserver/kvflowcontrol/kvflowcontroller/kvflowcontroller_test.go similarity index 62% rename from pkg/kv/kvserver/kvflowcontrol/kvflowcontroller/kvflowcontrol_token_adjustment_test.go rename to pkg/kv/kvserver/kvflowcontrol/kvflowcontroller/kvflowcontroller_test.go index ce00d7019b0d..1fdcf876aefd 100644 --- a/pkg/kv/kvserver/kvflowcontrol/kvflowcontroller/kvflowcontrol_token_adjustment_test.go +++ b/pkg/kv/kvserver/kvflowcontrol/kvflowcontroller/kvflowcontroller_test.go @@ -15,8 +15,10 @@ import ( "fmt" "strings" "testing" + "time" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol/kvflowinspectpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/util/admission/admissionpb" @@ -156,3 +158,75 @@ func (h adjustment) String() string { comment, ) } + +// TestInspectController tests the Inspect() API. +func TestInspectController(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + ctx := context.Background() + + makeStream := func(id uint64) kvflowcontrol.Stream { + return kvflowcontrol.Stream{ + TenantID: roachpb.MustMakeTenantID(id), + StoreID: roachpb.StoreID(id), + } + } + makeInspectStream := func(id uint64, availableElastic, availableRegular int64) kvflowinspectpb.Stream { + return kvflowinspectpb.Stream{ + TenantID: roachpb.MustMakeTenantID(id), + StoreID: roachpb.StoreID(id), + AvailableElasticTokens: availableElastic, + AvailableRegularTokens: availableRegular, + } + } + makeConnectedStream := func(id uint64) kvflowcontrol.ConnectedStream { + return &mockConnectedStream{ + stream: makeStream(id), + } + } + + st := cluster.MakeTestingClusterSettings() + elasticTokensPerStream.Override(ctx, &st.SV, 8<<20 /* 8 MiB */) + regularTokensPerStream.Override(ctx, &st.SV, 16<<20 /* 16 MiB */) + controller := New(metric.NewRegistry(), st, hlc.NewClockForTesting(nil)) + + // No streams connected -- inspect state should be empty. + require.Len(t, controller.Inspect(ctx), 0) + + // Set up a single connected stream, s1/t1, and ensure it shows up in the + // Inspect() state. + require.NoError(t, controller.Admit(ctx, admissionpb.NormalPri, time.Time{}, makeConnectedStream(1))) + require.Len(t, controller.Inspect(ctx), 1) + require.Equal(t, controller.Inspect(ctx)[0], + makeInspectStream(1, 8<<20 /* 8MiB */, 16<<20 /* 16 MiB */)) + + // Deduct some {regular,elastic} tokens from s1/t1 and verify that Inspect() + // renders the state correctly. + controller.DeductTokens(ctx, admissionpb.NormalPri, kvflowcontrol.Tokens(1<<20 /* 1 MiB */), makeStream(1)) + controller.DeductTokens(ctx, admissionpb.BulkNormalPri, kvflowcontrol.Tokens(2<<20 /* 2 MiB */), makeStream(1)) + require.Len(t, controller.Inspect(ctx), 1) + require.Equal(t, controller.Inspect(ctx)[0], + makeInspectStream(1, 5<<20 /* 8 MiB - 2 MiB - 1 MiB = 5 MiB */, 15<<20 /* 16 MiB - 1 MiB = 15 MiB */)) + + // Connect another stream, s1/s2, and ensure it shows up in the Inspect() + // state. + require.NoError(t, controller.Admit(ctx, admissionpb.BulkNormalPri, time.Time{}, makeConnectedStream(2))) + require.Len(t, controller.Inspect(ctx), 2) + require.Equal(t, controller.Inspect(ctx)[1], + makeInspectStream(2, 8<<20 /* 8MiB */, 16<<20 /* 16 MiB */)) +} + +type mockConnectedStream struct { + stream kvflowcontrol.Stream +} + +var _ kvflowcontrol.ConnectedStream = &mockConnectedStream{} + +func (m *mockConnectedStream) Stream() kvflowcontrol.Stream { + return m.stream +} + +func (m *mockConnectedStream) Disconnected() <-chan struct{} { + return nil +} diff --git a/pkg/kv/kvserver/kvflowcontrol/kvflowhandle/BUILD.bazel b/pkg/kv/kvserver/kvflowcontrol/kvflowhandle/BUILD.bazel index d54387329f89..9080e68a0686 100644 --- a/pkg/kv/kvserver/kvflowcontrol/kvflowhandle/BUILD.bazel +++ b/pkg/kv/kvserver/kvflowcontrol/kvflowhandle/BUILD.bazel @@ -30,6 +30,7 @@ go_test( name = "kvflowhandle_test", srcs = ["kvflowhandle_test.go"], args = ["-test.timeout=295s"], + data = glob(["testdata/**"]), deps = [ ":kvflowhandle", "//pkg/kv/kvserver/kvflowcontrol", @@ -37,11 +38,14 @@ go_test( "//pkg/kv/kvserver/kvflowcontrol/kvflowcontrolpb", "//pkg/roachpb", "//pkg/settings/cluster", + "//pkg/testutils/datapathutils", + "//pkg/testutils/echotest", "//pkg/util/admission/admissionpb", "//pkg/util/hlc", "//pkg/util/leaktest", "//pkg/util/log", "//pkg/util/metric", + "@com_github_gogo_protobuf//jsonpb", "@com_github_stretchr_testify//require", ], ) diff --git a/pkg/kv/kvserver/kvflowcontrol/kvflowhandle/kvflowhandle_test.go b/pkg/kv/kvserver/kvflowcontrol/kvflowhandle/kvflowhandle_test.go index f9a7dda82cc9..9dbffa17d400 100644 --- a/pkg/kv/kvserver/kvflowcontrol/kvflowhandle/kvflowhandle_test.go +++ b/pkg/kv/kvserver/kvflowcontrol/kvflowhandle/kvflowhandle_test.go @@ -12,6 +12,8 @@ package kvflowhandle_test import ( "context" + "fmt" + "strings" "testing" "time" @@ -21,11 +23,14 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol/kvflowhandle" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/testutils/datapathutils" + "github.com/cockroachdb/cockroach/pkg/testutils/echotest" "github.com/cockroachdb/cockroach/pkg/util/admission/admissionpb" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/metric" + "github.com/gogo/protobuf/jsonpb" "github.com/stretchr/testify/require" ) @@ -125,6 +130,7 @@ func TestHandleAdmit(t *testing.T) { } } +// TestFlowControlMode tests the behavior of kvadmission.flow_control.mode. func TestFlowControlMode(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) @@ -221,5 +227,75 @@ func TestFlowControlMode(t *testing.T) { } }) } +} + +// TestInspectHandle tests the Inspect() API. +func TestInspectHandle(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + ctx := context.Background() + registry := metric.NewRegistry() + clock := hlc.NewClockForTesting(nil) + st := cluster.MakeTestingClusterSettings() + kvflowcontrol.Enabled.Override(ctx, &st.SV, true) + kvflowcontrol.Mode.Override(ctx, &st.SV, int64(kvflowcontrol.ApplyToAll)) + + pos := func(d uint64) kvflowcontrolpb.RaftLogPosition { + return kvflowcontrolpb.RaftLogPosition{Term: 1, Index: d} + } + stream := func(i uint64) kvflowcontrol.Stream { + return kvflowcontrol.Stream{ + TenantID: roachpb.MustMakeTenantID(1), + StoreID: roachpb.StoreID(i), + } + } + + controller := kvflowcontroller.New(registry, st, clock) + handle := kvflowhandle.New( + controller, + kvflowhandle.NewMetrics(registry), + clock, + roachpb.RangeID(1), + roachpb.SystemTenantID, + ) + marshaller := jsonpb.Marshaler{ + Indent: " ", + EmitDefaults: true, + OrigName: true, + } + + var buf strings.Builder + defer func() { echotest.Require(t, buf.String(), datapathutils.TestDataPath(t, "handle_inspect")) }() + + record := func(header string) { + if buf.Len() > 0 { + buf.WriteString("\n\n") + } + state := handle.Inspect(ctx) + marshaled, err := marshaller.MarshalToString(&state) + require.NoError(t, err) + buf.WriteString(fmt.Sprintf("# %s\n", header)) + buf.WriteString(marshaled) + } + record("No connected streams.") + + handle.ConnectStream(ctx, pos(42), stream(1)) + record("Single connected stream with no tracked deductions.") + + handle.DeductTokensFor(ctx, admissionpb.NormalPri, pos(43), kvflowcontrol.Tokens(1<<20 /* 1 MiB */)) + handle.DeductTokensFor(ctx, admissionpb.BulkNormalPri, pos(44), kvflowcontrol.Tokens(1<<20 /* 1 MiB */)) + record("Single connected stream with one 1MiB tracked deduction per work class.") + + handle.ReturnTokensUpto(ctx, admissionpb.NormalPri, pos(43), stream(1)) + handle.ConnectStream(ctx, pos(45), stream(2)) + handle.ConnectStream(ctx, pos(46), stream(3)) + handle.DeductTokensFor(ctx, admissionpb.BulkNormalPri, pos(47), kvflowcontrol.Tokens(1<<20 /* 1 MiB */)) + record("Triply connected stream with 2MiB, 1MiB, and 1MiB tracked elastic deductions respectively.") + + handle.DisconnectStream(ctx, stream(2)) + record("Doubly connected stream with 2MiB and 1MiB tracked elastic deductions respectively.") + handle.ResetStreams(ctx) + record("Doubly connected stream with no tracked deductions.") } diff --git a/pkg/kv/kvserver/kvflowcontrol/kvflowhandle/testdata/handle_inspect b/pkg/kv/kvserver/kvflowcontrol/kvflowhandle/testdata/handle_inspect new file mode 100644 index 000000000000..5e8b4225726e --- /dev/null +++ b/pkg/kv/kvserver/kvflowcontrol/kvflowhandle/testdata/handle_inspect @@ -0,0 +1,228 @@ +echo +---- +---- +# No connected streams. +{ + "range_id": "1", + "connected_streams": [ + ] +} + +# Single connected stream with no tracked deductions. +{ + "range_id": "1", + "connected_streams": [ + { + "stream": { + "tenant_id": { + "id": "1" + }, + "store_id": 1, + "available_regular_tokens": "16777216", + "available_elastic_tokens": "8388608" + }, + "tracked_deductions": [ + ] + } + ] +} + +# Single connected stream with one 1MiB tracked deduction per work class. +{ + "range_id": "1", + "connected_streams": [ + { + "stream": { + "tenant_id": { + "id": "1" + }, + "store_id": 1, + "available_regular_tokens": "15728640", + "available_elastic_tokens": "6291456" + }, + "tracked_deductions": [ + { + "priority": -30, + "tokens": "1048576", + "raft_log_position": { + "term": "1", + "index": "44" + } + }, + { + "priority": 0, + "tokens": "1048576", + "raft_log_position": { + "term": "1", + "index": "43" + } + } + ] + } + ] +} + +# Triply connected stream with 2MiB, 1MiB, and 1MiB tracked elastic deductions respectively. +{ + "range_id": "1", + "connected_streams": [ + { + "stream": { + "tenant_id": { + "id": "1" + }, + "store_id": 1, + "available_regular_tokens": "16777216", + "available_elastic_tokens": "6291456" + }, + "tracked_deductions": [ + { + "priority": -30, + "tokens": "1048576", + "raft_log_position": { + "term": "1", + "index": "44" + } + }, + { + "priority": -30, + "tokens": "1048576", + "raft_log_position": { + "term": "1", + "index": "47" + } + } + ] + }, + { + "stream": { + "tenant_id": { + "id": "1" + }, + "store_id": 2, + "available_regular_tokens": "16777216", + "available_elastic_tokens": "7340032" + }, + "tracked_deductions": [ + { + "priority": -30, + "tokens": "1048576", + "raft_log_position": { + "term": "1", + "index": "47" + } + } + ] + }, + { + "stream": { + "tenant_id": { + "id": "1" + }, + "store_id": 3, + "available_regular_tokens": "16777216", + "available_elastic_tokens": "7340032" + }, + "tracked_deductions": [ + { + "priority": -30, + "tokens": "1048576", + "raft_log_position": { + "term": "1", + "index": "47" + } + } + ] + } + ] +} + +# Doubly connected stream with 2MiB and 1MiB tracked elastic deductions respectively. +{ + "range_id": "1", + "connected_streams": [ + { + "stream": { + "tenant_id": { + "id": "1" + }, + "store_id": 1, + "available_regular_tokens": "16777216", + "available_elastic_tokens": "6291456" + }, + "tracked_deductions": [ + { + "priority": -30, + "tokens": "1048576", + "raft_log_position": { + "term": "1", + "index": "44" + } + }, + { + "priority": -30, + "tokens": "1048576", + "raft_log_position": { + "term": "1", + "index": "47" + } + } + ] + }, + { + "stream": { + "tenant_id": { + "id": "1" + }, + "store_id": 3, + "available_regular_tokens": "16777216", + "available_elastic_tokens": "7340032" + }, + "tracked_deductions": [ + { + "priority": -30, + "tokens": "1048576", + "raft_log_position": { + "term": "1", + "index": "47" + } + } + ] + } + ] +} + +# Doubly connected stream with no tracked deductions. +{ + "range_id": "1", + "connected_streams": [ + { + "stream": { + "tenant_id": { + "id": "1" + }, + "store_id": 1, + "available_regular_tokens": "16777216", + "available_elastic_tokens": "8388608" + }, + "tracked_deductions": [ + ] + }, + { + "stream": { + "tenant_id": { + "id": "1" + }, + "store_id": 3, + "available_regular_tokens": "16777216", + "available_elastic_tokens": "8388608" + }, + "tracked_deductions": [ + ] + } + ] +} +---- +---- + +# vim:ft=conf diff --git a/pkg/kv/kvserver/kvflowcontrol/kvflowtokentracker/testdata/overview b/pkg/kv/kvserver/kvflowcontrol/kvflowtokentracker/testdata/overview index b7bc87f839e3..0a602a194151 100644 --- a/pkg/kv/kvserver/kvflowcontrol/kvflowtokentracker/testdata/overview +++ b/pkg/kv/kvserver/kvflowcontrol/kvflowtokentracker/testdata/overview @@ -32,6 +32,20 @@ pri=normal-pri tokens=1B log-position=4/27 tokens=1B log-position=4/29 +# The Inspect() state should surface the same thing. +inspect +---- +pri=low-pri tokens=1B log-position=4/24 +pri=low-pri tokens=1B log-position=4/26 +pri=low-pri tokens=1B log-position=4/28 +pri=normal-pri tokens=1B log-position=4/20 +pri=normal-pri tokens=1B log-position=4/21 +pri=normal-pri tokens=1B log-position=4/22 +pri=normal-pri tokens=1B log-position=4/23 +pri=normal-pri tokens=1B log-position=4/25 +pri=normal-pri tokens=1B log-position=4/27 +pri=normal-pri tokens=1B log-position=4/29 + # Untrack a subset of normal-pri tokens, up to 4/23. This should get rid of four # tracked tokens. untrack pri=normal-pri up-to-log-position=4/23 @@ -86,4 +100,11 @@ pri=normal-pri tokens=1B log-position=4/27 tokens=1B log-position=4/29 +# The Inspect() state should surface the same thing. +inspect +---- +pri=normal-pri tokens=1B log-position=4/25 +pri=normal-pri tokens=1B log-position=4/27 +pri=normal-pri tokens=1B log-position=4/29 + # vim:ft=sh diff --git a/pkg/kv/kvserver/kvflowcontrol/kvflowtokentracker/tracker.go b/pkg/kv/kvserver/kvflowcontrol/kvflowtokentracker/tracker.go index 3b5a528bd59b..b507ff26eee8 100644 --- a/pkg/kv/kvserver/kvflowcontrol/kvflowtokentracker/tracker.go +++ b/pkg/kv/kvserver/kvflowcontrol/kvflowtokentracker/tracker.go @@ -13,6 +13,7 @@ package kvflowtokentracker import ( "context" "fmt" + "sort" "strings" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol" @@ -201,6 +202,15 @@ func (dt *Tracker) Inspect(ctx context.Context) []kvflowinspectpb.TrackedDeducti }) return true }) + sort.Slice(deductions, func(i, j int) bool { // for determinism + if deductions[i].Priority != deductions[j].Priority { + return deductions[i].Priority < deductions[j].Priority + } + if deductions[i].RaftLogPosition != deductions[j].RaftLogPosition { + return deductions[i].RaftLogPosition.Less(deductions[j].RaftLogPosition) + } + return deductions[i].Tokens < deductions[j].Tokens + }) return deductions } diff --git a/pkg/kv/kvserver/kvflowcontrol/kvflowtokentracker/tracker_test.go b/pkg/kv/kvserver/kvflowcontrol/kvflowtokentracker/tracker_test.go index 58d4ed4063b5..17cf51f5520b 100644 --- a/pkg/kv/kvserver/kvflowcontrol/kvflowtokentracker/tracker_test.go +++ b/pkg/kv/kvserver/kvflowcontrol/kvflowtokentracker/tracker_test.go @@ -89,6 +89,17 @@ func TestTracker(t *testing.T) { require.NotNilf(t, tracker, "uninitialized tracker (did you use 'init'?)") return tracker.TestingPrintIter() + case "inspect": + var buf strings.Builder + for _, tracked := range tracker.Inspect(ctx) { + buf.WriteString(fmt.Sprintf("pri=%s tokens=%s %s\n", + admissionpb.WorkPriority(tracked.Priority), + testingPrintTrimmedTokens(kvflowcontrol.Tokens(tracked.Tokens)), + tracked.RaftLogPosition, + )) + } + return buf.String() + case "untrack": require.NotNilf(t, tracker, "uninitialized tracker (did you use 'init'?)") var priStr, logPositionStr string From 3da5c73e72ab27f93d0589980ee6feab6ed58e7b Mon Sep 17 00:00:00 2001 From: irfan sharif Date: Wed, 26 Apr 2023 03:55:33 -0400 Subject: [PATCH 06/12] kvserver: add end-to-end TestFlowControl* tests TestFlowControl* are end-to-end tests of the kvflowcontrol machinery, replicating + admitting individual writes. They make use of an echotest-backed test harness to make it possible to observe sophisticated KV interactions. We can now write tests that look like this: -- Flow token metrics, before issuing the regular 1MiB replicated -- write. SELECT name, crdb_internal.humanize_bytes(value::INT8) FROM crdb_internal.node_metrics WHERE name LIKE '%kvadmission%tokens%' ORDER BY name ASC; kvadmission.flow_controller.elastic_tokens_available | 0 B kvadmission.flow_controller.elastic_tokens_deducted | 0 B kvadmission.flow_controller.elastic_tokens_returned | 0 B kvadmission.flow_controller.elastic_tokens_unaccounted | 0 B kvadmission.flow_controller.regular_tokens_available | 0 B kvadmission.flow_controller.regular_tokens_deducted | 0 B kvadmission.flow_controller.regular_tokens_returned | 0 B kvadmission.flow_controller.regular_tokens_unaccounted | 0 B -- Flow token metrics from n1 after issuing the regular 1MiB -- replicated write, and it being admitted on n1, n2 and n3. We -- should see 3*1MiB = 3MiB of {regular,elastic} tokens deducted and -- returned, and {8*3=24MiB,16*3=48MiB} of {regular,elastic} tokens -- available. Everything should be accounted for. SELECT name, crdb_internal.humanize_bytes(value::INT8) FROM crdb_internal.node_metrics WHERE name LIKE '%kvadmission%tokens%' ORDER BY name ASC; kvadmission.flow_controller.elastic_tokens_available | 24 MiB kvadmission.flow_controller.elastic_tokens_deducted | 3.0 MiB kvadmission.flow_controller.elastic_tokens_returned | 3.0 MiB kvadmission.flow_controller.elastic_tokens_unaccounted | 0 B kvadmission.flow_controller.regular_tokens_available | 48 MiB kvadmission.flow_controller.regular_tokens_deducted | 3.0 MiB kvadmission.flow_controller.regular_tokens_returned | 3.0 MiB kvadmission.flow_controller.regular_tokens_unaccounted | 0 B ---- ---- Release note: None --- pkg/base/testing_knobs.go | 2 + pkg/keys/spans.go | 4 + pkg/kv/kvserver/BUILD.bazel | 4 + pkg/kv/kvserver/client_raft_helpers_test.go | 18 + .../kvserver/flow_control_integration_test.go | 2120 +++++++++++++++++ pkg/kv/kvserver/flow_control_replica.go | 10 +- .../flow_control_replica_integration.go | 35 +- .../flow_control_replica_integration_test.go | 2 +- pkg/kv/kvserver/flow_control_stores.go | 11 + pkg/kv/kvserver/kvflowcontrol/doc.go | 17 +- .../kvflowhandle/kvflowhandle.go | 12 +- .../kvflowhandle/kvflowhandle_test.go | 3 + .../kvflowsimulator/simulation_test.go | 2 +- .../kvserver/kvflowcontrol/testing_knobs.go | 13 + pkg/kv/kvserver/raft_transport.go | 54 +- pkg/kv/kvserver/replica.go | 15 + pkg/kv/kvserver/replica_init.go | 1 + pkg/kv/kvserver/replica_init_test.go | 2 +- .../testdata/flow_control_integration/basic | 68 + .../blocked_admission | 99 + .../class_prioritization | 67 + .../flow_control_integration/crashed_node | 66 + .../leader_not_leaseholder | 83 + .../flow_control_integration/quiesced_range | 60 + .../flow_control_integration/raft_membership | 112 + .../raft_membership_remove_self | 62 + .../flow_control_integration/raft_snapshot | 192 ++ .../raft_transport_break | 69 + .../raft_transport_culled | 71 + .../flow_control_integration/split_merge | 89 + .../flow_control_integration/transfer_lease | 38 + .../flow_control_integration/unquiesced_range | 63 + pkg/kv/kvserver/testing_knobs.go | 5 + pkg/kv/txn.go | 4 +- pkg/server/server.go | 13 +- pkg/server/testserver.go | 14 +- pkg/sql/catalog/lease/lease.go | 2 +- pkg/sql/run_control_test.go | 2 +- pkg/testutils/serverutils/test_server_shim.go | 7 + pkg/testutils/testcluster/testcluster.go | 17 + pkg/util/admission/grant_coordinator.go | 15 +- pkg/util/admission/granter_test.go | 4 +- pkg/util/admission/testing_knobs.go | 10 + pkg/util/admission/work_queue.go | 16 +- 44 files changed, 3518 insertions(+), 55 deletions(-) create mode 100644 pkg/kv/kvserver/flow_control_integration_test.go create mode 100644 pkg/kv/kvserver/testdata/flow_control_integration/basic create mode 100644 pkg/kv/kvserver/testdata/flow_control_integration/blocked_admission create mode 100644 pkg/kv/kvserver/testdata/flow_control_integration/class_prioritization create mode 100644 pkg/kv/kvserver/testdata/flow_control_integration/crashed_node create mode 100644 pkg/kv/kvserver/testdata/flow_control_integration/leader_not_leaseholder create mode 100644 pkg/kv/kvserver/testdata/flow_control_integration/quiesced_range create mode 100644 pkg/kv/kvserver/testdata/flow_control_integration/raft_membership create mode 100644 pkg/kv/kvserver/testdata/flow_control_integration/raft_membership_remove_self create mode 100644 pkg/kv/kvserver/testdata/flow_control_integration/raft_snapshot create mode 100644 pkg/kv/kvserver/testdata/flow_control_integration/raft_transport_break create mode 100644 pkg/kv/kvserver/testdata/flow_control_integration/raft_transport_culled create mode 100644 pkg/kv/kvserver/testdata/flow_control_integration/split_merge create mode 100644 pkg/kv/kvserver/testdata/flow_control_integration/transfer_lease create mode 100644 pkg/kv/kvserver/testdata/flow_control_integration/unquiesced_range diff --git a/pkg/base/testing_knobs.go b/pkg/base/testing_knobs.go index eb6bf757970a..cf8367f1f80b 100644 --- a/pkg/base/testing_knobs.go +++ b/pkg/base/testing_knobs.go @@ -48,7 +48,9 @@ type TestingKnobs struct { DialerKnobs ModuleTestingKnobs ProtectedTS ModuleTestingKnobs CapturedIndexUsageStatsKnobs ModuleTestingKnobs + AdmissionControlOptions ModuleTestingKnobs // TODO(irfansharif): Remove. AdmissionControl ModuleTestingKnobs + RaftTransport ModuleTestingKnobs UnusedIndexRecommendKnobs ModuleTestingKnobs ExternalConnection ModuleTestingKnobs EventExporter ModuleTestingKnobs diff --git a/pkg/keys/spans.go b/pkg/keys/spans.go index 7f6d813784d0..4f5a0a875572 100644 --- a/pkg/keys/spans.go +++ b/pkg/keys/spans.go @@ -40,6 +40,10 @@ var ( // TimeseriesSpan holds all the timeseries data in the cluster. TimeseriesSpan = roachpb.Span{Key: TimeseriesPrefix, EndKey: TimeseriesKeyMax} + // ScratchSpan is used in tests to write arbitrary data without + // overlapping with meta, system or tenant ranges. + ScratchSpan = roachpb.Span{Key: ScratchRangeMin, EndKey: ScratchRangeMax} + // SystemSpanConfigSpan is part of the system keyspace that is used to carve // out spans for system span configurations. No data is stored in these spans, // instead, special meaning is assigned to them when stored in diff --git a/pkg/kv/kvserver/BUILD.bazel b/pkg/kv/kvserver/BUILD.bazel index 8d9bea30c1ad..a91726f0d1e9 100644 --- a/pkg/kv/kvserver/BUILD.bazel +++ b/pkg/kv/kvserver/BUILD.bazel @@ -272,6 +272,7 @@ go_test( "consistency_queue_test.go", "debug_print_test.go", "errors_test.go", + "flow_control_integration_test.go", "flow_control_raft_transport_test.go", "flow_control_replica_integration_test.go", "gossip_test.go", @@ -460,6 +461,7 @@ go_test( "//pkg/ts/tspb", "//pkg/upgrade/upgradebase", "//pkg/util", + "//pkg/util/admission", "//pkg/util/admission/admissionpb", "//pkg/util/caller", "//pkg/util/circuit", @@ -494,11 +496,13 @@ go_test( "@com_github_cockroachdb_logtags//:logtags", "@com_github_cockroachdb_pebble//:pebble", "@com_github_cockroachdb_redact//:redact", + "@com_github_dustin_go_humanize//:go-humanize", "@com_github_gogo_protobuf//proto", "@com_github_google_btree//:btree", "@com_github_jackc_pgx_v4//:pgx", "@com_github_kr_pretty//:pretty", "@com_github_lib_pq//:pq", + "@com_github_olekukonko_tablewriter//:tablewriter", "@com_github_stretchr_testify//assert", "@com_github_stretchr_testify//require", "@io_etcd_go_raft_v3//:raft", diff --git a/pkg/kv/kvserver/client_raft_helpers_test.go b/pkg/kv/kvserver/client_raft_helpers_test.go index e705102338a7..fc00d6070b90 100644 --- a/pkg/kv/kvserver/client_raft_helpers_test.go +++ b/pkg/kv/kvserver/client_raft_helpers_test.go @@ -96,6 +96,24 @@ func (h *unreliableRaftHandler) HandleRaftRequest( return nil } + if !h.dropReq(req) && log.V(1) { + // Debug logging, even if requests aren't dropped. This is a + // convenient way to observe all raft messages in unit tests when + // run using --vmodule='client_raft_helpers_test=1'. + var prefix string + if h.name != "" { + prefix = fmt.Sprintf("[%s] ", h.name) + } + log.Infof( + ctx, + "%s [raft] r%d Raft message %s", + prefix, + req.RangeID, + raft.DescribeMessage(req.Message, func([]byte) string { + return "" + }), + ) + } } return h.RaftMessageHandler.HandleRaftRequest(ctx, req, respStream) } diff --git a/pkg/kv/kvserver/flow_control_integration_test.go b/pkg/kv/kvserver/flow_control_integration_test.go new file mode 100644 index 000000000000..7350aaa8b264 --- /dev/null +++ b/pkg/kv/kvserver/flow_control_integration_test.go @@ -0,0 +1,2120 @@ +// Copyright 2023 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package kvserver_test + +import ( + "context" + "fmt" + "math/rand" + "strconv" + "strings" + "sync/atomic" + "testing" + "time" + + "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/server" + "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/testutils" + "github.com/cockroachdb/cockroach/pkg/testutils/datapathutils" + "github.com/cockroachdb/cockroach/pkg/testutils/echotest" + "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" + "github.com/cockroachdb/cockroach/pkg/testutils/testcluster" + "github.com/cockroachdb/cockroach/pkg/util/admission" + "github.com/cockroachdb/cockroach/pkg/util/admission/admissionpb" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/cockroach/pkg/util/randutil" + "github.com/cockroachdb/errors" + "github.com/dustin/go-humanize" + "github.com/olekukonko/tablewriter" + "github.com/stretchr/testify/require" +) + +// Tests in this file use the echotest library, printing out progressive state +// in files under testdata/flow_control_integration/*. Every +// TestFlowControlFileName maps to testdata/flow_control_integration/file_name +// -- running TestFlowControl* will match everything. It's instructive to run +// these tests with the following vmodule, as of 04/23: +// +// --vmodule='replica_raft=1,kvflowcontroller=2,replica_proposal_buf=1, +// raft_transport=2,kvflowdispatch=1,kvadmission=1, +// kvflowhandle=1,work_queue=1,replica_flow_control=1, +// tracker=1,client_raft_helpers_test=1' +// +// TODO(irfansharif): Add end-to-end tests for the following: +// - [ ] Node with full RaftTransport receive queue. +// - [ ] Node with full RaftTransport send queue, with proposals dropped. +// - [ ] Raft commands getting reproposed, either due to timeouts or not having +// the right MLAI. See TestReplicaRefreshPendingCommandsTicks, +// TestLogGrowthWhenRefreshingPendingCommands. +// - [ ] Raft proposals getting dropped/abandoned. See +// (*Replica).cleanupFailedProposalLocked and its uses. + +// TestFlowControlBasic runs a basic end-to-end test of the kvflowcontrol +// machinery, replicating + admitting a single 1MiB regular write. +func TestFlowControlBasic(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + testutils.RunTrueAndFalse(t, "always-enqueue", func(t *testing.T, alwaysEnqueue bool) { + ctx := context.Background() + const numNodes = 3 + tc := testcluster.StartTestCluster(t, numNodes, base.TestClusterArgs{ + ReplicationMode: base.ReplicationManual, + ServerArgs: base.TestServerArgs{ + Knobs: base.TestingKnobs{ + Store: &kvserver.StoreTestingKnobs{ + FlowControlTestingKnobs: &kvflowcontrol.TestingKnobs{ + UseOnlyForScratchRanges: true, + }, + }, + AdmissionControl: &admission.TestingKnobs{ + DisableWorkQueueFastPath: alwaysEnqueue, + }, + }, + }, + }) + defer tc.Stopper().Stop(ctx) + + k := tc.ScratchRange(t) + tc.AddVotersOrFatal(t, k, tc.Targets(1, 2)...) + + desc, err := tc.LookupRange(k) + require.NoError(t, err) + + for i := 0; i < numNodes; i++ { + si, err := tc.Server(i).GetStores().(*kvserver.Stores).GetStore(tc.Server(i).GetFirstStoreID()) + require.NoError(t, err) + tc.Servers[i].RaftTransport().Listen(si.StoreID(), + &unreliableRaftHandler{ + rangeID: desc.RangeID, + RaftMessageHandler: si, + unreliableRaftHandlerFuncs: unreliableRaftHandlerFuncs{ + dropReq: func(req *kvserverpb.RaftMessageRequest) bool { + // Install a raft handler to get verbose raft logging. + // + // TODO(irfansharif): Make this a more ergonomic + // testing knob instead. + return false + }, + }, + }) + } + + n1 := sqlutils.MakeSQLRunner(tc.ServerConn(0)) + + h := newFlowControlTestHelper(t, tc) + h.init() + defer h.close("basic") // this test behaves identically with or without the fast path + + h.waitForConnectedStreams(ctx, desc.RangeID, 3) + + h.comment(`-- Flow token metrics, before issuing the regular 1MiB replicated write.`) + h.query(n1, ` + SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvadmission%tokens%' +ORDER BY name ASC; +`) + + h.comment(`-- (Issuing + admitting a regular 1MiB, triply replicated write...)`) + t.Log("sending put request") + h.put(ctx, k, 1<<20 /* 1MiB */, admissionpb.NormalPri) + t.Log("sent put request") + + h.waitForAllTokensReturned(ctx, 3) + h.comment(` +-- Stream counts as seen by n1 post-write. We should see three {regular,elastic} +-- streams given there are three nodes and we're using a replication factor of +-- three. +`) + h.query(n1, ` + SELECT name, value + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvadmission.flow_controller%stream%' +ORDER BY name ASC; +`) + + h.comment(`-- Another view of the stream count, using /inspectz-backed vtables.`) + h.query(n1, ` + SELECT range_id, count(*) AS streams + FROM crdb_internal.kv_flow_control_handles +GROUP BY (range_id) +ORDER BY streams DESC; +`, "range_id", "stream_count") + + h.comment(` +-- Flow token metrics from n1 after issuing the regular 1MiB replicated write, +-- and it being admitted on n1, n2 and n3. We should see 3*1MiB = 3MiB of +-- {regular,elastic} tokens deducted and returned, and {8*3=24MiB,16*3=48MiB} of +-- {regular,elastic} tokens available. Everything should be accounted for. +`) + h.query(n1, ` + SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvadmission%tokens%' +ORDER BY name ASC; +`) + + // When run using -v the vmodule described at the top of this file, this + // test demonstrates end-to-end flow control machinery in the happy + // path. + // + // 1. The request gets admitted for flow tokens for each of the three + // replication streams (one per replica). + // + // [T1,n1] admitted request (pri=normal-pri stream=t1/s1 tokens=+16 MiB wait-duration=18.834Āµs mode=apply_to_all) + // [T1,n1] admitted request (pri=normal-pri stream=t1/s2 tokens=+16 MiB wait-duration=1.042Āµs mode=apply_to_all) + // [T1,n1] admitted request (pri=normal-pri stream=t1/s3 tokens=+16 MiB wait-duration=834ns mode=apply_to_all) + // + // 2. We encode the raft metadata as part of the raft command. At the + // proposer, we track 1 MiB of flow token deductions for each stream, + // using the log position the proposal is to end up in. + // + // [T1,n1,s1,r64/1:/{Table/Max-Max}] encoded raft admission meta: pri=normal-pri create-time=1685129503765352000 proposer=n1 + // [T1,n1,s1,r64/1:/{Table/Max-Max},raft] bound index/log terms for proposal entry: 6/20 EntryNormal + // [T1,n1,s1,r64/1:/{Table/Max-Max}] 279 tracking +1.0 MiB flow control tokens for pri=normal-pri stream=t1/s1 pos=log-position=6/20 + // [T1,n1,s1,r64/1:/{Table/Max-Max}] 280 adjusted flow tokens (pri=normal-pri stream=t1/s1 delta=-1.0 MiB): regular=+15 MiB elastic=+7.0 MiB + // [T1,n1,s1,r64/1:/{Table/Max-Max}] 281 tracking +1.0 MiB flow control tokens for pri=normal-pri stream=t1/s2 pos=log-position=6/20 + // [T1,n1,s1,r64/1:/{Table/Max-Max}] 282 adjusted flow tokens (pri=normal-pri stream=t1/s2 delta=-1.0 MiB): regular=+15 MiB elastic=+7.0 MiB + // [T1,n1,s1,r64/1:/{Table/Max-Max}] 283 tracking +1.0 MiB flow control tokens for pri=normal-pri stream=t1/s3 pos=log-position=6/20 + // [T1,n1,s1,r64/1:/{Table/Max-Max}] 284 adjusted flow tokens (pri=normal-pri stream=t1/s3 delta=-1.0 MiB): regular=+15 MiB elastic=+7.0 MiB + // + // 3. We decode the raft metadata below-raft on each of the replicas. + // The local replica does it first, either enqueues it in below-raft + // work queues or just fast path if tokens are available, informs itself + // of entries being admitted, and releases the corresponding flow + // tokens. + // + // [T1,n1,s1,r64/1:/{Table/Max-Max},raft] decoded raft admission meta below-raft: pri=normal-pri create-time=1685129503765352000 proposer=n1 receiver=[n1,s1] tenant=t1 tokensā‰ˆ+1.0 MiB sideloaded=false raft-entry=6/20 + // Fast-path: + // [T1,n1,s1,r64/1:/{Table/Max-Max},raft] fast-path: admitting t1 pri=normal-pri r64 origin=n1 log-position=6/20 ingested=false + // Async-path: + // [T1,n1,s1,r64/1:/{Table/Max-Max},raft] async-path: len(waiting-work)=1: enqueued t1 pri=normal-pri r64 origin=n1 log-position=6/20 ingested=false + // [T1,n1] async-path: len(waiting-work)=0 dequeued t1 pri=normal-pri r64 origin=n1 log-position=6/20 ingested=false + // [T1,n1] dispatching admitted-entries (r64 s1 pri=normal-pri up-to-log-position=6/20) to n1 + // [T1,n1] released +1.0 MiB flow control tokens for 1 out of 1 tracked deductions for pri=normal-pri stream=t1/s1, up to log-position=6/20; 0 tracked deduction(s) remain + // [T1,n1] adjusted flow tokens (pri=normal-pri stream=t1/s1 delta=+1.0 MiB): regular=+16 MiB elastic=+8.0 MiB + // + // + // 4. We propagate MsgApps with encoded raft metadata to the follower + // replicas on n2 and n3. Each of them similarly decode, virtually + // enqueue/use the fast path, admit, and inform the origin node (n1) + // of admission. Below we just show the fast path. + // + // [T1,n2] [raft] r64 Raft message 1->2 MsgApp Term:6 Log:6/19 Commit:19 Entries:[6/20 EntryNormal ] + // [T1,n3] [raft] r64 Raft message 1->3 MsgApp Term:6 Log:6/19 Commit:19 Entries:[6/20 EntryNormal ] + // + // [T1,n2,s2,r64/2:/{Table/Max-Max},raft] decoded raft admission meta below-raft: pri=normal-pri create-time=1685129503765352000 proposer=n1 receiver=[n2,s2] tenant=t1 tokensā‰ˆ+1.0 MiB sideloaded=false raft-entry=6/20 + // [T1,n3,s3,r64/3:/{Table/Max-Max},raft] decoded raft admission meta below-raft: pri=normal-pri create-time=1685129503765352000 proposer=n1 receiver=[n3,s3] tenant=t1 tokensā‰ˆ+1.0 MiB sideloaded=false raft-entry=6/20 + // [T1,n2,s2,r64/2:/{Table/Max-Max},raft] 294 fast-path: admitting t1 pri=normal-pri r64 origin=n1 log-position=6/20 ingested=false + // [T1,n3,s3,r64/3:/{Table/Max-Max},raft] 298 fast-path: admitting t1 pri=normal-pri r64 origin=n1 log-position=6/20 ingested=false + // [T1,n2] dispatching admitted-entries (r64 s2 pri=normal-pri up-to-log-position=6/20) to n1 + // [T1,n3] dispatching admitted-entries (r64 s3 pri=normal-pri up-to-log-position=6/20) to n1 + // + // 5. On MsgAppResps (or really any raft messages bound for n1), we + // piggyback these token dispatches. n1, on hearing about them, + // returns relevant tokens back to the stream. + // + // [T1,n2] informing n1 of below-raft admitted-entries (r64 s2 pri=normal-pri up-to-log-position=6/20): 1 out of 1 dispatches + // [T1,n1] [raft] r64 Raft message 2->1 MsgAppResp Term:6 Log:0/20 + // [T1,n1] released +1.0 MiB flow control tokens for 1 out of 1 tracked deductions for pri=normal-pri stream=t1/s2, up to log-position=6/20; 0 tracked deduction(s) remain + // [T1,n1] adjusted flow tokens (pri=normal-pri stream=t1/s2 delta=+1.0 MiB): regular=+16 MiB elastic=+8.0 MiB + // [T1,n3] informing n1 of below-raft admitted-entries (r64 s3 pri=normal-pri up-to-log-position=6/20): 1 out of 1 dispatches + // [T1,n1] [raft] r64 Raft message 3->1 MsgAppResp Term:6 Log:0/20 + // [T1,n1] released +1.0 MiB flow control tokens for 1 out of 1 tracked deductions for pri=normal-pri stream=t1/s3, up to log-position=6/20; 0 tracked deduction(s) remain + // [T1,n1] adjusted flow tokens (pri=normal-pri stream=t1/s3 delta=+1.0 MiB): regular=+16 MiB elastic=+8.0 MiB + }) +} + +// TestFlowControlRangeSplitMerge walks through what happens to flow tokens when +// a range splits/merges. +func TestFlowControlRangeSplitMerge(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + ctx := context.Background() + const numNodes = 3 + tc := testcluster.StartTestCluster(t, numNodes, base.TestClusterArgs{ + ReplicationMode: base.ReplicationManual, + ServerArgs: base.TestServerArgs{ + Knobs: base.TestingKnobs{ + Store: &kvserver.StoreTestingKnobs{ + FlowControlTestingKnobs: &kvflowcontrol.TestingKnobs{ + UseOnlyForScratchRanges: true, + }, + }, + }, + }, + }) + defer tc.Stopper().Stop(ctx) + + k := tc.ScratchRange(t) + tc.AddVotersOrFatal(t, k, tc.Targets(1, 2)...) + + n1 := sqlutils.MakeSQLRunner(tc.ServerConn(0)) + + h := newFlowControlTestHelper(t, tc) + h.init() + defer h.close("split_merge") + + desc, err := tc.LookupRange(k) + require.NoError(t, err) + + h.waitForConnectedStreams(ctx, desc.RangeID, 3) + t.Log("sending put request to pre-split range") + h.put(ctx, k, 1<<20 /* 1MiB */, admissionpb.NormalPri) + t.Log("sent put request to pre-split range") + + h.waitForAllTokensReturned(ctx, 3) + h.comment(` +-- Flow token metrics from n1 after issuing + admitting the regular 1MiB 3x +-- replicated write to the pre-split range. There should be 3MiB of +-- {regular,elastic} tokens {deducted,returned}. +`) + h.query(n1, ` + SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvadmission%tokens%' +ORDER BY name ASC; +`) + + h.comment(`-- (Splitting range.)`) + left, right := tc.SplitRangeOrFatal(t, k.Next()) + h.waitForConnectedStreams(ctx, right.RangeID, 3) + // [T1,n1,s1,r63/1:/{Table/62-Max},*kvpb.AdminSplitRequest] initiating a split of this range at key /Table/Max [r64] (manual) + // [T1,n1,s1,r64/1:/{Table/Max-Max},raft] connected to stream: t1/s1 + // [T1,n1,s1,r64/1:/{Table/Max-Max},raft] connected to stream: t1/s2 + // [T1,n1,s1,r64/1:/{Table/Max-Max},raft] connected to stream: t1/s3 + + t.Log("sending 2MiB put request to post-split LHS") + h.put(ctx, k, 2<<20 /* 2MiB */, admissionpb.NormalPri) + t.Log("sent 2MiB put request to post-split LHS") + + t.Log("sending 3MiB put request to post-split RHS") + h.put(ctx, roachpb.Key(right.StartKey), 3<<20 /* 3MiB */, admissionpb.NormalPri) + t.Log("sent 3MiB put request to post-split RHS") + + h.waitForAllTokensReturned(ctx, 3) + h.comment(` +-- Flow token metrics from n1 after further issuing 2MiB and 3MiB writes to +-- post-split LHS and RHS ranges respectively. We should see 15MiB extra tokens +-- {deducted,returned}, which comes from (2MiB+3MiB)*3=15MiB. So we stand at +-- 3MiB+15MiB=18MiB now. +`) + h.query(n1, ` + SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvadmission%tokens%' +ORDER BY name ASC; +`) + + h.comment(`-- Observe the newly split off replica, with its own three streams.`) + h.query(n1, ` + SELECT range_id, count(*) AS streams + FROM crdb_internal.kv_flow_control_handles +GROUP BY (range_id) +ORDER BY streams DESC; +`, "range_id", "stream_count") + + h.comment(`-- (Merging ranges.)`) + merged := tc.MergeRangesOrFatal(t, left.StartKey.AsRawKey()) + + // [T1,n1,s1,r64/1:{/Table/Max-\xfa\x00},*kvpb.AdminMergeRequest] initiating a merge of r65:{\xfa\x00-/Max} [(n1,s1):1, (n2,s2):2, (n3,s3):3, next=4, gen=6, sticky=9223372036.854775807,2147483647] into this range (manual) + // [T1,n1,s1,r64/1:{/Table/Max-\xfa\x00},raft] 380 removing replica r65/1 + // [T1,n2,s2,r64/2:{/Table/Max-\xfa\x00},raft] 385 removing replica r65/2 + // [T1,n3,s3,r64/3:{/Table/Max-\xfa\x00},raft] 384 removing replica r65/3 + // [T1,n1,s1,r65/1:{\xfa\x00-/Max},raft] disconnected stream: t1/s1 + // [T1,n1,s1,r65/1:{\xfa\x00-/Max},raft] disconnected stream: t1/s2 + // [T1,n1,s1,r65/1:{\xfa\x00-/Max},raft] disconnected stream: t1/s3 + + t.Log("sending 4MiB put request to post-merge range") + h.put(ctx, roachpb.Key(merged.StartKey), 4<<20 /* 4MiB */, admissionpb.NormalPri) + t.Log("sent 4MiB put request to post-merged range") + + h.waitForAllTokensReturned(ctx, 3) + h.comment(` +-- Flow token metrics from n1 after issuing 4MiB of regular replicated writes to +-- the post-merged range. We should see 12MiB extra tokens {deducted,returned}, +-- which comes from 4MiB*3=12MiB. So we stand at 18MiB+12MiB=30MiB now. +`) + h.query(n1, ` + SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvadmission%tokens%' +ORDER BY name ASC; +`) + + h.comment(`-- Observe only the merged replica with its own three streams.`) + h.query(n1, ` + SELECT range_id, count(*) AS streams + FROM crdb_internal.kv_flow_control_handles +GROUP BY (range_id) +ORDER BY streams DESC; +`, "range_id", "stream_count") +} + +// TestFlowControlBlockedAdmission tests token tracking behavior by explicitly +// blocking below-raft admission. +func TestFlowControlBlockedAdmission(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + ctx := context.Background() + const numNodes = 3 + var disableWorkQueueGranting atomic.Bool + disableWorkQueueGranting.Store(true) + + tc := testcluster.StartTestCluster(t, numNodes, base.TestClusterArgs{ + ReplicationMode: base.ReplicationManual, + ServerArgs: base.TestServerArgs{ + Knobs: base.TestingKnobs{ + Store: &kvserver.StoreTestingKnobs{ + FlowControlTestingKnobs: &kvflowcontrol.TestingKnobs{ + UseOnlyForScratchRanges: true, + }, + }, + AdmissionControl: &admission.TestingKnobs{ + DisableWorkQueueFastPath: true, + DisableWorkQueueGranting: func() bool { + return disableWorkQueueGranting.Load() + }, + }, + }, + }, + }) + defer tc.Stopper().Stop(ctx) + + k := tc.ScratchRange(t) + tc.AddVotersOrFatal(t, k, tc.Targets(1, 2)...) + + n1 := sqlutils.MakeSQLRunner(tc.ServerConn(0)) + n2 := sqlutils.MakeSQLRunner(tc.ServerConn(1)) + + h := newFlowControlTestHelper(t, tc) + h.init() + defer h.close("blocked_admission") + + desc, err := tc.LookupRange(k) + require.NoError(t, err) + h.waitForConnectedStreams(ctx, desc.RangeID, 3) + + h.comment(`-- (Issuing regular 1MiB, 3x replicated write that's not admitted.)`) + t.Log("sending put requests") + for i := 0; i < 5; i++ { + h.put(ctx, k, 1<<20 /* 1MiB */, admissionpb.NormalPri) + } + t.Log("sent put requests") + + h.comment(` +-- Flow token metrics from n1 after issuing 5 regular 1MiB 3x replicated writes +-- that are yet to get admitted. We see 5*1MiB*3=15MiB deductions of +-- {regular,elastic} tokens with no corresponding returns. +`) + h.query(n1, ` + SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvadmission%tokens%' +ORDER BY name ASC; +`) + + h.comment(`-- Observe the total tracked tokens per-stream on n1.`) + h.query(n1, ` + SELECT range_id, store_id, crdb_internal.humanize_bytes(total_tracked_tokens::INT8) + FROM crdb_internal.kv_flow_control_handles +`, "range_id", "store_id", "total_tracked_tokens") + + h.comment(`-- Observe the individual tracked tokens per-stream on the scratch range.`) + h.query(n1, ` + SELECT range_id, store_id, priority, crdb_internal.humanize_bytes(tokens::INT8) + FROM crdb_internal.kv_flow_token_deductions +`, "range_id", "store_id", "priority", "tokens") + + h.comment(`-- (Allow below-raft admission to proceed.)`) + disableWorkQueueGranting.Store(false) + h.waitForAllTokensReturned(ctx, 3) // wait for admission + + h.comment(`-- Observe flow token dispatch metrics from n1.`) + h.query(n1, ` + SELECT name, value + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvadmission.flow_token_dispatch.local_regular%' +ORDER BY name ASC; +`) + + h.comment(`-- Observe flow token dispatch metrics from n2.`) + h.query(n2, ` + SELECT name, value + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvadmission.flow_token_dispatch.remote_regular%' +ORDER BY name ASC; +`) + + h.comment(` +-- Flow token metrics from n1 after work gets admitted. We see 15MiB returns of +-- {regular,elastic} tokens, and the available capacities going back to what +-- they were. +`) + h.query(n1, ` + SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvadmission%tokens%' +ORDER BY name ASC; +`) +} + +// TestFlowControlCrashedNode tests flow token behavior in the presence of +// crashed nodes. +func TestFlowControlCrashedNode(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + ctx := context.Background() + const numNodes = 2 + var maintainStreamsForBrokenRaftTransport atomic.Bool + + tc := testcluster.StartTestCluster(t, numNodes, base.TestClusterArgs{ + ReplicationMode: base.ReplicationManual, + ServerArgs: base.TestServerArgs{ + RaftConfig: base.RaftConfig{ + // Suppress timeout-based elections. This test doesn't want to + // deal with leadership changing hands. + RaftElectionTimeoutTicks: 1000000, + }, + Knobs: base.TestingKnobs{ + Store: &kvserver.StoreTestingKnobs{ + FlowControlTestingKnobs: &kvflowcontrol.TestingKnobs{ + UseOnlyForScratchRanges: true, + MaintainStreamsForBrokenRaftTransport: func() bool { + return maintainStreamsForBrokenRaftTransport.Load() + }, + }, + }, + AdmissionControl: &admission.TestingKnobs{ + DisableWorkQueueFastPath: true, + DisableWorkQueueGranting: func() bool { + return true + }, + }, + }, + }, + }) + defer tc.Stopper().Stop(ctx) + + k := tc.ScratchRange(t) + tc.AddVotersOrFatal(t, k, tc.Targets(1)...) + + n1 := sqlutils.MakeSQLRunner(tc.ServerConn(0)) + + h := newFlowControlTestHelper(t, tc) + h.init() + defer h.close("crashed_node") + + desc, err := tc.LookupRange(k) + require.NoError(t, err) + tc.TransferRangeLeaseOrFatal(t, desc, tc.Target(0)) + h.waitForConnectedStreams(ctx, desc.RangeID, 2) + + h.comment(`-- (Issuing regular 5x1MiB, 2x replicated writes that are not admitted.)`) + t.Log("sending put requests") + for i := 0; i < 5; i++ { + h.put(ctx, k, 1<<20 /* 1MiB */, admissionpb.NormalPri) + } + t.Log("sent put requests") + + h.comment(` +-- Flow token metrics from n1 after issuing 5 regular 1MiB 2x replicated writes +-- that are yet to get admitted. We see 5*1MiB*2=10MiB deductions of +-- {regular,elastic} tokens with no corresponding returns. +`) + h.query(n1, ` + SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvadmission%tokens%' +ORDER BY name ASC; +`) + h.comment(`-- Observe the per-stream tracked tokens on n1, before n2 is crashed.`) + h.query(n1, ` + SELECT range_id, store_id, crdb_internal.humanize_bytes(total_tracked_tokens::INT8) + FROM crdb_internal.kv_flow_control_handles +`, "range_id", "store_id", "total_tracked_tokens") + + // In this test we want to see how we integrate with nodes crashing -- we + // want to make sure that we return all held tokens when nodes crash. + // There's a secondary mechanism that would release flow tokens in such + // cases -- we also release tokens when the RaftTransport breaks. We don't + // want that to kick in here, so we disable it first. See + // TestFlowControlRaftTransportBreak where that mechanism is tested instead. + maintainStreamsForBrokenRaftTransport.Store(true) + + h.comment(`-- (Crashing n2 but disabling the raft-transport-break token return mechanism.)`) + tc.StopServer(1) + h.waitForConnectedStreams(ctx, desc.RangeID, 1) + + h.comment(` +-- Observe the per-stream tracked tokens on n1, after n2 crashed. We're no +-- longer tracking the 5MiB held by n2. +`) + h.query(n1, ` + SELECT range_id, store_id, crdb_internal.humanize_bytes(total_tracked_tokens::INT8) + FROM crdb_internal.kv_flow_control_handles +`, "range_id", "store_id", "total_tracked_tokens") + + h.comment(` +-- Flow token metrics from n1 after n2 crashed. Observe that we've returned the +-- 5MiB previously held by n2. +`) + h.query(n1, ` + SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvadmission%tokens%' +ORDER BY name ASC; +`) +} + +// TestFlowControlRaftSnapshot tests flow token behavior when one replica needs +// to be caught up via raft snapshot. +func TestFlowControlRaftSnapshot(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + stickyEngineRegistry := server.NewStickyInMemEnginesRegistry() + defer stickyEngineRegistry.CloseAllStickyInMemEngines() + + const numServers int = 5 + stickyServerArgs := make(map[int]base.TestServerArgs) + var maintainStreamsForInactiveFollowers atomic.Bool + var maintainStreamsForBrokenRaftTransport atomic.Bool + var disableWorkQueueGranting atomic.Bool + disableWorkQueueGranting.Store(true) + + for i := 0; i < numServers; i++ { + stickyServerArgs[i] = base.TestServerArgs{ + StoreSpecs: []base.StoreSpec{ + { + InMemory: true, + StickyInMemoryEngineID: strconv.FormatInt(int64(i), 10), + }, + }, + RaftConfig: base.RaftConfig{ + // Suppress timeout-based elections. This test doesn't want to + // deal with leadership changing hands. + RaftElectionTimeoutTicks: 1000000, + }, + Knobs: base.TestingKnobs{ + Server: &server.TestingKnobs{ + StickyEngineRegistry: stickyEngineRegistry, + }, + Store: &kvserver.StoreTestingKnobs{ + FlowControlTestingKnobs: &kvflowcontrol.TestingKnobs{ + UseOnlyForScratchRanges: true, + OverrideTokenDeduction: func() kvflowcontrol.Tokens { + // This test makes use of (small) increment + // requests, but wants to see large token + // deductions/returns. + return kvflowcontrol.Tokens(1 << 20 /* 1MiB */) + }, + MaintainStreamsForInactiveFollowers: func() bool { + return maintainStreamsForInactiveFollowers.Load() + }, + MaintainStreamsForBrokenRaftTransport: func() bool { + return maintainStreamsForBrokenRaftTransport.Load() + }, + }, + }, + AdmissionControl: &admission.TestingKnobs{ + DisableWorkQueueFastPath: true, + DisableWorkQueueGranting: func() bool { + return disableWorkQueueGranting.Load() + }, + }, + RaftTransport: &kvserver.RaftTransportTestingKnobs{ + OverrideIdleTimeout: func() time.Duration { + // Effectively disable token returns due to underlying + // raft transport streams disconnecting due to + // inactivity. + return time.Hour + }, + }, + }, + } + } + + ctx := context.Background() + tc := testcluster.StartTestCluster(t, numServers, + base.TestClusterArgs{ + ReplicationMode: base.ReplicationManual, + ServerArgsPerNode: stickyServerArgs, + }) + defer tc.Stopper().Stop(ctx) + + n1 := sqlutils.MakeSQLRunner(tc.ServerConn(0)) + n4 := sqlutils.MakeSQLRunner(tc.ServerConn(3)) + + h := newFlowControlTestHelper(t, tc) + h.init() + defer h.close("raft_snapshot") + + store := tc.GetFirstStoreFromServer(t, 0) + + incA := int64(5) + incB := int64(7) + incAB := incA + incB + + k := tc.ScratchRange(t) + tc.AddVotersOrFatal(t, k, tc.Targets(1, 2)...) + + tc.AddVotersOrFatal(t, k, tc.Targets(3, 4)...) + repl := store.LookupReplica(roachpb.RKey(k)) + require.NotNil(t, repl) + h.waitForConnectedStreams(ctx, repl.RangeID, 5) + + // Set up a key to replicate across the cluster. We're going to modify this + // key and truncate the raft logs from that command after killing one of the + // nodes to check that it gets the new value after it comes up. + incArgs := incrementArgs(k, incA) + if _, err := kv.SendWrappedWithAdmission(ctx, tc.Server(0).DB().NonTransactionalSender(), kvpb.Header{}, kvpb.AdmissionHeader{ + Priority: int32(admissionpb.HighPri), + Source: kvpb.AdmissionHeader_FROM_SQL, + }, incArgs); err != nil { + t.Fatal(err) + } + + h.comment(` +-- Flow token metrics from n1 after issuing 1 regular 1MiB 5x replicated write +-- that's not admitted. Since this test is ignoring crashed nodes for token +-- deduction purposes, we see a deduction of 5MiB {regular,elastic} tokens. + `) + h.query(n1, ` + SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvadmission%tokens%' +ORDER BY name ASC; + `) + h.comment(` +-- Observe the total tracked tokens per-stream on n1. 1MiB is tracked for n1-n5. + `) + h.query(n1, ` + SELECT range_id, store_id, crdb_internal.humanize_bytes(total_tracked_tokens::INT8) + FROM crdb_internal.kv_flow_control_handles + `, "range_id", "store_id", "total_tracked_tokens") + + tc.WaitForValues(t, k, []int64{incA, incA, incA, incA, incA}) + + h.comment(` +-- (Killing n2 and n3, but preventing their tokens from being returned + +-- artificially allowing tokens to get deducted.)`) + + // In this test we want to see how we integrate with raft progress state + // when nodes are down long enough to warrant snapshots (post + // log-truncation). We want to do this without responding to the node + // actually being down, which is tested separately in + // TestFlowControlCrashedNode and TestFlowControlRaftTransportBreak. + maintainStreamsForInactiveFollowers.Store(true) + maintainStreamsForBrokenRaftTransport.Store(true) + + // Now kill stores 1 + 2, increment the key on the other stores and + // truncate their logs to make sure that when store 1 + 2 comes back up + // they will require a snapshot from Raft. + tc.StopServer(1) + tc.StopServer(2) + + h.comment(` +-- Observe the total tracked tokens per-stream on n1. 1MiB is (still) tracked +-- for n1-n5. Typically n2, n3 would release their tokens, but this test is +-- intentionally suppressing that behavior to observe token returns only once +-- issuing a raft snapshot. + `) + h.query(n1, ` + SELECT range_id, store_id, crdb_internal.humanize_bytes(total_tracked_tokens::INT8) + FROM crdb_internal.kv_flow_control_handles + `, "range_id", "store_id", "total_tracked_tokens") + + h.comment(` +-- (Issuing another 1MiB of 5x replicated writes while n2 and n3 are down and +-- below-raft admission is paused.) +`) + incArgs = incrementArgs(k, incB) + if _, err := kv.SendWrappedWithAdmission(ctx, tc.Server(0).DB().NonTransactionalSender(), kvpb.Header{}, kvpb.AdmissionHeader{ + Priority: int32(admissionpb.HighPri), + Source: kvpb.AdmissionHeader_FROM_SQL, + }, incArgs); err != nil { + t.Fatal(err) + } + + h.comment(` +-- Flow token metrics from n1 after issuing 1 regular 1MiB 5x replicated write +-- that's not admitted. We'll have deducted another 5*1MiB=5MiB worth of tokens. +-- Normally we wouldn't deduct tokens for n2 and n3 since they're dead (both +-- according to the per-replica last-updated map, and according broken +-- RaftTransport streams). But this test is intentionally suppressing that +-- behavior to observe token returns when sending raft snapshots. + `) + h.query(n1, ` + SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvadmission%tokens%' +ORDER BY name ASC; + `) + h.comment(` +-- Observe the total tracked tokens per-stream on n1. 2MiB is tracked for n1-n5; +-- see last comment for an explanation why we're still deducting for n2, n3. +`) + h.query(n1, ` + SELECT range_id, store_id, crdb_internal.humanize_bytes(total_tracked_tokens::INT8) + FROM crdb_internal.kv_flow_control_handles + `, "range_id", "store_id", "total_tracked_tokens") + + tc.WaitForValues(t, k, []int64{incAB, incA, incA, incAB, incAB}) + + index := repl.GetLastIndex() + h.comment(`-- (Truncating raft log.)`) + + // Truncate the log at index+1 (log entries < N are removed, so this + // includes the increment). + truncArgs := truncateLogArgs(index+1, repl.GetRangeID()) + if _, err := kv.SendWrappedWithAdmission(ctx, tc.Server(0).DB().NonTransactionalSender(), kvpb.Header{}, kvpb.AdmissionHeader{ + Priority: int32(admissionpb.HighPri), + Source: kvpb.AdmissionHeader_FROM_SQL, + }, truncArgs); err != nil { + t.Fatal(err) + } + + h.comment(`-- (Restarting n2 and n3.)`) + require.NoError(t, tc.RestartServer(1)) + require.NoError(t, tc.RestartServer(2)) + + // Go back to the default kvflowcontrol integration behavior. + maintainStreamsForInactiveFollowers.Store(false) + maintainStreamsForBrokenRaftTransport.Store(false) + + tc.WaitForValues(t, k, []int64{incAB, incAB, incAB, incAB, incAB}) + + // [T1,n1,s1,r63/1:/{Table/Max-Max},raft] 646 adjusted flow tokens (pri=high-pri stream=t1/s2 delta=+1.0 MiB): regular=+16 MiB elastic=+8.0 MiB + // [T1,n1,s1,r63/1:/{Table/Max-Max},raft] 647 disconnected stream: t1/s2 + // [T1,n1,s1,r63/1:/{Table/Max-Max},raft] 648 tracked disconnected stream: t1/s2 (reason: not actively replicating) + // [T1,n1,s1,r63/1:/{Table/Max-Max},raft] 705 adjusted flow tokens (pri=high-pri stream=t1/s3 delta=+1.0 MiB): regular=+16 MiB elastic=+8.0 MiB + // [T1,n1,s1,r63/1:/{Table/Max-Max},raft] 706 disconnected stream: t1/s3 + // [T1,n1,s1,r63/1:/{Table/Max-Max},raft] 707 tracked disconnected stream: t1/s3 (reason: not actively replicating) + + h.comment(` +-- Flow token metrics from n1 after restarting n2 and n3. We've returned the +-- 2MiB previously held by those nodes (2MiB each). We're reacting to it's raft +-- progress state, noting that since we've truncated our log, we need to catch +-- it up via snapshot. So we release all held tokens. + `) + h.query(n1, ` + SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvadmission%tokens%' +ORDER BY name ASC; +`) + + h.comment(` +-- Observe the total tracked tokens per-stream on n1. There's nothing tracked +-- for n2 and n3 anymore. +`) + h.query(n1, ` + SELECT range_id, store_id, crdb_internal.humanize_bytes(total_tracked_tokens::INT8) + FROM crdb_internal.kv_flow_control_handles +`, "range_id", "store_id", "total_tracked_tokens") + + h.waitForConnectedStreams(ctx, repl.RangeID, 5) + // [T1,n1,s1,r63/1:/{Table/Max-Max},raft] 651 connected to stream: t1/s2 + // [T1,n1,s1,r63/1:/{Table/Max-Max},raft] 710 connected to stream: t1/s3 + + h.comment(`-- (Allow below-raft admission to proceed.)`) + disableWorkQueueGranting.Store(false) + + h.waitForAllTokensReturned(ctx, 5) + + h.comment(`-- Observe flow token dispatch metrics from n4.`) + h.query(n4, ` + SELECT name, value + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvadmission.flow_token_dispatch.pending_nodes%' +ORDER BY name ASC; +`) + + h.comment(` +-- Flow token metrics from n1 after work gets admitted. We see the remaining +-- 6MiB of {regular,elastic} tokens returned. +`) + h.query(n1, ` + SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvadmission%tokens%' +ORDER BY name ASC; +`) + + h.comment(` +-- Observe the total tracked tokens per-stream on n1; there should be nothing. +`) + h.query(n1, ` + SELECT range_id, store_id, crdb_internal.humanize_bytes(total_tracked_tokens::INT8) + FROM crdb_internal.kv_flow_control_handles +`, "range_id", "store_id", "total_tracked_tokens") + + h.comment(`-- Another view of tokens, using /inspectz-backed vtables.`) + h.query(n1, ` +SELECT store_id, + crdb_internal.humanize_bytes(available_regular_tokens), + crdb_internal.humanize_bytes(available_elastic_tokens) + FROM crdb_internal.kv_flow_controller + ORDER BY store_id ASC; +`, "range_id", "regular_available", "elastic_available") +} + +// TestFlowControlRaftTransportBreak tests flow token behavior when the raft +// transport breaks (due to crashed nodes, in this test). +func TestFlowControlRaftTransportBreak(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + ctx := context.Background() + const numNodes = 3 + var maintainStreamsForInactiveFollowers atomic.Bool + + tc := testcluster.StartTestCluster(t, numNodes, base.TestClusterArgs{ + ReplicationMode: base.ReplicationManual, + ServerArgs: base.TestServerArgs{ + RaftConfig: base.RaftConfig{ + // Suppress timeout-based elections. This test doesn't want to + // deal with leadership changing hands. + RaftElectionTimeoutTicks: 1000000, + }, + Knobs: base.TestingKnobs{ + Store: &kvserver.StoreTestingKnobs{ + FlowControlTestingKnobs: &kvflowcontrol.TestingKnobs{ + UseOnlyForScratchRanges: true, + MaintainStreamsForInactiveFollowers: func() bool { + return maintainStreamsForInactiveFollowers.Load() + }, + }, + }, + AdmissionControl: &admission.TestingKnobs{ + DisableWorkQueueFastPath: true, + DisableWorkQueueGranting: func() bool { + return true + }, + }, + }, + }, + }) + defer tc.Stopper().Stop(ctx) + + k := tc.ScratchRange(t) + tc.AddVotersOrFatal(t, k, tc.Targets(1, 2)...) + + n1 := sqlutils.MakeSQLRunner(tc.ServerConn(0)) + + h := newFlowControlTestHelper(t, tc) + h.init() + defer h.close("raft_transport_break") + + desc, err := tc.LookupRange(k) + require.NoError(t, err) + tc.TransferRangeLeaseOrFatal(t, desc, tc.Target(0)) + h.waitForConnectedStreams(ctx, desc.RangeID, 3) + + h.comment(`-- (Issuing regular 5x1MiB, 3x replicated writes that are not admitted.)`) + t.Log("sending put requests") + for i := 0; i < 5; i++ { + h.put(ctx, k, 1<<20 /* 1MiB */, admissionpb.NormalPri) + } + t.Log("sent put requests") + + h.comment(` +-- Flow token metrics from n1 after issuing 5 regular 1MiB 3x replicated writes +-- that are yet to get admitted. We see 5*1MiB*3=15MiB deductions of +-- {regular,elastic} tokens with no corresponding returns. +`) + h.query(n1, ` + SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvadmission%tokens%' +ORDER BY name ASC; +`) + h.comment(` +-- Observe the per-stream tracked tokens on n1, before n2 is crashed. +`) + h.query(n1, ` + SELECT range_id, store_id, crdb_internal.humanize_bytes(total_tracked_tokens::INT8) + FROM crdb_internal.kv_flow_control_handles +`, "range_id", "store_id", "total_tracked_tokens") + + // In this test we want to see how we integrate with raft transport + // breaking. To break the raft transport between n1 and n2, we just kill n2. + // There's a secondary mechanism that would release flow tokens when a node + // is down for long enough -- the per-replica last-updated map is used to + // prune out inactive replicas. We don't want that to kick in, so we disable + // it first. See TestFlowControlCrashedNode where that mechanism is tested + // instead. + maintainStreamsForInactiveFollowers.Store(true) + + h.comment(`-- (Crashing n2 but disabling the last-updated token return mechanism.)`) + tc.StopServer(1) + h.waitForConnectedStreams(ctx, desc.RangeID, 2) + + h.comment(` +-- Observe the per-stream tracked tokens on n1, after n2 crashed. We're no +-- longer tracking the 5MiB held by n2 because the raft transport between +-- n1<->n2 is broken. +`) + h.query(n1, ` + SELECT range_id, store_id, crdb_internal.humanize_bytes(total_tracked_tokens::INT8) + FROM crdb_internal.kv_flow_control_handles +`, "range_id", "store_id", "total_tracked_tokens") + + h.comment(` +-- Flow token metrics from n1 after n2 crashed. Observe that we've returned the +-- 5MiB previously held by n2. +`) + h.query(n1, ` + SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvadmission%tokens%' +ORDER BY name ASC; +`) +} + +// TestFlowControlRaftTransportCulled tests flow token behavior when the raft +// transport streams are culled due to inactivity. +func TestFlowControlRaftTransportCulled(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + ctx := context.Background() + const numNodes = 3 + workerTeardownCh := make(chan roachpb.NodeID, 1) + markSendQueueAsIdleCh := make(chan roachpb.NodeID) + + baseServerArgs := base.TestServerArgs{ + Knobs: base.TestingKnobs{ + Store: &kvserver.StoreTestingKnobs{ + FlowControlTestingKnobs: &kvflowcontrol.TestingKnobs{ + UseOnlyForScratchRanges: true, + OverrideTokenDeduction: func() kvflowcontrol.Tokens { + // This test asserts on the exact values of tracked + // tokens. In non-test code, the tokens deducted are + // a few bytes off (give or take) from the size of + // the proposals. We don't care about such + // differences. + return kvflowcontrol.Tokens(1 << 20 /* 1MiB */) + }, + }, + }, + AdmissionControl: &admission.TestingKnobs{ + DisableWorkQueueFastPath: true, + DisableWorkQueueGranting: func() bool { + return true + }, + }, + }, + } + baseServerArgsWithRaftTransportKnobs := baseServerArgs + baseServerArgsWithRaftTransportKnobs.Knobs.RaftTransport = &kvserver.RaftTransportTestingKnobs{ + MarkSendQueueAsIdleCh: markSendQueueAsIdleCh, + OnWorkerTeardown: func(nodeID roachpb.NodeID) { + workerTeardownCh <- nodeID + }, + } + serverArgsPerNode := map[int]base.TestServerArgs{ + 0: baseServerArgs, + 1: baseServerArgsWithRaftTransportKnobs, + 2: baseServerArgs, + } + + tc := testcluster.StartTestCluster(t, numNodes, base.TestClusterArgs{ + ReplicationMode: base.ReplicationManual, + ServerArgsPerNode: serverArgsPerNode, + }) + defer tc.Stopper().Stop(ctx) + + k := tc.ScratchRange(t) + tc.AddVotersOrFatal(t, k, tc.Targets(1, 2)...) + + n1 := sqlutils.MakeSQLRunner(tc.ServerConn(0)) + + h := newFlowControlTestHelper(t, tc) + h.init() + defer h.close("raft_transport_culled") + + desc, err := tc.LookupRange(k) + require.NoError(t, err) + tc.TransferRangeLeaseOrFatal(t, desc, tc.Target(0)) + h.waitForConnectedStreams(ctx, desc.RangeID, 3) + + h.comment(`-- (Issuing regular 5x1MiB, 3x replicated writes that are not admitted.)`) + t.Log("sending put requests") + for i := 0; i < 5; i++ { + h.put(ctx, k, 1<<20 /* 1MiB */, admissionpb.NormalPri) + } + t.Log("sent put requests") + + h.comment(` +-- Flow token metrics from n1 after issuing 5 regular 1MiB 3x replicated writes +-- that are yet to get admitted. We see 5*1MiB*3=15MiB deductions of +-- {regular,elastic} tokens with no corresponding returns. +`) + h.query(n1, ` + SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvadmission%tokens%' +ORDER BY name ASC; +`) + h.comment(` +-- Observe the per-stream tracked tokens on n1, before we cull the n1<->n2 raft +-- transport stream out of idleness. +`) + h.query(n1, ` + SELECT range_id, store_id, crdb_internal.humanize_bytes(total_tracked_tokens::INT8) + FROM crdb_internal.kv_flow_control_handles +`, "range_id", "store_id", "total_tracked_tokens") + + h.comment(`-- (Marking n2->n1 raft transport gRPC stream as idle.)`) + + select { + case markSendQueueAsIdleCh <- roachpb.NodeID(1): + case <-time.After(time.Second): + t.Fatalf("timed out") + } + select { + case gotNodeID := <-workerTeardownCh: + require.Equal(t, gotNodeID, roachpb.NodeID(1)) + case <-time.After(time.Second): + t.Fatalf("timed out") + } + + h.waitForTotalTrackedTokens(ctx, desc.RangeID, 10<<20 /* 5*1MiB*2=10MiB */) + + h.comment(` +-- Observe the per-stream tracked tokens on n1 after n2->n1 raft transport +-- stream is culled. We're no longer tracking the 5MiB held by n2 because the +-- raft transport between n1<->n2 is broken. +`) + h.query(n1, ` + SELECT range_id, store_id, crdb_internal.humanize_bytes(total_tracked_tokens::INT8) + FROM crdb_internal.kv_flow_control_handles + WHERE total_tracked_tokens > 0 +`, "range_id", "store_id", "total_tracked_tokens") + + h.comment(` +-- Flow token metrics from n1 after n2->n1 raft transport stream is culled. +-- Observe that we've returned the 5MiB previously held by n2. +`) + h.query(n1, ` + SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvadmission%tokens%' +ORDER BY name ASC; +`) +} + +// TestFlowControlRaftMembership tests flow token behavior when the raft +// membership changes. +func TestFlowControlRaftMembership(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + ctx := context.Background() + const numNodes = 5 + var disableWorkQueueGranting atomic.Bool + disableWorkQueueGranting.Store(true) + tc := testcluster.StartTestCluster(t, numNodes, base.TestClusterArgs{ + ReplicationMode: base.ReplicationManual, + ServerArgs: base.TestServerArgs{ + Knobs: base.TestingKnobs{ + Store: &kvserver.StoreTestingKnobs{ + FlowControlTestingKnobs: &kvflowcontrol.TestingKnobs{ + UseOnlyForScratchRanges: true, + }, + }, + AdmissionControl: &admission.TestingKnobs{ + DisableWorkQueueFastPath: true, + DisableWorkQueueGranting: func() bool { + return disableWorkQueueGranting.Load() + }, + }, + }, + }, + }) + defer tc.Stopper().Stop(ctx) + + k := tc.ScratchRange(t) + tc.AddVotersOrFatal(t, k, tc.Targets(1, 2)...) + + n1 := sqlutils.MakeSQLRunner(tc.ServerConn(0)) + + h := newFlowControlTestHelper(t, tc) + h.init() + defer h.close("raft_membership") + + desc, err := tc.LookupRange(k) + require.NoError(t, err) + h.waitForConnectedStreams(ctx, desc.RangeID, 3) + + h.comment(`-- (Issuing 1x1MiB, 3x replicated write that's not admitted.)`) + h.put(ctx, k, 1<<20 /* 1MiB */, admissionpb.NormalPri) + + h.comment(` +-- Flow token metrics from n1 after issuing 1x1MiB regular 3x replicated write +-- that's not admitted. We see 1*1MiB*3=3MiB deductions of regular tokens with +-- no corresponding returns. +`) + h.query(n1, ` + SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvadmission%regular_tokens%' +ORDER BY name ASC; +`) + + h.comment(`-- (Adding a voting replica on n4.)`) + tc.AddVotersOrFatal(t, k, tc.Target(3)) + h.waitForConnectedStreams(ctx, desc.RangeID, 4) + + h.comment(` +-- Observe the total tracked tokens per-stream on n1. s1-s3 should have 1MiB +-- tracked each, and s4 should have none.`) + h.query(n1, ` + SELECT range_id, store_id, crdb_internal.humanize_bytes(total_tracked_tokens::INT8) + FROM crdb_internal.kv_flow_control_handles +`, "range_id", "store_id", "total_tracked_tokens") + + h.comment(`-- (Issuing 1x1MiB, 4x replicated write that's not admitted.)`) + h.put(ctx, k, 1<<20 /* 1MiB */, admissionpb.NormalPri) + + h.comment(` +-- Observe the individual tracked tokens per-stream on the scratch range. s1-s3 +-- should have 2MiB tracked (they've observed 2x1MiB writes), s4 should have +-- 1MiB. +`) + h.query(n1, ` + SELECT range_id, store_id, crdb_internal.humanize_bytes(total_tracked_tokens::INT8) + FROM crdb_internal.kv_flow_control_handles +`, "range_id", "store_id", "total_tracked_tokens") + + h.comment(`-- (Removing voting replica from n3.)`) + tc.RemoveVotersOrFatal(t, k, tc.Target(2)) + h.waitForConnectedStreams(ctx, desc.RangeID, 3) + + h.comment(`-- (Adding non-voting replica to n5.)`) + tc.AddNonVotersOrFatal(t, k, tc.Target(4)) + h.waitForConnectedStreams(ctx, desc.RangeID, 4) + + h.comment(`-- (Issuing 1x1MiB, 4x replicated write (w/ one non-voter) that's not admitted.`) + h.put(ctx, k, 1<<20 /* 1MiB */, admissionpb.NormalPri) + + h.comment(` +-- Observe the individual tracked tokens per-stream on the scratch range. s1-s2 +-- should have 3MiB tracked (they've observed 3x1MiB writes), there should be +-- no s3 since it was removed, s4 and s5 should have 2MiB and 1MiB +-- respectively. +`) + h.query(n1, ` + SELECT range_id, store_id, crdb_internal.humanize_bytes(total_tracked_tokens::INT8) + FROM crdb_internal.kv_flow_control_handles +`, "range_id", "store_id", "total_tracked_tokens") + + h.comment(`-- (Allow below-raft admission to proceed.)`) + disableWorkQueueGranting.Store(false) + h.waitForAllTokensReturned(ctx, 5) + + h.comment(`-- Observe that there no tracked tokens across s1,s2,s4,s5.`) + h.query(n1, ` + SELECT range_id, store_id, crdb_internal.humanize_bytes(total_tracked_tokens::INT8) + FROM crdb_internal.kv_flow_control_handles +`, "range_id", "store_id", "total_tracked_tokens") + + h.comment(` +-- Flow token metrics from n1 after work gets admitted. All {regular,elastic} +-- tokens deducted are returned, including from when s3 was removed as a raft +-- member. +`) + h.query(n1, ` + SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvadmission%tokens%' +ORDER BY name ASC; +`) +} + +// TestFlowControlRaftMembershipRemoveSelf tests flow token behavior when the +// raft leader removes itself from the raft group. +func TestFlowControlRaftMembershipRemoveSelf(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + testutils.RunTrueAndFalse(t, "transfer-lease-first", func(t *testing.T, transferLeaseFirst bool) { + ctx := context.Background() + const numNodes = 4 + var disableWorkQueueGranting atomic.Bool + disableWorkQueueGranting.Store(true) + tc := testcluster.StartTestCluster(t, numNodes, base.TestClusterArgs{ + ReplicationMode: base.ReplicationManual, + ServerArgs: base.TestServerArgs{ + RaftConfig: base.RaftConfig{ + // Suppress timeout-based elections. This test doesn't want to + // deal with leadership changing hands. + RaftElectionTimeoutTicks: 1000000, + }, + Knobs: base.TestingKnobs{ + Store: &kvserver.StoreTestingKnobs{ + FlowControlTestingKnobs: &kvflowcontrol.TestingKnobs{ + UseOnlyForScratchRanges: true, + }, + }, + AdmissionControl: &admission.TestingKnobs{ + DisableWorkQueueFastPath: true, + DisableWorkQueueGranting: func() bool { + return disableWorkQueueGranting.Load() + }, + }, + }, + }, + }) + defer tc.Stopper().Stop(ctx) + + k := tc.ScratchRange(t) + tc.AddVotersOrFatal(t, k, tc.Targets(1, 2)...) + + n1 := sqlutils.MakeSQLRunner(tc.ServerConn(0)) + + h := newFlowControlTestHelper(t, tc) + h.init() + defer h.close("raft_membership_remove_self") // this test behaves identically independent of we transfer the lease first + + desc, err := tc.LookupRange(k) + require.NoError(t, err) + + // Make sure the lease is on n1 and that we're triply connected. + tc.TransferRangeLeaseOrFatal(t, desc, tc.Target(0)) + h.waitForConnectedStreams(ctx, desc.RangeID, 3) + + h.comment(`-- (Issuing 1x1MiB, 3x replicated write that's not admitted.)`) + h.put(ctx, k, 1<<20 /* 1MiB */, admissionpb.NormalPri) + + h.comment(` +-- Flow token metrics from n1 after issuing 1x1MiB regular 3x replicated write +-- that's not admitted. We see 1*1MiB*3=3MiB deductions of regular tokens with +-- no corresponding returns. +`) + h.query(n1, ` + SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvadmission%regular_tokens%' +ORDER BY name ASC; +`) + + h.comment(`-- (Replacing current raft leader on n1 in raft group with new n4 replica.)`) + testutils.SucceedsSoon(t, func() error { + // Relocate range from n1 -> n4. + if err := tc.Servers[2].DB(). + AdminRelocateRange( + context.Background(), desc.StartKey.AsRawKey(), + tc.Targets(1, 2, 3), nil, transferLeaseFirst); err != nil { + return err + } + leaseHolder, err := tc.FindRangeLeaseHolder(desc, nil) + if err != nil { + return err + } + if leaseHolder.Equal(tc.Target(0)) { + return errors.Errorf("expected leaseholder to not be on n1") + } + return nil + }) + h.waitForAllTokensReturned(ctx, 3) + + h.comment(` +-- Flow token metrics from n1 after raft leader removed itself from raft group. +-- All {regular,elastic} tokens deducted are returned. +`) + h.query(n1, ` + SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvadmission%tokens%' +ORDER BY name ASC; +`) + + h.comment(`-- (Allow below-raft admission to proceed.)`) + disableWorkQueueGranting.Store(false) + h.waitForAllTokensReturned(ctx, 3) + + h.comment(` +-- Flow token metrics from n1 after work gets admitted. Tokens were already +-- returned earlier, so there's no change. +`) + h.query(n1, ` + SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvadmission%tokens%' +ORDER BY name ASC; +`) + }) +} + +// TestFlowControlClassPrioritization shows how tokens are managed for both +// regular and elastic work. It does so by replicating + admitting a single 1MiB +// {regular,elastic} write. +func TestFlowControlClassPrioritization(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + ctx := context.Background() + const numNodes = 5 + var disableWorkQueueGranting atomic.Bool + disableWorkQueueGranting.Store(true) + + tc := testcluster.StartTestCluster(t, numNodes, base.TestClusterArgs{ + ReplicationMode: base.ReplicationManual, + ServerArgs: base.TestServerArgs{ + Knobs: base.TestingKnobs{ + Store: &kvserver.StoreTestingKnobs{ + FlowControlTestingKnobs: &kvflowcontrol.TestingKnobs{ + UseOnlyForScratchRanges: true, + }, + }, + AdmissionControl: &admission.TestingKnobs{ + DisableWorkQueueFastPath: true, + DisableWorkQueueGranting: func() bool { + return disableWorkQueueGranting.Load() + }, + }, + }, + }, + }) + defer tc.Stopper().Stop(ctx) + + k := tc.ScratchRange(t) + tc.AddVotersOrFatal(t, k, tc.Targets(1, 2)...) + + n1 := sqlutils.MakeSQLRunner(tc.ServerConn(0)) + + h := newFlowControlTestHelper(t, tc) + h.init() + defer h.close("class_prioritization") + + desc, err := tc.LookupRange(k) + require.NoError(t, err) + h.waitForConnectedStreams(ctx, desc.RangeID, 3) + + h.comment(`-- (Issuing 1x1MiB, 3x replicated elastic write that's not admitted.)`) + h.put(ctx, k, 1<<20 /* 1MiB */, admissionpb.BulkNormalPri) + + h.comment(` +-- Flow token metrics from n1 after issuing 1x1MiB elastic 3x replicated write +-- that's not admitted. We see 1*1MiB*3=3MiB deductions of elastic tokens with +-- no corresponding returns. +`) + h.query(n1, ` + SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvadmission%tokens%' +ORDER BY name ASC; +`) + + h.comment(`-- (Issuing 1x1MiB, 3x replicated regular write that's not admitted.)`) + h.put(ctx, k, 1<<20 /* 1MiB */, admissionpb.NormalPri) + + h.comment(` +-- Flow token metrics from n1 after issuing 1x1MiB regular 3x replicated write +-- that's not admitted. We see 1*1MiB*3=3MiB deductions of {regular,elastic} +-- tokens with no corresponding returns. +`) + h.query(n1, ` + SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvadmission%tokens%' +ORDER BY name ASC; +`) + + h.comment(`-- (Allow below-raft admission to proceed.)`) + disableWorkQueueGranting.Store(false) + h.waitForAllTokensReturned(ctx, 3) + + h.comment(` +-- Flow token metrics from n1 after work gets admitted. All {regular,elastic} +-- tokens deducted are returned. +`) + h.query(n1, ` + SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvadmission%tokens%' +ORDER BY name ASC; +`) +} + +// TestFlowControlQuiescedRange tests flow token behavior when ranges are +// quiesced. It ensures that we have timely returns of flow tokens even when +// there's no raft traffic to piggyback token returns on top of. +func TestFlowControlQuiescedRange(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + ctx := context.Background() + const numNodes = 3 + var disableWorkQueueGranting atomic.Bool + var disableFallbackTokenDispatch atomic.Bool + disableWorkQueueGranting.Store(true) + disableFallbackTokenDispatch.Store(true) + + st := cluster.MakeTestingClusterSettings() + kvserver.ExpirationLeasesOnly.Override(ctx, &st.SV, false) // override metamorphism + + tc := testcluster.StartTestCluster(t, numNodes, base.TestClusterArgs{ + ReplicationMode: base.ReplicationManual, + ServerArgs: base.TestServerArgs{ + Settings: st, + RaftConfig: base.RaftConfig{ + // Suppress timeout-based elections. This test doesn't want to + // deal with leadership changing hands. + RaftElectionTimeoutTicks: 1000000, + }, + Knobs: base.TestingKnobs{ + Store: &kvserver.StoreTestingKnobs{ + FlowControlTestingKnobs: &kvflowcontrol.TestingKnobs{ + OverrideTokenDeduction: func() kvflowcontrol.Tokens { + // This test asserts on the exact values of tracked + // tokens. In non-test code, the tokens deducted are + // a few bytes off (give or take) from the size of + // the proposals. We don't care about such + // differences. + return kvflowcontrol.Tokens(1 << 20 /* 1MiB */) + }, + UseOnlyForScratchRanges: true, + }, + }, + AdmissionControl: &admission.TestingKnobs{ + DisableWorkQueueFastPath: true, + DisableWorkQueueGranting: func() bool { + return disableWorkQueueGranting.Load() + }, + }, + RaftTransport: &kvserver.RaftTransportTestingKnobs{ + DisableFallbackFlowTokenDispatch: func() bool { + return disableFallbackTokenDispatch.Load() + }, + DisablePiggyBackedFlowTokenDispatch: func() bool { + return true // we'll only test using the fallback token mechanism + }, + }, + }, + }, + }) + defer tc.Stopper().Stop(ctx) + + k := tc.ScratchRange(t) + tc.AddVotersOrFatal(t, k, tc.Targets(1, 2)...) + + n1 := sqlutils.MakeSQLRunner(tc.ServerConn(0)) + + h := newFlowControlTestHelper(t, tc) + h.init() + defer h.close("quiesced_range") + + desc, err := tc.LookupRange(k) + require.NoError(t, err) + h.waitForConnectedStreams(ctx, desc.RangeID, 3) + + h.comment(`-- (Issuing 1x1MiB, 3x replicated elastic write that's not admitted.)`) + h.put(ctx, k, 1<<20 /* 1MiB */, admissionpb.BulkNormalPri) + + h.comment(` +-- Flow token metrics from n1 after issuing 1x1MiB elastic 3x replicated write +-- that's not admitted. We see 1*1MiB*3=3MiB deductions of elastic tokens with +-- no corresponding returns. +`) + h.query(n1, ` + SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvadmission%elastic_tokens%' +ORDER BY name ASC; +`) + + // Wait for the range to quiesce. + h.comment(`-- (Wait for range to quiesce.)`) + testutils.SucceedsSoon(t, func() error { + for i := range tc.Servers { + rep := tc.GetFirstStoreFromServer(t, i).LookupReplica(roachpb.RKey(k)) + require.NotNil(t, rep) + if !rep.IsQuiescent() { + return errors.Errorf("%s not quiescent", rep) + } + } + return nil + }) + + h.comment(` +-- (Allow below-raft admission to proceed. We've disabled the fallback token +-- dispatch mechanism so no tokens are returned yet -- quiesced ranges don't +-- use the piggy-backed token return mechanism since there's no raft traffic.)`) + disableWorkQueueGranting.Store(false) + + h.comment(` +-- Flow token metrics from n1 after work gets admitted but fallback token +-- dispatch mechanism is disabled. Deducted elastic tokens from remote stores +-- are yet to be returned. Tokens for the local store are. +`) + h.waitForTotalTrackedTokens(ctx, desc.RangeID, 2<<20 /* 2*1MiB=2MiB */) + h.query(n1, ` + SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvadmission%elastic_tokens%' +ORDER BY name ASC; +`) + + h.comment(`-- (Enable the fallback token dispatch mechanism.)`) + disableFallbackTokenDispatch.Store(false) + h.waitForAllTokensReturned(ctx, 3) + + h.comment(` +-- Flow token metrics from n1 after work gets admitted and all elastic tokens +-- are returned through the fallback mechanism. +`) + h.query(n1, ` + SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvadmission%elastic_tokens%' +ORDER BY name ASC; +`) +} + +// TestFlowControlUnquiescedRange tests flow token behavior when ranges are +// unquiesced. It's a sort of roundabout test to ensure that flow tokens are +// returned through the raft transport piggybacking mechanism, piggybacking on +// raft heartbeats. +func TestFlowControlUnquiescedRange(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + ctx := context.Background() + const numNodes = 3 + var disableWorkQueueGranting atomic.Bool + var disablePiggybackTokenDispatch atomic.Bool + disableWorkQueueGranting.Store(true) + disablePiggybackTokenDispatch.Store(true) + + st := cluster.MakeTestingClusterSettings() + kvserver.ExpirationLeasesOnly.Override(ctx, &st.SV, false) // override metamorphism + + tc := testcluster.StartTestCluster(t, numNodes, base.TestClusterArgs{ + ReplicationMode: base.ReplicationManual, + ServerArgs: base.TestServerArgs{ + Settings: st, + RaftConfig: base.RaftConfig{ + // Suppress timeout-based elections. This test doesn't want to + // deal with leadership changing hands or followers unquiescing + // ranges by calling elections. + RaftElectionTimeoutTicks: 1000000, + }, + Knobs: base.TestingKnobs{ + Store: &kvserver.StoreTestingKnobs{ + FlowControlTestingKnobs: &kvflowcontrol.TestingKnobs{ + OverrideTokenDeduction: func() kvflowcontrol.Tokens { + // This test asserts on the exact values of tracked + // tokens. In non-test code, the tokens deducted are + // a few bytes off (give or take) from the size of + // the proposals. We don't care about such + // differences. + return kvflowcontrol.Tokens(1 << 20 /* 1MiB */) + }, + UseOnlyForScratchRanges: true, + MaintainStreamsForInactiveFollowers: func() bool { + // This test deals with quiesced ranges where + // followers have no activity. We don't want to + // disconnect streams due to this inactivity. + return true + }, + }, + }, + AdmissionControl: &admission.TestingKnobs{ + DisableWorkQueueFastPath: true, + DisableWorkQueueGranting: func() bool { + return disableWorkQueueGranting.Load() + }, + }, + RaftTransport: &kvserver.RaftTransportTestingKnobs{ + DisableFallbackFlowTokenDispatch: func() bool { + return true // we'll only test using the piggy-back token mechanism + }, + DisablePiggyBackedFlowTokenDispatch: func() bool { + return disablePiggybackTokenDispatch.Load() + }, + }, + }, + }, + }) + defer tc.Stopper().Stop(ctx) + + k := tc.ScratchRange(t) + tc.AddVotersOrFatal(t, k, tc.Targets(1, 2)...) + + n1 := sqlutils.MakeSQLRunner(tc.ServerConn(0)) + + h := newFlowControlTestHelper(t, tc) + h.init() + defer h.close("unquiesced_range") + + desc, err := tc.LookupRange(k) + require.NoError(t, err) + + for i := 0; i < numNodes; i++ { + si, err := tc.Server(i).GetStores().(*kvserver.Stores).GetStore(tc.Server(i).GetFirstStoreID()) + require.NoError(t, err) + tc.Servers[i].RaftTransport().Listen(si.StoreID(), + &unreliableRaftHandler{ + rangeID: desc.RangeID, + RaftMessageHandler: si, + unreliableRaftHandlerFuncs: unreliableRaftHandlerFuncs{ + dropReq: func(req *kvserverpb.RaftMessageRequest) bool { + // Install a raft handler to get verbose raft logging. + // + // TODO(irfansharif): Make this a more ergonomic + // testing knob instead. + return false + }, + }, + }) + } + + h.waitForConnectedStreams(ctx, desc.RangeID, 3) + + h.comment(`-- (Issuing 1x1MiB, 3x replicated elastic write that's not admitted.)`) + h.put(ctx, k, 1<<20 /* 1MiB */, admissionpb.BulkNormalPri) + + h.comment(` +-- Flow token metrics from n1 after issuing 1x1MiB elastic 3x replicated write +-- that's not admitted. We see 1*1MiB*3=3MiB deductions of elastic tokens with +-- no corresponding returns. +`) + h.query(n1, ` + SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvadmission%elastic_tokens%' +ORDER BY name ASC; +`) + + // Wait for the range to quiesce. + h.comment(`-- (Wait for range to quiesce.)`) + testutils.SucceedsSoon(t, func() error { + leader := tc.GetRaftLeader(t, roachpb.RKey(k)) + require.NotNil(t, leader) + if !leader.IsQuiescent() { + return errors.Errorf("%s not quiescent", leader) + } + return nil + }) + + h.comment(` +-- (Allow below-raft admission to proceed. We've disabled the fallback token +-- dispatch mechanism so no tokens are returned yet -- quiesced ranges don't +-- use the piggy-backed token return mechanism since there's no raft traffic.)`) + disableWorkQueueGranting.Store(false) + + h.comment(` +-- Flow token metrics from n1 after work gets admitted but fallback token +-- dispatch mechanism is disabled. Deducted elastic tokens from remote stores +-- are yet to be returned. Tokens for the local store are. +`) + h.waitForTotalTrackedTokens(ctx, desc.RangeID, 2<<20 /* 2*1MiB=2MiB */) + h.query(n1, ` + SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvadmission%elastic_tokens%' +ORDER BY name ASC; +`) + + h.comment(`-- (Enable the piggyback token dispatch mechanism.)`) + disablePiggybackTokenDispatch.Store(false) + + h.comment(`-- (Unquiesce the range.)`) + testutils.SucceedsSoon(t, func() error { + tc.GetRaftLeader(t, roachpb.RKey(k)).MaybeUnquiesceAndWakeLeader() + return h.checkAllTokensReturned(ctx, 3) + }) + + h.comment(` +-- Flow token metrics from n1 after work gets admitted and all elastic tokens +-- are returned through the piggyback mechanism. +`) + h.query(n1, ` + SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvadmission%elastic_tokens%' +ORDER BY name ASC; +`) +} + +// TestFlowControlTransferLease tests flow control behavior when the range lease +// is transferred, and the raft leadership along with it. +func TestFlowControlTransferLease(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + ctx := context.Background() + const numNodes = 5 + var disableWorkQueueGranting atomic.Bool + disableWorkQueueGranting.Store(true) + tc := testcluster.StartTestCluster(t, numNodes, base.TestClusterArgs{ + ReplicationMode: base.ReplicationManual, + ServerArgs: base.TestServerArgs{ + Knobs: base.TestingKnobs{ + Store: &kvserver.StoreTestingKnobs{ + FlowControlTestingKnobs: &kvflowcontrol.TestingKnobs{ + UseOnlyForScratchRanges: true, + }, + }, + AdmissionControl: &admission.TestingKnobs{ + DisableWorkQueueFastPath: true, + DisableWorkQueueGranting: func() bool { + return disableWorkQueueGranting.Load() + }, + }, + }, + }, + }) + defer tc.Stopper().Stop(ctx) + + k := tc.ScratchRange(t) + tc.AddVotersOrFatal(t, k, tc.Targets(1, 2)...) + + n1 := sqlutils.MakeSQLRunner(tc.ServerConn(0)) + + h := newFlowControlTestHelper(t, tc) + h.init() + defer h.close("transfer_lease") + + desc, err := tc.LookupRange(k) + require.NoError(t, err) + h.waitForConnectedStreams(ctx, desc.RangeID, 3) + + h.comment(`-- (Issuing 1x1MiB, 3x replicated write that's not admitted.)`) + h.put(ctx, k, 1<<20 /* 1MiB */, admissionpb.NormalPri) + + h.comment(` +-- Flow token metrics from n1 after issuing 1x1MiB regular 3x replicated write +-- that's not admitted. We see 1*1MiB*3=3MiB deductions of regular tokens with +-- no corresponding returns. +`) + h.query(n1, ` + SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvadmission%regular_tokens%' +ORDER BY name ASC; +`) + + h.comment(`-- (Transferring range lease to n2 and allowing leadership to follow.)`) + tc.TransferRangeLeaseOrFatal(t, desc, tc.Target(1)) + testutils.SucceedsSoon(t, func() error { + if leader := tc.GetRaftLeader(t, roachpb.RKey(k)); leader.NodeID() != tc.Target(1).NodeID { + return errors.Errorf("expected raft leadership to transfer to n1, found n%d", leader.NodeID()) + } + return nil + }) + h.waitForAllTokensReturned(ctx, 3) + + h.comment(` +-- Flow token metrics from n1 having lost the lease and raft leadership. All +-- deducted tokens are returned. +`) + h.query(n1, ` + SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvadmission%regular_tokens%' +ORDER BY name ASC; +`) +} + +// TestFlowControlLeaderNotLeaseholder tests flow control behavior when the +// range leaseholder is not the raft leader. +func TestFlowControlLeaderNotLeaseholder(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + ctx := context.Background() + const numNodes = 5 + var disableWorkQueueGranting atomic.Bool + disableWorkQueueGranting.Store(true) + tc := testcluster.StartTestCluster(t, numNodes, base.TestClusterArgs{ + ReplicationMode: base.ReplicationManual, + ServerArgs: base.TestServerArgs{ + Knobs: base.TestingKnobs{ + Store: &kvserver.StoreTestingKnobs{ + // Disable leader transfers during leaseholder changes so + // that we can easily create leader-not-leaseholder + // scenarios. + DisableLeaderFollowsLeaseholder: true, + FlowControlTestingKnobs: &kvflowcontrol.TestingKnobs{ + UseOnlyForScratchRanges: true, + }, + }, + AdmissionControl: &admission.TestingKnobs{ + DisableWorkQueueFastPath: true, + DisableWorkQueueGranting: func() bool { + return disableWorkQueueGranting.Load() + }, + }, + }, + }, + }) + defer tc.Stopper().Stop(ctx) + + k := tc.ScratchRange(t) + tc.AddVotersOrFatal(t, k, tc.Targets(1, 2)...) + + n1 := sqlutils.MakeSQLRunner(tc.ServerConn(0)) + n2 := sqlutils.MakeSQLRunner(tc.ServerConn(1)) + + h := newFlowControlTestHelper(t, tc) + h.init() + defer h.close("leader_not_leaseholder") + + desc, err := tc.LookupRange(k) + require.NoError(t, err) + h.waitForConnectedStreams(ctx, desc.RangeID, 3) + + h.comment(`-- (Issuing 1x1MiB, 3x replicated write that's not admitted.)`) + h.put(ctx, k, 1<<20 /* 1MiB */, admissionpb.NormalPri) + + h.comment(` +-- Flow token metrics from n1 after issuing 1x1MiB regular 3x replicated write +-- that's not admitted. We see 1*1MiB*3=3MiB deductions of regular tokens with +-- no corresponding returns. +`) + h.query(n1, ` + SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvadmission%regular_tokens%' +ORDER BY name ASC; +`) + + h.comment(`-- (Transferring only range lease, not raft leadership, to n2.)`) + tc.TransferRangeLeaseOrFatal(t, desc, tc.Target(1)) + require.Equal(t, tc.GetRaftLeader(t, roachpb.RKey(k)).NodeID(), tc.Target(0).NodeID) + + h.comment(` +-- Flow token metrics from n1 having lost the lease but retained raft +-- leadership. No deducted tokens are released. +`) + h.query(n1, ` + SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvadmission%regular_tokens%' +ORDER BY name ASC; +`) + + h.comment(` +-- (Issuing another 1x1MiB, 3x replicated write that's not admitted while in +-- this leader != leaseholder state.) +`) + h.put(ctx, k, 1<<20 /* 1MiB */, admissionpb.NormalPri) + + h.comment(` +-- Looking at n1's flow token metrics, there's no change. No additional tokens +-- are deducted since the write is not being proposed here. +`) + h.query(n1, ` + SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvadmission%regular_tokens%' +ORDER BY name ASC; +`) + + h.comment(` +-- Looking at n2's flow token metrics, there's no activity. n2 never acquired +-- the raft leadership. +`) + h.query(n2, ` + SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvadmission%regular_tokens%' +ORDER BY name ASC; +`) + + h.comment(`-- (Allow below-raft admission to proceed.)`) + disableWorkQueueGranting.Store(false) + h.waitForAllTokensReturned(ctx, 3) // wait for admission + + h.comment(` +-- All deducted flow tokens are returned back to where the raft leader is. +`) + h.query(n1, ` + SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvadmission%regular_tokens%' +ORDER BY name ASC; +`) +} + +type flowControlTestHelper struct { + t *testing.T + tc *testcluster.TestCluster + buf *strings.Builder + rng *rand.Rand +} + +func newFlowControlTestHelper(t *testing.T, tc *testcluster.TestCluster) *flowControlTestHelper { + rng, _ := randutil.NewPseudoRand() + buf := &strings.Builder{} + return &flowControlTestHelper{ + t: t, + tc: tc, + buf: buf, + rng: rng, + } +} + +func (h *flowControlTestHelper) init() { + if _, err := h.tc.Conns[0].Exec(`SET CLUSTER SETTING kvadmission.flow_control.enabled = true`); err != nil { + h.t.Fatal(err) + } + if _, err := h.tc.Conns[0].Exec(`SET CLUSTER SETTING kvadmission.flow_control.mode = 'apply_to_all'`); err != nil { + h.t.Fatal(err) + } +} + +func (h *flowControlTestHelper) waitForAllTokensReturned(ctx context.Context, expStreamCount int) { + testutils.SucceedsSoon(h.t, func() error { + return h.checkAllTokensReturned(ctx, expStreamCount) + }) +} + +func (h *flowControlTestHelper) checkAllTokensReturned( + ctx context.Context, expStreamCount int, +) error { + kfc := h.tc.Server(0).KVFlowController().(kvflowcontrol.Controller) + streams := kfc.Inspect(ctx) + if len(streams) != expStreamCount { + return fmt.Errorf("expected %d replication streams, got %d", expStreamCount, len(streams)) + } + for _, stream := range streams { + if stream.AvailableRegularTokens != 16<<20 { + return fmt.Errorf("expected %s of regular flow tokens for %s, got %s", + humanize.IBytes(16<<20), + kvflowcontrol.Stream{ + TenantID: stream.TenantID, + StoreID: stream.StoreID, + }, + humanize.IBytes(uint64(stream.AvailableRegularTokens)), + ) + } + if stream.AvailableElasticTokens != 8<<20 { + return fmt.Errorf("expected %s of elastic flow tokens for %s, got %s", + humanize.IBytes(8<<20), + kvflowcontrol.Stream{ + TenantID: stream.TenantID, + StoreID: stream.StoreID, + }, + humanize.IBytes(uint64(stream.AvailableElasticTokens)), + ) + } + } + return nil +} + +func (h *flowControlTestHelper) waitForConnectedStreams( + ctx context.Context, rangeID roachpb.RangeID, expConnectedStreams int, +) { + testutils.SucceedsSoon(h.t, func() error { + kfh := h.tc.Server(0).KVFlowHandles().(kvflowcontrol.Handles) + handle, found := kfh.Lookup(rangeID) + if !found { + return fmt.Errorf("handle for %s not found", rangeID) + } + require.True(h.t, found) + state := handle.Inspect(ctx) + if len(state.ConnectedStreams) != expConnectedStreams { + return fmt.Errorf("expected %d connected streams, got %d", + expConnectedStreams, len(state.ConnectedStreams)) + } + return nil + }) +} + +func (h *flowControlTestHelper) waitForTotalTrackedTokens( + ctx context.Context, rangeID roachpb.RangeID, expTotalTrackedTokens int64, +) { + testutils.SucceedsSoon(h.t, func() error { + kfh := h.tc.Server(0).KVFlowHandles().(kvflowcontrol.Handles) + handle, found := kfh.Lookup(rangeID) + if !found { + return fmt.Errorf("handle for %s not found", rangeID) + } + require.True(h.t, found) + state := handle.Inspect(ctx) + var totalTracked int64 + for _, stream := range state.ConnectedStreams { + for _, tracked := range stream.TrackedDeductions { + totalTracked += tracked.Tokens + } + } + if totalTracked != expTotalTrackedTokens { + return fmt.Errorf("expected to track %d tokens in aggregate, got %d", + kvflowcontrol.Tokens(expTotalTrackedTokens), kvflowcontrol.Tokens(totalTracked)) + } + return nil + }) +} + +func (h *flowControlTestHelper) comment(comment string) { + if h.buf.Len() > 0 { + h.buf.WriteString("\n\n") + } + + comment = strings.TrimSpace(comment) + h.buf.WriteString(fmt.Sprintf("%s\n", comment)) + h.t.Log(comment) +} + +func (h *flowControlTestHelper) query(runner *sqlutils.SQLRunner, sql string, headers ...string) { + sql = strings.TrimSpace(sql) + h.t.Log(sql) + h.buf.WriteString(fmt.Sprintf("%s\n\n", sql)) + + rows := runner.Query(h.t, sql) + tbl := tablewriter.NewWriter(h.buf) + output, err := sqlutils.RowsToStrMatrix(rows) + require.NoError(h.t, err) + tbl.SetAlignment(tablewriter.ALIGN_LEFT) + tbl.AppendBulk(output) + tbl.SetBorder(false) + tbl.SetHeader(headers) + tbl.SetAutoFormatHeaders(false) + tbl.Render() +} + +func (h *flowControlTestHelper) put( + ctx context.Context, key roachpb.Key, size int, pri admissionpb.WorkPriority, +) *kvpb.BatchRequest { + value := roachpb.MakeValueFromString(randutil.RandString(h.rng, size, randutil.PrintableKeyAlphabet)) + ba := &kvpb.BatchRequest{} + ba.Add(kvpb.NewPut(key, value)) + ba.AdmissionHeader.Priority = int32(pri) + ba.AdmissionHeader.Source = kvpb.AdmissionHeader_FROM_SQL + if _, pErr := h.tc.Server(0).DB().NonTransactionalSender().Send( + ctx, ba, + ); pErr != nil { + h.t.Fatal(pErr.GoError()) + } + return ba +} + +func (h *flowControlTestHelper) close(filename string) { + echotest.Require(h.t, h.buf.String(), datapathutils.TestDataPath(h.t, "flow_control_integration", filename)) +} diff --git a/pkg/kv/kvserver/flow_control_replica.go b/pkg/kv/kvserver/flow_control_replica.go index 0f73e3241995..bc70455ad867 100644 --- a/pkg/kv/kvserver/flow_control_replica.go +++ b/pkg/kv/kvserver/flow_control_replica.go @@ -34,7 +34,8 @@ type replicaForFlowControl interface { isRaftTransportConnectedTo(roachpb.StoreID) bool withReplicaProgress(f func(roachpb.ReplicaID, rafttracker.Progress)) - assertLocked() // only affects test builds + assertLocked() // only affects test builds + isScratchRange() bool // only used in tests } // replicaFlowControl is a concrete implementation of the replicaForFlowControl @@ -105,3 +106,10 @@ func (rf *replicaFlowControl) withReplicaProgress(f func(roachpb.ReplicaID, raft f(roachpb.ReplicaID(id), progress) }) } + +func (rf *replicaFlowControl) isScratchRange() bool { + rf.mu.AssertHeld() + r := (*Replica)(rf) + return r.isScratchRangeRLocked() +} + diff --git a/pkg/kv/kvserver/flow_control_replica_integration.go b/pkg/kv/kvserver/flow_control_replica_integration.go index fe658e86f8ba..36260bfded16 100644 --- a/pkg/kv/kvserver/flow_control_replica_integration.go +++ b/pkg/kv/kvserver/flow_control_replica_integration.go @@ -20,17 +20,6 @@ import ( rafttracker "go.etcd.io/raft/v3/tracker" ) -// TODO(irfansharif): Write integration tests, walking through -// kvflowcontrol/doc.go. Do this as part of #95563. Randomize an in-memory -// workload with various chaos events, like nodes dying, streams breaking, -// splits, merges, etc. and assert stable flow tokens. Leader replica removing -// itself from descriptor. Uninitialized replica below raft for which we've -// deducted flow tokens for (dealt with by looking at StateReplicate). Dropped -// proposals -- we should only be deducting tokens once submitting to raft. But -// if leader's raft messages to follower get dropped (and not vice versa), -// leader will still see follower as active and not disconnect streams. Has this -// changed with us upgrading asymmetric partitions to bidirectional ones? - // replicaFlowControlIntegration is used to integrate with replication flow // control. It's intercepts various points in a replica's lifecycle, like it // acquiring raft leadership or losing it, or its raft membership changing, etc. @@ -57,14 +46,22 @@ type replicaFlowControlIntegrationImpl struct { innerHandle kvflowcontrol.Handle lastKnownReplicas roachpb.ReplicaSet disconnectedStreams map[roachpb.ReplicaID]kvflowcontrol.Stream + + knobs *kvflowcontrol.TestingKnobs } func newReplicaFlowControlIntegration( - replicaForFlowControl replicaForFlowControl, handleFactory kvflowcontrol.HandleFactory, + replicaForFlowControl replicaForFlowControl, + handleFactory kvflowcontrol.HandleFactory, + knobs *kvflowcontrol.TestingKnobs, ) *replicaFlowControlIntegrationImpl { + if knobs == nil { + knobs = &kvflowcontrol.TestingKnobs{} + } return &replicaFlowControlIntegrationImpl{ replicaForFlowControl: replicaForFlowControl, handleFactory: handleFactory, + knobs: knobs, } } @@ -82,6 +79,10 @@ func (f *replicaFlowControlIntegrationImpl) onBecameLeader(ctx context.Context) log.Fatal(ctx, "unset tenant ID") } + if f.knobs.UseOnlyForScratchRanges && !f.replicaForFlowControl.isScratchRange() { + return + } + // See I5 from kvflowcontrol/doc.go. The per-replica kvflowcontrol.Handle is // tied to the lifetime of a leaseholder replica having raft leadership. f.innerHandle = f.handleFactory.NewHandle( @@ -271,6 +272,9 @@ func (f *replicaFlowControlIntegrationImpl) onProposalQuotaUpdated(ctx context.C // likely unintentional, but we paper over it here. continue // nothing to do } + if fn := f.knobs.MaintainStreamsForInactiveFollowers; fn != nil && fn() { + continue // nothing to do + } toDisconnect = append(toDisconnect, repl) } f.disconnectStreams(ctx, toDisconnect, "inactive followers") @@ -331,6 +335,9 @@ func (f *replicaFlowControlIntegrationImpl) notActivelyReplicatingTo() []roachpb } if !f.replicaForFlowControl.isRaftTransportConnectedTo(repl.StoreID) { + if fn := f.knobs.MaintainStreamsForBrokenRaftTransport; fn != nil && fn() { + return // nothing to do + } res = append(res, replID) } }) @@ -362,6 +369,10 @@ func (f *replicaFlowControlIntegrationImpl) onRaftTransportDisconnected( return // nothing to do } + if fn := f.knobs.MaintainStreamsForBrokenRaftTransport; fn != nil && fn() { + return // nothing to do + } + disconnectedStores := make(map[roachpb.StoreID]struct{}) for _, storeID := range storeIDs { disconnectedStores[storeID] = struct{}{} diff --git a/pkg/kv/kvserver/flow_control_replica_integration_test.go b/pkg/kv/kvserver/flow_control_replica_integration_test.go index 4d501292531d..f5efd66670c3 100644 --- a/pkg/kv/kvserver/flow_control_replica_integration_test.go +++ b/pkg/kv/kvserver/flow_control_replica_integration_test.go @@ -116,7 +116,7 @@ func TestFlowControlReplicaIntegration(t *testing.T) { mockHandleFactory = newMockFlowHandleFactory(t, logger) mockReplica = newMockReplicaForFlowControl(t, rangeID, tenantID, replID) - integration = newReplicaFlowControlIntegration(mockReplica, mockHandleFactory) + integration = newReplicaFlowControlIntegration(mockReplica, mockHandleFactory, nil /* knobs */) return "" case "state": diff --git a/pkg/kv/kvserver/flow_control_stores.go b/pkg/kv/kvserver/flow_control_stores.go index e1f58c832d85..6981b06ac1b6 100644 --- a/pkg/kv/kvserver/flow_control_stores.go +++ b/pkg/kv/kvserver/flow_control_stores.go @@ -121,6 +121,12 @@ func (sh *storeForFlowControl) Lookup( return nil, false } + if knobs := s.TestingKnobs().FlowControlTestingKnobs; knobs != nil && + knobs.UseOnlyForScratchRanges && + !repl.IsScratchRange() { + return nil, false + } + repl.mu.Lock() defer repl.mu.Unlock() return repl.mu.replicaFlowControlIntegration.handle() @@ -181,12 +187,17 @@ func (shf *storeFlowControlHandleFactory) NewHandle( rangeID roachpb.RangeID, tenantID roachpb.TenantID, ) kvflowcontrol.Handle { s := (*Store)(shf) + var knobs *kvflowcontrol.TestingKnobs + if s.TestingKnobs() != nil { + knobs = s.TestingKnobs().FlowControlTestingKnobs + } return kvflowhandle.New( s.cfg.KVFlowController, s.cfg.KVFlowHandleMetrics, s.cfg.Clock, rangeID, tenantID, + knobs, ) } diff --git a/pkg/kv/kvserver/kvflowcontrol/doc.go b/pkg/kv/kvserver/kvflowcontrol/doc.go index c48e23e7ebfb..5882d6b80a1e 100644 --- a/pkg/kv/kvserver/kvflowcontrol/doc.go +++ b/pkg/kv/kvserver/kvflowcontrol/doc.go @@ -234,14 +234,19 @@ package kvflowcontrol // I5. What happens when the leaseholder and/or the raft leader changes? When // the raft leader is not the same as the leaseholder? // - The per-replica kvflowcontrol.Handle is tied to the lifetime of a -// leaseholder replica having raft leadership. When leadership is lost, or the -// lease changes hands, we release all held flow tokens. +// replica having raft leadership. When leadership is lost we release all held +// flow tokens. Tokens are only deducted at proposal time when the proposing +// replica is both the raft leader and leaseholder (the latter is tautological +// since only leaseholders propose). We're relying on timely acquisition of +// raft leadership by the leaseholder to not be persistently over admitting. +// - Even if the lease transfers but raft leadership remains, and there's +// later admission of writes for which tokens were originally deducted at +// the raft leader, it's the raft leader that's informed of that admission +// who then releases those tokens. // - Avoiding double returns on subsequent AdmittedRaftLogEntries for these -// already released flow tokens is easier for raft leadership changes since +// already released flow tokens is easy for raft leadership changes since // there's a term change, and all earlier/stale AdmittedRaftLogEntries with -// the lower term can be discarded. We do a similar thing for leases -- when -// being granted a lease, the low water mark in kvflowcontrol.Handle is at -// least as high as the command that transferred the lease. +// the lower term can be discarded. // // I6. What happens during replica GC? // - It's unlikely that a replica gets GC-ed without first going through the diff --git a/pkg/kv/kvserver/kvflowcontrol/kvflowhandle/kvflowhandle.go b/pkg/kv/kvserver/kvflowcontrol/kvflowhandle/kvflowhandle.go index 778768130ef5..e14b3e9b343f 100644 --- a/pkg/kv/kvserver/kvflowcontrol/kvflowhandle/kvflowhandle.go +++ b/pkg/kv/kvserver/kvflowcontrol/kvflowhandle/kvflowhandle.go @@ -46,6 +46,7 @@ type Handle struct { perStreamTokenTracker map[kvflowcontrol.Stream]*kvflowtokentracker.Tracker closed bool } + knobs *kvflowcontrol.TestingKnobs } // New constructs a new Handle. @@ -55,16 +56,21 @@ func New( clock *hlc.Clock, rangeID roachpb.RangeID, tenantID roachpb.TenantID, + knobs *kvflowcontrol.TestingKnobs, ) *Handle { if metrics == nil { // only nil in tests metrics = NewMetrics(nil) } + if knobs == nil { + knobs = &kvflowcontrol.TestingKnobs{} + } h := &Handle{ controller: controller, metrics: metrics, clock: clock, rangeID: rangeID, tenantID: tenantID, + knobs: knobs, } h.mu.perStreamTokenTracker = map[kvflowcontrol.Stream]*kvflowtokentracker.Tracker{} return h @@ -140,6 +146,10 @@ func (h *Handle) deductTokensForInner( return nil // unused return value in production code } + if h.knobs.OverrideTokenDeduction != nil { + tokens = h.knobs.OverrideTokenDeduction() + } + for _, c := range h.mu.connections { if h.mu.perStreamTokenTracker[c.Stream()].Track(ctx, pri, tokens, pos) { // Only deduct tokens if we're able to track them for subsequent @@ -234,7 +244,7 @@ func (h *Handle) connectStreamLocked( // that case, this sorting will help avoid deadlocks. return h.mu.connections[i].Stream().StoreID < h.mu.connections[j].Stream().StoreID }) - h.mu.perStreamTokenTracker[stream] = kvflowtokentracker.New(pos, stream, nil /* knobs */) + h.mu.perStreamTokenTracker[stream] = kvflowtokentracker.New(pos, stream, h.knobs) h.metrics.StreamsConnected.Inc(1) log.VInfof(ctx, 1, "connected to stream: %s", stream) } diff --git a/pkg/kv/kvserver/kvflowcontrol/kvflowhandle/kvflowhandle_test.go b/pkg/kv/kvserver/kvflowcontrol/kvflowhandle/kvflowhandle_test.go index 9dbffa17d400..2d0201acb203 100644 --- a/pkg/kv/kvserver/kvflowcontrol/kvflowhandle/kvflowhandle_test.go +++ b/pkg/kv/kvserver/kvflowcontrol/kvflowhandle/kvflowhandle_test.go @@ -98,6 +98,7 @@ func TestHandleAdmit(t *testing.T) { clock, roachpb.RangeID(1), roachpb.SystemTenantID, + nil, /* knobs */ ) // Connect a single stream at pos=0 and deplete all 16MiB of regular @@ -179,6 +180,7 @@ func TestFlowControlMode(t *testing.T) { clock, roachpb.RangeID(1), roachpb.SystemTenantID, + nil, /* knobs */ ) defer handle.Close(ctx) @@ -258,6 +260,7 @@ func TestInspectHandle(t *testing.T) { clock, roachpb.RangeID(1), roachpb.SystemTenantID, + nil, /* knobs */ ) marshaller := jsonpb.Marshaler{ Indent: " ", diff --git a/pkg/kv/kvserver/kvflowcontrol/kvflowsimulator/simulation_test.go b/pkg/kv/kvserver/kvflowcontrol/kvflowsimulator/simulation_test.go index 7bb66c32d45d..09c5491c0a48 100644 --- a/pkg/kv/kvserver/kvflowcontrol/kvflowsimulator/simulation_test.go +++ b/pkg/kv/kvserver/kvflowcontrol/kvflowsimulator/simulation_test.go @@ -176,7 +176,7 @@ func TestUsingSimulation(t *testing.T) { for _, line := range strings.Split(d.Input, "\n") { name := strings.TrimPrefix(strings.TrimSpace(line), "handle=") replicaHandles[name] = &replicaHandle{ - handle: kvflowhandle.New(controller, hmetrics, clock, roachpb.RangeID(0), roachpb.TenantID{}), + handle: kvflowhandle.New(controller, hmetrics, clock, roachpb.RangeID(0), roachpb.TenantID{}, nil /* knobs */), deductionTracker: make(map[kvflowcontrol.Stream]*kvflowtokentracker.Tracker), outstandingReturns: make(map[kvflowcontrol.Stream]kvflowcontrol.Tokens), snapshots: make([]snapshot, 0), diff --git a/pkg/kv/kvserver/kvflowcontrol/testing_knobs.go b/pkg/kv/kvserver/kvflowcontrol/testing_knobs.go index bfdf3a324126..d09c942107db 100644 --- a/pkg/kv/kvserver/kvflowcontrol/testing_knobs.go +++ b/pkg/kv/kvserver/kvflowcontrol/testing_knobs.go @@ -21,6 +21,19 @@ type TestingKnobs struct { // UntrackTokensInterceptor is invoked whenever tokens are untracked, along // with their corresponding log positions. UntrackTokensInterceptor func(Tokens, kvflowcontrolpb.RaftLogPosition) + // OverrideTokenDeduction is used to override how many tokens are deducted + // post-evaluation. + OverrideTokenDeduction func() Tokens + // MaintainStreamsForInactiveFollowers is used in tests to maintain + // replication streams for inactive followers. + MaintainStreamsForInactiveFollowers func() bool + // MaintainStreamsForBrokenRaftTransport is used in tests to maintain + // replication streams for followers we're no longer connected to via the + // RaftTransport. + MaintainStreamsForBrokenRaftTransport func() bool + // UseOnlyForScratchRanges enables the use of kvflowcontrol + // only for scratch ranges. + UseOnlyForScratchRanges bool } // ModuleTestingKnobs is part of the base.ModuleTestingKnobs interface. diff --git a/pkg/kv/kvserver/raft_transport.go b/pkg/kv/kvserver/raft_transport.go index 7892290fcb68..d6e0c1c505cf 100644 --- a/pkg/kv/kvserver/raft_transport.go +++ b/pkg/kv/kvserver/raft_transport.go @@ -676,6 +676,10 @@ func (t *RaftTransport) processQueue( var raftIdleTimer timeutil.Timer defer raftIdleTimer.Stop() + idleTimeout := raftIdleTimeout + if overrideFn := t.knobs.OverrideIdleTimeout; overrideFn != nil { + idleTimeout = overrideFn() + } var dispatchPendingFlowTokensTimer timeutil.Timer defer dispatchPendingFlowTokensTimer.Stop() @@ -688,7 +692,7 @@ func (t *RaftTransport) processQueue( batch := &kvserverpb.RaftMessageRequestBatch{} for { - raftIdleTimer.Reset(raftIdleTimeout) + raftIdleTimer.Reset(idleTimeout) select { case <-t.stopper.ShouldQuiesce(): @@ -706,21 +710,25 @@ func (t *RaftTransport) processQueue( q.bytes.Add(-size) budget := targetRaftOutgoingBatchSize.Get(&t.st.SV) - size - // Piggyback any pending flow token dispatches on raft transport - // messages already bound for the remote node. If the stream - // over which we're returning these flow tokens breaks, this is - // detected by the remote node, where tokens were originally - // deducted, who then frees up all held tokens (see I1 from - // kvflowcontrol/doc.go). If the stream is culled because it's - // idle, that's deducted remotely using the same stream-break - // mechanism. If there are no open streams to a given node and - // there's still pending flow tokens, we'll drop those tokens to - // reclaim memory in dropFlowTokensForDisconnectedNodes. For - // idle-but-not-culled connections, we have a fallback timer to - // periodically transmit one-off RaftMessageRequests for timely - // token returns. - pendingDispatches := t.kvflowControl.dispatchReader.PendingDispatchFor(q.nodeID) - maybeAnnotateWithAdmittedRaftLogEntries(req, pendingDispatches) + var pendingDispatches []kvflowcontrolpb.AdmittedRaftLogEntries + if disableFn := t.knobs.DisablePiggyBackedFlowTokenDispatch; disableFn == nil || !disableFn() { + // Piggyback any pending flow token dispatches on raft transport + // messages already bound for the remote node. If the stream + // over which we're returning these flow tokens breaks, this is + // detected by the remote node, where tokens were originally + // deducted, who then frees up all held tokens (see I1 from + // kvflowcontrol/doc.go). If the stream is culled because it's + // idle, that's deducted remotely using the same stream-break + // mechanism. If there are no open streams to a given node and + // there's still pending flow tokens, we'll drop those tokens to + // reclaim memory in dropFlowTokensForDisconnectedNodes. For + // idle-but-not-culled connections, we have a fallback timer to + // periodically transmit one-off RaftMessageRequests for timely + // token returns. + pendingDispatches = t.kvflowControl.dispatchReader.PendingDispatchFor(q.nodeID) + maybeAnnotateWithAdmittedRaftLogEntries(req, pendingDispatches) + } + batch.Requests = append(batch.Requests, *req) releaseRaftMessageRequest(req) @@ -750,6 +758,10 @@ func (t *RaftTransport) processQueue( dispatchPendingFlowTokensTimer.Read = true dispatchPendingFlowTokensTimer.Reset(kvadmission.FlowTokenDispatchInterval.Get(&t.st.SV)) + if disableFn := t.knobs.DisableFallbackFlowTokenDispatch; disableFn != nil && disableFn() { + continue // nothing to do + } + pendingDispatches := t.kvflowControl.dispatchReader.PendingDispatchFor(q.nodeID) if len(pendingDispatches) == 0 { continue // nothing to do @@ -1140,6 +1152,16 @@ type RaftTransportTestingKnobs struct { // OnFallbackDispatch is invoked whenever the fallback token dispatch // mechanism is used. OnFallbackDispatch func() + // OverrideIdleTimeout overrides the raftIdleTimeout, which controls how + // long until an instance of processQueue winds down after not observing any + // messages. + OverrideIdleTimeout func() time.Duration + // DisableFallbackFlowTokenDispatch disables the fallback mechanism when + // dispatching flow tokens. + DisableFallbackFlowTokenDispatch func() bool + // DisablePiggyBackedFlowTokenDispatch disables the piggybacked mechanism + // when dispatching flow tokens. + DisablePiggyBackedFlowTokenDispatch func() bool } // ModuleTestingKnobs is part of the base.ModuleTestingKnobs interface. diff --git a/pkg/kv/kvserver/replica.go b/pkg/kv/kvserver/replica.go index 0a3645cd89ba..026474c4e111 100644 --- a/pkg/kv/kvserver/replica.go +++ b/pkg/kv/kvserver/replica.go @@ -960,6 +960,21 @@ func (r *Replica) SetSpanConfig(conf roachpb.SpanConfig) { r.mu.conf, r.mu.spanConfigExplicitlySet = conf, true } +// IsScratchRange returns true if this is range is a scratch range (i.e. +// overlaps with the scratch span and has a start key <= keys.ScratchRangeMin). +func (r *Replica) IsScratchRange() bool { + r.mu.RLock() + defer r.mu.RUnlock() + return r.isScratchRangeRLocked() +} + +func (r *Replica) isScratchRangeRLocked() bool { + rangeKeySpan := r.descRLocked().KeySpan() + rangeStartKey := rangeKeySpan.Key + return rangeKeySpan.AsRawSpanWithNoLocals().Overlaps(keys.ScratchSpan) && + roachpb.RKey(keys.ScratchRangeMin).Compare(rangeStartKey) <= 0 +} + // IsFirstRange returns true if this is the first range. func (r *Replica) IsFirstRange() bool { return r.RangeID == 1 diff --git a/pkg/kv/kvserver/replica_init.go b/pkg/kv/kvserver/replica_init.go index f152596728b0..fcdd89a9063f 100644 --- a/pkg/kv/kvserver/replica_init.go +++ b/pkg/kv/kvserver/replica_init.go @@ -164,6 +164,7 @@ func newUninitializedReplica( r.mu.replicaFlowControlIntegration = newReplicaFlowControlIntegration( (*replicaFlowControl)(r), makeStoreFlowControlHandleFactory(r.store), + r.store.TestingKnobs().FlowControlTestingKnobs, ) return r } diff --git a/pkg/kv/kvserver/replica_init_test.go b/pkg/kv/kvserver/replica_init_test.go index c3109cfa39c3..a8792bb8fcee 100644 --- a/pkg/kv/kvserver/replica_init_test.go +++ b/pkg/kv/kvserver/replica_init_test.go @@ -64,7 +64,7 @@ func TestReplicaUpdateLastReplicaAdded(t *testing.T) { var r Replica r.mu.state.Desc = &c.oldDesc r.mu.lastReplicaAdded = c.lastReplicaAdded - r.mu.replicaFlowControlIntegration = newReplicaFlowControlIntegration((*replicaFlowControl)(&r), nil) + r.mu.replicaFlowControlIntegration = newReplicaFlowControlIntegration((*replicaFlowControl)(&r), nil, nil) r.store = tc.store r.concMgr = tc.repl.concMgr r.setDescRaftMuLocked(context.Background(), &c.newDesc) diff --git a/pkg/kv/kvserver/testdata/flow_control_integration/basic b/pkg/kv/kvserver/testdata/flow_control_integration/basic new file mode 100644 index 000000000000..3590a83f4b08 --- /dev/null +++ b/pkg/kv/kvserver/testdata/flow_control_integration/basic @@ -0,0 +1,68 @@ +echo +---- +---- +-- Flow token metrics, before issuing the regular 1MiB replicated write. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvadmission%tokens%' +ORDER BY name ASC; + + kvadmission.flow_controller.elastic_tokens_available | 24 MiB + kvadmission.flow_controller.elastic_tokens_deducted | 0 B + kvadmission.flow_controller.elastic_tokens_returned | 0 B + kvadmission.flow_controller.elastic_tokens_unaccounted | 0 B + kvadmission.flow_controller.regular_tokens_available | 48 MiB + kvadmission.flow_controller.regular_tokens_deducted | 0 B + kvadmission.flow_controller.regular_tokens_returned | 0 B + kvadmission.flow_controller.regular_tokens_unaccounted | 0 B + + +-- (Issuing + admitting a regular 1MiB, triply replicated write...) + + +-- Stream counts as seen by n1 post-write. We should see three {regular,elastic} +-- streams given there are three nodes and we're using a replication factor of +-- three. +SELECT name, value + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvadmission.flow_controller%stream%' +ORDER BY name ASC; + + kvadmission.flow_controller.elastic_blocked_stream_count | 0 + kvadmission.flow_controller.elastic_stream_count | 3 + kvadmission.flow_controller.regular_blocked_stream_count | 0 + kvadmission.flow_controller.regular_stream_count | 3 + + +-- Another view of the stream count, using /inspectz-backed vtables. +SELECT range_id, count(*) AS streams + FROM crdb_internal.kv_flow_control_handles +GROUP BY (range_id) +ORDER BY streams DESC; + + range_id | stream_count +-----------+--------------- + 64 | 3 + + +-- Flow token metrics from n1 after issuing the regular 1MiB replicated write, +-- and it being admitted on n1, n2 and n3. We should see 3*1MiB = 3MiB of +-- {regular,elastic} tokens deducted and returned, and {8*3=24MiB,16*3=48MiB} of +-- {regular,elastic} tokens available. Everything should be accounted for. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvadmission%tokens%' +ORDER BY name ASC; + + kvadmission.flow_controller.elastic_tokens_available | 24 MiB + kvadmission.flow_controller.elastic_tokens_deducted | 3.0 MiB + kvadmission.flow_controller.elastic_tokens_returned | 3.0 MiB + kvadmission.flow_controller.elastic_tokens_unaccounted | 0 B + kvadmission.flow_controller.regular_tokens_available | 48 MiB + kvadmission.flow_controller.regular_tokens_deducted | 3.0 MiB + kvadmission.flow_controller.regular_tokens_returned | 3.0 MiB + kvadmission.flow_controller.regular_tokens_unaccounted | 0 B +---- +---- + +# vim:ft=sql diff --git a/pkg/kv/kvserver/testdata/flow_control_integration/blocked_admission b/pkg/kv/kvserver/testdata/flow_control_integration/blocked_admission new file mode 100644 index 000000000000..d5be7a37d2d7 --- /dev/null +++ b/pkg/kv/kvserver/testdata/flow_control_integration/blocked_admission @@ -0,0 +1,99 @@ +echo +---- +---- +-- (Issuing regular 1MiB, 3x replicated write that's not admitted.) + + +-- Flow token metrics from n1 after issuing 5 regular 1MiB 3x replicated writes +-- that are yet to get admitted. We see 5*1MiB*3=15MiB deductions of +-- {regular,elastic} tokens with no corresponding returns. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvadmission%tokens%' +ORDER BY name ASC; + + kvadmission.flow_controller.elastic_tokens_available | 9.0 MiB + kvadmission.flow_controller.elastic_tokens_deducted | 15 MiB + kvadmission.flow_controller.elastic_tokens_returned | 0 B + kvadmission.flow_controller.elastic_tokens_unaccounted | 0 B + kvadmission.flow_controller.regular_tokens_available | 33 MiB + kvadmission.flow_controller.regular_tokens_deducted | 15 MiB + kvadmission.flow_controller.regular_tokens_returned | 0 B + kvadmission.flow_controller.regular_tokens_unaccounted | 0 B + + +-- Observe the total tracked tokens per-stream on n1. +SELECT range_id, store_id, crdb_internal.humanize_bytes(total_tracked_tokens::INT8) + FROM crdb_internal.kv_flow_control_handles + + range_id | store_id | total_tracked_tokens +-----------+----------+----------------------- + 64 | 1 | 5.0 MiB + 64 | 2 | 5.0 MiB + 64 | 3 | 5.0 MiB + + +-- Observe the individual tracked tokens per-stream on the scratch range. +SELECT range_id, store_id, priority, crdb_internal.humanize_bytes(tokens::INT8) + FROM crdb_internal.kv_flow_token_deductions + + range_id | store_id | priority | tokens +-----------+----------+------------+---------- + 64 | 1 | normal-pri | 1.0 MiB + 64 | 1 | normal-pri | 1.0 MiB + 64 | 1 | normal-pri | 1.0 MiB + 64 | 1 | normal-pri | 1.0 MiB + 64 | 1 | normal-pri | 1.0 MiB + 64 | 2 | normal-pri | 1.0 MiB + 64 | 2 | normal-pri | 1.0 MiB + 64 | 2 | normal-pri | 1.0 MiB + 64 | 2 | normal-pri | 1.0 MiB + 64 | 2 | normal-pri | 1.0 MiB + 64 | 3 | normal-pri | 1.0 MiB + 64 | 3 | normal-pri | 1.0 MiB + 64 | 3 | normal-pri | 1.0 MiB + 64 | 3 | normal-pri | 1.0 MiB + 64 | 3 | normal-pri | 1.0 MiB + + +-- (Allow below-raft admission to proceed.) + + +-- Observe flow token dispatch metrics from n1. +SELECT name, value + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvadmission.flow_token_dispatch.local_regular%' +ORDER BY name ASC; + + kvadmission.flow_token_dispatch.local_regular | 5 + + +-- Observe flow token dispatch metrics from n2. +SELECT name, value + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvadmission.flow_token_dispatch.remote_regular%' +ORDER BY name ASC; + + kvadmission.flow_token_dispatch.remote_regular | 5 + + +-- Flow token metrics from n1 after work gets admitted. We see 15MiB returns of +-- {regular,elastic} tokens, and the available capacities going back to what +-- they were. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvadmission%tokens%' +ORDER BY name ASC; + + kvadmission.flow_controller.elastic_tokens_available | 24 MiB + kvadmission.flow_controller.elastic_tokens_deducted | 15 MiB + kvadmission.flow_controller.elastic_tokens_returned | 15 MiB + kvadmission.flow_controller.elastic_tokens_unaccounted | 0 B + kvadmission.flow_controller.regular_tokens_available | 48 MiB + kvadmission.flow_controller.regular_tokens_deducted | 15 MiB + kvadmission.flow_controller.regular_tokens_returned | 15 MiB + kvadmission.flow_controller.regular_tokens_unaccounted | 0 B +---- +---- + +# vim:ft=sql diff --git a/pkg/kv/kvserver/testdata/flow_control_integration/class_prioritization b/pkg/kv/kvserver/testdata/flow_control_integration/class_prioritization new file mode 100644 index 000000000000..aa204e94a9ca --- /dev/null +++ b/pkg/kv/kvserver/testdata/flow_control_integration/class_prioritization @@ -0,0 +1,67 @@ +echo +---- +---- +-- (Issuing 1x1MiB, 3x replicated elastic write that's not admitted.) + + +-- Flow token metrics from n1 after issuing 1x1MiB elastic 3x replicated write +-- that's not admitted. We see 1*1MiB*3=3MiB deductions of elastic tokens with +-- no corresponding returns. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvadmission%tokens%' +ORDER BY name ASC; + + kvadmission.flow_controller.elastic_tokens_available | 21 MiB + kvadmission.flow_controller.elastic_tokens_deducted | 3.0 MiB + kvadmission.flow_controller.elastic_tokens_returned | 0 B + kvadmission.flow_controller.elastic_tokens_unaccounted | 0 B + kvadmission.flow_controller.regular_tokens_available | 48 MiB + kvadmission.flow_controller.regular_tokens_deducted | 0 B + kvadmission.flow_controller.regular_tokens_returned | 0 B + kvadmission.flow_controller.regular_tokens_unaccounted | 0 B + + +-- (Issuing 1x1MiB, 3x replicated regular write that's not admitted.) + + +-- Flow token metrics from n1 after issuing 1x1MiB regular 3x replicated write +-- that's not admitted. We see 1*1MiB*3=3MiB deductions of {regular,elastic} +-- tokens with no corresponding returns. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvadmission%tokens%' +ORDER BY name ASC; + + kvadmission.flow_controller.elastic_tokens_available | 18 MiB + kvadmission.flow_controller.elastic_tokens_deducted | 6.0 MiB + kvadmission.flow_controller.elastic_tokens_returned | 0 B + kvadmission.flow_controller.elastic_tokens_unaccounted | 0 B + kvadmission.flow_controller.regular_tokens_available | 45 MiB + kvadmission.flow_controller.regular_tokens_deducted | 3.0 MiB + kvadmission.flow_controller.regular_tokens_returned | 0 B + kvadmission.flow_controller.regular_tokens_unaccounted | 0 B + + +-- (Allow below-raft admission to proceed.) + + +-- Flow token metrics from n1 after work gets admitted. All {regular,elastic} +-- tokens deducted are returned. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvadmission%tokens%' +ORDER BY name ASC; + + kvadmission.flow_controller.elastic_tokens_available | 24 MiB + kvadmission.flow_controller.elastic_tokens_deducted | 6.0 MiB + kvadmission.flow_controller.elastic_tokens_returned | 6.0 MiB + kvadmission.flow_controller.elastic_tokens_unaccounted | 0 B + kvadmission.flow_controller.regular_tokens_available | 48 MiB + kvadmission.flow_controller.regular_tokens_deducted | 3.0 MiB + kvadmission.flow_controller.regular_tokens_returned | 3.0 MiB + kvadmission.flow_controller.regular_tokens_unaccounted | 0 B +---- +---- + +# vim:ft=sql diff --git a/pkg/kv/kvserver/testdata/flow_control_integration/crashed_node b/pkg/kv/kvserver/testdata/flow_control_integration/crashed_node new file mode 100644 index 000000000000..38ffc5b7e1a5 --- /dev/null +++ b/pkg/kv/kvserver/testdata/flow_control_integration/crashed_node @@ -0,0 +1,66 @@ +echo +---- +---- +-- (Issuing regular 5x1MiB, 2x replicated writes that are not admitted.) + + +-- Flow token metrics from n1 after issuing 5 regular 1MiB 2x replicated writes +-- that are yet to get admitted. We see 5*1MiB*2=10MiB deductions of +-- {regular,elastic} tokens with no corresponding returns. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvadmission%tokens%' +ORDER BY name ASC; + + kvadmission.flow_controller.elastic_tokens_available | 6.0 MiB + kvadmission.flow_controller.elastic_tokens_deducted | 10 MiB + kvadmission.flow_controller.elastic_tokens_returned | 0 B + kvadmission.flow_controller.elastic_tokens_unaccounted | 0 B + kvadmission.flow_controller.regular_tokens_available | 22 MiB + kvadmission.flow_controller.regular_tokens_deducted | 10 MiB + kvadmission.flow_controller.regular_tokens_returned | 0 B + kvadmission.flow_controller.regular_tokens_unaccounted | 0 B + + +-- Observe the per-stream tracked tokens on n1, before n2 is crashed. +SELECT range_id, store_id, crdb_internal.humanize_bytes(total_tracked_tokens::INT8) + FROM crdb_internal.kv_flow_control_handles + + range_id | store_id | total_tracked_tokens +-----------+----------+----------------------- + 64 | 1 | 5.0 MiB + 64 | 2 | 5.0 MiB + + +-- (Crashing n2 but disabling the raft-transport-break token return mechanism.) + + +-- Observe the per-stream tracked tokens on n1, after n2 crashed. We're no +-- longer tracking the 5MiB held by n2. +SELECT range_id, store_id, crdb_internal.humanize_bytes(total_tracked_tokens::INT8) + FROM crdb_internal.kv_flow_control_handles + + range_id | store_id | total_tracked_tokens +-----------+----------+----------------------- + 64 | 1 | 5.0 MiB + + +-- Flow token metrics from n1 after n2 crashed. Observe that we've returned the +-- 5MiB previously held by n2. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvadmission%tokens%' +ORDER BY name ASC; + + kvadmission.flow_controller.elastic_tokens_available | 11 MiB + kvadmission.flow_controller.elastic_tokens_deducted | 10 MiB + kvadmission.flow_controller.elastic_tokens_returned | 5.0 MiB + kvadmission.flow_controller.elastic_tokens_unaccounted | 0 B + kvadmission.flow_controller.regular_tokens_available | 27 MiB + kvadmission.flow_controller.regular_tokens_deducted | 10 MiB + kvadmission.flow_controller.regular_tokens_returned | 5.0 MiB + kvadmission.flow_controller.regular_tokens_unaccounted | 0 B +---- +---- + +# vim:ft=sql diff --git a/pkg/kv/kvserver/testdata/flow_control_integration/leader_not_leaseholder b/pkg/kv/kvserver/testdata/flow_control_integration/leader_not_leaseholder new file mode 100644 index 000000000000..3868d5ced08d --- /dev/null +++ b/pkg/kv/kvserver/testdata/flow_control_integration/leader_not_leaseholder @@ -0,0 +1,83 @@ +echo +---- +---- +-- (Issuing 1x1MiB, 3x replicated write that's not admitted.) + + +-- Flow token metrics from n1 after issuing 1x1MiB regular 3x replicated write +-- that's not admitted. We see 1*1MiB*3=3MiB deductions of regular tokens with +-- no corresponding returns. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvadmission%regular_tokens%' +ORDER BY name ASC; + + kvadmission.flow_controller.regular_tokens_available | 45 MiB + kvadmission.flow_controller.regular_tokens_deducted | 3.0 MiB + kvadmission.flow_controller.regular_tokens_returned | 0 B + kvadmission.flow_controller.regular_tokens_unaccounted | 0 B + + +-- (Transferring only range lease, not raft leadership, to n2.) + + +-- Flow token metrics from n1 having lost the lease but retained raft +-- leadership. No deducted tokens are released. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvadmission%regular_tokens%' +ORDER BY name ASC; + + kvadmission.flow_controller.regular_tokens_available | 45 MiB + kvadmission.flow_controller.regular_tokens_deducted | 3.0 MiB + kvadmission.flow_controller.regular_tokens_returned | 0 B + kvadmission.flow_controller.regular_tokens_unaccounted | 0 B + + +-- (Issuing another 1x1MiB, 3x replicated write that's not admitted while in +-- this leader != leaseholder state.) + + +-- Looking at n1's flow token metrics, there's no change. No additional tokens +-- are deducted since the write is not being proposed here. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvadmission%regular_tokens%' +ORDER BY name ASC; + + kvadmission.flow_controller.regular_tokens_available | 45 MiB + kvadmission.flow_controller.regular_tokens_deducted | 3.0 MiB + kvadmission.flow_controller.regular_tokens_returned | 0 B + kvadmission.flow_controller.regular_tokens_unaccounted | 0 B + + +-- Looking at n2's flow token metrics, there's no activity. n2 never acquired +-- the raft leadership. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvadmission%regular_tokens%' +ORDER BY name ASC; + + kvadmission.flow_controller.regular_tokens_available | 0 B + kvadmission.flow_controller.regular_tokens_deducted | 0 B + kvadmission.flow_controller.regular_tokens_returned | 0 B + kvadmission.flow_controller.regular_tokens_unaccounted | 0 B + + +-- (Allow below-raft admission to proceed.) + + +-- All deducted flow tokens are returned back to where the raft leader is. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvadmission%regular_tokens%' +ORDER BY name ASC; + + kvadmission.flow_controller.regular_tokens_available | 48 MiB + kvadmission.flow_controller.regular_tokens_deducted | 3.0 MiB + kvadmission.flow_controller.regular_tokens_returned | 3.0 MiB + kvadmission.flow_controller.regular_tokens_unaccounted | 0 B +---- +---- + +# vim:ft=sql diff --git a/pkg/kv/kvserver/testdata/flow_control_integration/quiesced_range b/pkg/kv/kvserver/testdata/flow_control_integration/quiesced_range new file mode 100644 index 000000000000..93fc93412776 --- /dev/null +++ b/pkg/kv/kvserver/testdata/flow_control_integration/quiesced_range @@ -0,0 +1,60 @@ +echo +---- +---- +-- (Issuing 1x1MiB, 3x replicated elastic write that's not admitted.) + + +-- Flow token metrics from n1 after issuing 1x1MiB elastic 3x replicated write +-- that's not admitted. We see 1*1MiB*3=3MiB deductions of elastic tokens with +-- no corresponding returns. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvadmission%elastic_tokens%' +ORDER BY name ASC; + + kvadmission.flow_controller.elastic_tokens_available | 21 MiB + kvadmission.flow_controller.elastic_tokens_deducted | 3.0 MiB + kvadmission.flow_controller.elastic_tokens_returned | 0 B + kvadmission.flow_controller.elastic_tokens_unaccounted | 0 B + + +-- (Wait for range to quiesce.) + + +-- (Allow below-raft admission to proceed. We've disabled the fallback token +-- dispatch mechanism so no tokens are returned yet -- quiesced ranges don't +-- use the piggy-backed token return mechanism since there's no raft traffic.) + + +-- Flow token metrics from n1 after work gets admitted but fallback token +-- dispatch mechanism is disabled. Deducted elastic tokens from remote stores +-- are yet to be returned. Tokens for the local store are. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvadmission%elastic_tokens%' +ORDER BY name ASC; + + kvadmission.flow_controller.elastic_tokens_available | 22 MiB + kvadmission.flow_controller.elastic_tokens_deducted | 3.0 MiB + kvadmission.flow_controller.elastic_tokens_returned | 1.0 MiB + kvadmission.flow_controller.elastic_tokens_unaccounted | 0 B + + +-- (Enable the fallback token dispatch mechanism.) + + +-- Flow token metrics from n1 after work gets admitted and all elastic tokens +-- are returned through the fallback mechanism. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvadmission%elastic_tokens%' +ORDER BY name ASC; + + kvadmission.flow_controller.elastic_tokens_available | 24 MiB + kvadmission.flow_controller.elastic_tokens_deducted | 3.0 MiB + kvadmission.flow_controller.elastic_tokens_returned | 3.0 MiB + kvadmission.flow_controller.elastic_tokens_unaccounted | 0 B +---- +---- + +# vim:ft=sql diff --git a/pkg/kv/kvserver/testdata/flow_control_integration/raft_membership b/pkg/kv/kvserver/testdata/flow_control_integration/raft_membership new file mode 100644 index 000000000000..c5119f9cd59c --- /dev/null +++ b/pkg/kv/kvserver/testdata/flow_control_integration/raft_membership @@ -0,0 +1,112 @@ +echo +---- +---- +-- (Issuing 1x1MiB, 3x replicated write that's not admitted.) + + +-- Flow token metrics from n1 after issuing 1x1MiB regular 3x replicated write +-- that's not admitted. We see 1*1MiB*3=3MiB deductions of regular tokens with +-- no corresponding returns. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvadmission%regular_tokens%' +ORDER BY name ASC; + + kvadmission.flow_controller.regular_tokens_available | 45 MiB + kvadmission.flow_controller.regular_tokens_deducted | 3.0 MiB + kvadmission.flow_controller.regular_tokens_returned | 0 B + kvadmission.flow_controller.regular_tokens_unaccounted | 0 B + + +-- (Adding a voting replica on n4.) + + +-- Observe the total tracked tokens per-stream on n1. s1-s3 should have 1MiB +-- tracked each, and s4 should have none. +SELECT range_id, store_id, crdb_internal.humanize_bytes(total_tracked_tokens::INT8) + FROM crdb_internal.kv_flow_control_handles + + range_id | store_id | total_tracked_tokens +-----------+----------+----------------------- + 64 | 1 | 1.0 MiB + 64 | 2 | 1.0 MiB + 64 | 3 | 1.0 MiB + 64 | 4 | 0 B + + +-- (Issuing 1x1MiB, 4x replicated write that's not admitted.) + + +-- Observe the individual tracked tokens per-stream on the scratch range. s1-s3 +-- should have 2MiB tracked (they've observed 2x1MiB writes), s4 should have +-- 1MiB. +SELECT range_id, store_id, crdb_internal.humanize_bytes(total_tracked_tokens::INT8) + FROM crdb_internal.kv_flow_control_handles + + range_id | store_id | total_tracked_tokens +-----------+----------+----------------------- + 64 | 1 | 2.0 MiB + 64 | 2 | 2.0 MiB + 64 | 3 | 2.0 MiB + 64 | 4 | 1.0 MiB + + +-- (Removing voting replica from n3.) + + +-- (Adding non-voting replica to n5.) + + +-- (Issuing 1x1MiB, 4x replicated write (w/ one non-voter) that's not admitted. + + +-- Observe the individual tracked tokens per-stream on the scratch range. s1-s2 +-- should have 3MiB tracked (they've observed 3x1MiB writes), there should be +-- no s3 since it was removed, s4 and s5 should have 2MiB and 1MiB +-- respectively. +SELECT range_id, store_id, crdb_internal.humanize_bytes(total_tracked_tokens::INT8) + FROM crdb_internal.kv_flow_control_handles + + range_id | store_id | total_tracked_tokens +-----------+----------+----------------------- + 64 | 1 | 3.0 MiB + 64 | 2 | 3.0 MiB + 64 | 4 | 2.0 MiB + 64 | 5 | 1.0 MiB + + +-- (Allow below-raft admission to proceed.) + + +-- Observe that there no tracked tokens across s1,s2,s4,s5. +SELECT range_id, store_id, crdb_internal.humanize_bytes(total_tracked_tokens::INT8) + FROM crdb_internal.kv_flow_control_handles + + range_id | store_id | total_tracked_tokens +-----------+----------+----------------------- + 64 | 1 | 0 B + 64 | 2 | 0 B + 64 | 4 | 0 B + 64 | 5 | 0 B + + +-- Flow token metrics from n1 after work gets admitted. All {regular,elastic} +-- tokens deducted are returned, including from when s3 was removed as a raft +-- member. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvadmission%tokens%' +ORDER BY name ASC; + + kvadmission.flow_controller.elastic_tokens_available | 40 MiB + kvadmission.flow_controller.elastic_tokens_deducted | 11 MiB + kvadmission.flow_controller.elastic_tokens_returned | 11 MiB + kvadmission.flow_controller.elastic_tokens_unaccounted | 0 B + kvadmission.flow_controller.regular_tokens_available | 80 MiB + kvadmission.flow_controller.regular_tokens_deducted | 11 MiB + kvadmission.flow_controller.regular_tokens_returned | 11 MiB + kvadmission.flow_controller.regular_tokens_unaccounted | 0 B +---- +---- + +# vim:ft=sql diff --git a/pkg/kv/kvserver/testdata/flow_control_integration/raft_membership_remove_self b/pkg/kv/kvserver/testdata/flow_control_integration/raft_membership_remove_self new file mode 100644 index 000000000000..2b424bbfcb56 --- /dev/null +++ b/pkg/kv/kvserver/testdata/flow_control_integration/raft_membership_remove_self @@ -0,0 +1,62 @@ +echo +---- +---- +-- (Issuing 1x1MiB, 3x replicated write that's not admitted.) + + +-- Flow token metrics from n1 after issuing 1x1MiB regular 3x replicated write +-- that's not admitted. We see 1*1MiB*3=3MiB deductions of regular tokens with +-- no corresponding returns. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvadmission%regular_tokens%' +ORDER BY name ASC; + + kvadmission.flow_controller.regular_tokens_available | 45 MiB + kvadmission.flow_controller.regular_tokens_deducted | 3.0 MiB + kvadmission.flow_controller.regular_tokens_returned | 0 B + kvadmission.flow_controller.regular_tokens_unaccounted | 0 B + + +-- (Replacing current raft leader on n1 in raft group with new n4 replica.) + + +-- Flow token metrics from n1 after raft leader removed itself from raft group. +-- All {regular,elastic} tokens deducted are returned. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvadmission%tokens%' +ORDER BY name ASC; + + kvadmission.flow_controller.elastic_tokens_available | 24 MiB + kvadmission.flow_controller.elastic_tokens_deducted | 3.0 MiB + kvadmission.flow_controller.elastic_tokens_returned | 3.0 MiB + kvadmission.flow_controller.elastic_tokens_unaccounted | 0 B + kvadmission.flow_controller.regular_tokens_available | 48 MiB + kvadmission.flow_controller.regular_tokens_deducted | 3.0 MiB + kvadmission.flow_controller.regular_tokens_returned | 3.0 MiB + kvadmission.flow_controller.regular_tokens_unaccounted | 0 B + + +-- (Allow below-raft admission to proceed.) + + +-- Flow token metrics from n1 after work gets admitted. Tokens were already +-- returned earlier, so there's no change. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvadmission%tokens%' +ORDER BY name ASC; + + kvadmission.flow_controller.elastic_tokens_available | 24 MiB + kvadmission.flow_controller.elastic_tokens_deducted | 3.0 MiB + kvadmission.flow_controller.elastic_tokens_returned | 3.0 MiB + kvadmission.flow_controller.elastic_tokens_unaccounted | 0 B + kvadmission.flow_controller.regular_tokens_available | 48 MiB + kvadmission.flow_controller.regular_tokens_deducted | 3.0 MiB + kvadmission.flow_controller.regular_tokens_returned | 3.0 MiB + kvadmission.flow_controller.regular_tokens_unaccounted | 0 B +---- +---- + +# vim:ft=sql diff --git a/pkg/kv/kvserver/testdata/flow_control_integration/raft_snapshot b/pkg/kv/kvserver/testdata/flow_control_integration/raft_snapshot new file mode 100644 index 000000000000..ac2cfe915a03 --- /dev/null +++ b/pkg/kv/kvserver/testdata/flow_control_integration/raft_snapshot @@ -0,0 +1,192 @@ +echo +---- +---- +-- Flow token metrics from n1 after issuing 1 regular 1MiB 5x replicated write +-- that's not admitted. Since this test is ignoring crashed nodes for token +-- deduction purposes, we see a deduction of 5MiB {regular,elastic} tokens. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvadmission%tokens%' +ORDER BY name ASC; + + kvadmission.flow_controller.elastic_tokens_available | 35 MiB + kvadmission.flow_controller.elastic_tokens_deducted | 5.0 MiB + kvadmission.flow_controller.elastic_tokens_returned | 0 B + kvadmission.flow_controller.elastic_tokens_unaccounted | 0 B + kvadmission.flow_controller.regular_tokens_available | 75 MiB + kvadmission.flow_controller.regular_tokens_deducted | 5.0 MiB + kvadmission.flow_controller.regular_tokens_returned | 0 B + kvadmission.flow_controller.regular_tokens_unaccounted | 0 B + + +-- Observe the total tracked tokens per-stream on n1. 1MiB is tracked for n1-n5. +SELECT range_id, store_id, crdb_internal.humanize_bytes(total_tracked_tokens::INT8) + FROM crdb_internal.kv_flow_control_handles + + range_id | store_id | total_tracked_tokens +-----------+----------+----------------------- + 64 | 1 | 1.0 MiB + 64 | 2 | 1.0 MiB + 64 | 3 | 1.0 MiB + 64 | 4 | 1.0 MiB + 64 | 5 | 1.0 MiB + + +-- (Killing n2 and n3, but preventing their tokens from being returned + +-- artificially allowing tokens to get deducted.) + + +-- Observe the total tracked tokens per-stream on n1. 1MiB is (still) tracked +-- for n1-n5. Typically n2, n3 would release their tokens, but this test is +-- intentionally suppressing that behavior to observe token returns only once +-- issuing a raft snapshot. +SELECT range_id, store_id, crdb_internal.humanize_bytes(total_tracked_tokens::INT8) + FROM crdb_internal.kv_flow_control_handles + + range_id | store_id | total_tracked_tokens +-----------+----------+----------------------- + 64 | 1 | 1.0 MiB + 64 | 2 | 1.0 MiB + 64 | 3 | 1.0 MiB + 64 | 4 | 1.0 MiB + 64 | 5 | 1.0 MiB + + +-- (Issuing another 1MiB of 5x replicated writes while n2 and n3 are down and +-- below-raft admission is paused.) + + +-- Flow token metrics from n1 after issuing 1 regular 1MiB 5x replicated write +-- that's not admitted. We'll have deducted another 5*1MiB=5MiB worth of tokens. +-- Normally we wouldn't deduct tokens for n2 and n3 since they're dead (both +-- according to the per-replica last-updated map, and according broken +-- RaftTransport streams). But this test is intentionally suppressing that +-- behavior to observe token returns when sending raft snapshots. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvadmission%tokens%' +ORDER BY name ASC; + + kvadmission.flow_controller.elastic_tokens_available | 30 MiB + kvadmission.flow_controller.elastic_tokens_deducted | 10 MiB + kvadmission.flow_controller.elastic_tokens_returned | 0 B + kvadmission.flow_controller.elastic_tokens_unaccounted | 0 B + kvadmission.flow_controller.regular_tokens_available | 70 MiB + kvadmission.flow_controller.regular_tokens_deducted | 10 MiB + kvadmission.flow_controller.regular_tokens_returned | 0 B + kvadmission.flow_controller.regular_tokens_unaccounted | 0 B + + +-- Observe the total tracked tokens per-stream on n1. 2MiB is tracked for n1-n5; +-- see last comment for an explanation why we're still deducting for n2, n3. +SELECT range_id, store_id, crdb_internal.humanize_bytes(total_tracked_tokens::INT8) + FROM crdb_internal.kv_flow_control_handles + + range_id | store_id | total_tracked_tokens +-----------+----------+----------------------- + 64 | 1 | 2.0 MiB + 64 | 2 | 2.0 MiB + 64 | 3 | 2.0 MiB + 64 | 4 | 2.0 MiB + 64 | 5 | 2.0 MiB + + +-- (Truncating raft log.) + + +-- (Restarting n2 and n3.) + + +-- Flow token metrics from n1 after restarting n2 and n3. We've returned the +-- 2MiB previously held by those nodes (2MiB each). We're reacting to it's raft +-- progress state, noting that since we've truncated our log, we need to catch +-- it up via snapshot. So we release all held tokens. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvadmission%tokens%' +ORDER BY name ASC; + + kvadmission.flow_controller.elastic_tokens_available | 34 MiB + kvadmission.flow_controller.elastic_tokens_deducted | 10 MiB + kvadmission.flow_controller.elastic_tokens_returned | 4.0 MiB + kvadmission.flow_controller.elastic_tokens_unaccounted | 0 B + kvadmission.flow_controller.regular_tokens_available | 74 MiB + kvadmission.flow_controller.regular_tokens_deducted | 10 MiB + kvadmission.flow_controller.regular_tokens_returned | 4.0 MiB + kvadmission.flow_controller.regular_tokens_unaccounted | 0 B + + +-- Observe the total tracked tokens per-stream on n1. There's nothing tracked +-- for n2 and n3 anymore. +SELECT range_id, store_id, crdb_internal.humanize_bytes(total_tracked_tokens::INT8) + FROM crdb_internal.kv_flow_control_handles + + range_id | store_id | total_tracked_tokens +-----------+----------+----------------------- + 64 | 1 | 2.0 MiB + 64 | 2 | 0 B + 64 | 3 | 0 B + 64 | 4 | 2.0 MiB + 64 | 5 | 2.0 MiB + + +-- (Allow below-raft admission to proceed.) + + +-- Observe flow token dispatch metrics from n4. +SELECT name, value + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvadmission.flow_token_dispatch.pending_nodes%' +ORDER BY name ASC; + + kvadmission.flow_token_dispatch.pending_nodes | 0 + + +-- Flow token metrics from n1 after work gets admitted. We see the remaining +-- 6MiB of {regular,elastic} tokens returned. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvadmission%tokens%' +ORDER BY name ASC; + + kvadmission.flow_controller.elastic_tokens_available | 40 MiB + kvadmission.flow_controller.elastic_tokens_deducted | 10 MiB + kvadmission.flow_controller.elastic_tokens_returned | 10 MiB + kvadmission.flow_controller.elastic_tokens_unaccounted | 0 B + kvadmission.flow_controller.regular_tokens_available | 80 MiB + kvadmission.flow_controller.regular_tokens_deducted | 10 MiB + kvadmission.flow_controller.regular_tokens_returned | 10 MiB + kvadmission.flow_controller.regular_tokens_unaccounted | 0 B + + +-- Observe the total tracked tokens per-stream on n1; there should be nothing. +SELECT range_id, store_id, crdb_internal.humanize_bytes(total_tracked_tokens::INT8) + FROM crdb_internal.kv_flow_control_handles + + range_id | store_id | total_tracked_tokens +-----------+----------+----------------------- + 64 | 1 | 0 B + 64 | 2 | 0 B + 64 | 3 | 0 B + 64 | 4 | 0 B + 64 | 5 | 0 B + + +-- Another view of tokens, using /inspectz-backed vtables. +SELECT store_id, + crdb_internal.humanize_bytes(available_regular_tokens), + crdb_internal.humanize_bytes(available_elastic_tokens) + FROM crdb_internal.kv_flow_controller + ORDER BY store_id ASC; + + range_id | regular_available | elastic_available +-----------+-------------------+-------------------- + 1 | 16 MiB | 8.0 MiB + 2 | 16 MiB | 8.0 MiB + 3 | 16 MiB | 8.0 MiB + 4 | 16 MiB | 8.0 MiB + 5 | 16 MiB | 8.0 MiB +---- +---- + +# vim:ft=sql diff --git a/pkg/kv/kvserver/testdata/flow_control_integration/raft_transport_break b/pkg/kv/kvserver/testdata/flow_control_integration/raft_transport_break new file mode 100644 index 000000000000..7d44a62afe74 --- /dev/null +++ b/pkg/kv/kvserver/testdata/flow_control_integration/raft_transport_break @@ -0,0 +1,69 @@ +echo +---- +---- +-- (Issuing regular 5x1MiB, 3x replicated writes that are not admitted.) + + +-- Flow token metrics from n1 after issuing 5 regular 1MiB 3x replicated writes +-- that are yet to get admitted. We see 5*1MiB*3=15MiB deductions of +-- {regular,elastic} tokens with no corresponding returns. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvadmission%tokens%' +ORDER BY name ASC; + + kvadmission.flow_controller.elastic_tokens_available | 9.0 MiB + kvadmission.flow_controller.elastic_tokens_deducted | 15 MiB + kvadmission.flow_controller.elastic_tokens_returned | 0 B + kvadmission.flow_controller.elastic_tokens_unaccounted | 0 B + kvadmission.flow_controller.regular_tokens_available | 33 MiB + kvadmission.flow_controller.regular_tokens_deducted | 15 MiB + kvadmission.flow_controller.regular_tokens_returned | 0 B + kvadmission.flow_controller.regular_tokens_unaccounted | 0 B + + +-- Observe the per-stream tracked tokens on n1, before n2 is crashed. +SELECT range_id, store_id, crdb_internal.humanize_bytes(total_tracked_tokens::INT8) + FROM crdb_internal.kv_flow_control_handles + + range_id | store_id | total_tracked_tokens +-----------+----------+----------------------- + 64 | 1 | 5.0 MiB + 64 | 2 | 5.0 MiB + 64 | 3 | 5.0 MiB + + +-- (Crashing n2 but disabling the last-updated token return mechanism.) + + +-- Observe the per-stream tracked tokens on n1, after n2 crashed. We're no +-- longer tracking the 5MiB held by n2 because the raft transport between +-- n1<->n2 is broken. +SELECT range_id, store_id, crdb_internal.humanize_bytes(total_tracked_tokens::INT8) + FROM crdb_internal.kv_flow_control_handles + + range_id | store_id | total_tracked_tokens +-----------+----------+----------------------- + 64 | 1 | 5.0 MiB + 64 | 3 | 5.0 MiB + + +-- Flow token metrics from n1 after n2 crashed. Observe that we've returned the +-- 5MiB previously held by n2. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvadmission%tokens%' +ORDER BY name ASC; + + kvadmission.flow_controller.elastic_tokens_available | 14 MiB + kvadmission.flow_controller.elastic_tokens_deducted | 15 MiB + kvadmission.flow_controller.elastic_tokens_returned | 5.0 MiB + kvadmission.flow_controller.elastic_tokens_unaccounted | 0 B + kvadmission.flow_controller.regular_tokens_available | 38 MiB + kvadmission.flow_controller.regular_tokens_deducted | 15 MiB + kvadmission.flow_controller.regular_tokens_returned | 5.0 MiB + kvadmission.flow_controller.regular_tokens_unaccounted | 0 B +---- +---- + +# vim:ft=sql diff --git a/pkg/kv/kvserver/testdata/flow_control_integration/raft_transport_culled b/pkg/kv/kvserver/testdata/flow_control_integration/raft_transport_culled new file mode 100644 index 000000000000..602d9b46ca01 --- /dev/null +++ b/pkg/kv/kvserver/testdata/flow_control_integration/raft_transport_culled @@ -0,0 +1,71 @@ +echo +---- +---- +-- (Issuing regular 5x1MiB, 3x replicated writes that are not admitted.) + + +-- Flow token metrics from n1 after issuing 5 regular 1MiB 3x replicated writes +-- that are yet to get admitted. We see 5*1MiB*3=15MiB deductions of +-- {regular,elastic} tokens with no corresponding returns. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvadmission%tokens%' +ORDER BY name ASC; + + kvadmission.flow_controller.elastic_tokens_available | 9.0 MiB + kvadmission.flow_controller.elastic_tokens_deducted | 15 MiB + kvadmission.flow_controller.elastic_tokens_returned | 0 B + kvadmission.flow_controller.elastic_tokens_unaccounted | 0 B + kvadmission.flow_controller.regular_tokens_available | 33 MiB + kvadmission.flow_controller.regular_tokens_deducted | 15 MiB + kvadmission.flow_controller.regular_tokens_returned | 0 B + kvadmission.flow_controller.regular_tokens_unaccounted | 0 B + + +-- Observe the per-stream tracked tokens on n1, before we cull the n1<->n2 raft +-- transport stream out of idleness. +SELECT range_id, store_id, crdb_internal.humanize_bytes(total_tracked_tokens::INT8) + FROM crdb_internal.kv_flow_control_handles + + range_id | store_id | total_tracked_tokens +-----------+----------+----------------------- + 64 | 1 | 5.0 MiB + 64 | 2 | 5.0 MiB + 64 | 3 | 5.0 MiB + + +-- (Marking n2->n1 raft transport gRPC stream as idle.) + + +-- Observe the per-stream tracked tokens on n1 after n2->n1 raft transport +-- stream is culled. We're no longer tracking the 5MiB held by n2 because the +-- raft transport between n1<->n2 is broken. +SELECT range_id, store_id, crdb_internal.humanize_bytes(total_tracked_tokens::INT8) + FROM crdb_internal.kv_flow_control_handles + WHERE total_tracked_tokens > 0 + + range_id | store_id | total_tracked_tokens +-----------+----------+----------------------- + 64 | 1 | 5.0 MiB + 64 | 3 | 5.0 MiB + + +-- Flow token metrics from n1 after n2->n1 raft transport stream is culled. +-- Observe that we've returned the 5MiB previously held by n2. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvadmission%tokens%' +ORDER BY name ASC; + + kvadmission.flow_controller.elastic_tokens_available | 14 MiB + kvadmission.flow_controller.elastic_tokens_deducted | 15 MiB + kvadmission.flow_controller.elastic_tokens_returned | 5.0 MiB + kvadmission.flow_controller.elastic_tokens_unaccounted | 0 B + kvadmission.flow_controller.regular_tokens_available | 38 MiB + kvadmission.flow_controller.regular_tokens_deducted | 15 MiB + kvadmission.flow_controller.regular_tokens_returned | 5.0 MiB + kvadmission.flow_controller.regular_tokens_unaccounted | 0 B +---- +---- + +# vim:ft=sql diff --git a/pkg/kv/kvserver/testdata/flow_control_integration/split_merge b/pkg/kv/kvserver/testdata/flow_control_integration/split_merge new file mode 100644 index 000000000000..4b5df38d6f32 --- /dev/null +++ b/pkg/kv/kvserver/testdata/flow_control_integration/split_merge @@ -0,0 +1,89 @@ +echo +---- +---- +-- Flow token metrics from n1 after issuing + admitting the regular 1MiB 3x +-- replicated write to the pre-split range. There should be 3MiB of +-- {regular,elastic} tokens {deducted,returned}. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvadmission%tokens%' +ORDER BY name ASC; + + kvadmission.flow_controller.elastic_tokens_available | 24 MiB + kvadmission.flow_controller.elastic_tokens_deducted | 3.0 MiB + kvadmission.flow_controller.elastic_tokens_returned | 3.0 MiB + kvadmission.flow_controller.elastic_tokens_unaccounted | 0 B + kvadmission.flow_controller.regular_tokens_available | 48 MiB + kvadmission.flow_controller.regular_tokens_deducted | 3.0 MiB + kvadmission.flow_controller.regular_tokens_returned | 3.0 MiB + kvadmission.flow_controller.regular_tokens_unaccounted | 0 B + + +-- (Splitting range.) + + +-- Flow token metrics from n1 after further issuing 2MiB and 3MiB writes to +-- post-split LHS and RHS ranges respectively. We should see 15MiB extra tokens +-- {deducted,returned}, which comes from (2MiB+3MiB)*3=15MiB. So we stand at +-- 3MiB+15MiB=18MiB now. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvadmission%tokens%' +ORDER BY name ASC; + + kvadmission.flow_controller.elastic_tokens_available | 24 MiB + kvadmission.flow_controller.elastic_tokens_deducted | 18 MiB + kvadmission.flow_controller.elastic_tokens_returned | 18 MiB + kvadmission.flow_controller.elastic_tokens_unaccounted | 0 B + kvadmission.flow_controller.regular_tokens_available | 48 MiB + kvadmission.flow_controller.regular_tokens_deducted | 18 MiB + kvadmission.flow_controller.regular_tokens_returned | 18 MiB + kvadmission.flow_controller.regular_tokens_unaccounted | 0 B + + +-- Observe the newly split off replica, with its own three streams. +SELECT range_id, count(*) AS streams + FROM crdb_internal.kv_flow_control_handles +GROUP BY (range_id) +ORDER BY streams DESC; + + range_id | stream_count +-----------+--------------- + 64 | 3 + 65 | 3 + + +-- (Merging ranges.) + + +-- Flow token metrics from n1 after issuing 4MiB of regular replicated writes to +-- the post-merged range. We should see 12MiB extra tokens {deducted,returned}, +-- which comes from 4MiB*3=12MiB. So we stand at 18MiB+12MiB=30MiB now. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvadmission%tokens%' +ORDER BY name ASC; + + kvadmission.flow_controller.elastic_tokens_available | 24 MiB + kvadmission.flow_controller.elastic_tokens_deducted | 30 MiB + kvadmission.flow_controller.elastic_tokens_returned | 30 MiB + kvadmission.flow_controller.elastic_tokens_unaccounted | 0 B + kvadmission.flow_controller.regular_tokens_available | 48 MiB + kvadmission.flow_controller.regular_tokens_deducted | 30 MiB + kvadmission.flow_controller.regular_tokens_returned | 30 MiB + kvadmission.flow_controller.regular_tokens_unaccounted | 0 B + + +-- Observe only the merged replica with its own three streams. +SELECT range_id, count(*) AS streams + FROM crdb_internal.kv_flow_control_handles +GROUP BY (range_id) +ORDER BY streams DESC; + + range_id | stream_count +-----------+--------------- + 64 | 3 +---- +---- + +# vim:ft=sql diff --git a/pkg/kv/kvserver/testdata/flow_control_integration/transfer_lease b/pkg/kv/kvserver/testdata/flow_control_integration/transfer_lease new file mode 100644 index 000000000000..209b73fce0c1 --- /dev/null +++ b/pkg/kv/kvserver/testdata/flow_control_integration/transfer_lease @@ -0,0 +1,38 @@ +echo +---- +---- +-- (Issuing 1x1MiB, 3x replicated write that's not admitted.) + + +-- Flow token metrics from n1 after issuing 1x1MiB regular 3x replicated write +-- that's not admitted. We see 1*1MiB*3=3MiB deductions of regular tokens with +-- no corresponding returns. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvadmission%regular_tokens%' +ORDER BY name ASC; + + kvadmission.flow_controller.regular_tokens_available | 45 MiB + kvadmission.flow_controller.regular_tokens_deducted | 3.0 MiB + kvadmission.flow_controller.regular_tokens_returned | 0 B + kvadmission.flow_controller.regular_tokens_unaccounted | 0 B + + +-- (Transferring range lease to n2 and allowing leadership to follow.) + + +-- Flow token metrics from n1 having lost the lease and raft leadership. All +-- deducted tokens are returned. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvadmission%regular_tokens%' +ORDER BY name ASC; + + kvadmission.flow_controller.regular_tokens_available | 48 MiB + kvadmission.flow_controller.regular_tokens_deducted | 3.0 MiB + kvadmission.flow_controller.regular_tokens_returned | 3.0 MiB + kvadmission.flow_controller.regular_tokens_unaccounted | 0 B +---- +---- + +# vim:ft=sql diff --git a/pkg/kv/kvserver/testdata/flow_control_integration/unquiesced_range b/pkg/kv/kvserver/testdata/flow_control_integration/unquiesced_range new file mode 100644 index 000000000000..fb29726c1a21 --- /dev/null +++ b/pkg/kv/kvserver/testdata/flow_control_integration/unquiesced_range @@ -0,0 +1,63 @@ +echo +---- +---- +-- (Issuing 1x1MiB, 3x replicated elastic write that's not admitted.) + + +-- Flow token metrics from n1 after issuing 1x1MiB elastic 3x replicated write +-- that's not admitted. We see 1*1MiB*3=3MiB deductions of elastic tokens with +-- no corresponding returns. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvadmission%elastic_tokens%' +ORDER BY name ASC; + + kvadmission.flow_controller.elastic_tokens_available | 21 MiB + kvadmission.flow_controller.elastic_tokens_deducted | 3.0 MiB + kvadmission.flow_controller.elastic_tokens_returned | 0 B + kvadmission.flow_controller.elastic_tokens_unaccounted | 0 B + + +-- (Wait for range to quiesce.) + + +-- (Allow below-raft admission to proceed. We've disabled the fallback token +-- dispatch mechanism so no tokens are returned yet -- quiesced ranges don't +-- use the piggy-backed token return mechanism since there's no raft traffic.) + + +-- Flow token metrics from n1 after work gets admitted but fallback token +-- dispatch mechanism is disabled. Deducted elastic tokens from remote stores +-- are yet to be returned. Tokens for the local store are. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvadmission%elastic_tokens%' +ORDER BY name ASC; + + kvadmission.flow_controller.elastic_tokens_available | 22 MiB + kvadmission.flow_controller.elastic_tokens_deducted | 3.0 MiB + kvadmission.flow_controller.elastic_tokens_returned | 1.0 MiB + kvadmission.flow_controller.elastic_tokens_unaccounted | 0 B + + +-- (Enable the piggyback token dispatch mechanism.) + + +-- (Unquiesce the range.) + + +-- Flow token metrics from n1 after work gets admitted and all elastic tokens +-- are returned through the piggyback mechanism. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvadmission%elastic_tokens%' +ORDER BY name ASC; + + kvadmission.flow_controller.elastic_tokens_available | 24 MiB + kvadmission.flow_controller.elastic_tokens_deducted | 3.0 MiB + kvadmission.flow_controller.elastic_tokens_returned | 3.0 MiB + kvadmission.flow_controller.elastic_tokens_unaccounted | 0 B +---- +---- + +# vim:ft=sql diff --git a/pkg/kv/kvserver/testing_knobs.go b/pkg/kv/kvserver/testing_knobs.go index 357f2dbaa8df..fbe8be7c0ae2 100644 --- a/pkg/kv/kvserver/testing_knobs.go +++ b/pkg/kv/kvserver/testing_knobs.go @@ -19,6 +19,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/allocator" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/allocator/plan" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/allocator/storepool" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness" @@ -481,6 +482,10 @@ type StoreTestingKnobs struct { // it can be easily extended to validate other properties of baseQueue if // required. BaseQueueInterceptor func(ctx context.Context, bq *baseQueue) + + // FlowControlTestingKnobs provide fine-grained control over the various + // kvflowcontrol components for testing. + FlowControlTestingKnobs *kvflowcontrol.TestingKnobs } // ModuleTestingKnobs is part of the base.ModuleTestingKnobs interface. diff --git a/pkg/kv/txn.go b/pkg/kv/txn.go index 71031d58616f..ff8abe97eeea 100644 --- a/pkg/kv/txn.go +++ b/pkg/kv/txn.go @@ -1557,7 +1557,9 @@ func (txn *Txn) AdmissionHeader() kvpb.AdmissionHeader { // the transaction throughput by 10+%. In that experiment 40% of the // BatchRequests evaluated by KV had been assigned high priority due to // locking. - h.Priority = int32(admissionpb.LockingPri) + if h.Priority < int32(admissionpb.LockingPri) { + h.Priority = int32(admissionpb.LockingPri) + } } return h } diff --git a/pkg/server/server.go b/pkg/server/server.go index b488079804f4..4b0ffe81dccc 100644 --- a/pkg/server/server.go +++ b/pkg/server/server.go @@ -274,7 +274,7 @@ func NewServer(cfg Config, stopper *stop.Stopper) (*Server, error) { idContainer := base.NewSQLIDContainerForNode(nodeIDContainer) admissionOptions := admission.DefaultOptions - if opts, ok := cfg.TestingKnobs.AdmissionControl.(*admission.Options); ok { + if opts, ok := cfg.TestingKnobs.AdmissionControlOptions.(*admission.Options); ok { admissionOptions.Override(opts) } @@ -549,12 +549,17 @@ func NewServer(cfg Config, stopper *stop.Stopper) (*Server, error) { storesForFlowControl := kvserver.MakeStoresForFlowControl(stores) kvflowTokenDispatch := kvflowdispatch.New(registry, storesForFlowControl, nodeIDContainer) admittedEntryAdaptor := newAdmittedLogEntryAdaptor(kvflowTokenDispatch) + admissionKnobs, ok := cfg.TestingKnobs.AdmissionControl.(*admission.TestingKnobs) + if !ok { + admissionKnobs = &admission.TestingKnobs{} + } gcoords := admission.NewGrantCoordinators( cfg.AmbientCtx, st, admissionOptions, registry, admittedEntryAdaptor, + admissionKnobs, ) db.SQLKVResponseAdmissionQ = gcoords.Regular.GetWorkQueue(admission.SQLKVResponseWork) cbID := goschedstats.RegisterRunnableCountCallback(gcoords.Regular.CPULoad) @@ -589,6 +594,10 @@ func NewServer(cfg Config, stopper *stop.Stopper) (*Server, error) { admissionControl.storesFlowControl.ResetStreams(ctx) }) + var raftTransportKnobs *kvserver.RaftTransportTestingKnobs + if knobs := cfg.TestingKnobs.RaftTransport; knobs != nil { + raftTransportKnobs = knobs.(*kvserver.RaftTransportTestingKnobs) + } raftTransport := kvserver.NewRaftTransport( cfg.AmbientCtx, st, @@ -599,7 +608,7 @@ func NewServer(cfg Config, stopper *stop.Stopper) (*Server, error) { admissionControl.kvflowTokenDispatch, admissionControl.storesFlowControl, admissionControl.storesFlowControl, - nil, /* knobs */ + raftTransportKnobs, ) registry.AddMetricStruct(raftTransport.Metrics()) diff --git a/pkg/server/testserver.go b/pkg/server/testserver.go index 09c34a0de48d..80ec0922cafd 100644 --- a/pkg/server/testserver.go +++ b/pkg/server/testserver.go @@ -309,8 +309,8 @@ func makeTestConfigFromParams(params base.TestServerArgs) Config { cfg.TestingKnobs.SQLExecutor = &sql.ExecutorTestingKnobs{} } - if params.Knobs.AdmissionControl == nil { - cfg.TestingKnobs.AdmissionControl = &admission.Options{} + if params.Knobs.AdmissionControlOptions == nil { + cfg.TestingKnobs.AdmissionControlOptions = &admission.Options{} } cfg.ObsServiceAddr = params.ObsServiceAddr @@ -1802,6 +1802,16 @@ func (ts *TestServer) SystemConfigProvider() config.SystemConfigProvider { return ts.node.storeCfg.SystemConfigProvider } +// KVFlowController is part of the TestServerInterface. +func (ts *TestServer) KVFlowController() interface{} { + return ts.node.storeCfg.KVFlowController +} + +// KVFlowHandles is part of the TestServerInterface. +func (ts *TestServer) KVFlowHandles() interface{} { + return ts.node.storeCfg.KVFlowHandles +} + func (ts *TestServer) Codec() keys.SQLCodec { return ts.ExecutorConfig().(sql.ExecutorConfig).Codec } diff --git a/pkg/sql/catalog/lease/lease.go b/pkg/sql/catalog/lease/lease.go index 3eb6e6a471c5..8c40b270bd83 100644 --- a/pkg/sql/catalog/lease/lease.go +++ b/pkg/sql/catalog/lease/lease.go @@ -1312,7 +1312,7 @@ func (m *Manager) refreshSomeLeases(ctx context.Context) { } if _, err := acquireNodeLease(ctx, m, id, AcquireBackground); err != nil { - log.Infof(ctx, "refreshing descriptor: %d lease failed: %s", id, err) + log.Errorf(ctx, "refreshing descriptor: %d lease failed: %s", id, err) if errors.Is(err, catalog.ErrDescriptorNotFound) { // Lease renewal failed due to removed descriptor; Remove this descriptor from cache. diff --git a/pkg/sql/run_control_test.go b/pkg/sql/run_control_test.go index c32c0c5a81c8..2d15a511e98b 100644 --- a/pkg/sql/run_control_test.go +++ b/pkg/sql/run_control_test.go @@ -881,7 +881,7 @@ func TestTenantStatementTimeoutAdmissionQueueCancelation(t *testing.T) { params := base.TestServerArgs{ DefaultTestTenant: base.TestTenantDisabled, Knobs: base.TestingKnobs{ - AdmissionControl: &admission.Options{ + AdmissionControlOptions: &admission.Options{ MaxCPUSlots: numBlockers, // During testing if CPU isn't responsive and skipEnforcement // turns off admission control queuing behavior, for this test diff --git a/pkg/testutils/serverutils/test_server_shim.go b/pkg/testutils/serverutils/test_server_shim.go index 6d1b907b9952..4617d453ba04 100644 --- a/pkg/testutils/serverutils/test_server_shim.go +++ b/pkg/testutils/serverutils/test_server_shim.go @@ -296,6 +296,13 @@ type TestServerInterface interface { // the server. SpanConfigKVSubscriber() interface{} + // KVFlowController returns the embedded kvflowcontrol.Controller for the + // server. + KVFlowController() interface{} + + // KVFlowHandles returns the embedded kvflowcontrol.Handles for the server. + KVFlowHandles() interface{} + // TestTenants returns the test tenants associated with the server TestTenants() []TestTenantInterface diff --git a/pkg/testutils/testcluster/testcluster.go b/pkg/testutils/testcluster/testcluster.go index 56ed144e4928..30f87d7a0415 100644 --- a/pkg/testutils/testcluster/testcluster.go +++ b/pkg/testutils/testcluster/testcluster.go @@ -719,6 +719,23 @@ func (tc *TestCluster) SplitRangeOrFatal( return lhsDesc, rhsDesc } +// MergeRanges merges the range containing leftKey with the range to its right. +func (tc *TestCluster) MergeRanges(leftKey roachpb.Key) (roachpb.RangeDescriptor, error) { + return tc.Servers[0].MergeRanges(leftKey) +} + +// MergeRangesOrFatal is the same as MergeRanges but will Fatal the test on +// error. +func (tc *TestCluster) MergeRangesOrFatal( + t testing.TB, leftKey roachpb.Key, +) roachpb.RangeDescriptor { + mergedDesc, err := tc.MergeRanges(leftKey) + if err != nil { + t.Fatalf(`merging at %s: %+v`, leftKey, err) + } + return mergedDesc +} + // Target returns a ReplicationTarget for the specified server. func (tc *TestCluster) Target(serverIdx int) roachpb.ReplicationTarget { s := tc.Servers[serverIdx] diff --git a/pkg/util/admission/grant_coordinator.go b/pkg/util/admission/grant_coordinator.go index c76c87720b09..ef219b6a38d8 100644 --- a/pkg/util/admission/grant_coordinator.go +++ b/pkg/util/admission/grant_coordinator.go @@ -64,7 +64,8 @@ type StoreGrantCoordinators struct { onLogEntryAdmitted OnLogEntryAdmitted closeCh chan struct{} - disableTickerForTesting bool + disableTickerForTesting bool // TODO(irfansharif): Fold into the testing knobs struct below. + knobs *TestingKnobs } // SetPebbleMetricsProvider sets a PebbleMetricsProvider and causes the load @@ -187,7 +188,7 @@ func (sgc *StoreGrantCoordinators) initGrantCoordinator(storeID roachpb.StoreID) sgc.settings, sgc.workQueueMetrics, opts, - nil, /* knobs */ + sgc.knobs, sgc.onLogEntryAdmitted, &coord.mu.Mutex, ) @@ -384,12 +385,13 @@ func NewGrantCoordinators( opts Options, registry *metric.Registry, onLogEntryAdmitted OnLogEntryAdmitted, + knobs *TestingKnobs, ) GrantCoordinators { metrics := makeGrantCoordinatorMetrics() registry.AddMetricStruct(metrics) return GrantCoordinators{ - Stores: makeStoresGrantCoordinators(ambientCtx, opts, st, metrics, registry, onLogEntryAdmitted), + Stores: makeStoresGrantCoordinators(ambientCtx, opts, st, metrics, registry, onLogEntryAdmitted, knobs), Regular: makeRegularGrantCoordinator(ambientCtx, opts, st, metrics, registry), Elastic: makeElasticGrantCoordinator(ambientCtx, st, registry), } @@ -412,7 +414,7 @@ func makeElasticGrantCoordinator( elasticCPUInternalWorkQueue := &WorkQueue{} initWorkQueue(elasticCPUInternalWorkQueue, ambientCtx, KVWork, elasticCPUGranter, st, elasticWorkQueueMetrics, - workQueueOptions{usesTokens: true}) // will be closed by the embedding *ElasticCPUWorkQueue + workQueueOptions{usesTokens: true}, nil /* knobs */) // will be closed by the embedding *ElasticCPUWorkQueue elasticCPUWorkQueue := makeElasticCPUWorkQueue(st, elasticCPUInternalWorkQueue, elasticCPUGranter, elasticCPUGranterMetrics) elasticCPUGrantCoordinator := makeElasticCPUGrantCoordinator(elasticCPUGranter, elasticCPUWorkQueue, schedulerLatencyListener) elasticCPUGranter.setRequester(elasticCPUInternalWorkQueue) @@ -427,7 +429,11 @@ func makeStoresGrantCoordinators( metrics GrantCoordinatorMetrics, registry *metric.Registry, onLogEntryAdmitted OnLogEntryAdmitted, + knobs *TestingKnobs, ) *StoreGrantCoordinators { + if knobs == nil { + knobs = &TestingKnobs{} + } // These metrics are shared across all stores and broken down by priority for // the common priorities. // TODO(baptist): Add per-store metrics. @@ -446,6 +452,7 @@ func makeStoresGrantCoordinators( kvIOTokensExhaustedDuration: metrics.KVIOTokensExhaustedDuration, workQueueMetrics: storeWorkQueueMetrics, onLogEntryAdmitted: onLogEntryAdmitted, + knobs: knobs, } return storeCoordinators } diff --git a/pkg/util/admission/granter_test.go b/pkg/util/admission/granter_test.go index 25fa20cd3bdf..9c312b76146c 100644 --- a/pkg/util/admission/granter_test.go +++ b/pkg/util/admission/granter_test.go @@ -98,7 +98,7 @@ func TestGranterBasic(t *testing.T) { return req } delayForGrantChainTermination = 0 - coords := NewGrantCoordinators(ambientCtx, settings, opts, registry, &noopOnLogEntryAdmitted{}) + coords := NewGrantCoordinators(ambientCtx, settings, opts, registry, &noopOnLogEntryAdmitted{}, nil) defer coords.Close() coord = coords.Regular return flushAndReset() @@ -323,7 +323,7 @@ func TestStoreCoordinators(t *testing.T) { return str }, } - coords := NewGrantCoordinators(ambientCtx, settings, opts, registry, &noopOnLogEntryAdmitted{}) + coords := NewGrantCoordinators(ambientCtx, settings, opts, registry, &noopOnLogEntryAdmitted{}, nil) // There is only 1 KVWork requester at this point in initialization, for the // Regular GrantCoordinator. require.Equal(t, 1, len(requesters)) diff --git a/pkg/util/admission/testing_knobs.go b/pkg/util/admission/testing_knobs.go index 659552c84dc5..7dfc9ca2426a 100644 --- a/pkg/util/admission/testing_knobs.go +++ b/pkg/util/admission/testing_knobs.go @@ -18,6 +18,9 @@ import ( // TestingKnobs provide fine-grained control over the various admission control // components for testing. +// +// TODO(irfansharif): Consolidate the various other testing-knob like things (in +// admission.Options, for example) into this one struct. type TestingKnobs struct { // AdmittedReplicatedWorkInterceptor is invoked whenever replicated work is // admitted. @@ -28,6 +31,13 @@ type TestingKnobs struct { originalTokens int64, createTime int64, ) + + // DisableWorkQueueFastPath disables the fast-path in work queues. + DisableWorkQueueFastPath bool + + // DisableWorkQueueGranting disables the work queue from granting admission + // to waiting work. + DisableWorkQueueGranting func() bool } // ModuleTestingKnobs is part of the base.ModuleTestingKnobs interface. diff --git a/pkg/util/admission/work_queue.go b/pkg/util/admission/work_queue.go index d7d2b344b731..3613a412cd9c 100644 --- a/pkg/util/admission/work_queue.go +++ b/pkg/util/admission/work_queue.go @@ -311,6 +311,7 @@ type WorkQueue struct { stopCh chan struct{} timeSource timeutil.TimeSource + knobs *TestingKnobs } var _ requester = &WorkQueue{} @@ -352,7 +353,7 @@ func makeWorkQueue( opts workQueueOptions, ) requester { q := &WorkQueue{} - initWorkQueue(q, ambientCtx, workKind, granter, settings, metrics, opts) + initWorkQueue(q, ambientCtx, workKind, granter, settings, metrics, opts, nil) return q } @@ -364,7 +365,11 @@ func initWorkQueue( settings *cluster.Settings, metrics *WorkQueueMetrics, opts workQueueOptions, + knobs *TestingKnobs, ) { + if knobs == nil { + knobs = &TestingKnobs{} + } stopCh := make(chan struct{}) timeSource := opts.timeSource @@ -383,6 +388,7 @@ func initWorkQueue( q.metrics = metrics q.stopCh = stopCh q.timeSource = timeSource + q.knobs = knobs func() { q.mu.Lock() @@ -609,7 +615,7 @@ func (q *WorkQueue) Admit(ctx context.Context, info WorkInfo) (enabled bool, err // threshold for LIFO queueing based on observed admission latency. tenant.priorityStates.requestAtPriority(info.Priority) - if len(q.mu.tenantHeap) == 0 { + if len(q.mu.tenantHeap) == 0 && !q.knobs.DisableWorkQueueFastPath { // Fast-path. Try to grab token/slot. // Optimistically update used to avoid locking again. tenant.used += uint64(info.RequestedCount) @@ -848,6 +854,10 @@ func (q *WorkQueue) granted(grantChainID grantChainID) int64 { q.mu.Unlock() return 0 } + if fn := q.knobs.DisableWorkQueueGranting; fn != nil && fn() { + q.mu.Unlock() + return 0 + } tenant := q.mu.tenantHeap[0] var item *waitingWork if len(tenant.waitingWorkHeap) > 0 { @@ -2134,7 +2144,7 @@ func makeStoreWorkQueue( opts.usesAsyncAdmit = true for i := range q.q { - initWorkQueue(&q.q[i], ambientCtx, KVWork, granters[i], settings, metrics, opts) + initWorkQueue(&q.q[i], ambientCtx, KVWork, granters[i], settings, metrics, opts, knobs) q.q[i].onAdmittedReplicatedWork = q } // Arbitrary initial value. This will be replaced before any meaningful From 87d6547dba9451e021e5a8872773fb821a5ea7cc Mon Sep 17 00:00:00 2001 From: irfan sharif Date: Fri, 5 May 2023 14:31:28 -0400 Subject: [PATCH 07/12] kvserver: document/simplify flow control integration MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit This commit documents the kvflowcontrol integration interfaces introduced in earlier commits across flow_control_*.go, grouping commentary and interfaces in a top-level flow_control_integration.go, and makes minor simplifications where applicable. It's helpful to read kvflowcontrol/{doc,kvflowcontrol}.go to understand the library components in question, and also the comment block on replicaFlowControlIntegration. Here's how the various pieces fit together: ā”Œā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā” ā”‚ Receiver (client) ā”‚ ā”œā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”“ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”¬ā”€ā”¬ā”€ā” ā”Œā”€ā”€ā—‹ kvflowcontrolpb.AdmittedRaftLogEntries ā”‚ ā”‚ ā”‚ ā”‚ ā””ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”“ā”€ā”“ā”€ā”˜ ā”‚ ā”Œā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā” ā”‚ ā”‚ Receiver (client) ā”‚ ā”‚ ā”œā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”“ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”¬ā”€ā”¬ā”€ā” ā”Œā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā–¶ā”€ā”¼ā”€ā”€ā—‹ kvflowcontrolpb.AdmittedRaftLogEntries ā”‚ ā”‚ ā”‚ ā”‚ ā”‚ ā””ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”“ā”€ā”“ā”€ā”˜ ['1] gRPC streams ā”‚ connecting/disconnecting [1] RaftMessageBatch ā”‚ ā”‚ ā”‚ ā”Œā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā” ā”‚ ā”‚ ā”‚ Sender (server) ā”‚ ā”‚ ā”‚ ā”œā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”“ā”€ā”€ā”‚ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā” ā”Œā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā” ā”‚ ā”‚ RaftTransport ā”‚ ā”‚ ā”‚ StoresForFlowControl ā”‚ ā”‚ ā”‚ ā”‚ ā”‚ ā”‚ ā”‚ ā”‚ ā”‚ ā”‚ ā”‚ ā”‚ ā”Œā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā” ā”‚ ā”‚ ā”‚ ā””ā”€ā”€ā”€ā”€[2] Lookup ā”€ā”¼ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”¼ā”€ā–¶ā”‚ kvflowcontrol.Handles ā—‹ā”€ā”¼ā”€ā”€ā” ā”‚ ā”‚ ā”‚ ā”‚ ā””ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”˜ ā”‚ ā”‚ ā”‚ ā”‚ ā”Œā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā” ā”‚ ā”‚ ā”Œā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā” ā”‚ ā”‚ ā””ā”€ā”€ā”¼ā–¶ā”‚ connectionTrackerForFlowControl ā”‚ ā”œā”€ā”€['2]ā”€ā”€ā”¼ā”€ā–¶ā”‚ RaftTransportDisconnectedListener ā”‚ ā”‚ ā”‚ ā”‚ ā””ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”˜ ā”‚ ā”‚ ā””ā”€ā”€ā”€ā”€ā”€ā”€ā—‹ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”˜ ā”‚ ā”‚ ā””ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā–²ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”˜ ā””ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”¼ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”˜ ā”‚ ā”‚ ā”‚ ā”‚ ā”‚ ['3] onRaftTransportDisconnected [3] ReturnTokensUpto ā”‚ ā”‚ ā”‚ ā”‚ ā”‚ ā”‚ ā”‚ ā”Œā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”¼ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”¼ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”¬ā”€ā”¬ā”€ā” ā”‚ ā”‚ replicaFlowControlIntegrationā”‚ ā”Œā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā–¼ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā” ā”‚ ā”‚ ā”‚ ā”‚ ā”‚ ā”‚ ā”‚ kvflowcontrol.Handle ā”‚ ā”‚ ā”‚ ā”‚ ā”‚ ā”‚ onBecameLeader() ā–¼ ā””ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā–²ā”€ā–²ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”˜ ā”‚ ā”‚ ā”‚ ā”‚ ā”‚ onBecameFollower() ā—‹ā”€ā”€ā”€ā”€['4] DisconnectStream ā”€ā”€ā”˜ ā”‚ ā”‚ ā”‚ ā”‚ ā”‚ ā”‚ onDescChanged() ā—€ā”€ā”€ā”€ ["5] tryReconnect ā”€ā”€ā”€ā”€ā”€ā”€ā” ā”‚ ā”‚ ā”‚ ā”‚ ā”‚ ā”‚ onFollowersPaused() ā—‹ā”€ā”€ā”€ ["7] ConnectStream ā”€ā”€ā”€ā”€ā”¼ā”€ā”˜ ā”‚ ā”‚ ā”‚ ā”‚ ā”‚ = onRaftTransportDisconnected() ā”Œā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā–¼ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā” ā”‚ ā”‚ ā”‚ ā”‚ ā”‚ = onRaftTicked() ā”‚ replicaForFlowControl ā”‚ ā”‚ ā”‚ ā”‚ ā”‚ ā”‚ onReplicaDestroyed() ā”‚ ā”‚ ā”‚ ā”‚ ā”‚ ā”‚ ā”‚ ā”‚ getDescriptor() ā”‚ ā”‚ ā”‚ ā”‚ ["6] isConnectedTo ā”‚ ā”‚ getPausedFollowers() ā”‚ ā”‚ ā”‚ ā”‚ ā”‚ ā”‚ ā”‚ getBehindFollowers() ā”‚ ā”‚ ā”‚ ā”‚ ā”‚ ā”‚ ā”‚ getInactiveFollowers() ā”‚ ā”‚ ā”‚ ā”‚ ā””ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”¼ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā–¶ = getDisconnectedFollowers() ā”‚ ā”‚ ā”‚ ā”‚ ā”‚ ā””ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”˜ ā”‚ ā”‚ ā”‚ ā””ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”“ā”€ā”“ā”€ā”˜ The "server" and "client" demarcations refer to the server and client-side of the RaftTransport stream. "Sender" and "Receiver" is kvflowcontrol verbiage, referring to where proposals originate (and flow tokens deducted) and the remote follower nodes where they're received. Below-raft admission happens asynchronously on the receiver nodes, of which the sender is informed, which in turn lets it release flow tokens and unblock further proposals. Notation: - Stacked boxes (with " ā”‚ā”‚ā”‚" on the right hand side) indicate that there are multiple of a kind. Like multiple replicaFlowControlIntegration implementations (one per locally held replica), multiple kvflowcontrolpb.AdmittedRaftLogEntries, etc. - [], ['], and ["] denote independent sequences, explained in text below. --- A. How are flow tokens returned after work is admitted below-raft on remote, receiver nodes? [1]: When work gets admitted below-raft on the receiver, the sender (where work originated, and flow tokens were deducted) is informed of the fact through the RaftMessageBatch gRPC stream. There are two bi-directional raft transport streams between a pair of nodes. We piggyback kvflowcontrolpb.AdmittedRaftLogEntries on raft messages being sent from the RaftMessageBatch client to the RaftMessageBatch server. [2]: We lookup the relevant kvflowcontrol.Handle from the set of kvflowcontrol.Handles, to inform it of below-raft admission. [3]: We use the relevant kvflowcontrol.Handle (hanging off of some locally held replica) to return relevant previously deducted flow tokens. The piggy-backing from [1] and the intercepting of piggy-backed messages and kvflowcontrol.Handle lookup from [2] both happen in the RaftTransport layer, in raft_transport.go. The set of local kvflowcontrol.Handles is exposed through the StoresForFlowControl interface, backed by local stores and their contained replicas. Each replica exposes the underlying handle through the replicaFlowControlIntegration interface. --- B. How do we react to raft transport streams breaking? (I1 from kvflowcontrol/doc.go) ['1]: The server-side of RaftMessageBatch observes every client-initiated stream breaking. The connectionTrackerForFlowControl, used within the RaftTransport layer, also monitors all live gRPC streams to understand exactly the set of clients we're connected to. ['2]: Whenever any raft transport gRPC stream breaks, we notify components of this fact through the RaftTransportDisconnectedListener interface. ['3]: This in turn informs all locally held replicas, through the replicaFlowControlIntegration interface. ['4]: We actively disconnect streams for replicas we just disconnected from as informed by the raft transport. Note that we actually plumb down information about exactly which raft transport streams broke. It's not enough to simply inform the various replicaFlowControlIntegrations of some transport stream breaking, and for them to then determine which streams to disconnect. This is because it's possible for the streams to be re-established in the interim, or for there to be another active stream from the same client but using a different RPC class. We still want to free up all tokens for that replication stream, lest we leak flow tokens in transit on the particular stream that broke. --- C. What happens when the raft transport streams reconnect? (I1 from kvflowcontrol/doc.go) ["5]: The replicaFlowControlIntegration interface is used to periodically reconnect previously disconnected streams. This is driven primarily through the onRaftTicked() API, but also happens opportunistically through onFollowersPaused(), onRaftTransportDisconnected(), etc. ["6]: We check whether we're connected to remote replicas via the raftTransportForFlowControl.isConnectedTo(). This is powered by the connectionTrackerForFlowControl embedded in the RaftTransport which monitors all active gRPC streams as seen on the server-side. ["7]: If we're now connected to previously disconnected replicas, we inform the underlying kvflowcontrol.Handle in order to deduct flow tokens for subsequent proposals. --- replicaFlowControlIntegration is used to integrate with replication flow control. It intercepts various points in a replica's lifecycle, like it acquiring raft leadership or losing it, or its raft membership changing, etc. Accessing it requires Replica.mu to be held, exclusively (this is asserted on in the canonical implementation). The "external" state is mediated by the replicaForFlowControl interface. The state transitions look as follows: ā”€ ā”€ ā”€ ā”€ ā”€ ā”€ ā”€ ā”Œā”€ā”€ā”€ā”€ā”€ onDestroyed ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā–¶ ā•³ā•³ā•³ā•³ā•³ā•³ā•³ā•³ā•³ā•³ā•³ā•³ā•³ ā”€ ā”€ ā”€ ā”€ ā”€ ā”€ ā” ā”‚ ā”‚ ā”Œā”€ā”€ā”€ onDescChanged(removed=self) ā”€ā”€ā–¶ ā•³ destroyed ā•³ ā”Œā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ onBecameLeader ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā” ā”‚ ā”‚ ā•³ā•³ā•³ā•³ā•³ā•³ā•³ā•³ā•³ā•³ā•³ā•³ā•³ ā”‚ ā”‚ ā”‚ ā”‚ ā”‚ ā”‚ ā—‹ ā—‹ ā—‹ ā–¼ ā—‹ ā—‹ ā”Œ ā”€ ā”€ ā”€ ā”€ ā”€ ā”€ ā”€ ā” ā”Œā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā” ā”€ ā”€ ā”€ ā—‹ follower ā”‚ leader ā”‚ ā—‹ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā” ā”” ā”€ ā”€ ā”€ ā”€ ā”€ ā”€ ā”€ ā”˜ ā””ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”˜ ā”‚ ā–² ā–² ā—‹ ā–² onDescChanged ā”‚ ā”‚ ā”‚ ā”‚ ā”‚ onFollowersPaused ā”‚ ā”€ ā”€ ā”€ ā”€ ā”€ ā”€ ā”€ ā””ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ onBecameFollower ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”˜ ā””ā”€ā”€ā”€ā”€ā”€ā”€ onRaftTransportDisconnected ā”€ā”˜ onRaftTicked We're primarily interested in transitions to/from the leader state -- the equivalent transitions from the follower state are no-ops. - onBecameLeader is when the replica acquires raft leadership. At this point we initialize the underlying kvflowcontrol.Handle and other internal tracking state to handle subsequent transitions. - onBecameFollower is when the replica loses raft leadership. We close the underlying kvflowcontrol.Handle and clear other tracking state. - onDescChanged is when the range descriptor changes. We react to changes by disconnecting streams for replicas no longer part of the range, connecting streams for newly members of the range, closing the underlying kvflowcontrol.Handle + clearing tracking state if we ourselves are no longer part of the range. - onFollowersPaused is when the set of paused followers have changed. We react to it by disconnecting streams for newly paused followers, or reconnecting to newly unpaused ones. - onRaftTransportDisconnected is when we're no longer connected to some replicas via the raft transport. We react to it by disconnecting relevant streams. - onRaftTicked is invoked periodically, and refreshes the set of streams we're connected to. It disconnects streams to inactive followers and/or reconnects to now-active followers. It also observes raft progress state for individual replicas, disconnecting from ones we're not actively replicating to (because they're too far behind on their raft log, in need of snapshots, or because we're unaware of their committed log indexes). It also reconnects streams if the raft progress changes. - onDestroyed is when the replica is destroyed. Like onBecameFollower, we close the underlying kvflowcontrol.Handle and clear other tracking state. Release note: None --- pkg/kv/kvserver/BUILD.bazel | 1 + pkg/kv/kvserver/flow_control_integration.go | 271 +++++++++++ .../kvserver/flow_control_integration_test.go | 104 +++- .../kvserver/flow_control_raft_transport.go | 16 - pkg/kv/kvserver/flow_control_replica.go | 95 ++-- .../flow_control_replica_integration.go | 460 +++++++++--------- .../flow_control_replica_integration_test.go | 78 +-- pkg/kv/kvserver/flow_control_stores.go | 10 +- pkg/kv/kvserver/helpers_test.go | 2 +- pkg/kv/kvserver/kvflowcontrol/doc.go | 60 ++- .../kvserver/kvflowcontrol/testing_knobs.go | 3 + pkg/kv/kvserver/raft_log_queue.go | 3 +- pkg/kv/kvserver/raft_log_queue_test.go | 2 +- pkg/kv/kvserver/replica_destroy.go | 2 +- pkg/kv/kvserver/replica_proposal_quota.go | 11 +- pkg/kv/kvserver/replica_raft.go | 2 +- pkg/kv/kvserver/replica_raft_overload.go | 2 +- pkg/kv/kvserver/split_delay_helper.go | 3 +- pkg/kv/kvserver/store.go | 5 + .../flow_control_integration/raft_snapshot | 3 +- .../desc_changed | 21 +- .../disconnected_followers | 45 ++ .../inactive_followers | 10 +- .../paused_followers | 4 +- .../replica_destroyed | 6 +- .../replica_raft_leadership | 16 +- .../replica_removed_self | 4 +- pkg/util/log/clog.go | 5 + 28 files changed, 824 insertions(+), 420 deletions(-) create mode 100644 pkg/kv/kvserver/flow_control_integration.go create mode 100644 pkg/kv/kvserver/testdata/flow_control_replica_integration/disconnected_followers diff --git a/pkg/kv/kvserver/BUILD.bazel b/pkg/kv/kvserver/BUILD.bazel index a91726f0d1e9..55c034eaa9c0 100644 --- a/pkg/kv/kvserver/BUILD.bazel +++ b/pkg/kv/kvserver/BUILD.bazel @@ -12,6 +12,7 @@ go_library( "consistency_queue.go", "debug_print.go", "doc.go", + "flow_control_integration.go", "flow_control_raft_transport.go", "flow_control_replica.go", "flow_control_replica_integration.go", diff --git a/pkg/kv/kvserver/flow_control_integration.go b/pkg/kv/kvserver/flow_control_integration.go new file mode 100644 index 000000000000..e1bde737c3e1 --- /dev/null +++ b/pkg/kv/kvserver/flow_control_integration.go @@ -0,0 +1,271 @@ +// Copyright 2023 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package kvserver + +import ( + "context" + + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol/kvflowcontrolpb" + "github.com/cockroachdb/cockroach/pkg/roachpb" +) + +// The various flow_control_*.go files in this package contain integration code +// between kvflowcontrol and kvserver. This file specifically houses the +// interfaces that are implemented elsewhere, and documents how they fit +// together. It's helpful to read kvflowcontrol/{doc,kvflowcontrol}.go to +// understand the library components in question, the comment blocks on +// replicaFlowControlIntegration and (*RaftTransport).kvflowControl. The +// integration interfaces here help address I1-I13 from kvflowcontrol/doc.go. +// +// Here's how the various pieces fit together: +// +// ā”Œā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā” +// ā”‚ Receiver (client) ā”‚ +// ā”œā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”“ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”¬ā”€ā”¬ā”€ā” +// ā”Œā”€ā”€ā—‹ kvflowcontrolpb.AdmittedRaftLogEntries ā”‚ ā”‚ ā”‚ +// ā”‚ ā””ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”“ā”€ā”“ā”€ā”˜ +// ā”‚ ā”Œā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā” +// ā”‚ ā”‚ Receiver (client) ā”‚ +// ā”‚ ā”œā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”“ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”¬ā”€ā”¬ā”€ā” +// ā”Œā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā–¶ā”€ā”¼ā”€ā”€ā—‹ kvflowcontrolpb.AdmittedRaftLogEntries ā”‚ ā”‚ ā”‚ +// ā”‚ ā”‚ ā””ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”“ā”€ā”“ā”€ā”˜ +// ['1] gRPC streams ā”‚ +// connecting/disconnecting [1] RaftMessageBatch +// ā”‚ ā”‚ +// ā”‚ ā”Œā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā” ā”‚ +// ā”‚ ā”‚ Sender (server) ā”‚ ā”‚ +// ā”‚ ā”œā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”“ā”€ā”€ā”‚ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā” ā”Œā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā” +// ā”‚ ā”‚ RaftTransport ā”‚ ā”‚ ā”‚ StoresForFlowControl ā”‚ +// ā”‚ ā”‚ ā”‚ ā”‚ ā”‚ ā”‚ +// ā”‚ ā”‚ ā”‚ ā”‚ ā”‚ ā”Œā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā” ā”‚ +// ā”‚ ā”‚ ā””ā”€ā”€ā”€ā”€[2] Lookup ā”€ā”¼ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”¼ā”€ā–¶ā”‚ kvflowcontrol.Handles ā—‹ā”€ā”¼ā”€ā”€ā” +// ā”‚ ā”‚ ā”‚ ā”‚ ā””ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”˜ ā”‚ ā”‚ +// ā”‚ ā”‚ ā”Œā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā” ā”‚ ā”‚ ā”Œā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā” ā”‚ ā”‚ +// ā””ā”€ā”€ā”¼ā–¶ā”‚ connectionTrackerForFlowControl ā”‚ ā”œā”€ā”€['2]ā”€ā”€ā”¼ā”€ā–¶ā”‚ RaftTransportDisconnectedListener ā”‚ ā”‚ ā”‚ +// ā”‚ ā””ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”˜ ā”‚ ā”‚ ā””ā”€ā”€ā”€ā”€ā”€ā”€ā—‹ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”˜ ā”‚ ā”‚ +// ā””ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā–²ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”˜ ā””ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”¼ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”˜ ā”‚ +// ā”‚ ā”‚ ā”‚ +// ā”‚ ['3] onRaftTransportDisconnected [3] ReturnTokensUpto +// ā”‚ ā”‚ ā”‚ +// ā”‚ ā”‚ ā”‚ +// ā”‚ ā”Œā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”¼ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”¼ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”¬ā”€ā”¬ā”€ā” +// ā”‚ ā”‚ replicaFlowControlIntegrationā”‚ ā”Œā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā–¼ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā” ā”‚ ā”‚ ā”‚ +// ā”‚ ā”‚ ā”‚ ā”‚ kvflowcontrol.Handle ā”‚ ā”‚ ā”‚ ā”‚ +// ā”‚ ā”‚ onBecameLeader() ā–¼ ā””ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā–²ā”€ā–²ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”˜ ā”‚ ā”‚ ā”‚ +// ā”‚ ā”‚ onBecameFollower() ā—‹ā”€ā”€ā”€ā”€['4] DisconnectStream ā”€ā”€ā”˜ ā”‚ ā”‚ ā”‚ ā”‚ +// ā”‚ ā”‚ onDescChanged() ā—€ā”€ā”€ā”€ ["5] tryReconnect ā”€ā”€ā”€ā”€ā”€ā”€ā” ā”‚ ā”‚ ā”‚ ā”‚ +// ā”‚ ā”‚ onFollowersPaused() ā—‹ā”€ā”€ā”€ ["7] ConnectStream ā”€ā”€ā”€ā”€ā”¼ā”€ā”˜ ā”‚ ā”‚ ā”‚ +// ā”‚ ā”‚ = onRaftTransportDisconnected() ā”Œā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā–¼ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā” ā”‚ ā”‚ ā”‚ +// ā”‚ ā”‚ = onRaftTicked() ā”‚ replicaForFlowControl ā”‚ ā”‚ ā”‚ ā”‚ +// ā”‚ ā”‚ onReplicaDestroyed() ā”‚ ā”‚ ā”‚ ā”‚ ā”‚ +// ā”‚ ā”‚ ā”‚ getDescriptor() ā”‚ ā”‚ ā”‚ ā”‚ +// ["6] isConnectedTo ā”‚ ā”‚ getPausedFollowers() ā”‚ ā”‚ ā”‚ ā”‚ +// ā”‚ ā”‚ ā”‚ getBehindFollowers() ā”‚ ā”‚ ā”‚ ā”‚ +// ā”‚ ā”‚ ā”‚ getInactiveFollowers() ā”‚ ā”‚ ā”‚ ā”‚ +// ā””ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”¼ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā–¶ = getDisconnectedFollowers() ā”‚ ā”‚ ā”‚ ā”‚ +// ā”‚ ā””ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”˜ ā”‚ ā”‚ ā”‚ +// ā””ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”“ā”€ā”“ā”€ā”˜ +// +// The "server" and "client" demarcations refer to the server and client-side of +// the RaftTransport stream. "Sender" and "Receiver" is kvflowcontrol verbiage, +// referring to where proposals originate (and flow tokens deducted) and the +// remote follower nodes where they're received. Below-raft admission happens +// asynchronously on the receiver nodes, of which the sender is informed, which +// in turn lets it release flow tokens and unblock further proposals. +// +// Notation: +// - Stacked boxes (with " ā”‚ā”‚ā”‚" on the right hand side) indicate that there are +// multiple of a kind. Like multiple replicaFlowControlIntegration +// implementations (one per locally held replica), multiple +// kvflowcontrolpb.AdmittedRaftLogEntries, etc. +// - [], ['], and ["] denote independent sequences, +// explained in text below. +// +// --- +// +// A. How are flow tokens returned after work is admitted below-raft on remote, +// receiver nodes? +// +// [1]: When work gets admitted below-raft on the receiver, the sender (where +// work originated, and flow tokens were deducted) is informed of the fact +// through the RaftMessageBatch gRPC stream. There are two bi-directional +// raft transport streams between a pair of nodes. We piggyback +// kvflowcontrolpb.AdmittedRaftLogEntries on raft messages being sent from +// the RaftMessageBatch client to the RaftMessageBatch server. +// [2]: We lookup the relevant kvflowcontrol.Handle from the set of +// kvflowcontrol.Handles, to inform it of below-raft admission. +// [3]: We use the relevant kvflowcontrol.Handle (hanging off of some locally +// held replica) to return relevant previously deducted flow tokens. +// +// The piggy-backing from [1] and the intercepting of piggy-backed messages and +// kvflowcontrol.Handle lookup from [2] both happen in the RaftTransport layer, +// in raft_transport.go. The set of local kvflowcontrol.Handles is exposed +// through the StoresForFlowControl interface, backed by local stores and their +// contained replicas. Each replica exposes the underlying handle through the +// replicaFlowControlIntegration interface. +// +// --- +// +// B. How do we react to raft transport streams breaking? (I1 from +// kvflowcontrol/doc.go) +// +// ['1]: The server-side of RaftMessageBatch observes every client-initiated +// stream breaking. The connectionTrackerForFlowControl, used within the +// RaftTransport layer, also monitors all live gRPC streams to understand +// exactly the set of clients we're connected to. +// ['2]: Whenever any raft transport gRPC stream breaks, we notify components of +// this fact through the RaftTransportDisconnectedListener interface. +// ['3]: This in turn informs all locally held replicas, through the +// replicaFlowControlIntegration interface. +// ['4]: We actively disconnect streams for replicas we just disconnected from +// as informed by the raft transport. +// +// Note that we actually plumb down information about exactly which raft +// transport streams broke. It's not enough to simply inform the various +// replicaFlowControlIntegrations of some transport stream breaking, and for +// them to then determine which streams to disconnect. This is because it's +// possible for the streams to be re-established in the interim, or for there to +// be another active stream from the same client but using a different RPC +// class. We still want to free up all tokens for that replication stream, lest +// we leak flow tokens in transit on the particular stream that broke. +// +// --- +// +// C. What happens when the raft transport streams reconnect? (I1 from +// kvflowcontrol/doc.go) +// +// ["5]: The replicaFlowControlIntegration interface is used to periodically +// reconnect previously disconnected streams. This is driven primarily +// through the onRaftTicked() API, but also happens opportunistically +// through onFollowersPaused(), onRaftTransportDisconnected(), etc. +// ["6]: We check whether we're connected to remote replicas via the +// raftTransportForFlowControl.isConnectedTo(). This is powered by the +// connectionTrackerForFlowControl embedded in the RaftTransport which +// monitors all active gRPC streams as seen on the server-side. +// ["7]: If we're now connected to previously disconnected replicas, we inform +// the underlying kvflowcontrol.Handle in order to deduct flow tokens for +// subsequent proposals. + +// replicaFlowControlIntegration is used to integrate with replication flow +// control. It intercepts various points in a replica's lifecycle, like it +// acquiring raft leadership or losing it, or its raft membership changing, etc. +// Accessing it requires Replica.mu to be held, exclusively (this is asserted on +// in the canonical implementation). The "external" state is mediated by the +// replicaForFlowControl interface. The state transitions look as follows: +// +// ā”€ ā”€ ā”€ ā”€ ā”€ ā”€ ā”€ ā”Œā”€ā”€ā”€ā”€ā”€ onDestroyed ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā–¶ ā•³ā•³ā•³ā•³ā•³ā•³ā•³ā•³ā•³ā•³ā•³ā•³ā•³ +// ā”€ ā”€ ā”€ ā”€ ā”€ ā”€ ā” ā”‚ ā”‚ ā”Œā”€ā”€ā”€ onDescChanged(removed=self) ā”€ā”€ā–¶ ā•³ destroyed ā•³ +// ā”Œā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ onBecameLeader ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā” ā”‚ ā”‚ ā•³ā•³ā•³ā•³ā•³ā•³ā•³ā•³ā•³ā•³ā•³ā•³ā•³ +// ā”‚ ā”‚ ā”‚ ā”‚ ā”‚ ā”‚ +// ā—‹ ā—‹ ā—‹ ā–¼ ā—‹ ā—‹ +// ā”Œ ā”€ ā”€ ā”€ ā”€ ā”€ ā”€ ā”€ ā” ā”Œā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā” +// ā”€ ā”€ ā”€ ā—‹ follower ā”‚ leader ā”‚ ā—‹ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā” +// ā”” ā”€ ā”€ ā”€ ā”€ ā”€ ā”€ ā”€ ā”˜ ā””ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”˜ ā”‚ +// ā–² ā–² ā—‹ ā–² onDescChanged ā”‚ +// ā”‚ ā”‚ ā”‚ ā”‚ onFollowersPaused ā”‚ +// ā”€ ā”€ ā”€ ā”€ ā”€ ā”€ ā”€ ā””ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ onBecameFollower ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”˜ ā””ā”€ā”€ā”€ā”€ā”€ā”€ onRaftTransportDisconnected ā”€ā”˜ +// onRaftTicked +// +// We're primarily interested in transitions to/from the leader state -- the +// equivalent transitions from the follower state are no-ops. +// +// - onBecameLeader is when the replica acquires raft leadership. At this +// point we initialize the underlying kvflowcontrol.Handle and other +// internal tracking state to handle subsequent transitions. +// +// - onBecameFollower is when the replica loses raft leadership. We close the +// underlying kvflowcontrol.Handle and clear other tracking state. +// +// - onDescChanged is when the range descriptor changes. We react to changes +// by disconnecting streams for replicas no longer part of the range, +// connecting streams for newly members of the range, closing the underlying +// kvflowcontrol.Handle + clearing tracking state if we ourselves are no +// longer part of the range. +// +// - onFollowersPaused is when the set of paused followers have changed. We +// react to it by disconnecting streams for newly paused followers, or +// reconnecting to newly unpaused ones. +// +// - onRaftTransportDisconnected is when we're no longer connected to some +// replicas via the raft transport. We react to it by disconnecting relevant +// streams. +// +// - onRaftTicked is invoked periodically, and refreshes the set of streams +// we're connected to. It disconnects streams to inactive followers and/or +// reconnects to now-active followers. It also observes raft progress state +// for individual replicas, disconnecting from ones we're not actively +// replicating to (because they're too far behind on their raft log, in need +// of snapshots, or because we're unaware of their committed log indexes). +// It also reconnects streams if the raft progress changes. +// +// - onDestroyed is when the replica is destroyed. Like onBecameFollower, we +// close the underlying kvflowcontrol.Handle and clear other tracking state. +// +// TODO(irfansharif): Today, whenever a raft transport stream breaks, we +// propagate O(replicas) notifications. We could do something simpler -- +// bump a sequence number for stores that have been disconnected and lazily +// release tokens the next time onRaftTicked() is invoked. Internally we'd +// track the last sequence number we observed for each replication stream. +type replicaFlowControlIntegration interface { + onBecameLeader(context.Context) + onBecameFollower(context.Context) + onDescChanged(context.Context) + onFollowersPaused(context.Context) + onRaftTransportDisconnected(context.Context, ...roachpb.StoreID) + onRaftTicked(context.Context) + onDestroyed(context.Context) + + handle() (kvflowcontrol.Handle, bool) +} + +// replicaForFlowControl abstracts the interface of an individual Replica, as +// needed by replicaFlowControlIntegration. +type replicaForFlowControl interface { + getTenantID() roachpb.TenantID + getReplicaID() roachpb.ReplicaID + getRangeID() roachpb.RangeID + getDescriptor() *roachpb.RangeDescriptor + getAppliedLogPosition() kvflowcontrolpb.RaftLogPosition + getPausedFollowers() map[roachpb.ReplicaID]struct{} + getBehindFollowers() map[roachpb.ReplicaID]struct{} + getInactiveFollowers() map[roachpb.ReplicaID]struct{} + getDisconnectedFollowers() map[roachpb.ReplicaID]struct{} + + annotateCtx(context.Context) context.Context + assertLocked() // only affects test builds + isScratchRange() bool // only used in tests +} + +// raftTransportForFlowControl abstracts the node-level raft transport, and is +// used by the canonical replicaForFlowControl implementation. It exposes the +// set of (remote) stores the raft transport is connected to. If the underlying +// gRPC streams break and don't reconnect, this indicates as much. Ditto if +// they're reconnected to. Also see RaftTransportDisconnectListener, which is +// used to observe every instance of gRPC streams breaking in order to free up +// tokens. +type raftTransportForFlowControl interface { + isConnectedTo(storeID roachpb.StoreID) bool +} + +// StoresForFlowControl is used to integrate with replication flow control. It +// exposes the underlying kvflowcontrol.Handles and is informed of (remote) +// stores we're no longer connected via the raft transport. +type StoresForFlowControl interface { + kvflowcontrol.Handles + RaftTransportDisconnectListener +} + +// RaftTransportDisconnectListener observes every instance of the raft +// transport disconnecting replication traffic to the given (remote) stores. +type RaftTransportDisconnectListener interface { + OnRaftTransportDisconnected(context.Context, ...roachpb.StoreID) +} diff --git a/pkg/kv/kvserver/flow_control_integration_test.go b/pkg/kv/kvserver/flow_control_integration_test.go index 7350aaa8b264..83da50effd3d 100644 --- a/pkg/kv/kvserver/flow_control_integration_test.go +++ b/pkg/kv/kvserver/flow_control_integration_test.go @@ -57,11 +57,11 @@ import ( // tracker=1,client_raft_helpers_test=1' // // TODO(irfansharif): Add end-to-end tests for the following: -// - [ ] Node with full RaftTransport receive queue. -// - [ ] Node with full RaftTransport send queue, with proposals dropped. +// - [ ] Node with full RaftTransport receive queue (I8). +// - [ ] Node with full RaftTransport send queue, with proposals dropped (I8). // - [ ] Raft commands getting reproposed, either due to timeouts or not having // the right MLAI. See TestReplicaRefreshPendingCommandsTicks, -// TestLogGrowthWhenRefreshingPendingCommands. +// TestLogGrowthWhenRefreshingPendingCommands. I7. // - [ ] Raft proposals getting dropped/abandoned. See // (*Replica).cleanupFailedProposalLocked and its uses. @@ -133,9 +133,9 @@ ORDER BY name ASC; `) h.comment(`-- (Issuing + admitting a regular 1MiB, triply replicated write...)`) - t.Log("sending put request") + h.log("sending put request") h.put(ctx, k, 1<<20 /* 1MiB */, admissionpb.NormalPri) - t.Log("sent put request") + h.log("sent put request") h.waitForAllTokensReturned(ctx, 3) h.comment(` @@ -277,9 +277,9 @@ func TestFlowControlRangeSplitMerge(t *testing.T) { require.NoError(t, err) h.waitForConnectedStreams(ctx, desc.RangeID, 3) - t.Log("sending put request to pre-split range") + h.log("sending put request to pre-split range") h.put(ctx, k, 1<<20 /* 1MiB */, admissionpb.NormalPri) - t.Log("sent put request to pre-split range") + h.log("sent put request to pre-split range") h.waitForAllTokensReturned(ctx, 3) h.comment(` @@ -302,13 +302,13 @@ ORDER BY name ASC; // [T1,n1,s1,r64/1:/{Table/Max-Max},raft] connected to stream: t1/s2 // [T1,n1,s1,r64/1:/{Table/Max-Max},raft] connected to stream: t1/s3 - t.Log("sending 2MiB put request to post-split LHS") + h.log("sending 2MiB put request to post-split LHS") h.put(ctx, k, 2<<20 /* 2MiB */, admissionpb.NormalPri) - t.Log("sent 2MiB put request to post-split LHS") + h.log("sent 2MiB put request to post-split LHS") - t.Log("sending 3MiB put request to post-split RHS") + h.log("sending 3MiB put request to post-split RHS") h.put(ctx, roachpb.Key(right.StartKey), 3<<20 /* 3MiB */, admissionpb.NormalPri) - t.Log("sent 3MiB put request to post-split RHS") + h.log("sent 3MiB put request to post-split RHS") h.waitForAllTokensReturned(ctx, 3) h.comment(` @@ -343,9 +343,9 @@ ORDER BY streams DESC; // [T1,n1,s1,r65/1:{\xfa\x00-/Max},raft] disconnected stream: t1/s2 // [T1,n1,s1,r65/1:{\xfa\x00-/Max},raft] disconnected stream: t1/s3 - t.Log("sending 4MiB put request to post-merge range") + h.log("sending 4MiB put request to post-merge range") h.put(ctx, roachpb.Key(merged.StartKey), 4<<20 /* 4MiB */, admissionpb.NormalPri) - t.Log("sent 4MiB put request to post-merged range") + h.log("sent 4MiB put request to post-merged range") h.waitForAllTokensReturned(ctx, 3) h.comment(` @@ -415,11 +415,11 @@ func TestFlowControlBlockedAdmission(t *testing.T) { h.waitForConnectedStreams(ctx, desc.RangeID, 3) h.comment(`-- (Issuing regular 1MiB, 3x replicated write that's not admitted.)`) - t.Log("sending put requests") + h.log("sending put requests") for i := 0; i < 5; i++ { h.put(ctx, k, 1<<20 /* 1MiB */, admissionpb.NormalPri) } - t.Log("sent put requests") + h.log("sent put requests") h.comment(` -- Flow token metrics from n1 after issuing 5 regular 1MiB 3x replicated writes @@ -488,13 +488,25 @@ func TestFlowControlCrashedNode(t *testing.T) { const numNodes = 2 var maintainStreamsForBrokenRaftTransport atomic.Bool + st := cluster.MakeTestingClusterSettings() + // See I13 from kvflowcontrol/doc.go. We disable the raft-transport-break + // mechanism below, and for quiesced ranges, that can effectively disable + // the last-updated mechanism since quiesced ranges aren't being ticked, and + // we only check the last-updated state when ticked. So we disable range + // quiescence. + kvserver.ExpirationLeasesOnly.Override(ctx, &st.SV, true) + tc := testcluster.StartTestCluster(t, numNodes, base.TestClusterArgs{ ReplicationMode: base.ReplicationManual, ServerArgs: base.TestServerArgs{ + Settings: st, RaftConfig: base.RaftConfig{ // Suppress timeout-based elections. This test doesn't want to // deal with leadership changing hands. RaftElectionTimeoutTicks: 1000000, + // Reduce the RangeLeaseDuration to speeds up failure detection + // below. + RangeLeaseDuration: time.Second, }, Knobs: base.TestingKnobs{ Store: &kvserver.StoreTestingKnobs{ @@ -531,11 +543,11 @@ func TestFlowControlCrashedNode(t *testing.T) { h.waitForConnectedStreams(ctx, desc.RangeID, 2) h.comment(`-- (Issuing regular 5x1MiB, 2x replicated writes that are not admitted.)`) - t.Log("sending put requests") + h.log("sending put requests") for i := 0; i < 5; i++ { h.put(ctx, k, 1<<20 /* 1MiB */, admissionpb.NormalPri) } - t.Log("sent put requests") + h.log("sent put requests") h.comment(` -- Flow token metrics from n1 after issuing 5 regular 1MiB 2x replicated writes @@ -598,6 +610,7 @@ func TestFlowControlRaftSnapshot(t *testing.T) { const numServers int = 5 stickyServerArgs := make(map[int]base.TestServerArgs) + var maintainStreamsForBehindFollowers atomic.Bool var maintainStreamsForInactiveFollowers atomic.Bool var maintainStreamsForBrokenRaftTransport atomic.Bool var disableWorkQueueGranting atomic.Bool @@ -629,6 +642,9 @@ func TestFlowControlRaftSnapshot(t *testing.T) { // deductions/returns. return kvflowcontrol.Tokens(1 << 20 /* 1MiB */) }, + MaintainStreamsForBehindFollowers: func() bool { + return maintainStreamsForBehindFollowers.Load() + }, MaintainStreamsForInactiveFollowers: func() bool { return maintainStreamsForInactiveFollowers.Load() }, @@ -728,6 +744,11 @@ ORDER BY name ASC; maintainStreamsForInactiveFollowers.Store(true) maintainStreamsForBrokenRaftTransport.Store(true) + // Depending on when the raft group gets ticked, we might notice than + // replicas on n2 and n3 are behind a bit too soon. Disable it first, and + // re-enable it right when this test wants to react to raft progress state. + maintainStreamsForBehindFollowers.Store(true) + // Now kill stores 1 + 2, increment the key on the other stores and // truncate their logs to make sure that when store 1 + 2 comes back up // they will require a snapshot from Raft. @@ -795,6 +816,9 @@ ORDER BY name ASC; t.Fatal(err) } + // Allow the flow control integration layer to react to raft progress state. + maintainStreamsForBehindFollowers.Store(false) + h.comment(`-- (Restarting n2 and n3.)`) require.NoError(t, tc.RestartServer(1)) require.NoError(t, tc.RestartServer(2)) @@ -832,6 +856,7 @@ ORDER BY name ASC; h.query(n1, ` SELECT range_id, store_id, crdb_internal.humanize_bytes(total_tracked_tokens::INT8) FROM crdb_internal.kv_flow_control_handles + WHERE total_tracked_tokens > 0 `, "range_id", "store_id", "total_tracked_tokens") h.waitForConnectedStreams(ctx, repl.RangeID, 5) @@ -933,11 +958,11 @@ func TestFlowControlRaftTransportBreak(t *testing.T) { h.waitForConnectedStreams(ctx, desc.RangeID, 3) h.comment(`-- (Issuing regular 5x1MiB, 3x replicated writes that are not admitted.)`) - t.Log("sending put requests") + h.log("sending put requests") for i := 0; i < 5; i++ { h.put(ctx, k, 1<<20 /* 1MiB */, admissionpb.NormalPri) } - t.Log("sent put requests") + h.log("sent put requests") h.comment(` -- Flow token metrics from n1 after issuing 5 regular 1MiB 3x replicated writes @@ -1003,6 +1028,7 @@ func TestFlowControlRaftTransportCulled(t *testing.T) { const numNodes = 3 workerTeardownCh := make(chan roachpb.NodeID, 1) markSendQueueAsIdleCh := make(chan roachpb.NodeID) + var disableWorkerTeardown atomic.Bool baseServerArgs := base.TestServerArgs{ Knobs: base.TestingKnobs{ @@ -1031,6 +1057,9 @@ func TestFlowControlRaftTransportCulled(t *testing.T) { baseServerArgsWithRaftTransportKnobs.Knobs.RaftTransport = &kvserver.RaftTransportTestingKnobs{ MarkSendQueueAsIdleCh: markSendQueueAsIdleCh, OnWorkerTeardown: func(nodeID roachpb.NodeID) { + if disableWorkerTeardown.Load() { + return + } workerTeardownCh <- nodeID }, } @@ -1061,11 +1090,11 @@ func TestFlowControlRaftTransportCulled(t *testing.T) { h.waitForConnectedStreams(ctx, desc.RangeID, 3) h.comment(`-- (Issuing regular 5x1MiB, 3x replicated writes that are not admitted.)`) - t.Log("sending put requests") + h.log("sending put requests") for i := 0; i < 5; i++ { h.put(ctx, k, 1<<20 /* 1MiB */, admissionpb.NormalPri) } - t.Log("sent put requests") + h.log("sent put requests") h.comment(` -- Flow token metrics from n1 after issuing 5 regular 1MiB 3x replicated writes @@ -1124,6 +1153,8 @@ ORDER BY name ASC; WHERE name LIKE '%kvadmission%tokens%' ORDER BY name ASC; `) + + disableWorkerTeardown.Store(true) } // TestFlowControlRaftMembership tests flow token behavior when the raft @@ -1269,9 +1300,24 @@ func TestFlowControlRaftMembershipRemoveSelf(t *testing.T) { ReplicationMode: base.ReplicationManual, ServerArgs: base.TestServerArgs{ RaftConfig: base.RaftConfig{ - // Suppress timeout-based elections. This test doesn't want to - // deal with leadership changing hands. - RaftElectionTimeoutTicks: 1000000, + // TODO(irfansharif): The AdminRelocateRange used below can + // occasionally flake if we suppress timeout-based + // elections. We get logging of the following form: + // + // I230507 19:47:03.143463 31 kv/kvserver_test/flow_control_integration_test.go:2065 [-] 349 -- (Replacing current raft leader on n1 in raft group with new n4 replica.) + // I230507 19:47:03.153105 5430 kv/kvserver/replica_raftstorage.go:514 [T1,n4,s4,r64/4:/{Table/Max-Max}] 352 applied INITIAL snapshot b8cdcb09 from (n1,s1):1 at applied index 23 (total=1ms data=1.0 MiB ingestion=6@1ms) + // I230507 19:47:03.167504 629 kv/kvserver/kvflowcontrol/kvflowhandle/kvflowhandle.go:249 [T1,n1,s1,r64/1:/{Table/Max-Max},raft] 353 connected to stream: t1/s4 + // W230507 19:47:03.186303 4268 go.etcd.io/raft/v3/raft.go:924 [T1,n4,s4,r64/4:/{Table/Max-Max}] 354 4 cannot campaign at term 6 since there are still 1 pending configuration changes to apply + // ... + // W230507 19:47:18.194829 5507 kv/kvserver/spanlatch/manager.go:559 [T1,n4,s4,r64/4:/{Table/Max-Max}] 378 have been waiting 15s to acquire read latch /Local/Range/Table/Max/RangeDescriptor@0,0, held by write latch /Local/Range/Table/Max/RangeDescriptor@0,0 + // W230507 19:47:19.082183 5891 kv/kvserver/spanlatch/manager.go:559 [T1,n4,s4,r64/4:/{Table/Max-Max}] 379 have been waiting 15s to acquire read latch /Local/Range/Table/Max/RangeDescriptor@0,0, held by write latch /Local/Range/Table/Max/RangeDescriptor@0,0 + // + // Followed by range unavailability. Are we relying on the + // new leader to be able to campaign immediately, in order + // to release the latch? And we're simultaneously preventing + // other replicas from calling elections? + // + // RaftElectionTimeoutTicks: 1000000, }, Knobs: base.TestingKnobs{ Store: &kvserver.StoreTestingKnobs{ @@ -2079,12 +2125,18 @@ func (h *flowControlTestHelper) comment(comment string) { comment = strings.TrimSpace(comment) h.buf.WriteString(fmt.Sprintf("%s\n", comment)) - h.t.Log(comment) + h.log(comment) +} + +func (h *flowControlTestHelper) log(msg string) { + if log.ShowLogs() { + log.Infof(context.Background(), "%s", msg) + } } func (h *flowControlTestHelper) query(runner *sqlutils.SQLRunner, sql string, headers ...string) { sql = strings.TrimSpace(sql) - h.t.Log(sql) + h.log(sql) h.buf.WriteString(fmt.Sprintf("%s\n\n", sql)) rows := runner.Query(h.t, sql) diff --git a/pkg/kv/kvserver/flow_control_raft_transport.go b/pkg/kv/kvserver/flow_control_raft_transport.go index f4535aa10f66..8bf818ac3768 100644 --- a/pkg/kv/kvserver/flow_control_raft_transport.go +++ b/pkg/kv/kvserver/flow_control_raft_transport.go @@ -20,16 +20,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/rpc" ) -// raftTransportForFlowControl abstracts the node-level raft transport, and is -// used by the canonical replicaFlowControlIntegration implementation. It -// exposes the set of (remote) stores the raft transport is connected to. If the -// underlying gRPC streams break and don't reconnect, this indicates as much. -// Ditto if they're reconnected to. Also see RaftTransportDisconnectListener, -// which is used to observe every instance of gRPC streams breaking. -type raftTransportForFlowControl interface { - isConnectedTo(storeID roachpb.StoreID) bool -} - var _ raftTransportForFlowControl = &RaftTransport{} // isConnectedTo implements the raftTransportForFlowControl interface. @@ -39,12 +29,6 @@ func (r *RaftTransport) isConnectedTo(storeID roachpb.StoreID) bool { return r.kvflowControl.mu.connectionTracker.isStoreConnected(storeID) } -// RaftTransportDisconnectListener observes every instance of the raft -// transport disconnecting replication traffic to the given (remote) stores. -type RaftTransportDisconnectListener interface { - OnRaftTransportDisconnected(context.Context, ...roachpb.StoreID) -} - // connectionTrackerForFlowControl tracks the set of client-side stores and // server-side nodes the raft transport is connected to. The "client" and // "server" refer to the client and server side of the RaftTransport stream (see diff --git a/pkg/kv/kvserver/flow_control_replica.go b/pkg/kv/kvserver/flow_control_replica.go index bc70455ad867..ad583c969353 100644 --- a/pkg/kv/kvserver/flow_control_replica.go +++ b/pkg/kv/kvserver/flow_control_replica.go @@ -20,24 +20,6 @@ import ( rafttracker "go.etcd.io/raft/v3/tracker" ) -// replicaForFlowControl abstracts the interface of an individual Replica, as -// needed by replicaFlowControlIntegration. -type replicaForFlowControl interface { - annotateCtx(context.Context) context.Context - getTenantID() roachpb.TenantID - getReplicaID() roachpb.ReplicaID - getRangeID() roachpb.RangeID - getDescriptor() *roachpb.RangeDescriptor - getAppliedLogPosition() kvflowcontrolpb.RaftLogPosition - getPausedFollowers() map[roachpb.ReplicaID]struct{} - isFollowerLive(context.Context, roachpb.ReplicaID) bool - isRaftTransportConnectedTo(roachpb.StoreID) bool - withReplicaProgress(f func(roachpb.ReplicaID, rafttracker.Progress)) - - assertLocked() // only affects test builds - isScratchRange() bool // only used in tests -} - // replicaFlowControl is a concrete implementation of the replicaForFlowControl // interface. type replicaFlowControl Replica @@ -76,23 +58,66 @@ func (rf *replicaFlowControl) getPausedFollowers() map[roachpb.ReplicaID]struct{ return rf.mu.pausedFollowers } -func (rf *replicaFlowControl) isFollowerLive(ctx context.Context, replID roachpb.ReplicaID) bool { - rf.mu.AssertHeld() - return rf.mu.lastUpdateTimes.isFollowerActiveSince( - ctx, - replID, - timeutil.Now(), - rf.store.cfg.RangeLeaseDuration, - ) +func (rf *replicaFlowControl) getBehindFollowers() map[roachpb.ReplicaID]struct{} { + rf.assertLocked() + behindFollowers := make(map[roachpb.ReplicaID]struct{}) + rf.mu.internalRaftGroup.WithProgress(func(id uint64, _ raft.ProgressType, progress rafttracker.Progress) { + if progress.State == rafttracker.StateReplicate { + return + } + + replID := roachpb.ReplicaID(id) + behindFollowers[replID] = struct{}{} + + // TODO(irfansharif): Integrating with these other progress fields + // from raft. For replicas exiting rafttracker.StateProbe, perhaps + // compare progress.Match against status.Commit to make sure it's + // sufficiently caught up with respect to its raft log before we + // start deducting tokens for it (lest we run into I3a from + // kvflowcontrol/doc.go). To play well with the replica-level + // proposal quota pool, maybe we also factor its base index? + // Replicas that crashed and came back could come back in + // StateReplicate but be behind on their logs. If we're deducting + // tokens right away for subsequent proposals, it would take some + // time for it to catch up and then later return those tokens to us. + // This is I3a again; do it as part of #95563. + _ = progress.RecentActive + _ = progress.MsgAppFlowPaused + _ = progress.Match + }) + return behindFollowers } -func (rf *replicaFlowControl) isRaftTransportConnectedTo(storeID roachpb.StoreID) bool { - rf.mu.AssertHeld() - return rf.store.cfg.Transport.isConnectedTo(storeID) +func (rf *replicaFlowControl) getInactiveFollowers() map[roachpb.ReplicaID]struct{} { + rf.assertLocked() + inactiveFollowers := make(map[roachpb.ReplicaID]struct{}) + for _, desc := range rf.getDescriptor().Replicas().Descriptors() { + if desc.ReplicaID == rf.getReplicaID() { + continue + } + if !rf.mu.lastUpdateTimes.isFollowerActiveSince(desc.ReplicaID, timeutil.Now(), rf.store.cfg.RangeLeaseDuration) { + inactiveFollowers[desc.ReplicaID] = struct{}{} + } + } + return inactiveFollowers +} + +func (rf *replicaFlowControl) getDisconnectedFollowers() map[roachpb.ReplicaID]struct{} { + rf.assertLocked() + disconnectedFollowers := make(map[roachpb.ReplicaID]struct{}) + for _, desc := range rf.getDescriptor().Replicas().Descriptors() { + if desc.ReplicaID == rf.getReplicaID() { + continue + } + if !rf.store.raftTransportForFlowControl.isConnectedTo(desc.StoreID) { + disconnectedFollowers[desc.ReplicaID] = struct{}{} + } + } + return disconnectedFollowers } func (rf *replicaFlowControl) getAppliedLogPosition() kvflowcontrolpb.RaftLogPosition { - rf.mu.AssertHeld() + rf.assertLocked() status := rf.mu.internalRaftGroup.BasicStatus() return kvflowcontrolpb.RaftLogPosition{ Term: status.Term, @@ -100,16 +125,8 @@ func (rf *replicaFlowControl) getAppliedLogPosition() kvflowcontrolpb.RaftLogPos } } -func (rf *replicaFlowControl) withReplicaProgress(f func(roachpb.ReplicaID, rafttracker.Progress)) { - rf.mu.AssertHeld() - rf.mu.internalRaftGroup.WithProgress(func(id uint64, _ raft.ProgressType, progress rafttracker.Progress) { - f(roachpb.ReplicaID(id), progress) - }) -} - func (rf *replicaFlowControl) isScratchRange() bool { - rf.mu.AssertHeld() + rf.assertLocked() r := (*Replica)(rf) return r.isScratchRangeRLocked() } - diff --git a/pkg/kv/kvserver/flow_control_replica_integration.go b/pkg/kv/kvserver/flow_control_replica_integration.go index 36260bfded16..35cbc1c354a9 100644 --- a/pkg/kv/kvserver/flow_control_replica_integration.go +++ b/pkg/kv/kvserver/flow_control_replica_integration.go @@ -16,40 +16,44 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/util/buildutil" "github.com/cockroachdb/cockroach/pkg/util/log" - rafttracker "go.etcd.io/raft/v3/tracker" ) -// replicaFlowControlIntegration is used to integrate with replication flow -// control. It's intercepts various points in a replica's lifecycle, like it -// acquiring raft leadership or losing it, or its raft membership changing, etc. -// -// Accessing it requires Replica.mu to be held, exclusively (this is asserted on -// in the canonical implementation). -type replicaFlowControlIntegration interface { - handle() (kvflowcontrol.Handle, bool) - onBecameLeader(context.Context) - onBecameFollower(context.Context) - onDescChanged(context.Context) - onFollowersPaused(context.Context) - onReplicaDestroyed(context.Context) - onProposalQuotaUpdated(context.Context) - onRaftTransportDisconnected(context.Context, ...roachpb.StoreID) -} - -var _ replicaFlowControlIntegration = &replicaFlowControlIntegrationImpl{} - +// replicaFlowControlIntegrationImpl is the canonical implementation of the +// replicaFlowControlIntegration interface. type replicaFlowControlIntegrationImpl struct { replicaForFlowControl replicaForFlowControl handleFactory kvflowcontrol.HandleFactory - - innerHandle kvflowcontrol.Handle - lastKnownReplicas roachpb.ReplicaSet + knobs *kvflowcontrol.TestingKnobs + + // The fields below are non-nil iff the replica is a raft leader and part of + // the range. + + // innerHandle is the underlying kvflowcontrol.Handle, which we + // deduct/return flow tokens to, and inform of connected/disconnected + // replication streams. + innerHandle kvflowcontrol.Handle + // lastKnownReplicas tracks the set of last know replicas in the range. This + // is updated whenever the range descriptor is changed, and we react to any + // deltas by disconnecting streams for replicas no longer part of the range, + // connecting streams for new members of the range, or closing innerHandle + // if we ourselves are no longer part of the range. + lastKnownReplicas roachpb.ReplicaSet + // disconnectedStreams tracks the set of replication streams we're not + // currently connected to, but want to in the near future should things + // change. This includes paused followers (who could be unpaused), + // inactive/dead followers (who could become active if the node they're on + // is restarted), followers we're not connected to via the raft transport + // (the transport streams could re-establish), replicas that are being + // caught up via snapshots or are being probed for the last committed index. + // This does not include replicas that are no longer part of the range, + // since we're not looking to reconnect to them in the future. disconnectedStreams map[roachpb.ReplicaID]kvflowcontrol.Stream - - knobs *kvflowcontrol.TestingKnobs } +var _ replicaFlowControlIntegration = &replicaFlowControlIntegrationImpl{} + func newReplicaFlowControlIntegration( replicaForFlowControl replicaForFlowControl, handleFactory kvflowcontrol.HandleFactory, @@ -65,11 +69,7 @@ func newReplicaFlowControlIntegration( } } -func (f *replicaFlowControlIntegrationImpl) handle() (kvflowcontrol.Handle, bool) { - f.replicaForFlowControl.assertLocked() - return f.innerHandle, f.innerHandle != nil -} - +// onBecameLeader is part of the replicaFlowControlIntegration interface. func (f *replicaFlowControlIntegrationImpl) onBecameLeader(ctx context.Context) { f.replicaForFlowControl.assertLocked() if f.innerHandle != nil { @@ -78,13 +78,18 @@ func (f *replicaFlowControlIntegrationImpl) onBecameLeader(ctx context.Context) if !f.replicaForFlowControl.getTenantID().IsSet() { log.Fatal(ctx, "unset tenant ID") } - if f.knobs.UseOnlyForScratchRanges && !f.replicaForFlowControl.isScratchRange() { - return + return // nothing to do } // See I5 from kvflowcontrol/doc.go. The per-replica kvflowcontrol.Handle is - // tied to the lifetime of a leaseholder replica having raft leadership. + // tied to the lifetime of a leaseholder replica having raft leadership. We + // don't intercept lease acquisitions/transfers -- simply raft leadership. + // When leadership is lost we release all held flow tokens. Tokens are only + // deducted at proposal time when the proposing replica is both the raft + // leader and leaseholder (the latter is tautological since only + // leaseholders propose). We're relying on timely acquisition of raft + // leadership by the leaseholder to not be persistently over admitting. f.innerHandle = f.handleFactory.NewHandle( f.replicaForFlowControl.getRangeID(), f.replicaForFlowControl.getTenantID(), @@ -92,21 +97,33 @@ func (f *replicaFlowControlIntegrationImpl) onBecameLeader(ctx context.Context) f.lastKnownReplicas = f.replicaForFlowControl.getDescriptor().Replicas() f.disconnectedStreams = make(map[roachpb.ReplicaID]kvflowcontrol.Stream) - appliedLogPosition := f.replicaForFlowControl.getAppliedLogPosition() - for _, desc := range f.replicaForFlowControl.getDescriptor().Replicas().Descriptors() { - // Start off every remote stream as disconnected. Later we'll try to - // reconnect them. - stream := kvflowcontrol.Stream{ + // Connect to the local stream. + localRepl, found := f.lastKnownReplicas.GetReplicaDescriptorByID(f.replicaForFlowControl.getReplicaID()) + if !found { + // This assertion relies on replicaForFlowControl being locked, so the + // descriptor could not have changed state while this callback is + // ongoing. We never disconnect the local stream until we lost raft + // leadership or are no longer a raft member. + log.Fatalf(ctx, "leader (replid=%d) didn't find self in last known replicas (%s)", + f.replicaForFlowControl.getReplicaID(), f.lastKnownReplicas) + } + f.innerHandle.ConnectStream(ctx, + f.replicaForFlowControl.getAppliedLogPosition(), + kvflowcontrol.Stream{ TenantID: f.replicaForFlowControl.getTenantID(), - StoreID: desc.StoreID, - } - if f.replicaForFlowControl.getReplicaID() != desc.ReplicaID { - f.disconnectedStreams[desc.ReplicaID] = stream - continue + StoreID: localRepl.StoreID, + }, + ) + + // Start off every remote stream as disconnected. Later we'll try to + // reconnect them. + var toDisconnect []roachpb.ReplicaDescriptor + for _, desc := range f.replicaForFlowControl.getDescriptor().Replicas().Descriptors() { + if desc.ReplicaID != localRepl.ReplicaID { + toDisconnect = append(toDisconnect, desc) } - // Connect to the local stream. - f.innerHandle.ConnectStream(ctx, appliedLogPosition, stream) } + f.disconnectStreams(ctx, toDisconnect, "unknown followers on new leader") f.tryReconnect(ctx) if log.V(1) { @@ -117,11 +134,15 @@ func (f *replicaFlowControlIntegrationImpl) onBecameLeader(ctx context.Context) sort.Slice(disconnected, func(i, j int) bool { return disconnected[i].StoreID < disconnected[j].StoreID }) - log.Infof(ctx, "assumed raft leadership: initializing flow handle for %s starting at %s (disconnected streams: %s)", - f.replicaForFlowControl.getDescriptor(), appliedLogPosition, disconnected) + log.VInfof(ctx, 1, "assumed raft leadership: initializing flow handle for %s starting at %s (disconnected streams: %s)", + f.replicaForFlowControl.getDescriptor(), + f.replicaForFlowControl.getAppliedLogPosition(), + disconnected, + ) } } +// onBecameFollower is part of the replicaFlowControlIntegration interface. func (f *replicaFlowControlIntegrationImpl) onBecameFollower(ctx context.Context) { f.replicaForFlowControl.assertLocked() if f.innerHandle == nil { @@ -135,12 +156,10 @@ func (f *replicaFlowControlIntegrationImpl) onBecameFollower(ctx context.Context // scenarios. log.VInfof(ctx, 1, "lost raft leadership: releasing flow tokens and closing handle for %s", f.replicaForFlowControl.getDescriptor()) - f.innerHandle.Close(ctx) - f.innerHandle = nil - f.lastKnownReplicas = roachpb.MakeReplicaSet(nil) - f.disconnectedStreams = nil + f.clearState(ctx) } +// onDescChanged is part of the replicaFlowControlIntegration interface. func (f *replicaFlowControlIntegrationImpl) onDescChanged(ctx context.Context) { f.replicaForFlowControl.assertLocked() if f.innerHandle == nil { @@ -150,45 +169,32 @@ func (f *replicaFlowControlIntegrationImpl) onDescChanged(ctx context.Context) { addedReplicas, removedReplicas := f.lastKnownReplicas.Difference( f.replicaForFlowControl.getDescriptor().Replicas(), ) + + ourReplicaID := f.replicaForFlowControl.getReplicaID() for _, repl := range removedReplicas { - if repl.ReplicaID == f.replicaForFlowControl.getReplicaID() { + if repl.ReplicaID == ourReplicaID { // We're observing ourselves get removed from the raft group, but // are still retaining raft leadership. Close the underlying handle - // and bail. - // - // TODO(irfansharif): Is this even possible? - f.innerHandle.Close(ctx) - f.innerHandle = nil - f.lastKnownReplicas = roachpb.MakeReplicaSet(nil) - f.disconnectedStreams = nil + // and bail. See TestFlowControlRaftMembershipRemoveSelf. + f.clearState(ctx) return } - // See I10 from kvflowcontrol/doc.go. We stop deducting flow tokens for - // replicas that are no longer part of the raft group, free-ing up all - // held tokens. - f.innerHandle.DisconnectStream(ctx, kvflowcontrol.Stream{ - TenantID: f.replicaForFlowControl.getTenantID(), - StoreID: repl.StoreID, - }) - delete(f.disconnectedStreams, repl.ReplicaID) } - for _, repl := range addedReplicas { - // Start off new replicas as disconnected. We'll subsequently try to - // re-add them, once we know their log positions and consider them - // sufficiently caught up. See I3a from kvflowcontrol/doc.go. - if repl.ReplicaID == f.replicaForFlowControl.getReplicaID() { - log.Fatalf(ctx, "observed replica adding itself to the range descriptor") - } - if _, found := f.disconnectedStreams[repl.ReplicaID]; found { - continue // already disconnected, nothing to do - } - stream := kvflowcontrol.Stream{ - TenantID: f.replicaForFlowControl.getTenantID(), - StoreID: repl.StoreID, - } - f.disconnectedStreams[repl.ReplicaID] = stream + // See I10 from kvflowcontrol/doc.go. We stop deducting flow tokens for + // replicas that are no longer part of the raft group, free-ing up all + // held tokens. + f.disconnectStreams(ctx, removedReplicas, "removed replicas") + for _, repl := range removedReplicas { + // We'll not reconnect to these replicas either, so untrack them. + delete(f.disconnectedStreams, repl.ReplicaID) } + + // Start off new replicas as disconnected. We'll subsequently try to + // re-add them, once we know their log positions and consider them + // sufficiently caught up. See I3a from kvflowcontrol/doc.go. + f.disconnectStreams(ctx, addedReplicas, "newly added replicas") + if len(addedReplicas) > 0 || len(removedReplicas) > 0 { log.VInfof(ctx, 1, "desc changed from %s to %s: added=%s removed=%s", f.lastKnownReplicas, f.replicaForFlowControl.getDescriptor(), addedReplicas, removedReplicas, @@ -197,35 +203,62 @@ func (f *replicaFlowControlIntegrationImpl) onDescChanged(ctx context.Context) { f.lastKnownReplicas = f.replicaForFlowControl.getDescriptor().Replicas() } +// onFollowersPaused is part of the replicaFlowControlIntegration interface. func (f *replicaFlowControlIntegrationImpl) onFollowersPaused(ctx context.Context) { f.replicaForFlowControl.assertLocked() if f.innerHandle == nil { return // nothing to do } - var toDisconnect []roachpb.ReplicaDescriptor // See I3 from kvflowcontrol/doc.go. We don't deduct flow tokens for // replication traffic that's not headed to paused replicas. - for replID := range f.replicaForFlowControl.getPausedFollowers() { - repl, ok := f.lastKnownReplicas.GetReplicaDescriptorByID(replID) - if !ok { - // As of 4/23, we don't make any strong guarantees around the set of - // paused followers we're tracking, nothing that ensures that what's - // tracked is guaranteed to be a member of the range descriptor. We - // treat the range descriptor derived state as authoritative. + f.refreshStreams(ctx, "paused followers") +} + +// onRaftTransportDisconnected is part of the replicaFlowControlIntegration interface. +func (f *replicaFlowControlIntegrationImpl) onRaftTransportDisconnected( + ctx context.Context, storeIDs ...roachpb.StoreID, +) { + f.replicaForFlowControl.assertLocked() + if f.innerHandle == nil { + return // nothing to do + } + + if fn := f.knobs.MaintainStreamsForBrokenRaftTransport; fn != nil && fn() { + return // nothing to do + } + + disconnectedStores := make(map[roachpb.StoreID]struct{}) + for _, storeID := range storeIDs { + disconnectedStores[storeID] = struct{}{} + } + + ourReplicaID := f.replicaForFlowControl.getReplicaID() + var toDisconnect []roachpb.ReplicaDescriptor + for _, repl := range f.lastKnownReplicas.Descriptors() { + if repl.ReplicaID == ourReplicaID { continue } - if repl.ReplicaID == f.replicaForFlowControl.getReplicaID() { - log.Fatalf(ctx, "observed replica pausing replication traffic to itself") + if _, found := disconnectedStores[repl.StoreID]; found { + toDisconnect = append(toDisconnect, repl) } - toDisconnect = append(toDisconnect, repl) } - - f.disconnectStreams(ctx, toDisconnect, "paused followers") + f.disconnectStreams(ctx, toDisconnect, "raft transport disconnected") f.tryReconnect(ctx) } -func (f *replicaFlowControlIntegrationImpl) onReplicaDestroyed(ctx context.Context) { +// onProposalQuotaUpdated is part of the replicaFlowControlIntegration interface. +func (f *replicaFlowControlIntegrationImpl) onRaftTicked(ctx context.Context) { + f.replicaForFlowControl.assertLocked() + if f.innerHandle == nil { + return // nothing to do + } + + f.refreshStreams(ctx, "refreshing streams") +} + +// onDestroyed is part of the replicaFlowControlIntegration interface. +func (f *replicaFlowControlIntegrationImpl) onDestroyed(ctx context.Context) { f.replicaForFlowControl.assertLocked() if f.innerHandle == nil { return // nothing to do @@ -240,114 +273,104 @@ func (f *replicaFlowControlIntegrationImpl) onReplicaDestroyed(ctx context.Conte // rebalanced away, is no longer part of the raft group, is being GC-ed, // destroyed as part of the EndTxn merge trigger, or subsumed if applying // the merge as part of an incoming snapshot. - f.innerHandle.Close(ctx) - f.innerHandle = nil - f.lastKnownReplicas = roachpb.MakeReplicaSet(nil) - f.disconnectedStreams = nil + f.clearState(ctx) } -func (f *replicaFlowControlIntegrationImpl) onProposalQuotaUpdated(ctx context.Context) { +// handle is part of the replicaFlowControlIntegration interface. +func (f *replicaFlowControlIntegrationImpl) handle() (kvflowcontrol.Handle, bool) { f.replicaForFlowControl.assertLocked() - if f.innerHandle == nil { - return // nothing to do - } + return f.innerHandle, f.innerHandle != nil +} - var toDisconnect []roachpb.ReplicaDescriptor +// refreshStreams disconnects any streams we're not actively replicating to, and +// reconnect previously disconnected streams if we're able. +func (f *replicaFlowControlIntegrationImpl) refreshStreams(ctx context.Context, reason string) { + f.disconnectStreams(ctx, f.notActivelyReplicatingTo(), reason) + f.tryReconnect(ctx) +} - // Disconnect any recently inactive followers. - // - // TODO(irfansharif): Experimentally this gets triggered quite often. It - // might be too sensitive and may result in ineffective flow control as - // a result. Fix as part of #95563. +// notActivelyReplicatingTo lists the replicas that aren't actively receiving +// log entries to append to its log. This encompasses newly added replicas that +// we're still probing to figure out its last index (I4), replicas that are +// pending raft snapshots because the leader has truncated away entries higher +// than its last position (I4), replicas on dead nodes (I2), replicas we're not +// connected to via the raft transport (I1), and paused followers (I3). +func (f *replicaFlowControlIntegrationImpl) notActivelyReplicatingTo() []roachpb.ReplicaDescriptor { + pausedFollowers := f.replicaForFlowControl.getPausedFollowers() + behindFollowers := f.replicaForFlowControl.getBehindFollowers() + inactiveFollowers := f.replicaForFlowControl.getInactiveFollowers() + disconnectedFollowers := f.replicaForFlowControl.getDisconnectedFollowers() + + maintainStreamsForBrokenRaftTransport := f.knobs.MaintainStreamsForBrokenRaftTransport != nil && + f.knobs.MaintainStreamsForBrokenRaftTransport() + maintainStreamsForInactiveFollowers := f.knobs.MaintainStreamsForInactiveFollowers != nil && + f.knobs.MaintainStreamsForInactiveFollowers() + maintainStreamsForBehindFollowers := f.knobs.MaintainStreamsForBehindFollowers != nil && + f.knobs.MaintainStreamsForBehindFollowers() + + notActivelyReplicatingTo := make(map[roachpb.ReplicaDescriptor]struct{}) + ourReplicaID := f.replicaForFlowControl.getReplicaID() for _, repl := range f.lastKnownReplicas.Descriptors() { - if f.replicaForFlowControl.isFollowerLive(ctx, repl.ReplicaID) { - continue // nothing to do - } - if repl.ReplicaID == f.replicaForFlowControl.getReplicaID() { - // NB: We ignore ourselves from this last-updated map. For followers - // we update the timestamps when we step a message from them into - // the local raft group, but for the leader we only update it - // whenever it ticks. So in workloads where the leader only sees - // occasional writes, it could see itself as non-live. This is - // likely unintentional, but we paper over it here. - continue // nothing to do - } - if fn := f.knobs.MaintainStreamsForInactiveFollowers; fn != nil && fn() { - continue // nothing to do + if repl.ReplicaID == ourReplicaID { + // NB: We ignore ourselves from the {paused,behind}-followers + // blocklist (we're the leader), the raft transport check (we're not + // connected to ourselves through the transport), and the + // last-updated map. The latter is a bit odd - for followers we + // update the timestamps when we step a message from them into the + // local raft group, but for the leader we only update it whenever + // it ticks. So in workloads where the leader only sees occasional + // writes, it could see itself as non-live. This is likely + // unintentional, but we paper over it here anyway. + continue } - toDisconnect = append(toDisconnect, repl) - } - f.disconnectStreams(ctx, toDisconnect, "inactive followers") - // Disconnect any streams we're not actively replicating to. - toDisconnect = nil - for _, replID := range f.notActivelyReplicatingTo() { - repl, ok := f.lastKnownReplicas.GetReplicaDescriptorByID(replID) - if !ok { - continue + if _, found := pausedFollowers[repl.ReplicaID]; found { + // As of 7/23, there are no strong guarantees around the set of + // paused followers we're tracking, nothing that ensures that what's + // tracked is guaranteed to be a member of the range descriptor. + // This is why we treat the range descriptor derived state as + // authoritative (we're using it in the loop iteration and only + // tracking replicas here that are both paused AND part of the + // descriptor). + notActivelyReplicatingTo[repl] = struct{}{} } - if repl.ReplicaID == f.replicaForFlowControl.getReplicaID() { - log.Fatalf(ctx, "leader replica observed that it was not being actively replicated to") + + if _, found := behindFollowers[repl.ReplicaID]; found && + !maintainStreamsForBehindFollowers { + notActivelyReplicatingTo[repl] = struct{}{} } - toDisconnect = append(toDisconnect, repl) - } - f.disconnectStreams(ctx, toDisconnect, "not actively replicating") - f.tryReconnect(ctx) -} + if _, found := inactiveFollowers[repl.ReplicaID]; found && + !maintainStreamsForInactiveFollowers { + notActivelyReplicatingTo[repl] = struct{}{} -// notActivelyReplicatingTo lists the replicas that aren't actively receiving -// log entries to append to its log, from raft's perspective (i.e. this is -// unrelated to CRDB-level follower pausing). This encompasses newly added -// replicas that we're still probing to figure out its last index, replicas -// that are pending raft snapshots because the leader has truncated away entries -// higher than its last position, and replicas we're not currently connected to -// via the raft transport. -func (f *replicaFlowControlIntegrationImpl) notActivelyReplicatingTo() []roachpb.ReplicaID { - var res []roachpb.ReplicaID - f.replicaForFlowControl.withReplicaProgress(func(replID roachpb.ReplicaID, progress rafttracker.Progress) { - if replID == f.replicaForFlowControl.getReplicaID() { - return - } - repl, ok := f.lastKnownReplicas.GetReplicaDescriptorByID(replID) - if !ok { - return + // TODO(irfansharif): Experimentally this gets triggered quite often. It + // might be too sensitive and may result in ineffective flow control as + // a result. Fix as part of #95563. } - if progress.State != rafttracker.StateReplicate { - res = append(res, replID) - // TODO(irfansharif): Integrating with these other progress fields - // from raft. For replicas exiting rafttracker.StateProbe, perhaps - // compare progress.Match against status.Commit to make sure it's - // sufficiently caught up with respect to its raft log before we - // start deducting tokens for it (lest we run into I3a from - // kvflowcontrol/doc.go). To play well with the replica-level - // proposal quota pool, maybe we also factor its base index? - // Replicas that crashed and came back could come back in - // StateReplicate but be behind on their logs. If we're deducting - // tokens right away for subsequent proposals, it would take some - // time for it to catch up and then later return those tokens to us. - // This is I3a again; do it as part of #95563. - _ = progress.RecentActive - _ = progress.MsgAppFlowPaused - _ = progress.Match - return + if _, found := disconnectedFollowers[repl.ReplicaID]; found && + !maintainStreamsForBrokenRaftTransport { + notActivelyReplicatingTo[repl] = struct{}{} } + } - if !f.replicaForFlowControl.isRaftTransportConnectedTo(repl.StoreID) { - if fn := f.knobs.MaintainStreamsForBrokenRaftTransport; fn != nil && fn() { - return // nothing to do - } - res = append(res, replID) - } - }) - return res + var repls []roachpb.ReplicaDescriptor + for repl := range notActivelyReplicatingTo { + repls = append(repls, repl) + } + return repls } +// disconnectStreams disconnects replication streams for the given replicas. func (f *replicaFlowControlIntegrationImpl) disconnectStreams( ctx context.Context, toDisconnect []roachpb.ReplicaDescriptor, reason string, ) { + ourReplicaID := f.replicaForFlowControl.getReplicaID() for _, repl := range toDisconnect { + if repl.ReplicaID == ourReplicaID { + log.Fatal(ctx, "replica attempting to disconnect from itself") + } if _, found := f.disconnectedStreams[repl.ReplicaID]; found { continue // already disconnected, nothing to do } @@ -361,81 +384,52 @@ func (f *replicaFlowControlIntegrationImpl) disconnectStreams( } } -func (f *replicaFlowControlIntegrationImpl) onRaftTransportDisconnected( - ctx context.Context, storeIDs ...roachpb.StoreID, -) { - f.replicaForFlowControl.assertLocked() - if f.innerHandle == nil { - return // nothing to do - } - - if fn := f.knobs.MaintainStreamsForBrokenRaftTransport; fn != nil && fn() { - return // nothing to do - } - - disconnectedStores := make(map[roachpb.StoreID]struct{}) - for _, storeID := range storeIDs { - disconnectedStores[storeID] = struct{}{} - } - - var toDisconnect []roachpb.ReplicaDescriptor - for _, repl := range f.lastKnownReplicas.Descriptors() { - if _, found := disconnectedStores[repl.StoreID]; found { - toDisconnect = append(toDisconnect, repl) - } - } - f.disconnectStreams(ctx, toDisconnect, "raft transport disconnected") - f.tryReconnect(ctx) -} - +// tryReconnect tries to reconnect to previously disconnected streams. func (f *replicaFlowControlIntegrationImpl) tryReconnect(ctx context.Context) { - // Try reconnecting streams we disconnected. - pausedFollowers := f.replicaForFlowControl.getPausedFollowers() - notActivelyReplicatingTo := f.notActivelyReplicatingTo() - appliedLogPosition := f.replicaForFlowControl.getAppliedLogPosition() - var disconnectedRepls []roachpb.ReplicaID for replID := range f.disconnectedStreams { disconnectedRepls = append(disconnectedRepls, replID) } - sort.Slice(disconnectedRepls, func(i, j int) bool { // for determinism in tests - return disconnectedRepls[i] < disconnectedRepls[j] - }) - for _, replID := range disconnectedRepls { - stream := f.disconnectedStreams[replID] - if _, ok := pausedFollowers[replID]; ok { - continue // still paused, nothing to reconnect - } + if buildutil.CrdbTestBuild { + sort.Slice(disconnectedRepls, func(i, j int) bool { // for determinism in tests + return disconnectedRepls[i] < disconnectedRepls[j] + }) + } - repl, ok := f.lastKnownReplicas.GetReplicaDescriptorByID(replID) - if !ok { + notActivelyReplicatingTo := f.notActivelyReplicatingTo() + appliedLogPosition := f.replicaForFlowControl.getAppliedLogPosition() + for _, replID := range disconnectedRepls { + if _, ok := f.lastKnownReplicas.GetReplicaDescriptorByID(replID); !ok { log.Fatalf(ctx, "%s: tracking %s in disconnected streams despite it not being in descriptor: %s", f.replicaForFlowControl.getReplicaID(), replID, f.lastKnownReplicas) } - if !f.replicaForFlowControl.isFollowerLive(ctx, replID) { - continue // still inactive, nothing to reconnect - } notReplicatedTo := false for _, notReplicatedToRepl := range notActivelyReplicatingTo { - if replID == notReplicatedToRepl { + if replID == notReplicatedToRepl.ReplicaID { notReplicatedTo = true break } } if notReplicatedTo { - continue // not actively replicated to, yet; nothing to reconnect - } - - if !f.replicaForFlowControl.isRaftTransportConnectedTo(repl.StoreID) { - continue // not connected to via raft transport + continue // not being actively replicated to, yet; nothing to reconnect } // See I1, I2, I3, I3a, I4 from kvflowcontrol/doc.go. Replica is // connected to via the RaftTransport (I1), on a live node (I2), not // paused (I3), and is being actively replicated to through log entries // (I3a, I4). Re-connect so we can start deducting tokens for it. + stream := f.disconnectedStreams[replID] f.innerHandle.ConnectStream(ctx, appliedLogPosition, stream) delete(f.disconnectedStreams, replID) } } + +// clearState closes the underlying kvflowcontrol.Handle and clears internal +// tracking state. +func (f *replicaFlowControlIntegrationImpl) clearState(ctx context.Context) { + f.innerHandle.Close(ctx) + f.innerHandle = nil + f.lastKnownReplicas = roachpb.MakeReplicaSet(nil) + f.disconnectedStreams = nil +} diff --git a/pkg/kv/kvserver/flow_control_replica_integration_test.go b/pkg/kv/kvserver/flow_control_replica_integration_test.go index f5efd66670c3..ad2bca7a3491 100644 --- a/pkg/kv/kvserver/flow_control_replica_integration_test.go +++ b/pkg/kv/kvserver/flow_control_replica_integration_test.go @@ -42,20 +42,21 @@ import ( // - "state" [applied=/] [descriptor=([,][,]*) [inactive=([][,]*) \ // [progress=([@:[probe | replicate | snapshot]:[!,]active:[!,]paused]*] \ -// [connected=([][,]*)] +// [disconnected=([][,]*)] // ---- // Set up relevant state of the underlying replica and/or raft transport. // // A. For replicas, we can control the applied state (term/index), // descriptor (set of replica IDs), paused and/or inactive replicas, and // per-replica raft progress. The raft progress syntax is structured as -// follows: -// => progress=(replid@match:::,...) -// Where is one of {probe,replicate,snapshot}, is -// {active,!inactive}, and is {paused,!paused}. +// follows: progress=(replid@match:::,...). +// is one of {probe,replicate,snapshot}, is +// {active,!inactive}, and is {paused,!paused}. The latter controls +// MsgAppFlowPaused in the raft library, not the CRDB-level follower +// pausing. // -// B. For the raft transport, we can specify the set of store IDs we're -// connected to. +// B. For the raft transport, we can specify the set of replica IDs we're +// not connected to. // // - "integration" op=[became-leader | became-follower | desc-changed | // followers-paused |replica-destroyed | @@ -123,7 +124,7 @@ func TestFlowControlReplicaIntegration(t *testing.T) { for _, arg := range d.CmdArgs { replicas := roachpb.MakeReplicaSet(nil) progress := make(map[roachpb.ReplicaID]tracker.Progress) - connected := make(map[roachpb.StoreID]struct{}) + disconnected := make(map[roachpb.ReplicaID]struct{}) for i := range arg.Vals { if arg.Vals[i] == "" { continue // we support syntax like inactive=(); there's nothing to do @@ -192,11 +193,11 @@ func TestFlowControlReplicaIntegration(t *testing.T) { case "applied": // Fall through. - case "connected": + case "disconnected": // Parse key=(,,...). var id uint64 arg.Scan(t, i, &id) - connected[roachpb.StoreID(id)] = struct{}{} + disconnected[roachpb.ReplicaID(id)] = struct{}{} default: t.Fatalf("unknown: %s", arg.Key) @@ -226,8 +227,8 @@ func TestFlowControlReplicaIntegration(t *testing.T) { // Parse applied=/. mockReplica.applied = parseLogPosition(t, arg.Vals[0]) - case "connected": - mockReplica.connected = connected + case "disconnected": + mockReplica.disconnected = disconnected default: t.Fatalf("unknown: %s", arg.Key) @@ -247,10 +248,10 @@ func TestFlowControlReplicaIntegration(t *testing.T) { integration.onDescChanged(ctx) case "followers-paused": integration.onFollowersPaused(ctx) - case "replica-destroyed": - integration.onReplicaDestroyed(ctx) - case "proposal-quota-updated": - integration.onProposalQuotaUpdated(ctx) + case "destroyed": + integration.onDestroyed(ctx) + case "raft-ticked": + integration.onRaftTicked(ctx) default: t.Fatalf("unknown op: %s", op) } @@ -270,12 +271,12 @@ type mockReplicaForFlowControl struct { tenantID roachpb.TenantID replicaID roachpb.ReplicaID - paused map[roachpb.ReplicaID]struct{} - inactive map[roachpb.ReplicaID]struct{} - progress map[roachpb.ReplicaID]tracker.Progress - connected map[roachpb.StoreID]struct{} - applied kvflowcontrolpb.RaftLogPosition - descriptor *roachpb.RangeDescriptor + paused map[roachpb.ReplicaID]struct{} + inactive map[roachpb.ReplicaID]struct{} + disconnected map[roachpb.ReplicaID]struct{} + progress map[roachpb.ReplicaID]tracker.Progress + applied kvflowcontrolpb.RaftLogPosition + descriptor *roachpb.RangeDescriptor } var _ replicaForFlowControl = &mockReplicaForFlowControl{} @@ -328,28 +329,29 @@ func (m *mockReplicaForFlowControl) getPausedFollowers() map[roachpb.ReplicaID]s return m.paused } -func (m *mockReplicaForFlowControl) isFollowerLive( - ctx context.Context, replID roachpb.ReplicaID, -) bool { - _, inactive := m.inactive[replID] - return !inactive +func (m *mockReplicaForFlowControl) getBehindFollowers() map[roachpb.ReplicaID]struct{} { + // NB: Keep this identical to the canonical implementation of + // getBehindFollowers. + behindFollowers := make(map[roachpb.ReplicaID]struct{}) + for replID, progress := range m.progress { + if progress.State == tracker.StateReplicate { + continue + } + behindFollowers[replID] = struct{}{} + } + return behindFollowers } -func (m *mockReplicaForFlowControl) isRaftTransportConnectedTo(storeID roachpb.StoreID) bool { - _, found := m.connected[storeID] - return found +func (m *mockReplicaForFlowControl) getInactiveFollowers() map[roachpb.ReplicaID]struct{} { + return m.inactive } -func (m *mockReplicaForFlowControl) getAppliedLogPosition() kvflowcontrolpb.RaftLogPosition { - return m.applied +func (m *mockReplicaForFlowControl) getDisconnectedFollowers() map[roachpb.ReplicaID]struct{} { + return m.disconnected } -func (m *mockReplicaForFlowControl) withReplicaProgress( - f func(roachpb.ReplicaID, tracker.Progress), -) { - for replID, progress := range m.progress { - f(replID, progress) - } +func (m *mockReplicaForFlowControl) getAppliedLogPosition() kvflowcontrolpb.RaftLogPosition { + return m.applied } func (m *mockReplicaForFlowControl) isScratchRange() bool { diff --git a/pkg/kv/kvserver/flow_control_stores.go b/pkg/kv/kvserver/flow_control_stores.go index 6981b06ac1b6..af5d4f6be056 100644 --- a/pkg/kv/kvserver/flow_control_stores.go +++ b/pkg/kv/kvserver/flow_control_stores.go @@ -19,14 +19,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/log" ) -// StoresForFlowControl is used to integrate with replication flow control. It -// exposes the underlying kvflowcontrol.Handles and is informed of (remote) -// stores we're no longer connected via the raft transport. -type StoresForFlowControl interface { - kvflowcontrol.Handles - RaftTransportDisconnectListener -} - // storesForFlowControl is a concrete implementation of the // StoresForFlowControl interface, backed by a set of Stores. type storesForFlowControl Stores @@ -221,7 +213,7 @@ func (l NoopStoresFlowControlIntegration) Inspect() []roachpb.RangeID { return nil } -// OnRaftTransportDisconnected is part of the RaftTransportDisconnectListener +// OnRaftTransportDisconnected is part of the StoresForFlowControl // interface. func (NoopStoresFlowControlIntegration) OnRaftTransportDisconnected( context.Context, ...roachpb.StoreID, diff --git a/pkg/kv/kvserver/helpers_test.go b/pkg/kv/kvserver/helpers_test.go index 7afb4ed47451..836f7f8cf44c 100644 --- a/pkg/kv/kvserver/helpers_test.go +++ b/pkg/kv/kvserver/helpers_test.go @@ -341,7 +341,7 @@ func (r *Replica) IsFollowerActiveSince( ) bool { r.mu.RLock() defer r.mu.RUnlock() - return r.mu.lastUpdateTimes.isFollowerActiveSince(ctx, followerID, timeutil.Now(), threshold) + return r.mu.lastUpdateTimes.isFollowerActiveSince(followerID, timeutil.Now(), threshold) } // GetTSCacheHighWater returns the high water mark of the replica's timestamp diff --git a/pkg/kv/kvserver/kvflowcontrol/doc.go b/pkg/kv/kvserver/kvflowcontrol/doc.go index 5882d6b80a1e..6cae3c70ef21 100644 --- a/pkg/kv/kvserver/kvflowcontrol/doc.go +++ b/pkg/kv/kvserver/kvflowcontrol/doc.go @@ -449,6 +449,42 @@ package kvflowcontrol // we're able to look up the right kvflowcontrol.Handle since the replica is // still around. When quiescing a range, we don't need to release all-held // tokens, or wait until there are no held flow tokens. +// - We use the replica ticking to periodically refresh the set of +// replication streams we're connected to, ticking that's disabled for +// quiesced ranges. The mechanism is responsible for disconnecting streams +// from paused, disconnected (per the raft transport), behind (per the raft +// log), and inactive (those we haven't heard from recently) followers. We +// don't quiesce ranges in the presence of paused followers, and if +// followers disconnect, we intercept that directly and release flow tokens. +// We don't quiesce if live followers are behind on the raft log (dead +// followers are allowed to be behind). Since the ticking is the only +// mechanism that frees up tokens for inactive followers, what happens if a +// range quiesces after observing caught up replicas, one of which is +// holding onto tokens due to slow below-raft admission, and post-quiescence +// the node crashes? We do react to the raft transport breaking, so that +// frees up tokens. But it's perhaps surprising that the replica-inactivity +// check is disabled. It's not clear what to do here. +// - We could observe node liveness directly, but we're already effectively +// doing that when reacting to raft transport streams breaking. +// - We could release all flow tokens whenever replicas quiesce (though +// risking over-admission). +// - We could make the last-updated map more evented, releasing tokens +// directly whenever replicas expire, to not need depend on this explicit +// ticking that's disabled when quiesced. We could continue ticking (at +// low frequency) even when quiesced. +// - If we only had expiration based leases/no quiescence, this would not be +// a problem. +// +// See kvserver/flow_control_*.go for where we address all the interactions +// above. The guiding principle is to 'only deduct flow tokens when actively +// replicating a proposal along specific streams', which excludes +// dead/paused/lagging/pre-split RHS/non-longer-group-member replicas, and +// explains why we only do it on replicas that are both leaseholder and leader. +// It also explains why we don't re-deduct on reproposals, or try to intercept +// raft-initiated re-transmissions. For each of these scenarios, we know when +// not to deduct flow tokens, and we simply free up all held tokens and +// safeguard against a subsequent double returns. We care about safety (no token +// leaks, no double returns) and liveness (eventual token returns). // // --- // @@ -510,27 +546,3 @@ package kvflowcontrol // [^13]: See admission.sequencer and its use in admission.StoreWorkQueue. // [^14]: See the high_create_time_low_position_different_range test case for // TestReplicatedWriteAdmission. -// -// TODO(irfansharif): These descriptions are too high-level, imprecise and -// possibly wrong. Fix that. After implementing these interfaces and integrating -// it into KV, write tests for each one of them and document the precise -// interactions/integration points. It needs to be distilled to crisper -// invariants. The guiding principle seems to be 'only grab flow tokens when -// actively replicating a proposal along specific streams', which excludes -// dead/paused/lagging/pre-split RHS/non-longer-group-member replicas, and -// explains why we only do it on replicas that are both leaseholder and leader. -// It also explains why we don't re-deduct on reproposals, or try to intercept -// raft-initiated re-transmissions. For each of these scenarios, we know when -// not to deduct flow tokens. If we observe getting into the scenarios, we -// simply free up all held tokens and safeguard against a subsequent double -// returns, relying entirely on low water marks or RangeIDs not being re-used. -// - When framing invariants, talk about how we're achieving safety (no token -// leaks, no double returns) and liveness (eventual token returns). -// - Other than I8 above, are there cases where the sender has deducted tokens -// and something happens on the receiver/sender/sender-receiver stream that: -// (a) doesn't cause the sender to "return all tokens", i.e. it's relying on -// the receiver to send messages to return tokens up to some point, and -// (b) the receiver has either not received the message for which we've -// deducted tokens, or forgotten about it. -// - Ensure that flow tokens aren't leaked, by checking that after the tests -// quiesce, flow tokens are back to their original limits. diff --git a/pkg/kv/kvserver/kvflowcontrol/testing_knobs.go b/pkg/kv/kvserver/kvflowcontrol/testing_knobs.go index d09c942107db..6231cfd1cc48 100644 --- a/pkg/kv/kvserver/kvflowcontrol/testing_knobs.go +++ b/pkg/kv/kvserver/kvflowcontrol/testing_knobs.go @@ -24,6 +24,9 @@ type TestingKnobs struct { // OverrideTokenDeduction is used to override how many tokens are deducted // post-evaluation. OverrideTokenDeduction func() Tokens + // MaintainStreamsForBehindFollowers is used in tests to maintain + // replication streams for behind followers. + MaintainStreamsForBehindFollowers func() bool // MaintainStreamsForInactiveFollowers is used in tests to maintain // replication streams for inactive followers. MaintainStreamsForInactiveFollowers func() bool diff --git a/pkg/kv/kvserver/raft_log_queue.go b/pkg/kv/kvserver/raft_log_queue.go index f0eef5c6507a..7c96c7381ec7 100644 --- a/pkg/kv/kvserver/raft_log_queue.go +++ b/pkg/kv/kvserver/raft_log_queue.go @@ -306,8 +306,7 @@ func newTruncateDecision(ctx context.Context, r *Replica) (truncateDecision, err updateRaftProgressFromActivity( ctx, raftStatus.Progress, r.descRLocked().Replicas().Descriptors(), func(replicaID roachpb.ReplicaID) bool { - return r.mu.lastUpdateTimes.isFollowerActiveSince( - ctx, replicaID, now, r.store.cfg.RangeLeaseDuration) + return r.mu.lastUpdateTimes.isFollowerActiveSince(replicaID, now, r.store.cfg.RangeLeaseDuration) }, ) log.Eventf(ctx, "raft status after lastUpdateTimes check: %+v", raftStatus.Progress) diff --git a/pkg/kv/kvserver/raft_log_queue_test.go b/pkg/kv/kvserver/raft_log_queue_test.go index c4ebd6df95f6..0dc14944acd8 100644 --- a/pkg/kv/kvserver/raft_log_queue_test.go +++ b/pkg/kv/kvserver/raft_log_queue_test.go @@ -405,7 +405,7 @@ func TestUpdateRaftStatusActivity(t *testing.T) { } updateRaftProgressFromActivity(ctx, prs, tc.replicas, func(replicaID roachpb.ReplicaID) bool { - return tc.lastUpdate.isFollowerActiveSince(ctx, replicaID, tc.now, inactivityThreashold) + return tc.lastUpdate.isFollowerActiveSince(replicaID, tc.now, inactivityThreashold) }, ) diff --git a/pkg/kv/kvserver/replica_destroy.go b/pkg/kv/kvserver/replica_destroy.go index 8a836067ac66..b53c19377d09 100644 --- a/pkg/kv/kvserver/replica_destroy.go +++ b/pkg/kv/kvserver/replica_destroy.go @@ -174,7 +174,7 @@ func (r *Replica) disconnectReplicationRaftMuLocked(ctx context.Context) { if pq := r.mu.proposalQuota; pq != nil { pq.Close("destroyed") } - r.mu.replicaFlowControlIntegration.onReplicaDestroyed(ctx) + r.mu.replicaFlowControlIntegration.onDestroyed(ctx) r.mu.proposalBuf.FlushLockedWithoutProposing(ctx) for _, p := range r.mu.proposals { r.cleanupFailedProposalLocked(p) diff --git a/pkg/kv/kvserver/replica_proposal_quota.go b/pkg/kv/kvserver/replica_proposal_quota.go index fb61391d8bf1..cece7b811f22 100644 --- a/pkg/kv/kvserver/replica_proposal_quota.go +++ b/pkg/kv/kvserver/replica_proposal_quota.go @@ -169,9 +169,7 @@ func (r *Replica) updateProposalQuotaRaftMuLocked( // such for the purposes of releasing quota can have bad consequences // (writes will stall), whereas for quiescing the downside is lower. - if !r.mu.lastUpdateTimes.isFollowerActiveSince( - ctx, rep.ReplicaID, now, r.store.cfg.RangeLeaseDuration, - ) { + if !r.mu.lastUpdateTimes.isFollowerActiveSince(rep.ReplicaID, now, r.store.cfg.RangeLeaseDuration) { return } // At this point, we know that either we communicated with this replica @@ -262,5 +260,10 @@ func (r *Replica) updateProposalQuotaRaftMuLocked( r.mu.proposalQuotaBaseIndex, len(r.mu.quotaReleaseQueue), releasableIndex, status.Applied) } - r.mu.replicaFlowControlIntegration.onProposalQuotaUpdated(ctx) + + // Tick the replicaFlowControlIntegration interface. This is as convenient a + // place to do it as any other. Much like the quota pool code above, the + // flow control integration layer considers raft progress state for + // individual replicas, and whether they've been recently active. + r.mu.replicaFlowControlIntegration.onRaftTicked(ctx) } diff --git a/pkg/kv/kvserver/replica_raft.go b/pkg/kv/kvserver/replica_raft.go index 22bbf796e7b4..cd43e7e778ad 100644 --- a/pkg/kv/kvserver/replica_raft.go +++ b/pkg/kv/kvserver/replica_raft.go @@ -2327,7 +2327,7 @@ func (m lastUpdateTimesMap) updateOnBecomeLeader(descs []roachpb.ReplicaDescript // isFollowerActiveSince returns whether the specified follower has made // communication with the leader recently (since threshold). func (m lastUpdateTimesMap) isFollowerActiveSince( - ctx context.Context, replicaID roachpb.ReplicaID, now time.Time, threshold time.Duration, + replicaID roachpb.ReplicaID, now time.Time, threshold time.Duration, ) bool { lastUpdateTime, ok := m[replicaID] if !ok { diff --git a/pkg/kv/kvserver/replica_raft_overload.go b/pkg/kv/kvserver/replica_raft_overload.go index 04d829151d3b..a545a4779d76 100644 --- a/pkg/kv/kvserver/replica_raft_overload.go +++ b/pkg/kv/kvserver/replica_raft_overload.go @@ -356,7 +356,7 @@ func (r *Replica) updatePausedFollowersLocked(ctx context.Context, ioThresholdMa getProgressMap: func(_ context.Context) map[uint64]tracker.Progress { prs := r.mu.internalRaftGroup.Status().Progress updateRaftProgressFromActivity(ctx, prs, r.descRLocked().Replicas().AsProto(), func(id roachpb.ReplicaID) bool { - return r.mu.lastUpdateTimes.isFollowerActiveSince(ctx, id, now, r.store.cfg.RangeLeaseDuration) + return r.mu.lastUpdateTimes.isFollowerActiveSince(id, now, r.store.cfg.RangeLeaseDuration) }) return prs }, diff --git a/pkg/kv/kvserver/split_delay_helper.go b/pkg/kv/kvserver/split_delay_helper.go index 148a16fa3c71..6f39144191de 100644 --- a/pkg/kv/kvserver/split_delay_helper.go +++ b/pkg/kv/kvserver/split_delay_helper.go @@ -40,8 +40,7 @@ func (sdh *splitDelayHelper) RaftStatus(ctx context.Context) (roachpb.RangeID, * updateRaftProgressFromActivity( ctx, raftStatus.Progress, r.descRLocked().Replicas().Descriptors(), func(replicaID roachpb.ReplicaID) bool { - return r.mu.lastUpdateTimes.isFollowerActiveSince( - ctx, replicaID, timeutil.Now(), r.store.cfg.RangeLeaseDuration) + return r.mu.lastUpdateTimes.isFollowerActiveSince(replicaID, timeutil.Now(), r.store.cfg.RangeLeaseDuration) }, ) } diff --git a/pkg/kv/kvserver/store.go b/pkg/kv/kvserver/store.go index e5b7da4adf43..6c25a43159e9 100644 --- a/pkg/kv/kvserver/store.go +++ b/pkg/kv/kvserver/store.go @@ -805,6 +805,10 @@ type Store struct { ctSender *sidetransport.Sender storeGossip *StoreGossip rebalanceObjManager *RebalanceObjectiveManager + // raftTransportForFlowControl exposes the set of (remote) stores the raft + // transport is connected to, and is used by the canonical + // replicaFlowControlIntegration implementation. + raftTransportForFlowControl raftTransportForFlowControl coalescedMu struct { syncutil.Mutex @@ -1437,6 +1441,7 @@ func NewStore( log.Warningf(ctx, "failed to clear snapshot storage: %v", err) } s.protectedtsReader = cfg.ProtectedTimestampReader + s.raftTransportForFlowControl = cfg.Transport // On low-CPU instances, a default limit value may still allow ExportRequests // to tie up all cores so cap limiter at cores-1 when setting value is higher. diff --git a/pkg/kv/kvserver/testdata/flow_control_integration/raft_snapshot b/pkg/kv/kvserver/testdata/flow_control_integration/raft_snapshot index ac2cfe915a03..e81112500f41 100644 --- a/pkg/kv/kvserver/testdata/flow_control_integration/raft_snapshot +++ b/pkg/kv/kvserver/testdata/flow_control_integration/raft_snapshot @@ -120,12 +120,11 @@ ORDER BY name ASC; -- for n2 and n3 anymore. SELECT range_id, store_id, crdb_internal.humanize_bytes(total_tracked_tokens::INT8) FROM crdb_internal.kv_flow_control_handles + WHERE total_tracked_tokens > 0 range_id | store_id | total_tracked_tokens -----------+----------+----------------------- 64 | 1 | 2.0 MiB - 64 | 2 | 0 B - 64 | 3 | 0 B 64 | 4 | 2.0 MiB 64 | 5 | 2.0 MiB diff --git a/pkg/kv/kvserver/testdata/flow_control_replica_integration/desc_changed b/pkg/kv/kvserver/testdata/flow_control_replica_integration/desc_changed index a8c16289dba3..56eb7a7d6398 100644 --- a/pkg/kv/kvserver/testdata/flow_control_replica_integration/desc_changed +++ b/pkg/kv/kvserver/testdata/flow_control_replica_integration/desc_changed @@ -1,18 +1,19 @@ -# Observe how the integration layer deals with change raft group members. Start -# off with a double replicated range r1/t1, with replicas on n1/s1 and n2/s2 -# (with replica IDs 1-2 respectively). +# Observe how the integration layer deals with changing raft group members. +# Start off with a double replicated range r1/t1, with replicas on n1/s1 and +# n2/s2 (with replica IDs 1-2 respectively). init tenant=t1 range=r1 replid=1 ---- -state applied=1/10 descriptor=(1,2) connected=(1,2,3,4) +state applied=1/10 descriptor=(1,2) ---- -# Set up replid=1 (declared in init above) to be the raft leader. It should -# connect to both replication streams. +# Set up replid=1 (declared in init above) to be the raft leader. It should end +# up connected to both replication streams. integration op=became-leader ---- initialized flow control handle for r1/t1 connected to replication stream t1/s1 starting at log-position=1/10 +disconnected from replication stream t1/s2 connected to replication stream t1/s2 starting at log-position=1/10 # Add replid=3. New replicas start off without a match index, in StateProbe, as @@ -25,8 +26,9 @@ state applied=1/11 descriptor=(1,2,3) progress=(1@11:replicate:active:!paused, 2 integration op=desc-changed ---- +disconnected from replication stream t1/s3 -integration op=proposal-quota-updated +integration op=raft-ticked ---- # replid=3 soon enters StateProbe with an up-to-date match index. It's also @@ -36,7 +38,7 @@ integration op=proposal-quota-updated state applied=1/11 descriptor=(1,2,3) progress=(1@11:replicate:active:!paused, 2@11:replicate:active:!paused, 3@11:replicate:active:!paused) ---- -integration op=proposal-quota-updated +integration op=raft-ticked ---- connected to replication stream t1/s3 starting at log-position=1/11 @@ -55,8 +57,9 @@ state applied=1/12 descriptor=(1,3,4) progress=(1@12:replicate:active:!paused, 3 integration op=desc-changed ---- disconnected from replication stream t1/s2 +disconnected from replication stream t1/s4 -integration op=proposal-quota-updated +integration op=raft-ticked ---- connected to replication stream t1/s4 starting at log-position=1/12 diff --git a/pkg/kv/kvserver/testdata/flow_control_replica_integration/disconnected_followers b/pkg/kv/kvserver/testdata/flow_control_replica_integration/disconnected_followers new file mode 100644 index 000000000000..caff593e0744 --- /dev/null +++ b/pkg/kv/kvserver/testdata/flow_control_replica_integration/disconnected_followers @@ -0,0 +1,45 @@ +# Observe how the integration layer deals with disconnected followers. Start off +# with a triply replicated range r1/t1, with replicas on n1/s1, n2/s2, and +# n3/s3 (with replica IDs 1-3 respectively). +init tenant=t1 range=r1 replid=1 +---- + +state descriptor=(1,2,3) applied=1/10 +---- + +# Set up replid=1 (declared in init above) to be the raft leader. It should +# connect to all three replication streams. +integration op=became-leader +---- +initialized flow control handle for r1/t1 +connected to replication stream t1/s1 starting at log-position=1/10 +disconnected from replication stream t1/s2 +disconnected from replication stream t1/s3 +connected to replication stream t1/s2 starting at log-position=1/10 +connected to replication stream t1/s3 starting at log-position=1/10 + +# Mark replid=2 as inactive. Observe that we disconnect the stream to t1/s2. +state descriptor=(1,2,3) disconnected=(2) +---- + +integration op=raft-ticked +---- +disconnected from replication stream t1/s2 + +# The integration layer maintains internal state about the set of disconnected +# followers we've already disconnected from. Informing it of disconnected +# followers again simply no-ops. +integration op=raft-ticked +---- + +# Mark replid=2 as connected and replid=3 as disconnected. We should see us disconnect +# from t1/s3 and reconnect to t1/s2 using our current applied state. +state descriptor=(1,2,3) disconnected=(3) applied=1/11 +---- + +integration op=raft-ticked +---- +disconnected from replication stream t1/s3 +connected to replication stream t1/s2 starting at log-position=1/11 + +# vim:ft=sh diff --git a/pkg/kv/kvserver/testdata/flow_control_replica_integration/inactive_followers b/pkg/kv/kvserver/testdata/flow_control_replica_integration/inactive_followers index be95ae1b6911..4a047a100bf9 100644 --- a/pkg/kv/kvserver/testdata/flow_control_replica_integration/inactive_followers +++ b/pkg/kv/kvserver/testdata/flow_control_replica_integration/inactive_followers @@ -4,7 +4,7 @@ init tenant=t1 range=r1 replid=1 ---- -state descriptor=(1,2,3) applied=1/10 connected=(1,2,3) +state descriptor=(1,2,3) applied=1/10 ---- # Set up replid=1 (declared in init above) to be the raft leader. It should @@ -13,6 +13,8 @@ integration op=became-leader ---- initialized flow control handle for r1/t1 connected to replication stream t1/s1 starting at log-position=1/10 +disconnected from replication stream t1/s2 +disconnected from replication stream t1/s3 connected to replication stream t1/s2 starting at log-position=1/10 connected to replication stream t1/s3 starting at log-position=1/10 @@ -20,14 +22,14 @@ connected to replication stream t1/s3 starting at log-position=1/10 state descriptor=(1,2,3) inactive=(2) ---- -integration op=proposal-quota-updated +integration op=raft-ticked ---- disconnected from replication stream t1/s2 # The integration layer maintains internal state about the set of inactive # followers we've already disconnected from. Informing it of inactive followers # again simply no-ops. -integration op=proposal-quota-updated +integration op=raft-ticked ---- # Mark replid=2 as active and replid=3 as inactive. We should see us disconnect @@ -35,7 +37,7 @@ integration op=proposal-quota-updated state descriptor=(1,2,3) inactive=(3) applied=1/11 ---- -integration op=proposal-quota-updated +integration op=raft-ticked ---- disconnected from replication stream t1/s3 connected to replication stream t1/s2 starting at log-position=1/11 diff --git a/pkg/kv/kvserver/testdata/flow_control_replica_integration/paused_followers b/pkg/kv/kvserver/testdata/flow_control_replica_integration/paused_followers index 21f90b8d29f6..eb8a244e44a8 100644 --- a/pkg/kv/kvserver/testdata/flow_control_replica_integration/paused_followers +++ b/pkg/kv/kvserver/testdata/flow_control_replica_integration/paused_followers @@ -4,7 +4,7 @@ init tenant=t1 range=r1 replid=1 ---- -state descriptor=(1,2,3) applied=1/10 connected=(1,2,3) +state descriptor=(1,2,3) applied=1/10 ---- # Set up replid=1 (declared in init above) to be the raft leader. It should @@ -13,6 +13,8 @@ integration op=became-leader ---- initialized flow control handle for r1/t1 connected to replication stream t1/s1 starting at log-position=1/10 +disconnected from replication stream t1/s2 +disconnected from replication stream t1/s3 connected to replication stream t1/s2 starting at log-position=1/10 connected to replication stream t1/s3 starting at log-position=1/10 diff --git a/pkg/kv/kvserver/testdata/flow_control_replica_integration/replica_destroyed b/pkg/kv/kvserver/testdata/flow_control_replica_integration/replica_destroyed index 4e835803cac5..470ccb15a152 100644 --- a/pkg/kv/kvserver/testdata/flow_control_replica_integration/replica_destroyed +++ b/pkg/kv/kvserver/testdata/flow_control_replica_integration/replica_destroyed @@ -8,7 +8,7 @@ init tenant=t1 range=r1 replid=1 ---- -state descriptor=(1,2,3) applied=1/10 connected=(1,2,3) +state descriptor=(1,2,3) applied=1/10 ---- # Set up replid=1 (declared in init above) to be the raft leader. It should @@ -17,6 +17,8 @@ integration op=became-leader ---- initialized flow control handle for r1/t1 connected to replication stream t1/s1 starting at log-position=1/10 +disconnected from replication stream t1/s2 +disconnected from replication stream t1/s3 connected to replication stream t1/s2 starting at log-position=1/10 connected to replication stream t1/s3 starting at log-position=1/10 @@ -24,7 +26,7 @@ connected to replication stream t1/s3 starting at log-position=1/10 # 'integration op=became-follower') or observing itself being removed from the # range ('integration op=desc-changed'). We should just close the handle, # disconnecting all underlying streams. -integration op=replica-destroyed +integration op=destroyed ---- closed flow control handle for r1/t1 diff --git a/pkg/kv/kvserver/testdata/flow_control_replica_integration/replica_raft_leadership b/pkg/kv/kvserver/testdata/flow_control_replica_integration/replica_raft_leadership index ab7a5dfc6a0f..1f0a84058583 100644 --- a/pkg/kv/kvserver/testdata/flow_control_replica_integration/replica_raft_leadership +++ b/pkg/kv/kvserver/testdata/flow_control_replica_integration/replica_raft_leadership @@ -4,7 +4,7 @@ init tenant=t1 range=r1 replid=1 ---- -state descriptor=(1,2,3) applied=1/10 connected=(1,2,3,4,5) +state descriptor=(1,2,3) applied=1/10 ---- # Set up replid=1 (declared in init above) to be the raft leader. It should @@ -15,6 +15,8 @@ integration op=became-leader ---- initialized flow control handle for r1/t1 connected to replication stream t1/s1 starting at log-position=1/10 +disconnected from replication stream t1/s2 +disconnected from replication stream t1/s3 connected to replication stream t1/s2 starting at log-position=1/10 connected to replication stream t1/s3 starting at log-position=1/10 @@ -24,7 +26,7 @@ integration op=became-follower ---- closed flow control handle for r1/t1 -# state applied=1/12 descriptor=(1,2,3,4,5) paused=(2) inactive=(3) +# state applied=1/12 descriptor=(1,2,3,4,5) paused=(2) inactive=(3) # Modify the descriptor. When re-acquiring leadership, we should connect to all # current members of the raft group. @@ -35,6 +37,8 @@ integration op=became-leader ---- initialized flow control handle for r1/t1 connected to replication stream t1/s1 starting at log-position=1/12 +disconnected from replication stream t1/s2 +disconnected from replication stream t1/s4 connected to replication stream t1/s2 starting at log-position=1/12 connected to replication stream t1/s4 starting at log-position=1/12 @@ -51,6 +55,9 @@ integration op=became-leader ---- initialized flow control handle for r1/t1 connected to replication stream t1/s1 starting at log-position=1/14 +disconnected from replication stream t1/s2 +disconnected from replication stream t1/s4 +disconnected from replication stream t1/s5 connected to replication stream t1/s2 starting at log-position=1/14 integration op=became-follower @@ -63,13 +70,16 @@ closed flow control handle for r1/t1 # - replid=5 just being recently added to the raft group, where we don't know # its match index just yet. # When acquiring raft leadership, we shouldn't connect to t1/s4 and t1/s5. -state applied=1/16 descriptor=(1,2,4,5) progress=(1@16:replicate:active:!paused, 2@16:replicate:active:!paused, 4@14:snapshot:!active:paused, 5@0:probe:!active:paused) +state applied=1/16 descriptor=(1,2,4,5) progress=(1@16:replicate:active:!paused, 2@16:replicate:active:!paused, 4@14:snapshot:!active:paused, 5@0:probe:!active:paused) ---- integration op=became-leader ---- initialized flow control handle for r1/t1 connected to replication stream t1/s1 starting at log-position=1/16 +disconnected from replication stream t1/s2 +disconnected from replication stream t1/s4 +disconnected from replication stream t1/s5 connected to replication stream t1/s2 starting at log-position=1/16 # vim:ft=sh diff --git a/pkg/kv/kvserver/testdata/flow_control_replica_integration/replica_removed_self b/pkg/kv/kvserver/testdata/flow_control_replica_integration/replica_removed_self index 7ccfe9ffd8de..af83de1a0770 100644 --- a/pkg/kv/kvserver/testdata/flow_control_replica_integration/replica_removed_self +++ b/pkg/kv/kvserver/testdata/flow_control_replica_integration/replica_removed_self @@ -6,7 +6,7 @@ init tenant=t1 range=r1 replid=1 ---- -state descriptor=(1,2,3) applied=1/10 connected=(1,2,3) +state descriptor=(1,2,3) applied=1/10 ---- # Set up replid=1 (declared in init above) to be the raft leader. It should @@ -15,6 +15,8 @@ integration op=became-leader ---- initialized flow control handle for r1/t1 connected to replication stream t1/s1 starting at log-position=1/10 +disconnected from replication stream t1/s2 +disconnected from replication stream t1/s3 connected to replication stream t1/s2 starting at log-position=1/10 connected to replication stream t1/s3 starting at log-position=1/10 diff --git a/pkg/util/log/clog.go b/pkg/util/log/clog.go index 5b9f83071460..d0206c66cf0c 100644 --- a/pkg/util/log/clog.go +++ b/pkg/util/log/clog.go @@ -419,6 +419,11 @@ func setActive() { } } +// ShowLogs returns whether -show-logs was passed (used for testing). +func ShowLogs() bool { + return logging.showLogs +} + const fatalErrorPostamble = ` **************************************************************************** From 0bafe3aec3d5b0e5ee3266c32a818e0e723bc2b7 Mon Sep 17 00:00:00 2001 From: irfan sharif Date: Fri, 26 May 2023 19:24:50 -0400 Subject: [PATCH 08/12] kvserver: add TestFlowControlAdmissionPostSplitMerge TestFlowControlAdmissionPostSplitMerge walks through what happens admission happens after undergoes splits/merges. It does this by blocking and later unblocking below-raft admission, verifying: - tokens for the RHS are released at the post-merge leaseholder, - admission for the RHS post-merge does not cause a double return of tokens, - admission for the LHS can happen post-merge, - admission for the LHS and RHS can happen post-split. Release note: None --- .../kvserver/flow_control_integration_test.go | 149 ++++++++++++++++++ .../admission_post_split_merge | 110 +++++++++++++ 2 files changed, 259 insertions(+) create mode 100644 pkg/kv/kvserver/testdata/flow_control_integration/admission_post_split_merge diff --git a/pkg/kv/kvserver/flow_control_integration_test.go b/pkg/kv/kvserver/flow_control_integration_test.go index 83da50effd3d..af262a08821c 100644 --- a/pkg/kv/kvserver/flow_control_integration_test.go +++ b/pkg/kv/kvserver/flow_control_integration_test.go @@ -478,6 +478,155 @@ ORDER BY name ASC; `) } +// TestFlowControlAdmissionPostSplitMerge walks through what happens with flow +// tokens when a range after undergoes splits/merges. It does this by blocking +// and later unblocking below-raft admission, verifying: +// - tokens for the RHS are released at the post-merge subsuming leaseholder, +// - admission for the RHS post-merge does not cause a double return of tokens, +// - admission for the LHS can happen post-merge, +// - admission for the LHS and RHS can happen post-split. +func TestFlowControlAdmissionPostSplitMerge(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + ctx := context.Background() + const numNodes = 3 + + var disableWorkQueueGranting atomic.Bool + disableWorkQueueGranting.Store(true) + + tc := testcluster.StartTestCluster(t, numNodes, base.TestClusterArgs{ + ReplicationMode: base.ReplicationManual, + ServerArgs: base.TestServerArgs{ + Knobs: base.TestingKnobs{ + Store: &kvserver.StoreTestingKnobs{ + FlowControlTestingKnobs: &kvflowcontrol.TestingKnobs{ + UseOnlyForScratchRanges: true, + }, + }, + AdmissionControl: &admission.TestingKnobs{ + DisableWorkQueueFastPath: true, + DisableWorkQueueGranting: func() bool { + return disableWorkQueueGranting.Load() + }, + }, + }, + }, + }) + defer tc.Stopper().Stop(ctx) + + k := tc.ScratchRange(t) + tc.AddVotersOrFatal(t, k, tc.Targets(1, 2)...) + + n1 := sqlutils.MakeSQLRunner(tc.ServerConn(0)) + + h := newFlowControlTestHelper(t, tc) + h.init() + defer h.close("admission_post_split_merge") + + desc, err := tc.LookupRange(k) + require.NoError(t, err) + + h.waitForConnectedStreams(ctx, desc.RangeID, 3) + + h.log("sending put request to pre-split range") + h.put(ctx, k, 1<<20 /* 1MiB */, admissionpb.NormalPri) + h.put(ctx, k.Next(), 1<<20 /* 1MiB */, admissionpb.NormalPri) + h.log("sent put request to pre-split range") + + h.comment(` +-- Flow token metrics from n1 after issuing a regular 2*1MiB 3x replicated write +-- that are yet to get admitted. We see 2*3*1MiB=6MiB deductions of +-- {regular,elastic} tokens with no corresponding returns. The 2*1MiB writes +-- happened on what is soon going to be the LHS and RHS of a range being split. +`) + h.query(n1, ` + SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvadmission%tokens%' +ORDER BY name ASC; +`) + + h.comment(`-- (Splitting range.)`) + left, right := tc.SplitRangeOrFatal(t, k.Next()) + h.waitForConnectedStreams(ctx, right.RangeID, 3) + + h.log("sending 2MiB put request to post-split LHS") + h.put(ctx, k, 2<<20 /* 2MiB */, admissionpb.NormalPri) + h.log("sent 2MiB put request to post-split LHS") + + h.log("sending 3MiB put request to post-split RHS") + h.put(ctx, roachpb.Key(right.StartKey), 3<<20 /* 3MiB */, admissionpb.NormalPri) + h.log("sent 3MiB put request to post-split RHS") + + h.comment(` +-- Flow token metrics from n1 after further issuing 2MiB and 3MiB writes to +-- post-split LHS and RHS ranges respectively. We should see 15MiB extra tokens +-- deducted which comes from (2MiB+3MiB)*3=15MiB. So we stand at +-- 6MiB+15MiB=21MiB now. +`) + h.query(n1, ` + SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvadmission%tokens%' +ORDER BY name ASC; +`) + + h.comment(`-- Observe the newly split off replica, with its own three streams.`) + h.query(n1, ` + SELECT range_id, count(*) AS streams + FROM crdb_internal.kv_flow_control_handles +GROUP BY (range_id) +ORDER BY streams DESC; +`, "range_id", "stream_count") + + h.comment(`-- (Merging ranges.)`) + merged := tc.MergeRangesOrFatal(t, left.StartKey.AsRawKey()) + + h.log("sending 4MiB put request to post-merge range") + h.put(ctx, roachpb.Key(merged.StartKey), 4<<20 /* 4MiB */, admissionpb.NormalPri) + h.log("sent 4MiB put request to post-merged range") + + h.comment(` +-- Flow token metrics from n1 after issuing 4MiB of regular replicated writes to +-- the post-merged range. We should see 12MiB extra tokens deducted which comes +-- from 4MiB*3=12MiB. So we stand at 21MiB+12MiB=33MiB tokens deducted now. The +-- RHS of the range is gone now, and the previously 3*3MiB=9MiB of tokens +-- deducted for it are released at the subsuming LHS leaseholder. +`) + h.query(n1, ` + SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvadmission%regular_tokens%' +ORDER BY name ASC; +`) + + h.comment(`-- Observe only the merged replica with its own three streams.`) + h.query(n1, ` + SELECT range_id, count(*) AS streams + FROM crdb_internal.kv_flow_control_handles +GROUP BY (range_id) +ORDER BY streams DESC; +`, "range_id", "stream_count") + + h.comment(`-- (Allow below-raft admission to proceed.)`) + disableWorkQueueGranting.Store(false) + h.waitForAllTokensReturned(ctx, 3) // wait for admission + + h.comment(` +-- Flow token metrics from n1 after work gets admitted. We see all outstanding +-- {regular,elastic} tokens returned, including those from: +-- - the LHS before the merge, and +-- - the LHS and RHS before the original split. +`) + h.query(n1, ` + SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvadmission%tokens%' +ORDER BY name ASC; +`) +} + // TestFlowControlCrashedNode tests flow token behavior in the presence of // crashed nodes. func TestFlowControlCrashedNode(t *testing.T) { diff --git a/pkg/kv/kvserver/testdata/flow_control_integration/admission_post_split_merge b/pkg/kv/kvserver/testdata/flow_control_integration/admission_post_split_merge new file mode 100644 index 000000000000..67b66820ff84 --- /dev/null +++ b/pkg/kv/kvserver/testdata/flow_control_integration/admission_post_split_merge @@ -0,0 +1,110 @@ +echo +---- +---- +-- Flow token metrics from n1 after issuing a regular 2*1MiB 3x replicated write +-- that are yet to get admitted. We see 2*3*1MiB=6MiB deductions of +-- {regular,elastic} tokens with no corresponding returns. The 2*1MiB writes +-- happened on what is soon going to be the LHS and RHS of a range being split. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvadmission%tokens%' +ORDER BY name ASC; + + kvadmission.flow_controller.elastic_tokens_available | 18 MiB + kvadmission.flow_controller.elastic_tokens_deducted | 6.0 MiB + kvadmission.flow_controller.elastic_tokens_returned | 0 B + kvadmission.flow_controller.elastic_tokens_unaccounted | 0 B + kvadmission.flow_controller.regular_tokens_available | 42 MiB + kvadmission.flow_controller.regular_tokens_deducted | 6.0 MiB + kvadmission.flow_controller.regular_tokens_returned | 0 B + kvadmission.flow_controller.regular_tokens_unaccounted | 0 B + + +-- (Splitting range.) + + +-- Flow token metrics from n1 after further issuing 2MiB and 3MiB writes to +-- post-split LHS and RHS ranges respectively. We should see 15MiB extra tokens +-- deducted which comes from (2MiB+3MiB)*3=15MiB. So we stand at +-- 6MiB+15MiB=21MiB now. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvadmission%tokens%' +ORDER BY name ASC; + + kvadmission.flow_controller.elastic_tokens_available | 3.0 MiB + kvadmission.flow_controller.elastic_tokens_deducted | 21 MiB + kvadmission.flow_controller.elastic_tokens_returned | 0 B + kvadmission.flow_controller.elastic_tokens_unaccounted | 0 B + kvadmission.flow_controller.regular_tokens_available | 27 MiB + kvadmission.flow_controller.regular_tokens_deducted | 21 MiB + kvadmission.flow_controller.regular_tokens_returned | 0 B + kvadmission.flow_controller.regular_tokens_unaccounted | 0 B + + +-- Observe the newly split off replica, with its own three streams. +SELECT range_id, count(*) AS streams + FROM crdb_internal.kv_flow_control_handles +GROUP BY (range_id) +ORDER BY streams DESC; + + range_id | stream_count +-----------+--------------- + 64 | 3 + 65 | 3 + + +-- (Merging ranges.) + + +-- Flow token metrics from n1 after issuing 4MiB of regular replicated writes to +-- the post-merged range. We should see 12MiB extra tokens deducted which comes +-- from 4MiB*3=12MiB. So we stand at 21MiB+12MiB=33MiB tokens deducted now. The +-- RHS of the range is gone now, and the previously 3*3MiB=9MiB of tokens +-- deducted for it are released at the subsuming LHS leaseholder. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvadmission%regular_tokens%' +ORDER BY name ASC; + + kvadmission.flow_controller.regular_tokens_available | 24 MiB + kvadmission.flow_controller.regular_tokens_deducted | 33 MiB + kvadmission.flow_controller.regular_tokens_returned | 9.0 MiB + kvadmission.flow_controller.regular_tokens_unaccounted | 0 B + + +-- Observe only the merged replica with its own three streams. +SELECT range_id, count(*) AS streams + FROM crdb_internal.kv_flow_control_handles +GROUP BY (range_id) +ORDER BY streams DESC; + + range_id | stream_count +-----------+--------------- + 64 | 3 + + +-- (Allow below-raft admission to proceed.) + + +-- Flow token metrics from n1 after work gets admitted. We see all outstanding +-- {regular,elastic} tokens returned, including those from: +-- - the LHS before the merge, and +-- - the LHS and RHS before the original split. +SELECT name, crdb_internal.humanize_bytes(value::INT8) + FROM crdb_internal.node_metrics + WHERE name LIKE '%kvadmission%tokens%' +ORDER BY name ASC; + + kvadmission.flow_controller.elastic_tokens_available | 24 MiB + kvadmission.flow_controller.elastic_tokens_deducted | 33 MiB + kvadmission.flow_controller.elastic_tokens_returned | 33 MiB + kvadmission.flow_controller.elastic_tokens_unaccounted | 0 B + kvadmission.flow_controller.regular_tokens_available | 48 MiB + kvadmission.flow_controller.regular_tokens_deducted | 33 MiB + kvadmission.flow_controller.regular_tokens_returned | 33 MiB + kvadmission.flow_controller.regular_tokens_unaccounted | 0 B +---- +---- + +# vim:ft=sql From d352355e8d7caca7920772f1d241eaf222947d96 Mon Sep 17 00:00:00 2001 From: irfan sharif Date: Tue, 6 Jun 2023 18:07:02 -0400 Subject: [PATCH 09/12] kvflowcontrol: disable by default Disable kvadmission.flow_control.enabled by default. We'll re-enable it on master shortly after some baking time while it's switched off. We need to ensure that there are zero performance regressions when switched off, and that the integration code does not exercise new machinery when turned off. Merging this as turned-off-by-default also reduces revert likelihood. Release note: None --- .../kvserver/flow_control_integration_test.go | 47 ++++++++++++++++++- .../kvserver/kvflowcontrol/kvflowcontrol.go | 2 +- 2 files changed, 47 insertions(+), 2 deletions(-) diff --git a/pkg/kv/kvserver/flow_control_integration_test.go b/pkg/kv/kvserver/flow_control_integration_test.go index af262a08821c..e03972f38a30 100644 --- a/pkg/kv/kvserver/flow_control_integration_test.go +++ b/pkg/kv/kvserver/flow_control_integration_test.go @@ -380,9 +380,13 @@ func TestFlowControlBlockedAdmission(t *testing.T) { var disableWorkQueueGranting atomic.Bool disableWorkQueueGranting.Store(true) + st := cluster.MakeTestingClusterSettings() + kvflowcontrol.Enabled.Override(ctx, &st.SV, true) + tc := testcluster.StartTestCluster(t, numNodes, base.TestClusterArgs{ ReplicationMode: base.ReplicationManual, ServerArgs: base.TestServerArgs{ + Settings: st, Knobs: base.TestingKnobs{ Store: &kvserver.StoreTestingKnobs{ FlowControlTestingKnobs: &kvflowcontrol.TestingKnobs{ @@ -495,9 +499,13 @@ func TestFlowControlAdmissionPostSplitMerge(t *testing.T) { var disableWorkQueueGranting atomic.Bool disableWorkQueueGranting.Store(true) + st := cluster.MakeTestingClusterSettings() + kvflowcontrol.Enabled.Override(ctx, &st.SV, true) + tc := testcluster.StartTestCluster(t, numNodes, base.TestClusterArgs{ ReplicationMode: base.ReplicationManual, ServerArgs: base.TestServerArgs{ + Settings: st, Knobs: base.TestingKnobs{ Store: &kvserver.StoreTestingKnobs{ FlowControlTestingKnobs: &kvflowcontrol.TestingKnobs{ @@ -644,6 +652,7 @@ func TestFlowControlCrashedNode(t *testing.T) { // we only check the last-updated state when ticked. So we disable range // quiescence. kvserver.ExpirationLeasesOnly.Override(ctx, &st.SV, true) + kvflowcontrol.Enabled.Override(ctx, &st.SV, true) tc := testcluster.StartTestCluster(t, numNodes, base.TestClusterArgs{ ReplicationMode: base.ReplicationManual, @@ -765,8 +774,13 @@ func TestFlowControlRaftSnapshot(t *testing.T) { var disableWorkQueueGranting atomic.Bool disableWorkQueueGranting.Store(true) + ctx := context.Background() + st := cluster.MakeTestingClusterSettings() + kvflowcontrol.Enabled.Override(ctx, &st.SV, true) + for i := 0; i < numServers; i++ { stickyServerArgs[i] = base.TestServerArgs{ + Settings: st, StoreSpecs: []base.StoreSpec{ { InMemory: true, @@ -820,7 +834,6 @@ func TestFlowControlRaftSnapshot(t *testing.T) { } } - ctx := context.Background() tc := testcluster.StartTestCluster(t, numServers, base.TestClusterArgs{ ReplicationMode: base.ReplicationManual, @@ -1064,9 +1077,13 @@ func TestFlowControlRaftTransportBreak(t *testing.T) { const numNodes = 3 var maintainStreamsForInactiveFollowers atomic.Bool + st := cluster.MakeTestingClusterSettings() + kvflowcontrol.Enabled.Override(ctx, &st.SV, true) + tc := testcluster.StartTestCluster(t, numNodes, base.TestClusterArgs{ ReplicationMode: base.ReplicationManual, ServerArgs: base.TestServerArgs{ + Settings: st, RaftConfig: base.RaftConfig{ // Suppress timeout-based elections. This test doesn't want to // deal with leadership changing hands. @@ -1179,7 +1196,11 @@ func TestFlowControlRaftTransportCulled(t *testing.T) { markSendQueueAsIdleCh := make(chan roachpb.NodeID) var disableWorkerTeardown atomic.Bool + st := cluster.MakeTestingClusterSettings() + kvflowcontrol.Enabled.Override(ctx, &st.SV, true) + baseServerArgs := base.TestServerArgs{ + Settings: st, Knobs: base.TestingKnobs{ Store: &kvserver.StoreTestingKnobs{ FlowControlTestingKnobs: &kvflowcontrol.TestingKnobs{ @@ -1313,12 +1334,16 @@ func TestFlowControlRaftMembership(t *testing.T) { defer log.Scope(t).Close(t) ctx := context.Background() + st := cluster.MakeTestingClusterSettings() + kvflowcontrol.Enabled.Override(ctx, &st.SV, true) + const numNodes = 5 var disableWorkQueueGranting atomic.Bool disableWorkQueueGranting.Store(true) tc := testcluster.StartTestCluster(t, numNodes, base.TestClusterArgs{ ReplicationMode: base.ReplicationManual, ServerArgs: base.TestServerArgs{ + Settings: st, Knobs: base.TestingKnobs{ Store: &kvserver.StoreTestingKnobs{ FlowControlTestingKnobs: &kvflowcontrol.TestingKnobs{ @@ -1442,12 +1467,16 @@ func TestFlowControlRaftMembershipRemoveSelf(t *testing.T) { testutils.RunTrueAndFalse(t, "transfer-lease-first", func(t *testing.T, transferLeaseFirst bool) { ctx := context.Background() + st := cluster.MakeTestingClusterSettings() + kvflowcontrol.Enabled.Override(ctx, &st.SV, true) + const numNodes = 4 var disableWorkQueueGranting atomic.Bool disableWorkQueueGranting.Store(true) tc := testcluster.StartTestCluster(t, numNodes, base.TestClusterArgs{ ReplicationMode: base.ReplicationManual, ServerArgs: base.TestServerArgs{ + Settings: st, RaftConfig: base.RaftConfig{ // TODO(irfansharif): The AdminRelocateRange used below can // occasionally flake if we suppress timeout-based @@ -1576,9 +1605,13 @@ func TestFlowControlClassPrioritization(t *testing.T) { var disableWorkQueueGranting atomic.Bool disableWorkQueueGranting.Store(true) + st := cluster.MakeTestingClusterSettings() + kvflowcontrol.Enabled.Override(ctx, &st.SV, true) + tc := testcluster.StartTestCluster(t, numNodes, base.TestClusterArgs{ ReplicationMode: base.ReplicationManual, ServerArgs: base.TestServerArgs{ + Settings: st, Knobs: base.TestingKnobs{ Store: &kvserver.StoreTestingKnobs{ FlowControlTestingKnobs: &kvflowcontrol.TestingKnobs{ @@ -1671,6 +1704,7 @@ func TestFlowControlQuiescedRange(t *testing.T) { st := cluster.MakeTestingClusterSettings() kvserver.ExpirationLeasesOnly.Override(ctx, &st.SV, false) // override metamorphism + kvflowcontrol.Enabled.Override(ctx, &st.SV, true) tc := testcluster.StartTestCluster(t, numNodes, base.TestClusterArgs{ ReplicationMode: base.ReplicationManual, @@ -1807,6 +1841,7 @@ func TestFlowControlUnquiescedRange(t *testing.T) { st := cluster.MakeTestingClusterSettings() kvserver.ExpirationLeasesOnly.Override(ctx, &st.SV, false) // override metamorphism + kvflowcontrol.Enabled.Override(ctx, &st.SV, true) tc := testcluster.StartTestCluster(t, numNodes, base.TestClusterArgs{ ReplicationMode: base.ReplicationManual, @@ -1966,9 +2001,14 @@ func TestFlowControlTransferLease(t *testing.T) { const numNodes = 5 var disableWorkQueueGranting atomic.Bool disableWorkQueueGranting.Store(true) + + st := cluster.MakeTestingClusterSettings() + kvflowcontrol.Enabled.Override(ctx, &st.SV, true) + tc := testcluster.StartTestCluster(t, numNodes, base.TestClusterArgs{ ReplicationMode: base.ReplicationManual, ServerArgs: base.TestServerArgs{ + Settings: st, Knobs: base.TestingKnobs{ Store: &kvserver.StoreTestingKnobs{ FlowControlTestingKnobs: &kvflowcontrol.TestingKnobs{ @@ -2046,9 +2086,14 @@ func TestFlowControlLeaderNotLeaseholder(t *testing.T) { const numNodes = 5 var disableWorkQueueGranting atomic.Bool disableWorkQueueGranting.Store(true) + + st := cluster.MakeTestingClusterSettings() + kvflowcontrol.Enabled.Override(ctx, &st.SV, true) + tc := testcluster.StartTestCluster(t, numNodes, base.TestClusterArgs{ ReplicationMode: base.ReplicationManual, ServerArgs: base.TestServerArgs{ + Settings: st, Knobs: base.TestingKnobs{ Store: &kvserver.StoreTestingKnobs{ // Disable leader transfers during leaseholder changes so diff --git a/pkg/kv/kvserver/kvflowcontrol/kvflowcontrol.go b/pkg/kv/kvserver/kvflowcontrol/kvflowcontrol.go index 7e8c5d9fdb32..875636b47546 100644 --- a/pkg/kv/kvserver/kvflowcontrol/kvflowcontrol.go +++ b/pkg/kv/kvserver/kvflowcontrol/kvflowcontrol.go @@ -30,7 +30,7 @@ var Enabled = settings.RegisterBoolSetting( settings.SystemOnly, "kvadmission.flow_control.enabled", "determines whether we use flow control for replication traffic in KV", - true, + false, ) // Mode determines the 'mode' of flow control we use for replication traffic in From a5296ac4b173289e3aff2b1618595e86c78eb692 Mon Sep 17 00:00:00 2001 From: irfan sharif Date: Thu, 8 Jun 2023 22:28:33 -0400 Subject: [PATCH 10/12] bors: update timeout to 1h20m We've seen the per-PR stress jobs take upwards of 55m and be successful. This happens for PRs with a wide surface area. Release note: None --- .github/bors.toml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/.github/bors.toml b/.github/bors.toml index 58ff381edf2a..c08e72fde8be 100644 --- a/.github/bors.toml +++ b/.github/bors.toml @@ -16,8 +16,8 @@ block_labels = ["do-not-merge"] # Number of seconds from when a merge commit is created to when its statuses # must pass. # -# Set to 40 minutes -timeout_sec = 2400 +# Set to 1 hour 20 minutes. +timeout_sec = 4800 required_approvals = 1 [committer] From 765c46425732f7213025084d5aaf3ae3e829abca Mon Sep 17 00:00:00 2001 From: irfan sharif Date: Thu, 8 Jun 2023 23:04:16 -0400 Subject: [PATCH 11/12] backupccl: skip TestIngest/batch=default It's flakey. See #104639. Release note: None --- pkg/ccl/backupccl/restore_data_processor_test.go | 3 +++ 1 file changed, 3 insertions(+) diff --git a/pkg/ccl/backupccl/restore_data_processor_test.go b/pkg/ccl/backupccl/restore_data_processor_test.go index f770a65f7f8d..9d25c9efa050 100644 --- a/pkg/ccl/backupccl/restore_data_processor_test.go +++ b/pkg/ccl/backupccl/restore_data_processor_test.go @@ -43,6 +43,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" + "github.com/cockroachdb/cockroach/pkg/testutils/skip" "github.com/cockroachdb/cockroach/pkg/util/encoding" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/leaktest" @@ -153,6 +154,8 @@ func TestIngest(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) + skip.WithIssue(t, 104639, "flaky test") + ctx := context.Background() t.Run("batch=default", func(t *testing.T) { runTestIngest(t, func(_ *cluster.Settings) {}) From 0a6aee0af97c9477f0eeb796c38032222ff6c65b Mon Sep 17 00:00:00 2001 From: irfan sharif Date: Thu, 8 Jun 2023 23:38:10 -0400 Subject: [PATCH 12/12] admission: squash datarace in (*StoreWorkqueue).gcSequencers() We see this (benign) data race in under --stress --race. Squash it by targeted use of atomics. Read at 0x00c0039c06a8 by goroutine 31161: github.com/cockroachdb/cockroach/pkg/util/admission.(*StoreWorkQueue).gcSequencers() github.com/cockroachdb/cockroach/pkg/util/admission/work_queue.go:2176 +0x196 github.com/cockroachdb/cockroach/pkg/util/admission.makeStoreWorkQueue.func1() github.com/cockroachdb/cockroach/pkg/util/admission/work_queue.go:2162 +0x5b Previous write at 0x00c0039c06a8 by goroutine 31105: github.com/cockroachdb/cockroach/pkg/util/admission.(*sequencer).sequence() github.com/cockroachdb/cockroach/pkg/util/admission/sequencer.go:61 +0x190 github.com/cockroachdb/cockroach/pkg/util/admission.(*StoreWorkQueue).sequenceReplicatedWork() github.com/cockroachdb/cockroach/pkg/util/admission/work_queue.go:2193 +0x145 github.com/cockroachdb/cockroach/pkg/util/admission.(*StoreWorkQueue).Admit() Release note: None --- pkg/util/admission/sequencer.go | 6 +++++- pkg/util/admission/work_queue.go | 3 ++- 2 files changed, 7 insertions(+), 2 deletions(-) diff --git a/pkg/util/admission/sequencer.go b/pkg/util/admission/sequencer.go index 05b389cc9310..fe836080b2d8 100644 --- a/pkg/util/admission/sequencer.go +++ b/pkg/util/admission/sequencer.go @@ -10,6 +10,8 @@ package admission +import "sync/atomic" + // sequencer issues monotonic sequencing timestamps derived from observed // CreateTimes. This is a purpose-built data structure for replication admission // control where we want to assign each AC-queued work below-raft a "sequence @@ -58,6 +60,8 @@ func (s *sequencer) sequence(createTime int64) int64 { if createTime <= s.maxCreateTime { createTime = s.maxCreateTime + 1 } - s.maxCreateTime = createTime + // This counter is read concurrently in gcSequencers. We use an atomic store + // here and an atomic load there, to avoid tripping up the race detector. + atomic.StoreInt64(&s.maxCreateTime, createTime) return createTime } diff --git a/pkg/util/admission/work_queue.go b/pkg/util/admission/work_queue.go index 3613a412cd9c..3afd5d7bc458 100644 --- a/pkg/util/admission/work_queue.go +++ b/pkg/util/admission/work_queue.go @@ -17,6 +17,7 @@ import ( "math" "sort" "sync" + "sync/atomic" "time" "github.com/cockroachdb/cockroach/pkg/base" @@ -2173,7 +2174,7 @@ func (q *StoreWorkQueue) gcSequencers() { defer q.sequencersMu.Unlock() for rangeID, seq := range q.sequencersMu.s { - maxCreateTime := timeutil.FromUnixNanos(seq.maxCreateTime) + maxCreateTime := timeutil.FromUnixNanos(atomic.LoadInt64(&seq.maxCreateTime)) if q.timeSource.Now().Sub(maxCreateTime) > rangeSequencerGCThreshold.Get(&q.settings.SV) { delete(q.sequencersMu.s, rangeID) }