From 7ca1c565fd41912dfae85c402f2cb93c4a0ec7ec Mon Sep 17 00:00:00 2001 From: Tobias Grieger Date: Wed, 30 Sep 2020 17:52:42 +0200 Subject: [PATCH 1/5] kvserver: remove unused parameter Release note: None --- pkg/kv/kvserver/raft_transport.go | 10 ++++------ pkg/kv/kvserver/replica_command.go | 1 - pkg/kv/kvserver/replica_sideload_test.go | 2 -- pkg/kv/kvserver/store_snapshot.go | 7 +------ pkg/kv/kvserver/store_test.go | 10 ++++------ 5 files changed, 9 insertions(+), 21 deletions(-) diff --git a/pkg/kv/kvserver/raft_transport.go b/pkg/kv/kvserver/raft_transport.go index a5277f64ea2e..ae438f388687 100644 --- a/pkg/kv/kvserver/raft_transport.go +++ b/pkg/kv/kvserver/raft_transport.go @@ -20,7 +20,6 @@ import ( "time" "unsafe" - "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/rpc" "github.com/cockroachdb/cockroach/pkg/rpc/nodedialer" @@ -653,23 +652,20 @@ func (t *RaftTransport) startProcessNewQueue( // for closing the OutgoingSnapshot. func (t *RaftTransport) SendSnapshot( ctx context.Context, - raftCfg *base.RaftConfig, storePool *StorePool, header SnapshotRequest_Header, snap *OutgoingSnapshot, newBatch func() storage.Batch, sent func(), ) error { - var stream MultiRaft_RaftSnapshotClient nodeID := header.RaftMessageRequest.ToReplica.NodeID conn, err := t.dialer.Dial(ctx, nodeID, rpc.DefaultClass) if err != nil { return err } - client := NewMultiRaftClient(conn) - stream, err = client.RaftSnapshot(ctx) + stream, err := client.RaftSnapshot(ctx) if err != nil { return err } @@ -679,5 +675,7 @@ func (t *RaftTransport) SendSnapshot( log.Warningf(ctx, "failed to close snapshot stream: %+v", err) } }() - return sendSnapshot(ctx, raftCfg, t.st, stream, storePool, header, snap, newBatch, sent) + return sendSnapshot( + ctx, t.st, stream, storePool, header, snap, newBatch, sent, + ) } diff --git a/pkg/kv/kvserver/replica_command.go b/pkg/kv/kvserver/replica_command.go index 533da82098d6..7538a1eb0f38 100644 --- a/pkg/kv/kvserver/replica_command.go +++ b/pkg/kv/kvserver/replica_command.go @@ -1928,7 +1928,6 @@ func (r *Replica) sendSnapshot( } if err := r.store.cfg.Transport.SendSnapshot( ctx, - &r.store.cfg.RaftConfig, r.store.allocator.storePool, req, snap, diff --git a/pkg/kv/kvserver/replica_sideload_test.go b/pkg/kv/kvserver/replica_sideload_test.go index a6922d5013ff..9cbe1cd335d1 100644 --- a/pkg/kv/kvserver/replica_sideload_test.go +++ b/pkg/kv/kvserver/replica_sideload_test.go @@ -825,7 +825,6 @@ func TestRaftSSTableSideloadingSnapshot(t *testing.T) { mockSender := &mockSender{} if err := sendSnapshot( ctx, - &tc.store.cfg.RaftConfig, tc.store.cfg.Settings, mockSender, &fakeStorePool{}, @@ -947,7 +946,6 @@ func TestRaftSSTableSideloadingSnapshot(t *testing.T) { mockSender := &mockSender{} err = sendSnapshot( ctx, - &tc.store.cfg.RaftConfig, tc.store.cfg.Settings, mockSender, &fakeStorePool{}, diff --git a/pkg/kv/kvserver/store_snapshot.go b/pkg/kv/kvserver/store_snapshot.go index c90ffa9f4068..36865204369c 100644 --- a/pkg/kv/kvserver/store_snapshot.go +++ b/pkg/kv/kvserver/store_snapshot.go @@ -16,7 +16,6 @@ import ( "io" "time" - "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/rditer" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings" @@ -91,8 +90,7 @@ func assertStrategy( // kvBatchSnapshotStrategy is an implementation of snapshotStrategy that streams // batches of KV pairs in the BatchRepr format. type kvBatchSnapshotStrategy struct { - raftCfg *base.RaftConfig - status string + status string // The size of the batches of PUT operations to send to the receiver of the // snapshot. Only used on the sender side. @@ -781,7 +779,6 @@ func (s *Store) receiveSnapshot( } ss = &kvBatchSnapshotStrategy{ - raftCfg: &s.cfg.RaftConfig, scratch: s.sstSnapshotStorage.NewScratchSpace(header.State.Desc.RangeID, snapUUID), sstChunkSize: snapshotSSTWriteSyncRate.Get(&s.cfg.Settings.SV), } @@ -901,7 +898,6 @@ func (e *errMustRetrySnapshotDueToTruncation) Error() string { // sendSnapshot sends an outgoing snapshot via a pre-opened GRPC stream. func sendSnapshot( ctx context.Context, - raftCfg *base.RaftConfig, st *cluster.Settings, stream outgoingSnapshotStream, storePool SnapshotStorePool, @@ -976,7 +972,6 @@ func sendSnapshot( switch header.Strategy { case SnapshotRequest_KV_BATCH: ss = &kvBatchSnapshotStrategy{ - raftCfg: raftCfg, batchSize: batchSize, limiter: limiter, newBatch: newBatch, diff --git a/pkg/kv/kvserver/store_test.go b/pkg/kv/kvserver/store_test.go index faa190f775ac..5e4c50a8d4c7 100644 --- a/pkg/kv/kvserver/store_test.go +++ b/pkg/kv/kvserver/store_test.go @@ -3059,8 +3059,6 @@ func TestSendSnapshotThrottling(t *testing.T) { defer e.Close() ctx := context.Background() - var cfg base.RaftConfig - cfg.SetDefaults() st := cluster.MakeTestingClusterSettings() header := SnapshotRequest_Header{ @@ -3076,7 +3074,7 @@ func TestSendSnapshotThrottling(t *testing.T) { sp := &fakeStorePool{} expectedErr := errors.New("") c := fakeSnapshotStream{nil, expectedErr} - err := sendSnapshot(ctx, &cfg, st, c, sp, header, nil, newBatch, nil) + err := sendSnapshot(ctx, st, c, sp, header, nil, newBatch, nil) if sp.failedThrottles != 1 { t.Fatalf("expected 1 failed throttle, but found %d", sp.failedThrottles) } @@ -3092,7 +3090,7 @@ func TestSendSnapshotThrottling(t *testing.T) { Status: SnapshotResponse_DECLINED, } c := fakeSnapshotStream{resp, nil} - err := sendSnapshot(ctx, &cfg, st, c, sp, header, nil, newBatch, nil) + err := sendSnapshot(ctx, st, c, sp, header, nil, newBatch, nil) if sp.declinedThrottles != 1 { t.Fatalf("expected 1 declined throttle, but found %d", sp.declinedThrottles) } @@ -3109,7 +3107,7 @@ func TestSendSnapshotThrottling(t *testing.T) { Status: SnapshotResponse_DECLINED, } c := fakeSnapshotStream{resp, nil} - err := sendSnapshot(ctx, &cfg, st, c, sp, header, nil, newBatch, nil) + err := sendSnapshot(ctx, st, c, sp, header, nil, newBatch, nil) if sp.failedThrottles != 1 { t.Fatalf("expected 1 failed throttle, but found %d", sp.failedThrottles) } @@ -3125,7 +3123,7 @@ func TestSendSnapshotThrottling(t *testing.T) { Status: SnapshotResponse_ERROR, } c := fakeSnapshotStream{resp, nil} - err := sendSnapshot(ctx, &cfg, st, c, sp, header, nil, newBatch, nil) + err := sendSnapshot(ctx, st, c, sp, header, nil, newBatch, nil) if sp.failedThrottles != 1 { t.Fatalf("expected 1 failed throttle, but found %d", sp.failedThrottles) } From 7abe7d7df800b9d2b852c651d3c58206ab4b4730 Mon Sep 17 00:00:00 2001 From: Tobias Grieger Date: Wed, 7 Oct 2020 10:24:38 +0200 Subject: [PATCH 2/5] server: create engines in NewServer We were jumping through a number of hoops to create the engines only in `(*Server).Start` since that seems to be the "idiomatic" place to start moving parts. However, it creates a lot of complexity since various callbacks have to be registered with access to engines. Move engine creation to `NewServer`. This unblocks #54936. Release note: None --- pkg/server/server.go | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/pkg/server/server.go b/pkg/server/server.go index 69f089afdd0c..5ffc5446b58e 100644 --- a/pkg/server/server.go +++ b/pkg/server/server.go @@ -123,6 +123,7 @@ type Server struct { mux http.ServeMux clock *hlc.Clock rpcContext *rpc.Context + engines Engines // The gRPC server on which the different RPC handlers will be registered. grpc *grpcServer gossip *gossip.Gossip @@ -158,9 +159,7 @@ type Server struct { externalStorageBuilder *externalStorageBuilder // The following fields are populated at start time, i.e. in `(*Server).Start`. - startTime time.Time - engines Engines } // externalStorageBuilder is a wrapper around the ExternalStorage factory @@ -597,12 +596,19 @@ func NewServer(cfg Config, stopper *stop.Stopper) (*Server, error) { debugServer := debug.NewServer(st, sqlServer.pgServer.HBADebugFn()) node.InitLogger(sqlServer.execCfg) + engines, err := cfg.CreateEngines(ctx) + if err != nil { + return nil, errors.Wrap(err, "failed to create engines") + } + stopper.AddCloser(&engines) + *lateBoundServer = Server{ nodeIDContainer: nodeIDContainer, cfg: cfg, st: st, clock: clock, rpcContext: rpcContext, + engines: engines, grpc: grpcServer, gossip: g, nodeDialer: nodeDialer, @@ -1048,12 +1054,6 @@ func (s *Server) PreStart(ctx context.Context) error { return err } - s.engines, err = s.cfg.CreateEngines(ctx) - if err != nil { - return errors.Wrap(err, "failed to create engines") - } - s.stopper.AddCloser(&s.engines) - // Initialize the external storage builders configuration params now that the // engines have been created. The object can be used to create ExternalStorage // objects hereafter. From 18be6a78a90a996940cc33db5ec54f9483fc3bf9 Mon Sep 17 00:00:00 2001 From: Raphael 'kena' Poss Date: Wed, 7 Oct 2020 14:06:43 +0200 Subject: [PATCH 3/5] roachtest: recognize GEOS dlls on more platforms This makes roachtest work on the BSDs again. Release note: None --- pkg/cmd/roachtest/cluster.go | 3 +++ 1 file changed, 3 insertions(+) diff --git a/pkg/cmd/roachtest/cluster.go b/pkg/cmd/roachtest/cluster.go index fc0f633b7243..25e3b12e9ee1 100644 --- a/pkg/cmd/roachtest/cluster.go +++ b/pkg/cmd/roachtest/cluster.go @@ -136,6 +136,9 @@ func findLibrary(libraryName string) (string, error) { if local { switch runtime.GOOS { case "linux": + case "freebsd": + case "openbsd": + case "dragonfly": case "windows": suffix = ".dll" case "darwin": From e587e13b623c83e77963049d58fb7cb1848acba8 Mon Sep 17 00:00:00 2001 From: arulajmani Date: Sat, 3 Oct 2020 20:45:54 -0400 Subject: [PATCH 4/5] sql: fix performance regression due to increased hash allocations MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit We recently increased allocations in the `appStats` structure when adding support for transaction level statistics in #52704. This was because the interactions with the `fnv` library were expensive in terms of allocations. This patch aims to claw back the regression by: - Using our own implementation of the FNV algorithm instead of the library, which is significantly lighter weight (microbenchmarks below). - Reorganizing the code to only construct a statement ID from scratch - if required. When comparing the difference between the commit that introduced the regression and the changes proposed by this diff, I got the following improvements on the KV workload: ``` name old ops/s new ops/s delta kv95-throughput 34.5k ± 6% 35.7k ± 4% +3.42% (p=0.023 n=10+10) ``` Microbenchmarks for the new hashing algorithm (written/run by @knz): ``` name old time/op new time/op delta ConstructStatementID-32 405ns ±17% 39ns ±12% -90.34% (p=0.008 n=5+5) name old alloc/op new alloc/op delta ConstructStatementID-32 120B ± 0% 16B ± 0% -86.67% (p=0.008 n=5+5) name old allocs/op new allocs/op delta ConstructStatementID-32 6.00 ± 0% 1.00 ± 0% -83.33% (p=0.008 n=5+5) ``` Closes #54515 Release note: None --- docs/generated/http/full.md | 2 +- pkg/roachpb/app_stats.go | 22 +- pkg/roachpb/app_stats.pb.go | 167 ++++--- pkg/roachpb/app_stats.proto | 4 +- pkg/server/serverpb/status.pb.go | 410 +++++++++--------- pkg/server/serverpb/status.proto | 6 +- pkg/server/status_test.go | 2 +- pkg/sql/app_stats.go | 60 ++- pkg/sql/conn_executor.go | 3 +- pkg/sql/conn_executor_exec.go | 10 +- pkg/sql/crdb_internal.go | 8 +- pkg/sql/executor_statement_metrics.go | 6 +- .../testdata/logic_test/crdb_internal | 12 +- pkg/util/hash.go | 32 ++ 14 files changed, 403 insertions(+), 341 deletions(-) diff --git a/docs/generated/http/full.md b/docs/generated/http/full.md index 59e3f5c6ddf0..b17a14af6f15 100644 --- a/docs/generated/http/full.md +++ b/docs/generated/http/full.md @@ -1813,7 +1813,7 @@ information about the resources on a node used by that table. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | key | [StatementsResponse.ExtendedStatementStatisticsKey](#cockroach.server.serverpb.StatementsResponse-cockroach.server.serverpb.StatementsResponse.ExtendedStatementStatisticsKey) | | | -| id | [string](#cockroach.server.serverpb.StatementsResponse-string) | | | +| id | [uint64](#cockroach.server.serverpb.StatementsResponse-uint64) | | | | stats | [cockroach.sql.StatementStatistics](#cockroach.server.serverpb.StatementsResponse-cockroach.sql.StatementStatistics) | | | diff --git a/pkg/roachpb/app_stats.go b/pkg/roachpb/app_stats.go index 0967a66838e9..70096ebca5af 100644 --- a/pkg/roachpb/app_stats.go +++ b/pkg/roachpb/app_stats.go @@ -11,28 +11,34 @@ package roachpb import ( - "fmt" - "hash/fnv" "math" + + "github.com/cockroachdb/cockroach/pkg/util" ) // StmtID is the type of a Statement ID. -type StmtID string +type StmtID uint64 // ConstructStatementID constructs an ID by hashing an anonymized query, it's // failure status, and if it was part of an implicit txn. At the time of writing, // these are the axis' we use to bucket queries for stats collection // (see stmtKey). func ConstructStatementID(anonymizedStmt string, failed bool, implicitTxn bool) StmtID { - h := fnv.New128() - h.Write([]byte(anonymizedStmt)) + fnv := util.MakeFNV64() + for _, c := range anonymizedStmt { + fnv.Add(uint64(c)) + } if failed { - h.Write([]byte("failed")) + fnv.Add('F') + } else { + fnv.Add('S') } if implicitTxn { - h.Write([]byte("implicit_txn")) + fnv.Add('I') + } else { + fnv.Add('E') } - return StmtID(fmt.Sprintf("%x", h.Sum(nil))) + return StmtID(fnv.Sum()) } // GetVariance retrieves the variance of the values. diff --git a/pkg/roachpb/app_stats.pb.go b/pkg/roachpb/app_stats.pb.go index f4fb111c18a2..e3106609cbd7 100644 --- a/pkg/roachpb/app_stats.pb.go +++ b/pkg/roachpb/app_stats.pb.go @@ -85,7 +85,7 @@ func (m *StatementStatistics) Reset() { *m = StatementStatistics{} } func (m *StatementStatistics) String() string { return proto.CompactTextString(m) } func (*StatementStatistics) ProtoMessage() {} func (*StatementStatistics) Descriptor() ([]byte, []int) { - return fileDescriptor_app_stats_ba11ef7e1d77017c, []int{0} + return fileDescriptor_app_stats_6c6ddfe12bf47ebd, []int{0} } func (m *StatementStatistics) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -134,7 +134,7 @@ func (m *TransactionStatistics) Reset() { *m = TransactionStatistics{} } func (m *TransactionStatistics) String() string { return proto.CompactTextString(m) } func (*TransactionStatistics) ProtoMessage() {} func (*TransactionStatistics) Descriptor() ([]byte, []int) { - return fileDescriptor_app_stats_ba11ef7e1d77017c, []int{1} + return fileDescriptor_app_stats_6c6ddfe12bf47ebd, []int{1} } func (m *TransactionStatistics) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -174,7 +174,7 @@ func (m *SensitiveInfo) Reset() { *m = SensitiveInfo{} } func (m *SensitiveInfo) String() string { return proto.CompactTextString(m) } func (*SensitiveInfo) ProtoMessage() {} func (*SensitiveInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_app_stats_ba11ef7e1d77017c, []int{2} + return fileDescriptor_app_stats_6c6ddfe12bf47ebd, []int{2} } func (m *SensitiveInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -216,7 +216,7 @@ func (m *NumericStat) Reset() { *m = NumericStat{} } func (m *NumericStat) String() string { return proto.CompactTextString(m) } func (*NumericStat) ProtoMessage() {} func (*NumericStat) Descriptor() ([]byte, []int) { - return fileDescriptor_app_stats_ba11ef7e1d77017c, []int{3} + return fileDescriptor_app_stats_6c6ddfe12bf47ebd, []int{3} } func (m *NumericStat) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -255,7 +255,7 @@ func (m *StatementStatisticsKey) Reset() { *m = StatementStatisticsKey{} func (m *StatementStatisticsKey) String() string { return proto.CompactTextString(m) } func (*StatementStatisticsKey) ProtoMessage() {} func (*StatementStatisticsKey) Descriptor() ([]byte, []int) { - return fileDescriptor_app_stats_ba11ef7e1d77017c, []int{4} + return fileDescriptor_app_stats_6c6ddfe12bf47ebd, []int{4} } func (m *StatementStatisticsKey) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -286,7 +286,7 @@ type CollectedStatementStatistics struct { // ID is a hash of the statement key (query fingerprint, failure status, // implicit txn or not) which can be used to identify the statement // for instance in transaction statistics. - ID StmtID `protobuf:"bytes,3,opt,name=id,casttype=StmtID" json:"id"` + ID StmtID `protobuf:"varint,3,opt,name=id,casttype=StmtID" json:"id"` Key StatementStatisticsKey `protobuf:"bytes,1,opt,name=key" json:"key"` Stats StatementStatistics `protobuf:"bytes,2,opt,name=stats" json:"stats"` } @@ -295,7 +295,7 @@ func (m *CollectedStatementStatistics) Reset() { *m = CollectedStatement func (m *CollectedStatementStatistics) String() string { return proto.CompactTextString(m) } func (*CollectedStatementStatistics) ProtoMessage() {} func (*CollectedStatementStatistics) Descriptor() ([]byte, []int) { - return fileDescriptor_app_stats_ba11ef7e1d77017c, []int{5} + return fileDescriptor_app_stats_6c6ddfe12bf47ebd, []int{5} } func (m *CollectedStatementStatistics) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -325,7 +325,7 @@ var xxx_messageInfo_CollectedStatementStatistics proto.InternalMessageInfo type CollectedTransactionStatistics struct { // StatementIDs is the IDs of the statements which this transaction comprises, // in order. - StatementIDs []StmtID `protobuf:"bytes,1,rep,name=statement_ids,json=statementIds,casttype=StmtID" json:"statement_ids,omitempty"` + StatementIDs []StmtID `protobuf:"varint,1,rep,name=statement_ids,json=statementIds,casttype=StmtID" json:"statement_ids,omitempty"` // App is the name of the app which executed the transaction. App string `protobuf:"bytes,2,opt,name=app" json:"app"` Stats TransactionStatistics `protobuf:"bytes,3,opt,name=stats" json:"stats"` @@ -335,7 +335,7 @@ func (m *CollectedTransactionStatistics) Reset() { *m = CollectedTransac func (m *CollectedTransactionStatistics) String() string { return proto.CompactTextString(m) } func (*CollectedTransactionStatistics) ProtoMessage() {} func (*CollectedTransactionStatistics) Descriptor() ([]byte, []int) { - return fileDescriptor_app_stats_ba11ef7e1d77017c, []int{6} + return fileDescriptor_app_stats_6c6ddfe12bf47ebd, []int{6} } func (m *CollectedTransactionStatistics) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -375,7 +375,7 @@ func (m *ExplainTreePlanNode) Reset() { *m = ExplainTreePlanNode{} } func (m *ExplainTreePlanNode) String() string { return proto.CompactTextString(m) } func (*ExplainTreePlanNode) ProtoMessage() {} func (*ExplainTreePlanNode) Descriptor() ([]byte, []int) { - return fileDescriptor_app_stats_ba11ef7e1d77017c, []int{7} + return fileDescriptor_app_stats_6c6ddfe12bf47ebd, []int{7} } func (m *ExplainTreePlanNode) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -409,7 +409,7 @@ func (m *ExplainTreePlanNode_Attr) Reset() { *m = ExplainTreePlanNode_At func (m *ExplainTreePlanNode_Attr) String() string { return proto.CompactTextString(m) } func (*ExplainTreePlanNode_Attr) ProtoMessage() {} func (*ExplainTreePlanNode_Attr) Descriptor() ([]byte, []int) { - return fileDescriptor_app_stats_ba11ef7e1d77017c, []int{7, 0} + return fileDescriptor_app_stats_6c6ddfe12bf47ebd, []int{7, 0} } func (m *ExplainTreePlanNode_Attr) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -448,7 +448,7 @@ func (m *TxnStats) Reset() { *m = TxnStats{} } func (m *TxnStats) String() string { return proto.CompactTextString(m) } func (*TxnStats) ProtoMessage() {} func (*TxnStats) Descriptor() ([]byte, []int) { - return fileDescriptor_app_stats_ba11ef7e1d77017c, []int{8} + return fileDescriptor_app_stats_6c6ddfe12bf47ebd, []int{8} } func (m *TxnStats) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -908,10 +908,9 @@ func (m *CollectedStatementStatistics) MarshalTo(dAtA []byte) (int, error) { return 0, err } i += n17 - dAtA[i] = 0x1a + dAtA[i] = 0x18 i++ - i = encodeVarintAppStats(dAtA, i, uint64(len(m.ID))) - i += copy(dAtA[i:], m.ID) + i = encodeVarintAppStats(dAtA, i, uint64(m.ID)) return i, nil } @@ -931,18 +930,10 @@ func (m *CollectedTransactionStatistics) MarshalTo(dAtA []byte) (int, error) { var l int _ = l if len(m.StatementIDs) > 0 { - for _, s := range m.StatementIDs { - dAtA[i] = 0xa + for _, num := range m.StatementIDs { + dAtA[i] = 0x8 i++ - l = len(s) - for l >= 1<<7 { - dAtA[i] = uint8(uint64(l)&0x7f | 0x80) - l >>= 7 - i++ - } - dAtA[i] = uint8(l) - i++ - i += copy(dAtA[i:], s) + i = encodeVarintAppStats(dAtA, i, uint64(num)) } } dAtA[i] = 0x12 @@ -1183,8 +1174,7 @@ func (m *CollectedStatementStatistics) Size() (n int) { n += 1 + l + sovAppStats(uint64(l)) l = m.Stats.Size() n += 1 + l + sovAppStats(uint64(l)) - l = len(m.ID) - n += 1 + l + sovAppStats(uint64(l)) + n += 1 + sovAppStats(uint64(m.ID)) return n } @@ -1195,9 +1185,8 @@ func (m *CollectedTransactionStatistics) Size() (n int) { var l int _ = l if len(m.StatementIDs) > 0 { - for _, s := range m.StatementIDs { - l = len(s) - n += 1 + l + sovAppStats(uint64(l)) + for _, e := range m.StatementIDs { + n += 1 + sovAppStats(uint64(e)) } } l = len(m.App) @@ -2422,10 +2411,10 @@ func (m *CollectedStatementStatistics) Unmarshal(dAtA []byte) error { } iNdEx = postIndex case 3: - if wireType != 2 { + if wireType != 0 { return fmt.Errorf("proto: wrong wireType = %d for field ID", wireType) } - var stringLen uint64 + m.ID = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowAppStats @@ -2435,21 +2424,11 @@ func (m *CollectedStatementStatistics) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + m.ID |= (StmtID(b) & 0x7F) << shift if b < 0x80 { break } } - intStringLen := int(stringLen) - if intStringLen < 0 { - return ErrInvalidLengthAppStats - } - postIndex := iNdEx + intStringLen - if postIndex > l { - return io.ErrUnexpectedEOF - } - m.ID = StmtID(dAtA[iNdEx:postIndex]) - iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipAppStats(dAtA[iNdEx:]) @@ -2501,34 +2480,78 @@ func (m *CollectedTransactionStatistics) Unmarshal(dAtA []byte) error { } switch fieldNum { case 1: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field StatementIDs", wireType) - } - var stringLen uint64 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowAppStats + if wireType == 0 { + var v StmtID + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowAppStats + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= (StmtID(b) & 0x7F) << shift + if b < 0x80 { + break + } } - if iNdEx >= l { + m.StatementIDs = append(m.StatementIDs, v) + } else if wireType == 2 { + var packedLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowAppStats + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + packedLen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if packedLen < 0 { + return ErrInvalidLengthAppStats + } + postIndex := iNdEx + packedLen + if postIndex > l { return io.ErrUnexpectedEOF } - b := dAtA[iNdEx] - iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift - if b < 0x80 { - break + var elementCount int + var count int + for _, integer := range dAtA { + if integer < 128 { + count++ + } } + elementCount = count + if elementCount != 0 && len(m.StatementIDs) == 0 { + m.StatementIDs = make([]StmtID, 0, elementCount) + } + for iNdEx < postIndex { + var v StmtID + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowAppStats + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= (StmtID(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + m.StatementIDs = append(m.StatementIDs, v) + } + } else { + return fmt.Errorf("proto: wrong wireType = %d for field StatementIDs", wireType) } - intStringLen := int(stringLen) - if intStringLen < 0 { - return ErrInvalidLengthAppStats - } - postIndex := iNdEx + intStringLen - if postIndex > l { - return io.ErrUnexpectedEOF - } - m.StatementIDs = append(m.StatementIDs, StmtID(dAtA[iNdEx:postIndex])) - iNdEx = postIndex case 2: if wireType != 2 { return fmt.Errorf("proto: wrong wireType = %d for field App", wireType) @@ -3100,9 +3123,9 @@ var ( ErrIntOverflowAppStats = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("roachpb/app_stats.proto", fileDescriptor_app_stats_ba11ef7e1d77017c) } +func init() { proto.RegisterFile("roachpb/app_stats.proto", fileDescriptor_app_stats_6c6ddfe12bf47ebd) } -var fileDescriptor_app_stats_ba11ef7e1d77017c = []byte{ +var fileDescriptor_app_stats_6c6ddfe12bf47ebd = []byte{ // 1113 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x9c, 0x56, 0xbd, 0x6e, 0x1b, 0x47, 0x17, 0xd5, 0xf2, 0x47, 0x24, 0x2f, 0x49, 0xc9, 0xdf, 0xf8, 0x6f, 0x3f, 0x42, 0x20, 0x15, 0xc2, @@ -3155,10 +3178,10 @@ var fileDescriptor_app_stats_ba11ef7e1d77017c = []byte{ 0x4a, 0x60, 0x80, 0xae, 0x1e, 0xcb, 0x13, 0x81, 0x01, 0xba, 0xcd, 0x1f, 0x2d, 0xd8, 0x38, 0x88, 0x7c, 0x1f, 0xd5, 0x08, 0x5a, 0xf4, 0xee, 0xed, 0x42, 0xf6, 0x02, 0xcd, 0x49, 0xcb, 0x3b, 0x8f, 0xe7, 0x47, 0xd1, 0xc2, 0xfa, 0xa4, 0xfa, 0x17, 0x38, 0x22, 0xcf, 0x20, 0xaf, 0xdf, 0xe8, 0x5b, - 0x6c, 0xb5, 0x40, 0x20, 0x2d, 0xa7, 0xa6, 0x91, 0x47, 0x90, 0x61, 0x9e, 0xae, 0x58, 0xa9, 0x7d, + 0x6c, 0xb5, 0x40, 0x20, 0x2d, 0xa7, 0xa6, 0x91, 0x47, 0x90, 0x61, 0x9e, 0xae, 0x58, 0xae, 0x7d, 0x4f, 0x25, 0xc6, 0xd7, 0x8d, 0x4c, 0xe7, 0xf0, 0xaf, 0xeb, 0xc6, 0xea, 0xb1, 0x0c, 0x64, 0xe7, 0xb0, 0x9b, 0x61, 0x5e, 0xf3, 0x07, 0x0b, 0xea, 0x93, 0x53, 0x2c, 0x1e, 0x36, 0x7b, 0x50, 0x15, - 0xe9, 0x62, 0x0e, 0xf3, 0x84, 0x6d, 0x6d, 0x66, 0xb7, 0x4a, 0xed, 0xda, 0xf8, 0xba, 0x51, 0x99, + 0xe9, 0x62, 0x0e, 0xf3, 0x84, 0x6d, 0x6d, 0x66, 0xb7, 0x72, 0xed, 0xda, 0xf8, 0xba, 0x51, 0x99, 0xec, 0xa2, 0x73, 0x28, 0xa6, 0x94, 0x2b, 0x13, 0x42, 0xc7, 0x13, 0xb7, 0x5e, 0xec, 0xf3, 0xf4, 0x84, 0xc6, 0xcf, 0x8f, 0xe6, 0x4e, 0xb8, 0x70, 0x37, 0x33, 0x67, 0x6c, 0xfe, 0x69, 0xc1, 0xdd, 0x05, 0xfd, 0xa5, 0x6c, 0x1c, 0xd2, 0x00, 0x67, 0x5c, 0xa6, 0x23, 0x64, 0x17, 0xf2, 0x54, 0x4a, @@ -3173,5 +3196,5 @@ var fileDescriptor_app_stats_ba11ef7e1d77017c = []byte{ 0x14, 0xba, 0x9a, 0xe6, 0x34, 0xb8, 0xfd, 0xf4, 0xea, 0xf7, 0xfa, 0xca, 0xd5, 0xb8, 0x6e, 0xbd, 0x1d, 0xd7, 0xad, 0x77, 0xe3, 0xba, 0xf5, 0xdb, 0xb8, 0x6e, 0xbd, 0xb9, 0xa9, 0xaf, 0xbc, 0xbd, 0xa9, 0xaf, 0xbc, 0xbb, 0xa9, 0xaf, 0x7c, 0x5b, 0x48, 0x3e, 0x5e, 0xff, 0x0e, 0x00, 0x00, 0xff, - 0xff, 0xf7, 0xd0, 0x18, 0x82, 0xc6, 0x0a, 0x00, 0x00, + 0xff, 0xea, 0x08, 0x8c, 0xbf, 0xc6, 0x0a, 0x00, 0x00, } diff --git a/pkg/roachpb/app_stats.proto b/pkg/roachpb/app_stats.proto index 685b53d0411d..c69a90666a3e 100644 --- a/pkg/roachpb/app_stats.proto +++ b/pkg/roachpb/app_stats.proto @@ -158,7 +158,7 @@ message CollectedStatementStatistics { // ID is a hash of the statement key (query fingerprint, failure status, // implicit txn or not) which can be used to identify the statement // for instance in transaction statistics. - optional string id = 3 [(gogoproto.nullable) = false, (gogoproto.customname) = "ID", (gogoproto.casttype) = "StmtID"]; + optional uint64 id = 3 [(gogoproto.nullable) = false, (gogoproto.customname) = "ID", (gogoproto.casttype) = "StmtID"]; optional StatementStatisticsKey key = 1 [(gogoproto.nullable) = false]; optional StatementStatistics stats = 2 [(gogoproto.nullable) = false]; } @@ -169,7 +169,7 @@ message CollectedStatementStatistics { message CollectedTransactionStatistics { // StatementIDs is the IDs of the statements which this transaction comprises, // in order. - repeated string statement_ids = 1 [(gogoproto.customname) = "StatementIDs", (gogoproto.casttype) = "StmtID"] ; + repeated uint64 statement_ids = 1 [(gogoproto.customname) = "StatementIDs", (gogoproto.casttype) = "StmtID"] ; // App is the name of the app which executed the transaction. optional string app = 2 [(gogoproto.nullable) = false]; optional TransactionStatistics stats = 3[(gogoproto.nullable) = false]; diff --git a/pkg/server/serverpb/status.pb.go b/pkg/server/serverpb/status.pb.go index 2a709b800a49..4288d6ed6215 100644 --- a/pkg/server/serverpb/status.pb.go +++ b/pkg/server/serverpb/status.pb.go @@ -65,7 +65,7 @@ func (x StacksType) String() string { return proto.EnumName(StacksType_name, int32(x)) } func (StacksType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_status_d9798677f74f0e55, []int{0} + return fileDescriptor_status_fdd87e929f38276a, []int{0} } // Represents the type of file. @@ -92,7 +92,7 @@ func (x FileType) String() string { return proto.EnumName(FileType_name, int32(x)) } func (FileType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_status_d9798677f74f0e55, []int{1} + return fileDescriptor_status_fdd87e929f38276a, []int{1} } // We use an enum to allow reporting of client certs and potential others (eg: @@ -129,7 +129,7 @@ func (x CertificateDetails_CertificateType) String() string { return proto.EnumName(CertificateDetails_CertificateType_name, int32(x)) } func (CertificateDetails_CertificateType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_status_d9798677f74f0e55, []int{1, 0} + return fileDescriptor_status_fdd87e929f38276a, []int{1, 0} } type ProfileRequest_Type int32 @@ -152,7 +152,7 @@ func (x ProfileRequest_Type) String() string { return proto.EnumName(ProfileRequest_Type_name, int32(x)) } func (ProfileRequest_Type) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_status_d9798677f74f0e55, []int{37, 0} + return fileDescriptor_status_fdd87e929f38276a, []int{37, 0} } // Enum for phase of execution. @@ -176,7 +176,7 @@ func (x ActiveQuery_Phase) String() string { return proto.EnumName(ActiveQuery_Phase_name, int32(x)) } func (ActiveQuery_Phase) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_status_d9798677f74f0e55, []int{45, 0} + return fileDescriptor_status_fdd87e929f38276a, []int{45, 0} } type CertificatesRequest struct { @@ -189,7 +189,7 @@ func (m *CertificatesRequest) Reset() { *m = CertificatesRequest{} } func (m *CertificatesRequest) String() string { return proto.CompactTextString(m) } func (*CertificatesRequest) ProtoMessage() {} func (*CertificatesRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_status_d9798677f74f0e55, []int{0} + return fileDescriptor_status_fdd87e929f38276a, []int{0} } func (m *CertificatesRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -228,7 +228,7 @@ func (m *CertificateDetails) Reset() { *m = CertificateDetails{} } func (m *CertificateDetails) String() string { return proto.CompactTextString(m) } func (*CertificateDetails) ProtoMessage() {} func (*CertificateDetails) Descriptor() ([]byte, []int) { - return fileDescriptor_status_d9798677f74f0e55, []int{1} + return fileDescriptor_status_fdd87e929f38276a, []int{1} } func (m *CertificateDetails) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -269,7 +269,7 @@ func (m *CertificateDetails_Fields) Reset() { *m = CertificateDetails_Fi func (m *CertificateDetails_Fields) String() string { return proto.CompactTextString(m) } func (*CertificateDetails_Fields) ProtoMessage() {} func (*CertificateDetails_Fields) Descriptor() ([]byte, []int) { - return fileDescriptor_status_d9798677f74f0e55, []int{1, 0} + return fileDescriptor_status_fdd87e929f38276a, []int{1, 0} } func (m *CertificateDetails_Fields) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -302,7 +302,7 @@ func (m *CertificatesResponse) Reset() { *m = CertificatesResponse{} } func (m *CertificatesResponse) String() string { return proto.CompactTextString(m) } func (*CertificatesResponse) ProtoMessage() {} func (*CertificatesResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_status_d9798677f74f0e55, []int{2} + return fileDescriptor_status_fdd87e929f38276a, []int{2} } func (m *CertificatesResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -339,7 +339,7 @@ func (m *DetailsRequest) Reset() { *m = DetailsRequest{} } func (m *DetailsRequest) String() string { return proto.CompactTextString(m) } func (*DetailsRequest) ProtoMessage() {} func (*DetailsRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_status_d9798677f74f0e55, []int{3} + return fileDescriptor_status_fdd87e929f38276a, []int{3} } func (m *DetailsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -376,7 +376,7 @@ func (m *SystemInfo) Reset() { *m = SystemInfo{} } func (m *SystemInfo) String() string { return proto.CompactTextString(m) } func (*SystemInfo) ProtoMessage() {} func (*SystemInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_status_d9798677f74f0e55, []int{4} + return fileDescriptor_status_fdd87e929f38276a, []int{4} } func (m *SystemInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -413,7 +413,7 @@ func (m *DetailsResponse) Reset() { *m = DetailsResponse{} } func (m *DetailsResponse) String() string { return proto.CompactTextString(m) } func (*DetailsResponse) ProtoMessage() {} func (*DetailsResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_status_d9798677f74f0e55, []int{5} + return fileDescriptor_status_fdd87e929f38276a, []int{5} } func (m *DetailsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -445,7 +445,7 @@ func (m *NodesRequest) Reset() { *m = NodesRequest{} } func (m *NodesRequest) String() string { return proto.CompactTextString(m) } func (*NodesRequest) ProtoMessage() {} func (*NodesRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_status_d9798677f74f0e55, []int{6} + return fileDescriptor_status_fdd87e929f38276a, []int{6} } func (m *NodesRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -479,7 +479,7 @@ func (m *NodesResponse) Reset() { *m = NodesResponse{} } func (m *NodesResponse) String() string { return proto.CompactTextString(m) } func (*NodesResponse) ProtoMessage() {} func (*NodesResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_status_d9798677f74f0e55, []int{7} + return fileDescriptor_status_fdd87e929f38276a, []int{7} } func (m *NodesResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -514,7 +514,7 @@ func (m *NodeRequest) Reset() { *m = NodeRequest{} } func (m *NodeRequest) String() string { return proto.CompactTextString(m) } func (*NodeRequest) ProtoMessage() {} func (*NodeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_status_d9798677f74f0e55, []int{8} + return fileDescriptor_status_fdd87e929f38276a, []int{8} } func (m *NodeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -559,7 +559,7 @@ func (m *RaftState) Reset() { *m = RaftState{} } func (m *RaftState) String() string { return proto.CompactTextString(m) } func (*RaftState) ProtoMessage() {} func (*RaftState) Descriptor() ([]byte, []int) { - return fileDescriptor_status_d9798677f74f0e55, []int{9} + return fileDescriptor_status_fdd87e929f38276a, []int{9} } func (m *RaftState) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -596,7 +596,7 @@ func (m *RaftState_Progress) Reset() { *m = RaftState_Progress{} } func (m *RaftState_Progress) String() string { return proto.CompactTextString(m) } func (*RaftState_Progress) ProtoMessage() {} func (*RaftState_Progress) Descriptor() ([]byte, []int) { - return fileDescriptor_status_d9798677f74f0e55, []int{9, 0} + return fileDescriptor_status_fdd87e929f38276a, []int{9, 0} } func (m *RaftState_Progress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -641,7 +641,7 @@ func (m *RangeProblems) Reset() { *m = RangeProblems{} } func (m *RangeProblems) String() string { return proto.CompactTextString(m) } func (*RangeProblems) ProtoMessage() {} func (*RangeProblems) Descriptor() ([]byte, []int) { - return fileDescriptor_status_d9798677f74f0e55, []int{10} + return fileDescriptor_status_fdd87e929f38276a, []int{10} } func (m *RangeProblems) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -677,7 +677,7 @@ func (m *RangeStatistics) Reset() { *m = RangeStatistics{} } func (m *RangeStatistics) String() string { return proto.CompactTextString(m) } func (*RangeStatistics) ProtoMessage() {} func (*RangeStatistics) Descriptor() ([]byte, []int) { - return fileDescriptor_status_d9798677f74f0e55, []int{11} + return fileDescriptor_status_fdd87e929f38276a, []int{11} } func (m *RangeStatistics) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -711,7 +711,7 @@ func (m *PrettySpan) Reset() { *m = PrettySpan{} } func (m *PrettySpan) String() string { return proto.CompactTextString(m) } func (*PrettySpan) ProtoMessage() {} func (*PrettySpan) Descriptor() ([]byte, []int) { - return fileDescriptor_status_d9798677f74f0e55, []int{12} + return fileDescriptor_status_fdd87e929f38276a, []int{12} } func (m *PrettySpan) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -757,7 +757,7 @@ func (m *RangeInfo) Reset() { *m = RangeInfo{} } func (m *RangeInfo) String() string { return proto.CompactTextString(m) } func (*RangeInfo) ProtoMessage() {} func (*RangeInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_status_d9798677f74f0e55, []int{13} + return fileDescriptor_status_fdd87e929f38276a, []int{13} } func (m *RangeInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -793,7 +793,7 @@ func (m *RangesRequest) Reset() { *m = RangesRequest{} } func (m *RangesRequest) String() string { return proto.CompactTextString(m) } func (*RangesRequest) ProtoMessage() {} func (*RangesRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_status_d9798677f74f0e55, []int{14} + return fileDescriptor_status_fdd87e929f38276a, []int{14} } func (m *RangesRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -826,7 +826,7 @@ func (m *RangesResponse) Reset() { *m = RangesResponse{} } func (m *RangesResponse) String() string { return proto.CompactTextString(m) } func (*RangesResponse) ProtoMessage() {} func (*RangesResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_status_d9798677f74f0e55, []int{15} + return fileDescriptor_status_fdd87e929f38276a, []int{15} } func (m *RangesResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -861,7 +861,7 @@ func (m *GossipRequest) Reset() { *m = GossipRequest{} } func (m *GossipRequest) String() string { return proto.CompactTextString(m) } func (*GossipRequest) ProtoMessage() {} func (*GossipRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_status_d9798677f74f0e55, []int{16} + return fileDescriptor_status_fdd87e929f38276a, []int{16} } func (m *GossipRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -896,7 +896,7 @@ func (m *EngineStatsInfo) Reset() { *m = EngineStatsInfo{} } func (m *EngineStatsInfo) String() string { return proto.CompactTextString(m) } func (*EngineStatsInfo) ProtoMessage() {} func (*EngineStatsInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_status_d9798677f74f0e55, []int{17} + return fileDescriptor_status_fdd87e929f38276a, []int{17} } func (m *EngineStatsInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -931,7 +931,7 @@ func (m *EngineStatsRequest) Reset() { *m = EngineStatsRequest{} } func (m *EngineStatsRequest) String() string { return proto.CompactTextString(m) } func (*EngineStatsRequest) ProtoMessage() {} func (*EngineStatsRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_status_d9798677f74f0e55, []int{18} + return fileDescriptor_status_fdd87e929f38276a, []int{18} } func (m *EngineStatsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -964,7 +964,7 @@ func (m *EngineStatsResponse) Reset() { *m = EngineStatsResponse{} } func (m *EngineStatsResponse) String() string { return proto.CompactTextString(m) } func (*EngineStatsResponse) ProtoMessage() {} func (*EngineStatsResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_status_d9798677f74f0e55, []int{19} + return fileDescriptor_status_fdd87e929f38276a, []int{19} } func (m *EngineStatsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -998,7 +998,7 @@ func (m *TraceEvent) Reset() { *m = TraceEvent{} } func (m *TraceEvent) String() string { return proto.CompactTextString(m) } func (*TraceEvent) ProtoMessage() {} func (*TraceEvent) Descriptor() ([]byte, []int) { - return fileDescriptor_status_d9798677f74f0e55, []int{20} + return fileDescriptor_status_fdd87e929f38276a, []int{20} } func (m *TraceEvent) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1032,7 +1032,7 @@ func (m *AllocatorDryRun) Reset() { *m = AllocatorDryRun{} } func (m *AllocatorDryRun) String() string { return proto.CompactTextString(m) } func (*AllocatorDryRun) ProtoMessage() {} func (*AllocatorDryRun) Descriptor() ([]byte, []int) { - return fileDescriptor_status_d9798677f74f0e55, []int{21} + return fileDescriptor_status_fdd87e929f38276a, []int{21} } func (m *AllocatorDryRun) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1065,7 +1065,7 @@ func (m *AllocatorRangeRequest) Reset() { *m = AllocatorRangeRequest{} } func (m *AllocatorRangeRequest) String() string { return proto.CompactTextString(m) } func (*AllocatorRangeRequest) ProtoMessage() {} func (*AllocatorRangeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_status_d9798677f74f0e55, []int{22} + return fileDescriptor_status_fdd87e929f38276a, []int{22} } func (m *AllocatorRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1101,7 +1101,7 @@ func (m *AllocatorRangeResponse) Reset() { *m = AllocatorRangeResponse{} func (m *AllocatorRangeResponse) String() string { return proto.CompactTextString(m) } func (*AllocatorRangeResponse) ProtoMessage() {} func (*AllocatorRangeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_status_d9798677f74f0e55, []int{23} + return fileDescriptor_status_fdd87e929f38276a, []int{23} } func (m *AllocatorRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1135,7 +1135,7 @@ func (m *AllocatorRequest) Reset() { *m = AllocatorRequest{} } func (m *AllocatorRequest) String() string { return proto.CompactTextString(m) } func (*AllocatorRequest) ProtoMessage() {} func (*AllocatorRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_status_d9798677f74f0e55, []int{24} + return fileDescriptor_status_fdd87e929f38276a, []int{24} } func (m *AllocatorRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1168,7 +1168,7 @@ func (m *AllocatorResponse) Reset() { *m = AllocatorResponse{} } func (m *AllocatorResponse) String() string { return proto.CompactTextString(m) } func (*AllocatorResponse) ProtoMessage() {} func (*AllocatorResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_status_d9798677f74f0e55, []int{25} + return fileDescriptor_status_fdd87e929f38276a, []int{25} } func (m *AllocatorResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1201,7 +1201,7 @@ func (m *JSONResponse) Reset() { *m = JSONResponse{} } func (m *JSONResponse) String() string { return proto.CompactTextString(m) } func (*JSONResponse) ProtoMessage() {} func (*JSONResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_status_d9798677f74f0e55, []int{26} + return fileDescriptor_status_fdd87e929f38276a, []int{26} } func (m *JSONResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1237,7 +1237,7 @@ func (m *ResponseError) Reset() { *m = ResponseError{} } func (m *ResponseError) String() string { return proto.CompactTextString(m) } func (*ResponseError) ProtoMessage() {} func (*ResponseError) Descriptor() ([]byte, []int) { - return fileDescriptor_status_d9798677f74f0e55, []int{27} + return fileDescriptor_status_fdd87e929f38276a, []int{27} } func (m *ResponseError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1289,7 +1289,7 @@ func (m *LogsRequest) Reset() { *m = LogsRequest{} } func (m *LogsRequest) String() string { return proto.CompactTextString(m) } func (*LogsRequest) ProtoMessage() {} func (*LogsRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_status_d9798677f74f0e55, []int{28} + return fileDescriptor_status_fdd87e929f38276a, []int{28} } func (m *LogsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1322,7 +1322,7 @@ func (m *LogEntriesResponse) Reset() { *m = LogEntriesResponse{} } func (m *LogEntriesResponse) String() string { return proto.CompactTextString(m) } func (*LogEntriesResponse) ProtoMessage() {} func (*LogEntriesResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_status_d9798677f74f0e55, []int{29} + return fileDescriptor_status_fdd87e929f38276a, []int{29} } func (m *LogEntriesResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1357,7 +1357,7 @@ func (m *LogFilesListRequest) Reset() { *m = LogFilesListRequest{} } func (m *LogFilesListRequest) String() string { return proto.CompactTextString(m) } func (*LogFilesListRequest) ProtoMessage() {} func (*LogFilesListRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_status_d9798677f74f0e55, []int{30} + return fileDescriptor_status_fdd87e929f38276a, []int{30} } func (m *LogFilesListRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1390,7 +1390,7 @@ func (m *LogFilesListResponse) Reset() { *m = LogFilesListResponse{} } func (m *LogFilesListResponse) String() string { return proto.CompactTextString(m) } func (*LogFilesListResponse) ProtoMessage() {} func (*LogFilesListResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_status_d9798677f74f0e55, []int{31} + return fileDescriptor_status_fdd87e929f38276a, []int{31} } func (m *LogFilesListResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1438,7 +1438,7 @@ func (m *LogFileRequest) Reset() { *m = LogFileRequest{} } func (m *LogFileRequest) String() string { return proto.CompactTextString(m) } func (*LogFileRequest) ProtoMessage() {} func (*LogFileRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_status_d9798677f74f0e55, []int{32} + return fileDescriptor_status_fdd87e929f38276a, []int{32} } func (m *LogFileRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1474,7 +1474,7 @@ func (m *StacksRequest) Reset() { *m = StacksRequest{} } func (m *StacksRequest) String() string { return proto.CompactTextString(m) } func (*StacksRequest) ProtoMessage() {} func (*StacksRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_status_d9798677f74f0e55, []int{33} + return fileDescriptor_status_fdd87e929f38276a, []int{33} } func (m *StacksRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1510,7 +1510,7 @@ func (m *File) Reset() { *m = File{} } func (m *File) String() string { return proto.CompactTextString(m) } func (*File) ProtoMessage() {} func (*File) Descriptor() ([]byte, []int) { - return fileDescriptor_status_d9798677f74f0e55, []int{34} + return fileDescriptor_status_fdd87e929f38276a, []int{34} } func (m *File) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1554,7 +1554,7 @@ func (m *GetFilesRequest) Reset() { *m = GetFilesRequest{} } func (m *GetFilesRequest) String() string { return proto.CompactTextString(m) } func (*GetFilesRequest) ProtoMessage() {} func (*GetFilesRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_status_d9798677f74f0e55, []int{35} + return fileDescriptor_status_fdd87e929f38276a, []int{35} } func (m *GetFilesRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1587,7 +1587,7 @@ func (m *GetFilesResponse) Reset() { *m = GetFilesResponse{} } func (m *GetFilesResponse) String() string { return proto.CompactTextString(m) } func (*GetFilesResponse) ProtoMessage() {} func (*GetFilesResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_status_d9798677f74f0e55, []int{36} + return fileDescriptor_status_fdd87e929f38276a, []int{36} } func (m *GetFilesResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1625,7 +1625,7 @@ func (m *ProfileRequest) Reset() { *m = ProfileRequest{} } func (m *ProfileRequest) String() string { return proto.CompactTextString(m) } func (*ProfileRequest) ProtoMessage() {} func (*ProfileRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_status_d9798677f74f0e55, []int{37} + return fileDescriptor_status_fdd87e929f38276a, []int{37} } func (m *ProfileRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1660,7 +1660,7 @@ func (m *MetricsRequest) Reset() { *m = MetricsRequest{} } func (m *MetricsRequest) String() string { return proto.CompactTextString(m) } func (*MetricsRequest) ProtoMessage() {} func (*MetricsRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_status_d9798677f74f0e55, []int{38} + return fileDescriptor_status_fdd87e929f38276a, []int{38} } func (m *MetricsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1694,7 +1694,7 @@ func (m *RaftRangeNode) Reset() { *m = RaftRangeNode{} } func (m *RaftRangeNode) String() string { return proto.CompactTextString(m) } func (*RaftRangeNode) ProtoMessage() {} func (*RaftRangeNode) Descriptor() ([]byte, []int) { - return fileDescriptor_status_d9798677f74f0e55, []int{39} + return fileDescriptor_status_fdd87e929f38276a, []int{39} } func (m *RaftRangeNode) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1727,7 +1727,7 @@ func (m *RaftRangeError) Reset() { *m = RaftRangeError{} } func (m *RaftRangeError) String() string { return proto.CompactTextString(m) } func (*RaftRangeError) ProtoMessage() {} func (*RaftRangeError) Descriptor() ([]byte, []int) { - return fileDescriptor_status_d9798677f74f0e55, []int{40} + return fileDescriptor_status_fdd87e929f38276a, []int{40} } func (m *RaftRangeError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1762,7 +1762,7 @@ func (m *RaftRangeStatus) Reset() { *m = RaftRangeStatus{} } func (m *RaftRangeStatus) String() string { return proto.CompactTextString(m) } func (*RaftRangeStatus) ProtoMessage() {} func (*RaftRangeStatus) Descriptor() ([]byte, []int) { - return fileDescriptor_status_d9798677f74f0e55, []int{41} + return fileDescriptor_status_fdd87e929f38276a, []int{41} } func (m *RaftRangeStatus) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1795,7 +1795,7 @@ func (m *RaftDebugRequest) Reset() { *m = RaftDebugRequest{} } func (m *RaftDebugRequest) String() string { return proto.CompactTextString(m) } func (*RaftDebugRequest) ProtoMessage() {} func (*RaftDebugRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_status_d9798677f74f0e55, []int{42} + return fileDescriptor_status_fdd87e929f38276a, []int{42} } func (m *RaftDebugRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1829,7 +1829,7 @@ func (m *RaftDebugResponse) Reset() { *m = RaftDebugResponse{} } func (m *RaftDebugResponse) String() string { return proto.CompactTextString(m) } func (*RaftDebugResponse) ProtoMessage() {} func (*RaftDebugResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_status_d9798677f74f0e55, []int{43} + return fileDescriptor_status_fdd87e929f38276a, []int{43} } func (m *RaftDebugResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1887,7 +1887,7 @@ func (m *TxnInfo) Reset() { *m = TxnInfo{} } func (m *TxnInfo) String() string { return proto.CompactTextString(m) } func (*TxnInfo) ProtoMessage() {} func (*TxnInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_status_d9798677f74f0e55, []int{44} + return fileDescriptor_status_fdd87e929f38276a, []int{44} } func (m *TxnInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1935,7 +1935,7 @@ func (m *ActiveQuery) Reset() { *m = ActiveQuery{} } func (m *ActiveQuery) String() string { return proto.CompactTextString(m) } func (*ActiveQuery) ProtoMessage() {} func (*ActiveQuery) Descriptor() ([]byte, []int) { - return fileDescriptor_status_d9798677f74f0e55, []int{45} + return fileDescriptor_status_fdd87e929f38276a, []int{45} } func (m *ActiveQuery) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1970,7 +1970,7 @@ func (m *ListSessionsRequest) Reset() { *m = ListSessionsRequest{} } func (m *ListSessionsRequest) String() string { return proto.CompactTextString(m) } func (*ListSessionsRequest) ProtoMessage() {} func (*ListSessionsRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_status_d9798677f74f0e55, []int{46} + return fileDescriptor_status_fdd87e929f38276a, []int{46} } func (m *ListSessionsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2029,7 +2029,7 @@ func (m *Session) Reset() { *m = Session{} } func (m *Session) String() string { return proto.CompactTextString(m) } func (*Session) ProtoMessage() {} func (*Session) Descriptor() ([]byte, []int) { - return fileDescriptor_status_d9798677f74f0e55, []int{47} + return fileDescriptor_status_fdd87e929f38276a, []int{47} } func (m *Session) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2066,7 +2066,7 @@ func (m *ListSessionsError) Reset() { *m = ListSessionsError{} } func (m *ListSessionsError) String() string { return proto.CompactTextString(m) } func (*ListSessionsError) ProtoMessage() {} func (*ListSessionsError) Descriptor() ([]byte, []int) { - return fileDescriptor_status_d9798677f74f0e55, []int{48} + return fileDescriptor_status_fdd87e929f38276a, []int{48} } func (m *ListSessionsError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2103,7 +2103,7 @@ func (m *ListSessionsResponse) Reset() { *m = ListSessionsResponse{} } func (m *ListSessionsResponse) String() string { return proto.CompactTextString(m) } func (*ListSessionsResponse) ProtoMessage() {} func (*ListSessionsResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_status_d9798677f74f0e55, []int{49} + return fileDescriptor_status_fdd87e929f38276a, []int{49} } func (m *ListSessionsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2149,7 +2149,7 @@ func (m *CancelQueryRequest) Reset() { *m = CancelQueryRequest{} } func (m *CancelQueryRequest) String() string { return proto.CompactTextString(m) } func (*CancelQueryRequest) ProtoMessage() {} func (*CancelQueryRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_status_d9798677f74f0e55, []int{50} + return fileDescriptor_status_fdd87e929f38276a, []int{50} } func (m *CancelQueryRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2186,7 +2186,7 @@ func (m *CancelQueryResponse) Reset() { *m = CancelQueryResponse{} } func (m *CancelQueryResponse) String() string { return proto.CompactTextString(m) } func (*CancelQueryResponse) ProtoMessage() {} func (*CancelQueryResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_status_d9798677f74f0e55, []int{51} + return fileDescriptor_status_fdd87e929f38276a, []int{51} } func (m *CancelQueryResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2228,7 +2228,7 @@ func (m *CancelSessionRequest) Reset() { *m = CancelSessionRequest{} } func (m *CancelSessionRequest) String() string { return proto.CompactTextString(m) } func (*CancelSessionRequest) ProtoMessage() {} func (*CancelSessionRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_status_d9798677f74f0e55, []int{52} + return fileDescriptor_status_fdd87e929f38276a, []int{52} } func (m *CancelSessionRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2262,7 +2262,7 @@ func (m *CancelSessionResponse) Reset() { *m = CancelSessionResponse{} } func (m *CancelSessionResponse) String() string { return proto.CompactTextString(m) } func (*CancelSessionResponse) ProtoMessage() {} func (*CancelSessionResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_status_d9798677f74f0e55, []int{53} + return fileDescriptor_status_fdd87e929f38276a, []int{53} } func (m *CancelSessionResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2297,7 +2297,7 @@ func (m *SpanStatsRequest) Reset() { *m = SpanStatsRequest{} } func (m *SpanStatsRequest) String() string { return proto.CompactTextString(m) } func (*SpanStatsRequest) ProtoMessage() {} func (*SpanStatsRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_status_d9798677f74f0e55, []int{54} + return fileDescriptor_status_fdd87e929f38276a, []int{54} } func (m *SpanStatsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2332,7 +2332,7 @@ func (m *SpanStatsResponse) Reset() { *m = SpanStatsResponse{} } func (m *SpanStatsResponse) String() string { return proto.CompactTextString(m) } func (*SpanStatsResponse) ProtoMessage() {} func (*SpanStatsResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_status_d9798677f74f0e55, []int{55} + return fileDescriptor_status_fdd87e929f38276a, []int{55} } func (m *SpanStatsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2366,7 +2366,7 @@ func (m *ProblemRangesRequest) Reset() { *m = ProblemRangesRequest{} } func (m *ProblemRangesRequest) String() string { return proto.CompactTextString(m) } func (*ProblemRangesRequest) ProtoMessage() {} func (*ProblemRangesRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_status_d9798677f74f0e55, []int{56} + return fileDescriptor_status_fdd87e929f38276a, []int{56} } func (m *ProblemRangesRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2401,7 +2401,7 @@ func (m *ProblemRangesResponse) Reset() { *m = ProblemRangesResponse{} } func (m *ProblemRangesResponse) String() string { return proto.CompactTextString(m) } func (*ProblemRangesResponse) ProtoMessage() {} func (*ProblemRangesResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_status_d9798677f74f0e55, []int{57} + return fileDescriptor_status_fdd87e929f38276a, []int{57} } func (m *ProblemRangesResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2442,7 +2442,7 @@ func (m *ProblemRangesResponse_NodeProblems) Reset() { *m = ProblemRange func (m *ProblemRangesResponse_NodeProblems) String() string { return proto.CompactTextString(m) } func (*ProblemRangesResponse_NodeProblems) ProtoMessage() {} func (*ProblemRangesResponse_NodeProblems) Descriptor() ([]byte, []int) { - return fileDescriptor_status_d9798677f74f0e55, []int{57, 0} + return fileDescriptor_status_fdd87e929f38276a, []int{57, 0} } func (m *ProblemRangesResponse_NodeProblems) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2486,7 +2486,7 @@ func (m *HotRangesRequest) Reset() { *m = HotRangesRequest{} } func (m *HotRangesRequest) String() string { return proto.CompactTextString(m) } func (*HotRangesRequest) ProtoMessage() {} func (*HotRangesRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_status_d9798677f74f0e55, []int{58} + return fileDescriptor_status_fdd87e929f38276a, []int{58} } func (m *HotRangesRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2526,7 +2526,7 @@ func (m *HotRangesResponse) Reset() { *m = HotRangesResponse{} } func (m *HotRangesResponse) String() string { return proto.CompactTextString(m) } func (*HotRangesResponse) ProtoMessage() {} func (*HotRangesResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_status_d9798677f74f0e55, []int{59} + return fileDescriptor_status_fdd87e929f38276a, []int{59} } func (m *HotRangesResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2570,7 +2570,7 @@ func (m *HotRangesResponse_HotRange) Reset() { *m = HotRangesResponse_Ho func (m *HotRangesResponse_HotRange) String() string { return proto.CompactTextString(m) } func (*HotRangesResponse_HotRange) ProtoMessage() {} func (*HotRangesResponse_HotRange) Descriptor() ([]byte, []int) { - return fileDescriptor_status_d9798677f74f0e55, []int{59, 0} + return fileDescriptor_status_fdd87e929f38276a, []int{59, 0} } func (m *HotRangesResponse_HotRange) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2610,7 +2610,7 @@ func (m *HotRangesResponse_StoreResponse) Reset() { *m = HotRangesRespon func (m *HotRangesResponse_StoreResponse) String() string { return proto.CompactTextString(m) } func (*HotRangesResponse_StoreResponse) ProtoMessage() {} func (*HotRangesResponse_StoreResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_status_d9798677f74f0e55, []int{59, 1} + return fileDescriptor_status_fdd87e929f38276a, []int{59, 1} } func (m *HotRangesResponse_StoreResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2652,7 +2652,7 @@ func (m *HotRangesResponse_NodeResponse) Reset() { *m = HotRangesRespons func (m *HotRangesResponse_NodeResponse) String() string { return proto.CompactTextString(m) } func (*HotRangesResponse_NodeResponse) ProtoMessage() {} func (*HotRangesResponse_NodeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_status_d9798677f74f0e55, []int{59, 2} + return fileDescriptor_status_fdd87e929f38276a, []int{59, 2} } func (m *HotRangesResponse_NodeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2685,7 +2685,7 @@ func (m *RangeRequest) Reset() { *m = RangeRequest{} } func (m *RangeRequest) String() string { return proto.CompactTextString(m) } func (*RangeRequest) ProtoMessage() {} func (*RangeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_status_d9798677f74f0e55, []int{60} + return fileDescriptor_status_fdd87e929f38276a, []int{60} } func (m *RangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2721,7 +2721,7 @@ func (m *RangeResponse) Reset() { *m = RangeResponse{} } func (m *RangeResponse) String() string { return proto.CompactTextString(m) } func (*RangeResponse) ProtoMessage() {} func (*RangeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_status_d9798677f74f0e55, []int{61} + return fileDescriptor_status_fdd87e929f38276a, []int{61} } func (m *RangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2756,7 +2756,7 @@ func (m *RangeResponse_NodeResponse) Reset() { *m = RangeResponse_NodeRe func (m *RangeResponse_NodeResponse) String() string { return proto.CompactTextString(m) } func (*RangeResponse_NodeResponse) ProtoMessage() {} func (*RangeResponse_NodeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_status_d9798677f74f0e55, []int{61, 0} + return fileDescriptor_status_fdd87e929f38276a, []int{61, 0} } func (m *RangeResponse_NodeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2792,7 +2792,7 @@ func (m *DiagnosticsRequest) Reset() { *m = DiagnosticsRequest{} } func (m *DiagnosticsRequest) String() string { return proto.CompactTextString(m) } func (*DiagnosticsRequest) ProtoMessage() {} func (*DiagnosticsRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_status_d9798677f74f0e55, []int{62} + return fileDescriptor_status_fdd87e929f38276a, []int{62} } func (m *DiagnosticsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2827,7 +2827,7 @@ func (m *StoresRequest) Reset() { *m = StoresRequest{} } func (m *StoresRequest) String() string { return proto.CompactTextString(m) } func (*StoresRequest) ProtoMessage() {} func (*StoresRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_status_d9798677f74f0e55, []int{63} + return fileDescriptor_status_fdd87e929f38276a, []int{63} } func (m *StoresRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2870,7 +2870,7 @@ func (m *StoreDetails) Reset() { *m = StoreDetails{} } func (m *StoreDetails) String() string { return proto.CompactTextString(m) } func (*StoreDetails) ProtoMessage() {} func (*StoreDetails) Descriptor() ([]byte, []int) { - return fileDescriptor_status_d9798677f74f0e55, []int{64} + return fileDescriptor_status_fdd87e929f38276a, []int{64} } func (m *StoreDetails) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2903,7 +2903,7 @@ func (m *StoresResponse) Reset() { *m = StoresResponse{} } func (m *StoresResponse) String() string { return proto.CompactTextString(m) } func (*StoresResponse) ProtoMessage() {} func (*StoresResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_status_d9798677f74f0e55, []int{65} + return fileDescriptor_status_fdd87e929f38276a, []int{65} } func (m *StoresResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2936,7 +2936,7 @@ func (m *StatementsRequest) Reset() { *m = StatementsRequest{} } func (m *StatementsRequest) String() string { return proto.CompactTextString(m) } func (*StatementsRequest) ProtoMessage() {} func (*StatementsRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_status_d9798677f74f0e55, []int{66} + return fileDescriptor_status_fdd87e929f38276a, []int{66} } func (m *StatementsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2977,7 +2977,7 @@ func (m *StatementsResponse) Reset() { *m = StatementsResponse{} } func (m *StatementsResponse) String() string { return proto.CompactTextString(m) } func (*StatementsResponse) ProtoMessage() {} func (*StatementsResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_status_d9798677f74f0e55, []int{67} + return fileDescriptor_status_fdd87e929f38276a, []int{67} } func (m *StatementsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3015,7 +3015,7 @@ func (m *StatementsResponse_ExtendedStatementStatisticsKey) String() string { } func (*StatementsResponse_ExtendedStatementStatisticsKey) ProtoMessage() {} func (*StatementsResponse_ExtendedStatementStatisticsKey) Descriptor() ([]byte, []int) { - return fileDescriptor_status_d9798677f74f0e55, []int{67, 0} + return fileDescriptor_status_fdd87e929f38276a, []int{67, 0} } func (m *StatementsResponse_ExtendedStatementStatisticsKey) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3042,7 +3042,7 @@ var xxx_messageInfo_StatementsResponse_ExtendedStatementStatisticsKey proto.Inte type StatementsResponse_CollectedStatementStatistics struct { Key StatementsResponse_ExtendedStatementStatisticsKey `protobuf:"bytes,1,opt,name=key,proto3" json:"key"` - ID github_com_cockroachdb_cockroach_pkg_roachpb.StmtID `protobuf:"bytes,3,opt,name=id,proto3,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.StmtID" json:"id,omitempty"` + ID github_com_cockroachdb_cockroach_pkg_roachpb.StmtID `protobuf:"varint,3,opt,name=id,proto3,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.StmtID" json:"id,omitempty"` Stats roachpb.StatementStatistics `protobuf:"bytes,2,opt,name=stats,proto3" json:"stats"` } @@ -3054,7 +3054,7 @@ func (m *StatementsResponse_CollectedStatementStatistics) String() string { } func (*StatementsResponse_CollectedStatementStatistics) ProtoMessage() {} func (*StatementsResponse_CollectedStatementStatistics) Descriptor() ([]byte, []int) { - return fileDescriptor_status_d9798677f74f0e55, []int{67, 1} + return fileDescriptor_status_fdd87e929f38276a, []int{67, 1} } func (m *StatementsResponse_CollectedStatementStatistics) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3092,7 +3092,7 @@ func (m *StatementsResponse_ExtendedCollectedTransactionStatistics) String() str } func (*StatementsResponse_ExtendedCollectedTransactionStatistics) ProtoMessage() {} func (*StatementsResponse_ExtendedCollectedTransactionStatistics) Descriptor() ([]byte, []int) { - return fileDescriptor_status_d9798677f74f0e55, []int{67, 2} + return fileDescriptor_status_fdd87e929f38276a, []int{67, 2} } func (m *StatementsResponse_ExtendedCollectedTransactionStatistics) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3129,7 +3129,7 @@ func (m *StatementDiagnosticsReport) Reset() { *m = StatementDiagnostics func (m *StatementDiagnosticsReport) String() string { return proto.CompactTextString(m) } func (*StatementDiagnosticsReport) ProtoMessage() {} func (*StatementDiagnosticsReport) Descriptor() ([]byte, []int) { - return fileDescriptor_status_d9798677f74f0e55, []int{68} + return fileDescriptor_status_fdd87e929f38276a, []int{68} } func (m *StatementDiagnosticsReport) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3164,7 +3164,7 @@ func (m *CreateStatementDiagnosticsReportRequest) Reset() { func (m *CreateStatementDiagnosticsReportRequest) String() string { return proto.CompactTextString(m) } func (*CreateStatementDiagnosticsReportRequest) ProtoMessage() {} func (*CreateStatementDiagnosticsReportRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_status_d9798677f74f0e55, []int{69} + return fileDescriptor_status_fdd87e929f38276a, []int{69} } func (m *CreateStatementDiagnosticsReportRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3199,7 +3199,7 @@ func (m *CreateStatementDiagnosticsReportResponse) Reset() { func (m *CreateStatementDiagnosticsReportResponse) String() string { return proto.CompactTextString(m) } func (*CreateStatementDiagnosticsReportResponse) ProtoMessage() {} func (*CreateStatementDiagnosticsReportResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_status_d9798677f74f0e55, []int{70} + return fileDescriptor_status_fdd87e929f38276a, []int{70} } func (m *CreateStatementDiagnosticsReportResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3231,7 +3231,7 @@ func (m *StatementDiagnosticsReportsRequest) Reset() { *m = StatementDia func (m *StatementDiagnosticsReportsRequest) String() string { return proto.CompactTextString(m) } func (*StatementDiagnosticsReportsRequest) ProtoMessage() {} func (*StatementDiagnosticsReportsRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_status_d9798677f74f0e55, []int{71} + return fileDescriptor_status_fdd87e929f38276a, []int{71} } func (m *StatementDiagnosticsReportsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3264,7 +3264,7 @@ func (m *StatementDiagnosticsReportsResponse) Reset() { *m = StatementDi func (m *StatementDiagnosticsReportsResponse) String() string { return proto.CompactTextString(m) } func (*StatementDiagnosticsReportsResponse) ProtoMessage() {} func (*StatementDiagnosticsReportsResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_status_d9798677f74f0e55, []int{72} + return fileDescriptor_status_fdd87e929f38276a, []int{72} } func (m *StatementDiagnosticsReportsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3300,7 +3300,7 @@ func (m *StatementDiagnostics) Reset() { *m = StatementDiagnostics{} } func (m *StatementDiagnostics) String() string { return proto.CompactTextString(m) } func (*StatementDiagnostics) ProtoMessage() {} func (*StatementDiagnostics) Descriptor() ([]byte, []int) { - return fileDescriptor_status_d9798677f74f0e55, []int{73} + return fileDescriptor_status_fdd87e929f38276a, []int{73} } func (m *StatementDiagnostics) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3333,7 +3333,7 @@ func (m *StatementDiagnosticsRequest) Reset() { *m = StatementDiagnostic func (m *StatementDiagnosticsRequest) String() string { return proto.CompactTextString(m) } func (*StatementDiagnosticsRequest) ProtoMessage() {} func (*StatementDiagnosticsRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_status_d9798677f74f0e55, []int{74} + return fileDescriptor_status_fdd87e929f38276a, []int{74} } func (m *StatementDiagnosticsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3366,7 +3366,7 @@ func (m *StatementDiagnosticsResponse) Reset() { *m = StatementDiagnosti func (m *StatementDiagnosticsResponse) String() string { return proto.CompactTextString(m) } func (*StatementDiagnosticsResponse) ProtoMessage() {} func (*StatementDiagnosticsResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_status_d9798677f74f0e55, []int{75} + return fileDescriptor_status_fdd87e929f38276a, []int{75} } func (m *StatementDiagnosticsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3399,7 +3399,7 @@ func (m *JobRegistryStatusRequest) Reset() { *m = JobRegistryStatusReque func (m *JobRegistryStatusRequest) String() string { return proto.CompactTextString(m) } func (*JobRegistryStatusRequest) ProtoMessage() {} func (*JobRegistryStatusRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_status_d9798677f74f0e55, []int{76} + return fileDescriptor_status_fdd87e929f38276a, []int{76} } func (m *JobRegistryStatusRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3433,7 +3433,7 @@ func (m *JobRegistryStatusResponse) Reset() { *m = JobRegistryStatusResp func (m *JobRegistryStatusResponse) String() string { return proto.CompactTextString(m) } func (*JobRegistryStatusResponse) ProtoMessage() {} func (*JobRegistryStatusResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_status_d9798677f74f0e55, []int{77} + return fileDescriptor_status_fdd87e929f38276a, []int{77} } func (m *JobRegistryStatusResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3466,7 +3466,7 @@ func (m *JobRegistryStatusResponse_Job) Reset() { *m = JobRegistryStatus func (m *JobRegistryStatusResponse_Job) String() string { return proto.CompactTextString(m) } func (*JobRegistryStatusResponse_Job) ProtoMessage() {} func (*JobRegistryStatusResponse_Job) Descriptor() ([]byte, []int) { - return fileDescriptor_status_d9798677f74f0e55, []int{77, 0} + return fileDescriptor_status_fdd87e929f38276a, []int{77, 0} } func (m *JobRegistryStatusResponse_Job) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3499,7 +3499,7 @@ func (m *JobStatusRequest) Reset() { *m = JobStatusRequest{} } func (m *JobStatusRequest) String() string { return proto.CompactTextString(m) } func (*JobStatusRequest) ProtoMessage() {} func (*JobStatusRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_status_d9798677f74f0e55, []int{78} + return fileDescriptor_status_fdd87e929f38276a, []int{78} } func (m *JobStatusRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3532,7 +3532,7 @@ func (m *JobStatusResponse) Reset() { *m = JobStatusResponse{} } func (m *JobStatusResponse) String() string { return proto.CompactTextString(m) } func (*JobStatusResponse) ProtoMessage() {} func (*JobStatusResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_status_d9798677f74f0e55, []int{79} + return fileDescriptor_status_fdd87e929f38276a, []int{79} } func (m *JobStatusResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8127,11 +8127,10 @@ func (m *StatementsResponse_CollectedStatementStatistics) MarshalTo(dAtA []byte) return 0, err } i += n57 - if len(m.ID) > 0 { - dAtA[i] = 0x1a + if m.ID != 0 { + dAtA[i] = 0x18 i++ - i = encodeVarintStatus(dAtA, i, uint64(len(m.ID))) - i += copy(dAtA[i:], m.ID) + i = encodeVarintStatus(dAtA, i, uint64(m.ID)) } return i, nil } @@ -10136,9 +10135,8 @@ func (m *StatementsResponse_CollectedStatementStatistics) Size() (n int) { n += 1 + l + sovStatus(uint64(l)) l = m.Stats.Size() n += 1 + l + sovStatus(uint64(l)) - l = len(m.ID) - if l > 0 { - n += 1 + l + sovStatus(uint64(l)) + if m.ID != 0 { + n += 1 + sovStatus(uint64(m.ID)) } return n } @@ -21031,10 +21029,10 @@ func (m *StatementsResponse_CollectedStatementStatistics) Unmarshal(dAtA []byte) } iNdEx = postIndex case 3: - if wireType != 2 { + if wireType != 0 { return fmt.Errorf("proto: wrong wireType = %d for field ID", wireType) } - var stringLen uint64 + m.ID = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowStatus @@ -21044,21 +21042,11 @@ func (m *StatementsResponse_CollectedStatementStatistics) Unmarshal(dAtA []byte) } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + m.ID |= (github_com_cockroachdb_cockroach_pkg_roachpb.StmtID(b) & 0x7F) << shift if b < 0x80 { break } } - intStringLen := int(stringLen) - if intStringLen < 0 { - return ErrInvalidLengthStatus - } - postIndex := iNdEx + intStringLen - if postIndex > l { - return io.ErrUnexpectedEOF - } - m.ID = github_com_cockroachdb_cockroach_pkg_roachpb.StmtID(dAtA[iNdEx:postIndex]) - iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipStatus(dAtA[iNdEx:]) @@ -22454,10 +22442,10 @@ var ( ) func init() { - proto.RegisterFile("server/serverpb/status.proto", fileDescriptor_status_d9798677f74f0e55) + proto.RegisterFile("server/serverpb/status.proto", fileDescriptor_status_fdd87e929f38276a) } -var fileDescriptor_status_d9798677f74f0e55 = []byte{ +var fileDescriptor_status_fdd87e929f38276a = []byte{ // 6184 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xcc, 0x7c, 0x5d, 0x6c, 0x5b, 0x47, 0x76, 0xbf, 0x2f, 0x49, 0x51, 0xe4, 0xa1, 0x3e, 0xa8, 0xd1, 0x87, 0x69, 0xda, 0x91, 0x9c, 0xeb, @@ -22754,96 +22742,96 @@ var fileDescriptor_status_d9798677f74f0e55 = []byte{ 0x67, 0x87, 0xb6, 0x96, 0x75, 0x4f, 0x97, 0x77, 0xaa, 0xd8, 0x47, 0xbc, 0x53, 0xe5, 0xbf, 0x15, 0x83, 0x73, 0x51, 0xf3, 0x4e, 0x8c, 0xb6, 0xdd, 0xcb, 0x5c, 0x5d, 0x3b, 0x9e, 0x78, 0x23, 0x87, 0x89, 0xb6, 0xf4, 0x4d, 0x3f, 0xdb, 0x32, 0xd6, 0x1d, 0x32, 0xef, 0x2d, 0xa7, 0x70, 0xce, 0xf5, - 0xdb, 0x18, 0xd8, 0xc7, 0x08, 0xe5, 0xe2, 0x1b, 0x3c, 0xb0, 0x7f, 0x64, 0xc9, 0x6c, 0x78, 0x75, - 0x8f, 0xdf, 0x07, 0xe4, 0x7f, 0xa2, 0xc0, 0xc5, 0xc1, 0x74, 0x83, 0x68, 0x7c, 0xdd, 0xb9, 0xf2, - 0x34, 0xbf, 0xd4, 0x01, 0x7f, 0x20, 0xf5, 0x4a, 0x23, 0x9b, 0xa7, 0x3d, 0xe1, 0xea, 0x97, 0x62, - 0x90, 0x0f, 0xc4, 0x19, 0xda, 0x68, 0x1a, 0xb6, 0xe3, 0x91, 0xb1, 0xe0, 0xce, 0x31, 0x8e, 0x37, - 0x23, 0xe7, 0x20, 0x5d, 0xb1, 0xeb, 0x8d, 0x1a, 0xf5, 0xa8, 0x21, 0x32, 0x83, 0xda, 0x05, 0xe4, - 0x1a, 0x4c, 0x07, 0x06, 0xa1, 0xb4, 0x6d, 0x5a, 0x55, 0xea, 0x34, 0x1c, 0xd3, 0xf2, 0x44, 0xac, - 0x78, 0x2a, 0xa8, 0xbc, 0xd9, 0xae, 0x23, 0x6f, 0x42, 0xae, 0x4d, 0x24, 0x3d, 0x74, 0x67, 0xc3, - 0xc5, 0xa7, 0xbb, 0x28, 0x10, 0x45, 0x9b, 0x71, 0x7b, 0xc0, 0xc4, 0x7c, 0x8e, 0x11, 0x87, 0x1b, - 0x4c, 0x6a, 0x94, 0x74, 0x4f, 0xbc, 0x4a, 0x1c, 0xcc, 0xf2, 0x64, 0x02, 0xca, 0x05, 0x4f, 0xfd, - 0x3c, 0xbc, 0xb0, 0xe4, 0x50, 0xdd, 0xa3, 0x87, 0xcb, 0xc3, 0xb7, 0xcc, 0x87, 0x0e, 0x54, 0x39, - 0x7c, 0xa0, 0x6a, 0x0b, 0xe6, 0xfb, 0xf3, 0x17, 0xe6, 0xfb, 0x6d, 0x48, 0x3a, 0x58, 0x22, 0x54, - 0xe8, 0xfa, 0x20, 0x2b, 0xad, 0x9b, 0x9d, 0x60, 0xa2, 0x3e, 0x07, 0xea, 0xe1, 0xad, 0x82, 0x97, - 0x97, 0xbf, 0x00, 0x17, 0x22, 0x5b, 0x09, 0x6c, 0x5b, 0x30, 0xcc, 0xd9, 0xfa, 0xfb, 0xca, 0xf1, - 0xc0, 0xf9, 0x66, 0x4d, 0xf0, 0x52, 0xff, 0x5c, 0x81, 0xa9, 0x5e, 0xad, 0xbb, 0x74, 0xf0, 0x50, - 0xe1, 0xc7, 0x22, 0xb4, 0xec, 0x16, 0x8c, 0x54, 0xfc, 0x65, 0xc7, 0xb4, 0xe4, 0x28, 0xfb, 0x53, - 0x26, 0xa0, 0x5c, 0xc0, 0x17, 0x77, 0x9e, 0xa3, 0x57, 0x82, 0x77, 0x85, 0xf8, 0xa1, 0x7e, 0x06, - 0xce, 0xf6, 0x1e, 0x29, 0xd7, 0x97, 0x1b, 0x11, 0x3a, 0xce, 0x07, 0x76, 0x88, 0x76, 0xab, 0x8f, - 0xe1, 0x5c, 0x6f, 0xc6, 0x62, 0x32, 0x1e, 0x40, 0x46, 0xe2, 0x27, 0xec, 0x65, 0xf1, 0xa8, 0x13, - 0x22, 0xf3, 0x50, 0xaf, 0x41, 0xee, 0x8e, 0x5d, 0xd6, 0x68, 0xd5, 0x74, 0x3d, 0xa7, 0x25, 0x72, - 0x5c, 0xfa, 0x79, 0x93, 0xff, 0xa5, 0xc0, 0x99, 0x1e, 0x54, 0x1f, 0xc3, 0xe1, 0xea, 0xb3, 0x30, - 0xe2, 0x34, 0x2d, 0xcb, 0xb4, 0xaa, 0xa5, 0x47, 0x76, 0xd9, 0x3f, 0xd0, 0x46, 0x25, 0x2e, 0x1d, - 0x8a, 0x13, 0x6b, 0x32, 0x82, 0xdb, 0x1d, 0xbb, 0xec, 0xe6, 0xa7, 0x21, 0x7e, 0xc7, 0x2e, 0x77, - 0xaa, 0xa0, 0x7a, 0x09, 0xb2, 0x77, 0xec, 0x72, 0x58, 0x34, 0xd3, 0x90, 0x7c, 0x64, 0x97, 0xdb, - 0x33, 0x3a, 0xf4, 0xc8, 0x2e, 0xaf, 0x1a, 0xea, 0x0a, 0x4c, 0x48, 0x4d, 0x85, 0x3c, 0x5e, 0x86, - 0xf8, 0x23, 0xbb, 0x2c, 0xd6, 0xf6, 0x6c, 0xc7, 0xf6, 0x80, 0x7f, 0x52, 0x84, 0xff, 0x79, 0x11, - 0x04, 0xc4, 0x9a, 0x5e, 0xbe, 0x0e, 0xd0, 0x4e, 0x14, 0x25, 0x53, 0x90, 0xbd, 0x75, 0x5f, 0xbb, - 0xbf, 0xb5, 0xb9, 0x7a, 0x6f, 0xa5, 0xb4, 0xb1, 0xb9, 0xb0, 0x74, 0x77, 0x23, 0x7b, 0x8a, 0x4c, - 0xc0, 0xe8, 0xe6, 0x6d, 0x6d, 0x65, 0x61, 0xd9, 0x2f, 0x52, 0x2e, 0x3f, 0x07, 0x29, 0x3f, 0x0b, - 0x53, 0x4a, 0x43, 0x1c, 0x03, 0x08, 0xc8, 0x37, 0xb2, 0xca, 0xd5, 0x1f, 0x3d, 0x0f, 0x49, 0xe1, - 0x77, 0x7f, 0x47, 0x81, 0x11, 0xf9, 0xef, 0x01, 0x90, 0xc2, 0x60, 0x2f, 0xfe, 0x7d, 0x31, 0xe4, - 0x8b, 0x03, 0xb7, 0xe7, 0xb2, 0x50, 0x5f, 0x78, 0xff, 0x9f, 0xfe, 0xe3, 0x5b, 0xb1, 0x67, 0xc9, - 0x5c, 0x51, 0x9c, 0x09, 0x8a, 0xf2, 0x9f, 0x0b, 0x28, 0xbe, 0x2b, 0x14, 0xe7, 0x09, 0xf9, 0x15, - 0x05, 0x86, 0xfd, 0xb3, 0x4a, 0x54, 0x1e, 0x58, 0xf8, 0xaf, 0x0b, 0xe4, 0x2f, 0x0f, 0xd2, 0x54, - 0x60, 0x51, 0x11, 0xcb, 0x39, 0x92, 0x0f, 0xb0, 0x88, 0x94, 0x71, 0x09, 0x86, 0x03, 0x43, 0xf8, - 0x18, 0x9c, 0xbc, 0xd0, 0xff, 0xb9, 0x38, 0x47, 0x30, 0x3f, 0xe8, 0xbb, 0x72, 0x75, 0x06, 0xfb, - 0xcf, 0x92, 0xb1, 0xa0, 0x7f, 0xfe, 0x6e, 0xfd, 0x3d, 0x48, 0x60, 0xe6, 0xe7, 0xc5, 0x3e, 0x9c, - 0xfc, 0x1e, 0x8f, 0xf4, 0x44, 0x5e, 0x3d, 0x8f, 0xbd, 0xe6, 0x49, 0x2e, 0xdc, 0xab, 0x34, 0xe6, - 0x27, 0xfc, 0x4d, 0x37, 0x66, 0xfb, 0x91, 0x17, 0x07, 0xcb, 0x09, 0x3c, 0x1c, 0xc9, 0xa1, 0x09, - 0x84, 0xea, 0x34, 0x22, 0x19, 0x27, 0xa3, 0x01, 0x12, 0x47, 0xdf, 0xf6, 0xc8, 0x17, 0x15, 0x48, - 0xf2, 0x18, 0x10, 0xe9, 0xfb, 0x9e, 0x2f, 0x90, 0xfa, 0xa5, 0x01, 0x5a, 0x8a, 0x6e, 0x9f, 0xc5, - 0x6e, 0xcf, 0x92, 0x33, 0x52, 0xb7, 0xac, 0x81, 0x24, 0x01, 0x17, 0x92, 0xfc, 0x51, 0x56, 0x24, - 0x82, 0xd0, 0xbb, 0xad, 0xbc, 0x9c, 0xd8, 0x2e, 0xfe, 0x82, 0xd0, 0xaa, 0xb5, 0x6d, 0x0b, 0xa9, - 0x77, 0x77, 0x2a, 0xfe, 0xd8, 0x50, 0xbb, 0xd3, 0x6f, 0x2b, 0x90, 0x91, 0x5e, 0x13, 0x91, 0x97, - 0x06, 0x7b, 0x75, 0xe4, 0xf7, 0x5f, 0x18, 0xb4, 0xb9, 0x10, 0xc3, 0x45, 0x44, 0x74, 0x9e, 0xcc, - 0x06, 0x88, 0xf8, 0x5d, 0x33, 0x7a, 0xa2, 0x12, 0xac, 0x6f, 0x28, 0x90, 0x0e, 0x9e, 0x7b, 0x44, - 0xaa, 0x43, 0xe7, 0x23, 0x97, 0x48, 0x75, 0xe8, 0x7a, 0x81, 0xa2, 0x5e, 0x42, 0x40, 0x17, 0xc8, - 0xb3, 0x01, 0x20, 0xdd, 0x6f, 0x83, 0x2a, 0x2a, 0x61, 0xfa, 0x9e, 0x02, 0x63, 0xe1, 0xe7, 0x40, - 0xe4, 0xe5, 0x81, 0xfa, 0x92, 0x02, 0x86, 0xf9, 0x57, 0x8e, 0x40, 0x21, 0x20, 0xbe, 0x88, 0x10, - 0x9f, 0x27, 0x17, 0x7a, 0x40, 0x44, 0x25, 0x2a, 0xbe, 0xeb, 0x87, 0xfe, 0x9e, 0x90, 0x2f, 0x2b, - 0x30, 0x22, 0x67, 0x9f, 0x44, 0x1a, 0xd7, 0x1e, 0xf9, 0x65, 0x91, 0xc6, 0xb5, 0x57, 0x76, 0x8d, - 0x7a, 0x06, 0xe1, 0x4d, 0x92, 0x89, 0x00, 0x5e, 0x90, 0x32, 0xf3, 0xdb, 0x22, 0x3b, 0x08, 0xdf, - 0x91, 0x7e, 0x7c, 0x88, 0xe6, 0x10, 0xd1, 0x19, 0x72, 0x3a, 0x40, 0x84, 0xaf, 0x62, 0x4b, 0x01, - 0xae, 0xef, 0x28, 0x90, 0x91, 0x92, 0x61, 0x22, 0x95, 0xbe, 0x3b, 0x4f, 0x27, 0x52, 0xe9, 0x7b, - 0xe4, 0xd8, 0xa8, 0x97, 0x11, 0xcf, 0x73, 0xaa, 0xb4, 0xfd, 0x60, 0x2b, 0x9e, 0x69, 0xd5, 0xd6, - 0xb0, 0xd7, 0x95, 0xcb, 0xe4, 0xbb, 0x0a, 0x8c, 0x86, 0x72, 0x5c, 0x48, 0xb1, 0x6f, 0x6f, 0xe1, - 0x44, 0x9c, 0xfc, 0xcb, 0x83, 0x13, 0x08, 0x80, 0x57, 0x10, 0xe0, 0x45, 0xf5, 0xd9, 0x4e, 0x80, - 0x42, 0x62, 0x61, 0x88, 0x5f, 0x54, 0x20, 0x1d, 0x24, 0x97, 0x44, 0xae, 0xcd, 0xce, 0x04, 0x9b, - 0xc8, 0xb5, 0xd9, 0x95, 0xaf, 0xa2, 0xe6, 0x10, 0x16, 0x51, 0xdb, 0xa6, 0xda, 0x6d, 0xe8, 0x16, - 0x83, 0xf0, 0x1e, 0x3a, 0x13, 0x95, 0x9d, 0x68, 0x63, 0x1d, 0x7a, 0x47, 0x93, 0x8f, 0xda, 0x4b, - 0xe5, 0xd7, 0x63, 0x3d, 0xac, 0xa6, 0x8b, 0x8c, 0x24, 0x53, 0xf0, 0x4b, 0x0a, 0x0c, 0x8b, 0xe7, - 0x1a, 0x91, 0x7e, 0x42, 0xf8, 0x49, 0xc7, 0xe0, 0x10, 0xba, 0x9d, 0x84, 0x06, 0xe7, 0xd4, 0x81, - 0x41, 0x3c, 0xf0, 0x88, 0xc4, 0x10, 0x7e, 0x04, 0x72, 0x12, 0x0c, 0x75, 0xce, 0x49, 0xc2, 0xf0, - 0xab, 0x0a, 0xa4, 0xfc, 0x47, 0x35, 0x24, 0xca, 0x0b, 0xea, 0x78, 0x17, 0x94, 0x7f, 0x71, 0xa0, - 0xb6, 0x02, 0x49, 0xb7, 0xf3, 0x80, 0xa1, 0xd6, 0xf0, 0x2e, 0x36, 0x22, 0xbf, 0x03, 0x8b, 0xb6, - 0x31, 0xdd, 0x0f, 0xcc, 0xa2, 0x6d, 0x4c, 0x8f, 0x07, 0x66, 0xea, 0x05, 0xc4, 0xf4, 0x0c, 0x39, - 0x2b, 0xd9, 0x98, 0x6a, 0x27, 0xac, 0xaf, 0x29, 0x30, 0x2c, 0xa8, 0x23, 0xa7, 0x28, 0xfc, 0xe0, - 0x2c, 0xff, 0x52, 0x74, 0xd3, 0x8e, 0xe7, 0x76, 0xbe, 0x79, 0x21, 0x6a, 0x04, 0x94, 0xe2, 0xbb, - 0xac, 0xe0, 0x09, 0xf3, 0xf2, 0xd6, 0xec, 0xaa, 0x1b, 0xe9, 0xe5, 0x49, 0xcf, 0x16, 0x8f, 0x0a, - 0xa5, 0x97, 0xe5, 0xad, 0xca, 0x12, 0xf9, 0xa6, 0x82, 0x7f, 0xf4, 0xa5, 0x7d, 0x7b, 0x1e, 0x69, - 0xde, 0x7a, 0x25, 0x82, 0x45, 0x9a, 0xb7, 0x9e, 0x17, 0xf3, 0xea, 0x2c, 0xa2, 0xca, 0x91, 0x19, - 0x79, 0x35, 0xb1, 0x76, 0xe2, 0x05, 0xc8, 0xfb, 0x0a, 0xa4, 0x83, 0x2b, 0xc0, 0x48, 0x83, 0xd6, - 0x79, 0x83, 0x1f, 0x69, 0xd0, 0xba, 0x6e, 0x15, 0xd5, 0x3c, 0x02, 0x99, 0x22, 0x24, 0x00, 0xf2, - 0xd0, 0xf6, 0x04, 0x88, 0x27, 0x30, 0xc4, 0x7d, 0x8a, 0x17, 0xfa, 0xdf, 0xea, 0xf4, 0xf7, 0xf9, - 0xc3, 0x1e, 0xc4, 0x21, 0xce, 0xa7, 0xec, 0x37, 0xfc, 0x8e, 0x02, 0x19, 0x39, 0x22, 0x12, 0x35, - 0xf1, 0xdd, 0xd1, 0x87, 0x5e, 0x93, 0x12, 0xfa, 0xf3, 0x92, 0x12, 0x0d, 0x0f, 0xcd, 0xf4, 0xf0, - 0x04, 0x25, 0x02, 0x49, 0x63, 0x98, 0x63, 0xce, 0x2f, 0x44, 0xfa, 0xd8, 0x7a, 0xe9, 0x46, 0x2a, - 0xd2, 0x31, 0x0f, 0xdf, 0xae, 0xf4, 0xb4, 0xf6, 0xac, 0x81, 0x04, 0xe1, 0xd7, 0x14, 0x3c, 0x19, - 0xfb, 0xb7, 0x10, 0x57, 0x06, 0x0c, 0x49, 0xf7, 0x5f, 0x41, 0xdd, 0x01, 0x6c, 0xf5, 0x2c, 0xc2, - 0x99, 0x26, 0x93, 0xf2, 0xe6, 0xe3, 0xf7, 0xfc, 0x63, 0x05, 0xce, 0xf7, 0x8b, 0xef, 0x91, 0xc5, - 0xa8, 0xed, 0x7f, 0xb0, 0xe0, 0x63, 0x7e, 0xe9, 0x44, 0x3c, 0xc2, 0x26, 0x52, 0xcd, 0x49, 0x43, - 0xa9, 0x7b, 0x6c, 0x96, 0x45, 0x3c, 0x8e, 0xed, 0xe4, 0x7f, 0xad, 0x1c, 0x16, 0x7d, 0x42, 0x24, - 0x2e, 0x79, 0xe3, 0x58, 0x91, 0xbf, 0x40, 0xfc, 0x6f, 0x1e, 0x97, 0xfc, 0xd0, 0xbd, 0xa7, 0x63, - 0x10, 0xe4, 0x2f, 0x0f, 0x0b, 0x2a, 0x7e, 0xe2, 0xc8, 0x5d, 0x73, 0xc8, 0xaf, 0x1d, 0x99, 0x4e, - 0x60, 0x7d, 0x15, 0xb1, 0x16, 0xc8, 0x95, 0x2e, 0xac, 0xc5, 0x77, 0x0f, 0x8b, 0x0d, 0x3e, 0x21, - 0xdf, 0x57, 0x30, 0x7c, 0x14, 0x0e, 0x57, 0x91, 0x6b, 0x47, 0x0b, 0x6e, 0x71, 0xe4, 0xaf, 0x1e, - 0x27, 0x22, 0xd6, 0x23, 0x3a, 0xf3, 0xc8, 0x2e, 0x97, 0x1c, 0xd1, 0x38, 0xec, 0x6d, 0xa4, 0x83, - 0x40, 0x57, 0xa4, 0x9d, 0xee, 0x8c, 0x9c, 0x45, 0xda, 0xe9, 0xae, 0xd8, 0x99, 0xfa, 0x0c, 0x22, - 0x3a, 0x4d, 0xa6, 0x65, 0x44, 0xc5, 0x77, 0x79, 0xec, 0xed, 0xc9, 0xe2, 0xe5, 0x1f, 0xfe, 0xfb, - 0xec, 0xa9, 0x1f, 0x1e, 0xcc, 0x2a, 0x3f, 0x3e, 0x98, 0x55, 0x7e, 0x72, 0x30, 0xab, 0xfc, 0xdb, - 0xc1, 0xac, 0xf2, 0xf5, 0x0f, 0x67, 0x4f, 0xfd, 0xf8, 0xc3, 0xd9, 0x53, 0x3f, 0xf9, 0x70, 0xf6, - 0xd4, 0x3b, 0x29, 0x9f, 0x79, 0x39, 0x89, 0x61, 0xdf, 0x6b, 0xff, 0x1b, 0x00, 0x00, 0xff, 0xff, - 0x01, 0x83, 0xa5, 0x0e, 0x32, 0x58, 0x00, 0x00, + 0xdb, 0x18, 0xd8, 0x47, 0xa3, 0xb1, 0xf8, 0x06, 0x0f, 0xec, 0x1f, 0x59, 0x32, 0x1b, 0x5e, 0xdd, + 0xe3, 0xf7, 0x01, 0xf9, 0x9f, 0x28, 0x70, 0x71, 0x30, 0xdd, 0x20, 0x1a, 0x5f, 0x77, 0xae, 0x3c, + 0xcd, 0x2f, 0x75, 0xc0, 0x1f, 0x48, 0xbd, 0xd2, 0xc8, 0xe6, 0x69, 0x4f, 0xb8, 0xfa, 0xa5, 0x18, + 0xe4, 0x03, 0x71, 0x86, 0x36, 0x9a, 0x86, 0xed, 0x78, 0x64, 0x2c, 0xb8, 0x73, 0x8c, 0xe3, 0xcd, + 0xc8, 0x39, 0x48, 0x57, 0xec, 0x7a, 0xa3, 0x46, 0x3d, 0x6a, 0x88, 0xcc, 0xa0, 0x76, 0x01, 0xb9, + 0x06, 0xd3, 0x81, 0x41, 0x28, 0x6d, 0x9b, 0x56, 0x95, 0x3a, 0x0d, 0xc7, 0xb4, 0x3c, 0x11, 0x2b, + 0x9e, 0x0a, 0x2a, 0x6f, 0xb6, 0xeb, 0xc8, 0x9b, 0x90, 0x6b, 0x13, 0x49, 0x0f, 0xdd, 0xd9, 0x70, + 0xf1, 0xe9, 0x2e, 0x0a, 0x44, 0xd1, 0x66, 0xdc, 0x1e, 0x30, 0x31, 0x9f, 0x63, 0xc4, 0xe1, 0x06, + 0x93, 0x1a, 0x25, 0xdd, 0x13, 0xaf, 0x12, 0x07, 0xb3, 0x3c, 0x99, 0x80, 0x72, 0xc1, 0x53, 0x3f, + 0x0f, 0x2f, 0x2c, 0x39, 0x54, 0xf7, 0xe8, 0xe1, 0xf2, 0xf0, 0x2d, 0xf3, 0xa1, 0x03, 0x55, 0x0e, + 0x1f, 0xa8, 0xda, 0x82, 0xf9, 0xfe, 0xfc, 0x85, 0xf9, 0x7e, 0x1b, 0x92, 0x0e, 0x96, 0x08, 0x15, + 0xba, 0x3e, 0xc8, 0x4a, 0xeb, 0x66, 0x27, 0x98, 0xa8, 0xcf, 0x81, 0x7a, 0x78, 0xab, 0xe0, 0xe5, + 0xe5, 0x2f, 0xc0, 0x85, 0xc8, 0x56, 0x02, 0xdb, 0x16, 0x0c, 0x73, 0xb6, 0xfe, 0xbe, 0x72, 0x3c, + 0x70, 0xbe, 0x59, 0x13, 0xbc, 0xd4, 0x3f, 0x57, 0x60, 0xaa, 0x57, 0xeb, 0x2e, 0x1d, 0x3c, 0x54, + 0xf8, 0xb1, 0x08, 0x2d, 0xbb, 0x05, 0x23, 0x15, 0x7f, 0xd9, 0x31, 0x2d, 0x39, 0xca, 0xfe, 0x94, + 0x09, 0x28, 0x17, 0xf0, 0xc5, 0x9d, 0xe7, 0xe8, 0x95, 0xe0, 0x5d, 0x21, 0x7e, 0xa8, 0x9f, 0x81, + 0xb3, 0xbd, 0x47, 0xca, 0xf5, 0xe5, 0x46, 0x84, 0x8e, 0xf3, 0x81, 0x1d, 0xa2, 0xdd, 0xea, 0x63, + 0x38, 0xd7, 0x9b, 0xb1, 0x98, 0x8c, 0x07, 0x90, 0x91, 0xf8, 0x09, 0x7b, 0x59, 0x3c, 0xea, 0x84, + 0xc8, 0x3c, 0xd4, 0x6b, 0x90, 0xbb, 0x63, 0x97, 0x35, 0x5a, 0x35, 0x5d, 0xcf, 0x69, 0x89, 0x1c, + 0x97, 0x7e, 0xde, 0xe4, 0x7f, 0x29, 0x70, 0xa6, 0x07, 0xd5, 0xc7, 0x70, 0xb8, 0xfa, 0x2c, 0x8c, + 0x38, 0x4d, 0xcb, 0x32, 0xad, 0x6a, 0xe9, 0x91, 0x5d, 0xf6, 0x0f, 0xb4, 0x51, 0x89, 0x4b, 0x87, + 0xe2, 0xc4, 0x9a, 0x8c, 0xe0, 0x76, 0xc7, 0x2e, 0xbb, 0xf9, 0x69, 0x88, 0xdf, 0xb1, 0xcb, 0x9d, + 0x2a, 0xa8, 0x5e, 0x82, 0xec, 0x1d, 0xbb, 0x1c, 0x16, 0xcd, 0x34, 0x24, 0x1f, 0xd9, 0xe5, 0xf6, + 0x8c, 0x0e, 0x3d, 0xb2, 0xcb, 0xab, 0x86, 0xba, 0x02, 0x13, 0x52, 0x53, 0x21, 0x8f, 0x97, 0x21, + 0xfe, 0xc8, 0x2e, 0x8b, 0xb5, 0x3d, 0xdb, 0xb1, 0x3d, 0xe0, 0x9f, 0x14, 0xe1, 0x7f, 0x5e, 0x04, + 0x01, 0xb1, 0xa6, 0x97, 0xaf, 0x03, 0xb4, 0x13, 0x45, 0xc9, 0x14, 0x64, 0x6f, 0xdd, 0xd7, 0xee, + 0x6f, 0x6d, 0xae, 0xde, 0x5b, 0x29, 0x6d, 0x6c, 0x2e, 0x2c, 0xdd, 0xdd, 0xc8, 0x9e, 0x22, 0x13, + 0x30, 0xba, 0x79, 0x5b, 0x5b, 0x59, 0x58, 0xf6, 0x8b, 0x94, 0xcb, 0xcf, 0x41, 0xca, 0xcf, 0xc2, + 0x94, 0xd2, 0x10, 0xc7, 0x00, 0x02, 0xf2, 0x8d, 0xac, 0x72, 0xf5, 0x47, 0xcf, 0x43, 0x52, 0xf8, + 0xdd, 0xdf, 0x51, 0x60, 0x44, 0xfe, 0x7b, 0x00, 0xa4, 0x30, 0xd8, 0x8b, 0x7f, 0x5f, 0x0c, 0xf9, + 0xe2, 0xc0, 0xed, 0xb9, 0x2c, 0xd4, 0x17, 0xde, 0xff, 0xa7, 0xff, 0xf8, 0x56, 0xec, 0x59, 0x32, + 0x57, 0x14, 0x67, 0x82, 0xa2, 0xfc, 0xe7, 0x02, 0x8a, 0xef, 0x0a, 0xc5, 0x79, 0x42, 0x7e, 0x45, + 0x81, 0x61, 0xff, 0xac, 0x12, 0x95, 0x07, 0x16, 0xfe, 0xeb, 0x02, 0xf9, 0xcb, 0x83, 0x34, 0x15, + 0x58, 0x54, 0xc4, 0x72, 0x8e, 0xe4, 0x03, 0x2c, 0x22, 0x65, 0x5c, 0x82, 0xe1, 0xc0, 0x10, 0x3e, + 0x06, 0x27, 0x2f, 0xf4, 0x7f, 0x2e, 0xce, 0x11, 0xcc, 0x0f, 0xfa, 0xae, 0x5c, 0x9d, 0xc1, 0xfe, + 0xb3, 0x64, 0x2c, 0xe8, 0x9f, 0xbf, 0x5b, 0x7f, 0x0f, 0x12, 0x98, 0xf9, 0x79, 0xb1, 0x0f, 0x27, + 0xbf, 0xc7, 0x23, 0x3d, 0x91, 0x57, 0xcf, 0x63, 0xaf, 0x79, 0x92, 0x0b, 0xf7, 0x2a, 0x8d, 0xf9, + 0x09, 0x7f, 0xd3, 0x8d, 0xd9, 0x7e, 0xe4, 0xc5, 0xc1, 0x72, 0x02, 0x0f, 0x47, 0x72, 0x68, 0x02, + 0xa1, 0x3a, 0x8d, 0x48, 0xc6, 0xc9, 0x68, 0x80, 0xc4, 0xd1, 0xb7, 0x3d, 0xf2, 0x45, 0x05, 0x92, + 0x3c, 0x06, 0x44, 0xfa, 0xbe, 0xe7, 0x0b, 0xa4, 0x7e, 0x69, 0x80, 0x96, 0xa2, 0xdb, 0x67, 0xb1, + 0xdb, 0xb3, 0xe4, 0x8c, 0xd4, 0x2d, 0x6b, 0x20, 0x49, 0xc0, 0x85, 0x24, 0x7f, 0x94, 0x15, 0x89, + 0x20, 0xf4, 0x6e, 0x2b, 0x2f, 0x27, 0xb6, 0x8b, 0xbf, 0x20, 0xb4, 0x6a, 0x6d, 0xdb, 0x42, 0xea, + 0xdd, 0x9d, 0x8a, 0x3f, 0x36, 0xd4, 0xee, 0xf4, 0xdb, 0x0a, 0x64, 0xa4, 0xd7, 0x44, 0xe4, 0xa5, + 0xc1, 0x5e, 0x1d, 0xf9, 0xfd, 0x17, 0x06, 0x6d, 0x2e, 0xc4, 0x70, 0x11, 0x11, 0x9d, 0x27, 0xb3, + 0x01, 0x22, 0x7e, 0xd7, 0x8c, 0x9e, 0xa8, 0x04, 0xeb, 0x1b, 0x0a, 0xa4, 0x83, 0xe7, 0x1e, 0x91, + 0xea, 0xd0, 0xf9, 0xc8, 0x25, 0x52, 0x1d, 0xba, 0x5e, 0xa0, 0xa8, 0x97, 0x10, 0xd0, 0x05, 0xf2, + 0x6c, 0x00, 0x48, 0xf7, 0xdb, 0xa0, 0x8a, 0x4a, 0x98, 0xbe, 0xa7, 0xc0, 0x58, 0xf8, 0x39, 0x10, + 0x79, 0x79, 0xa0, 0xbe, 0xa4, 0x80, 0x61, 0xfe, 0x95, 0x23, 0x50, 0x08, 0x88, 0x2f, 0x22, 0xc4, + 0xe7, 0xc9, 0x85, 0x1e, 0x10, 0x51, 0x89, 0x8a, 0xef, 0xfa, 0xa1, 0xbf, 0x27, 0xe4, 0xcb, 0x0a, + 0x8c, 0xc8, 0xd9, 0x27, 0x91, 0xc6, 0xb5, 0x47, 0x7e, 0x59, 0xa4, 0x71, 0xed, 0x95, 0x5d, 0xa3, + 0x9e, 0x41, 0x78, 0x93, 0x64, 0x22, 0x80, 0x17, 0xa4, 0xcc, 0xfc, 0xb6, 0xc8, 0x0e, 0xc2, 0x77, + 0xa4, 0x1f, 0x1f, 0xa2, 0x39, 0x44, 0x74, 0x86, 0x9c, 0x0e, 0x10, 0xe1, 0xab, 0xd8, 0x52, 0x80, + 0xeb, 0x3b, 0x0a, 0x64, 0xa4, 0x64, 0x98, 0x48, 0xa5, 0xef, 0xce, 0xd3, 0x89, 0x54, 0xfa, 0x1e, + 0x39, 0x36, 0xea, 0x65, 0xc4, 0xf3, 0x9c, 0x2a, 0x6d, 0x3f, 0xd8, 0x8a, 0x67, 0x5a, 0xb5, 0x35, + 0xec, 0x75, 0xe5, 0x32, 0xf9, 0xae, 0x02, 0xa3, 0xa1, 0x1c, 0x17, 0x52, 0xec, 0xdb, 0x5b, 0x38, + 0x11, 0x27, 0xff, 0xf2, 0xe0, 0x04, 0x02, 0xe0, 0x15, 0x04, 0x78, 0x51, 0x7d, 0xb6, 0x13, 0xa0, + 0x90, 0x58, 0x18, 0xe2, 0x17, 0x15, 0x48, 0x07, 0xc9, 0x25, 0x91, 0x6b, 0xb3, 0x33, 0xc1, 0x26, + 0x72, 0x6d, 0x76, 0xe5, 0xab, 0xa8, 0x39, 0x84, 0x45, 0xd4, 0xb6, 0xa9, 0x76, 0x1b, 0xba, 0xc5, + 0x20, 0xbc, 0x87, 0xce, 0x44, 0x65, 0x27, 0xda, 0x58, 0x87, 0xde, 0xd1, 0xe4, 0xa3, 0xf6, 0x52, + 0xf9, 0xf5, 0x58, 0x0f, 0xab, 0xe9, 0x22, 0x23, 0xc9, 0x14, 0xfc, 0x92, 0x02, 0xc3, 0xe2, 0xb9, + 0x46, 0xa4, 0x9f, 0x10, 0x7e, 0xd2, 0x31, 0x38, 0x84, 0x6e, 0x27, 0xa1, 0xc1, 0x39, 0x75, 0x60, + 0x10, 0x0f, 0x3c, 0x22, 0x31, 0x84, 0x1f, 0x81, 0x9c, 0x04, 0x43, 0x9d, 0x73, 0x92, 0x30, 0xfc, + 0xaa, 0x02, 0x29, 0xff, 0x51, 0x0d, 0x89, 0xf2, 0x82, 0x3a, 0xde, 0x05, 0xe5, 0x5f, 0x1c, 0xa8, + 0xad, 0x40, 0xd2, 0xed, 0x3c, 0x60, 0xa8, 0x35, 0xbc, 0x8b, 0x8d, 0xc8, 0xef, 0xc0, 0xa2, 0x6d, + 0x4c, 0xf7, 0x03, 0xb3, 0x68, 0x1b, 0xd3, 0xe3, 0x81, 0x99, 0x7a, 0x01, 0x31, 0x3d, 0x43, 0xce, + 0x4a, 0x36, 0xa6, 0xda, 0x09, 0xeb, 0x6b, 0x0a, 0x0c, 0x0b, 0xea, 0xc8, 0x29, 0x0a, 0x3f, 0x38, + 0xcb, 0xbf, 0x14, 0xdd, 0xb4, 0xe3, 0xb9, 0x9d, 0x6f, 0x5e, 0x88, 0x1a, 0x01, 0xa5, 0xf8, 0x2e, + 0x2b, 0x78, 0xc2, 0xbc, 0xbc, 0x35, 0xbb, 0xea, 0x46, 0x7a, 0x79, 0xd2, 0xb3, 0xc5, 0xa3, 0x42, + 0xe9, 0x65, 0x79, 0xab, 0xb2, 0x44, 0xbe, 0xa9, 0xe0, 0x1f, 0x7d, 0x69, 0xdf, 0x9e, 0x47, 0x9a, + 0xb7, 0x5e, 0x89, 0x60, 0x91, 0xe6, 0xad, 0xe7, 0xc5, 0xbc, 0x3a, 0x8b, 0xa8, 0x72, 0x64, 0x46, + 0x5e, 0x4d, 0xac, 0x9d, 0x78, 0x01, 0xf2, 0xbe, 0x02, 0xe9, 0xe0, 0x0a, 0x30, 0xd2, 0xa0, 0x75, + 0xde, 0xe0, 0x47, 0x1a, 0xb4, 0xae, 0x5b, 0x45, 0x35, 0x8f, 0x40, 0xa6, 0x08, 0x09, 0x80, 0x3c, + 0xb4, 0x3d, 0x01, 0xe2, 0x09, 0x0c, 0x71, 0x9f, 0xe2, 0x85, 0xfe, 0xb7, 0x3a, 0xfd, 0x7d, 0xfe, + 0xb0, 0x07, 0x71, 0x88, 0xf3, 0x29, 0xfb, 0x0d, 0xbf, 0xa3, 0x40, 0x46, 0x8e, 0x88, 0x44, 0x4d, + 0x7c, 0x77, 0xf4, 0xa1, 0xd7, 0xa4, 0x84, 0xfe, 0xbc, 0xa4, 0x44, 0xc3, 0x43, 0x33, 0x3d, 0x3c, + 0x41, 0x89, 0x40, 0xd2, 0x18, 0xe6, 0x98, 0xf3, 0x0b, 0x91, 0x3e, 0xb6, 0x5e, 0xba, 0x91, 0x8a, + 0x74, 0xcc, 0xc3, 0xb7, 0x2b, 0x3d, 0xad, 0x3d, 0x6b, 0x20, 0x41, 0xf8, 0x35, 0x05, 0x4f, 0xc6, + 0xfe, 0x2d, 0xc4, 0x95, 0x01, 0x43, 0xd2, 0xfd, 0x57, 0x50, 0x77, 0x00, 0x5b, 0x3d, 0x8b, 0x70, + 0xa6, 0xc9, 0xa4, 0xbc, 0xf9, 0xf8, 0x3d, 0xff, 0x58, 0x81, 0xf3, 0xfd, 0xe2, 0x7b, 0x64, 0x31, + 0x6a, 0xfb, 0x1f, 0x2c, 0xf8, 0x98, 0x5f, 0x3a, 0x11, 0x8f, 0xb0, 0x89, 0x54, 0x73, 0xd2, 0x50, + 0xea, 0x1e, 0x9b, 0x65, 0x11, 0x8f, 0x63, 0x3b, 0xf9, 0x5f, 0x2b, 0x87, 0x45, 0x9f, 0x10, 0x89, + 0x4b, 0xde, 0x38, 0x56, 0xe4, 0x2f, 0x10, 0xff, 0x9b, 0xc7, 0x25, 0x3f, 0x74, 0xef, 0xe9, 0x18, + 0x04, 0xf9, 0xcb, 0xc3, 0x82, 0x8a, 0x9f, 0x38, 0x72, 0xd7, 0x1c, 0xf2, 0x6b, 0x47, 0xa6, 0x13, + 0x58, 0x5f, 0x45, 0xac, 0x05, 0x72, 0xa5, 0x0b, 0x6b, 0xf1, 0xdd, 0xc3, 0x62, 0x83, 0x4f, 0xc8, + 0xf7, 0x15, 0x0c, 0x1f, 0x85, 0xc3, 0x55, 0xe4, 0xda, 0xd1, 0x82, 0x5b, 0x1c, 0xf9, 0xab, 0xc7, + 0x89, 0x88, 0xf5, 0x88, 0xce, 0x3c, 0xb2, 0xcb, 0x25, 0x47, 0x34, 0x0e, 0x7b, 0x1b, 0xe9, 0x20, + 0xd0, 0x15, 0x69, 0xa7, 0x3b, 0x23, 0x67, 0x91, 0x76, 0xba, 0x2b, 0x76, 0xa6, 0x3e, 0x83, 0x88, + 0x4e, 0x93, 0x69, 0x19, 0x51, 0xf1, 0x5d, 0x1e, 0x7b, 0x7b, 0xb2, 0x78, 0xf9, 0x87, 0xff, 0x3e, + 0x7b, 0xea, 0x87, 0x07, 0xb3, 0xca, 0x8f, 0x0f, 0x66, 0x95, 0x9f, 0x1c, 0xcc, 0x2a, 0xff, 0x76, + 0x30, 0xab, 0x7c, 0xfd, 0xc3, 0xd9, 0x53, 0x3f, 0xfe, 0x70, 0xf6, 0xd4, 0x4f, 0x3e, 0x9c, 0x3d, + 0xf5, 0x4e, 0xca, 0x67, 0x5e, 0x4e, 0x62, 0xd8, 0xf7, 0xda, 0xff, 0x06, 0x00, 0x00, 0xff, 0xff, + 0x7a, 0x6a, 0x2f, 0x42, 0x32, 0x58, 0x00, 0x00, } diff --git a/pkg/server/serverpb/status.proto b/pkg/server/serverpb/status.proto index 0e1817b96328..66a9967abe52 100644 --- a/pkg/server/serverpb/status.proto +++ b/pkg/server/serverpb/status.proto @@ -885,12 +885,12 @@ message StatementsResponse { message ExtendedStatementStatisticsKey { cockroach.sql.StatementStatisticsKey key_data = 1 [(gogoproto.nullable) = false]; int32 node_id = 2 [(gogoproto.customname) = "NodeID", - (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.NodeID"]; + (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.NodeID"]; } message CollectedStatementStatistics { ExtendedStatementStatisticsKey key = 1 [(gogoproto.nullable) = false]; - string id = 3 [(gogoproto.customname) = "ID", + uint64 id = 3 [(gogoproto.customname) = "ID", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.StmtID"]; cockroach.sql.StatementStatistics stats = 2 [(gogoproto.nullable) = false]; } @@ -905,7 +905,7 @@ message StatementsResponse { message ExtendedCollectedTransactionStatistics { cockroach.sql.CollectedTransactionStatistics stats_data = 1 [(gogoproto.nullable) = false]; int32 node_id = 2 [(gogoproto.customname) = "NodeID", - (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.NodeID"]; + (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.NodeID"]; } // Transactions is transaction-level statistics for the collection of diff --git a/pkg/server/status_test.go b/pkg/server/status_test.go index c3ba764a2ee7..470440dfaa6c 100644 --- a/pkg/server/status_test.go +++ b/pkg/server/status_test.go @@ -1649,7 +1649,7 @@ func TestStatusAPITransactions(t *testing.T) { // linked to StatementIDs for statements in the response. for _, stmtID := range respTransaction.StatsData.StatementIDs { if _, found := statementIDs[stmtID]; !found { - t.Fatalf("app: %s, expected stmtID: %s not found in StatementResponse.", appName, stmtID) + t.Fatalf("app: %s, expected stmtID: %d not found in StatementResponse.", appName, stmtID) } } stats := respTransaction.StatsData.Stats diff --git a/pkg/sql/app_stats.go b/pkg/sql/app_stats.go index d5c700dab4af..1dd96ff89124 100644 --- a/pkg/sql/app_stats.go +++ b/pkg/sql/app_stats.go @@ -44,14 +44,16 @@ type stmtKey struct { implicitTxn bool } -const invalidStmtID = "" +const invalidStmtID = 0 // txnKey is the hashed string constructed using the individual statement IDs // that comprise the transaction. -type txnKey string +type txnKey uint64 // appStats holds per-application statistics. type appStats struct { + // TODO(arul): This can be refactored to have a RWLock instead, and have all + // usages acquire a read lock whenever appropriate. See #55285. syncutil.Mutex st *cluster.Settings @@ -155,7 +157,10 @@ func (s stmtKey) String() string { // recordStatement saves per-statement statistics. // -// samplePlanDescription can be nil, as these are only sampled periodically per unique fingerprint. +// samplePlanDescription can be nil, as these are only sampled periodically +// per unique fingerprint. +// recordStatement always returns a valid stmtID corresponding to the given +// stmt regardless of whether the statement is actually recorded or not. func (a *appStats) recordStatement( stmt *Statement, samplePlanDescription *roachpb.ExplainTreePlanNode, @@ -168,28 +173,27 @@ func (a *appStats) recordStatement( parseLat, planLat, runLat, svcLat, ovhLat float64, stats topLevelQueryStats, ) roachpb.StmtID { - if !stmtStatsEnable.Get(&a.st.SV) { - return invalidStmtID - } - createIfNonExistent := true - // If the statement is below the latency threshold, we don't need to create - // an entry in the stmts map for it. - if t := sqlStatsCollectionLatencyThreshold.Get(&a.st.SV); t > 0 && t.Seconds() >= svcLat { + // If the statement is below the latency threshold, or stats aren't being + // recorded we don't need to create an entry in the stmts map for it. We do + // still need stmtID for transaction level metrics tracking. + t := sqlStatsCollectionLatencyThreshold.Get(&a.st.SV) + if !stmtStatsEnable.Get(&a.st.SV) || (t > 0 && t.Seconds() >= svcLat) { createIfNonExistent = false } // Get the statistics object. - s := a.getStatsForStmt( + s, stmtID := a.getStatsForStmt( stmt, implicitTxn, err, createIfNonExistent, ) - // This statement was below the latency threshold, and therefore shouldn't - // be recorded. We still need to return the statement ID though, for - // transaction level metrics tracking. + // This statement was below the latency threshold or sql stats aren't being + // recorded. Either way, we don't need to record anything in the stats object + // for this statement, though we do need to return the statement ID for + // transaction level metrics collection. if !createIfNonExistent { - return s.ID + return stmtID } // Collect the per-statement statistics. @@ -223,10 +227,12 @@ func (a *appStats) recordStatement( return s.ID } -// getStatsForStmt retrieves the per-stmt stat object. +// getStatsForStmt retrieves the per-stmt stat object. Regardless of if a valid +// stat object is returned or not, we always return the correct stmtID +// for the given stmt. func (a *appStats) getStatsForStmt( stmt *Statement, implicitTxn bool, err error, createIfNonexistent bool, -) *stmtStats { +) (*stmtStats, roachpb.StmtID) { // Extend the statement key with various characteristics, so // that we use separate buckets for the different situations. key := stmtKey{ @@ -240,13 +246,19 @@ func (a *appStats) getStatsForStmt( key.anonymizedStmt = anonymizeStmt(stmt.AST) } - stmtID := constructStatementIDFromStmtKey(key) - - return a.getStatsForStmtWithKey(key, stmtID, createIfNonexistent) + // We first try and see if we can get by without creating a new entry for this + // key, as this allows us to not construct the statementID from scratch (which + // is an expensive operation) + s := a.getStatsForStmtWithKey(key, invalidStmtID, false /* createIfNonexistent */) + if s == nil { + stmtID := constructStatementIDFromStmtKey(key) + return a.getStatsForStmtWithKey(key, stmtID, createIfNonexistent), stmtID + } + return s, s.ID } func (a *appStats) getStatsForStmtWithKey( - key stmtKey, ID roachpb.StmtID, createIfNonexistent bool, + key stmtKey, stmtID roachpb.StmtID, createIfNonexistent bool, ) *stmtStats { a.Lock() // Retrieve the per-statement statistic object, and create it if it @@ -254,7 +266,7 @@ func (a *appStats) getStatsForStmtWithKey( s, ok := a.stmts[key] if !ok && createIfNonexistent { s = &stmtStats{} - s.ID = ID + s.ID = stmtID a.stmts[key] = s } a.Unlock() @@ -299,7 +311,7 @@ func (a *appStats) Add(other *appStats) { // Merge the statement stats. for k, v := range statMap { - s := a.getStatsForStmtWithKey(k, v.ID, true) + s := a.getStatsForStmtWithKey(k, v.ID, true /* createIfNonexistent */) s.mu.Lock() // Note that we don't need to take a lock on v because // no other thread knows about v yet. @@ -436,7 +448,7 @@ func (a *appStats) shouldSaveLogicalPlanDescription(stmt *Statement, implicitTxn // We don't know yet if we will hit an error, so we assume we don't. The worst // that can happen is that for statements that always error out, we will // always save the tree plan. - stats := a.getStatsForStmt(stmt, implicitTxn, nil /* error */, false /* createIfNonexistent */) + stats, _ := a.getStatsForStmt(stmt, implicitTxn, nil /* error */, false /* createIfNonexistent */) if stats == nil { // Save logical plan the first time we see new statement fingerprint. return true diff --git a/pkg/sql/conn_executor.go b/pkg/sql/conn_executor.go index 84d58162fb92..f41dc79983a3 100644 --- a/pkg/sql/conn_executor.go +++ b/pkg/sql/conn_executor.go @@ -13,7 +13,6 @@ package sql import ( "context" "fmt" - "hash" "io" "math" "strings" @@ -1058,7 +1057,7 @@ type connExecutor struct { // transactionStatementIDs are capped to prevent unbound expansion, but we // still need the statementID hash to disambiguate beyond the capped // statements. - transactionStatementsHash hash.Hash + transactionStatementsHash util.FNV64 } // sessionData contains the user-configurable connection variables. diff --git a/pkg/sql/conn_executor_exec.go b/pkg/sql/conn_executor_exec.go index 1ac99ac83b3e..9efd6790e18d 100644 --- a/pkg/sql/conn_executor_exec.go +++ b/pkg/sql/conn_executor_exec.go @@ -13,7 +13,6 @@ package sql import ( "context" "fmt" - "hash/fnv" "runtime/pprof" "strings" "time" @@ -34,6 +33,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sqlerrors" "github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry" "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/cockroach/pkg/util" "github.com/cockroachdb/cockroach/pkg/util/cancelchecker" "github.com/cockroachdb/cockroach/pkg/util/duration" "github.com/cockroachdb/cockroach/pkg/util/errorutil/unimplemented" @@ -1432,7 +1432,7 @@ func (ex *connExecutor) recordTransactionStart() (onTxnFinish func(txnEvent), on ex.phaseTimes[sessionTransactionReceived] = ex.phaseTimes[sessionQueryReceived] ex.phaseTimes[sessionFirstStartExecTransaction] = timeutil.Now() ex.phaseTimes[sessionMostRecentStartExecTransaction] = ex.phaseTimes[sessionFirstStartExecTransaction] - ex.extraTxnState.transactionStatementsHash = fnv.New128() + ex.extraTxnState.transactionStatementsHash = util.MakeFNV64() ex.extraTxnState.transactionStatementIDs = nil ex.extraTxnState.numRows = 0 @@ -1443,7 +1443,7 @@ func (ex *connExecutor) recordTransactionStart() (onTxnFinish func(txnEvent), on onTxnRestart = func() { ex.phaseTimes[sessionMostRecentStartExecTransaction] = timeutil.Now() ex.extraTxnState.transactionStatementIDs = nil - ex.extraTxnState.transactionStatementsHash = fnv.New128() + ex.extraTxnState.transactionStatementsHash = util.MakeFNV64() ex.extraTxnState.numRows = 0 } return onTxnFinish, onTxnRestart @@ -1458,10 +1458,8 @@ func (ex *connExecutor) recordTransaction(ev txnEvent, implicit bool, txnStart t txnRetryLat := ex.phaseTimes.getTransactionRetryLatency() commitLat := ex.phaseTimes.getCommitLatency() - key := txnKey(fmt.Sprintf("%x", ex.extraTxnState.transactionStatementsHash.Sum(nil))) - ex.statsCollector.recordTransaction( - key, + txnKey(ex.extraTxnState.transactionStatementsHash.Sum()), txnTime.Seconds(), ev, implicit, diff --git a/pkg/sql/crdb_internal.go b/pkg/sql/crdb_internal.go index e8ec655f1e92..fd4b9cf4529f 100644 --- a/pkg/sql/crdb_internal.go +++ b/pkg/sql/crdb_internal.go @@ -17,6 +17,7 @@ import ( "net" "net/url" "sort" + "strconv" "strings" "time" @@ -876,6 +877,9 @@ CREATE TABLE crdb_internal.node_statement_statistics ( }, } +// TODO(arul): Explore updating the schema below to have key be an INT and +// statement_ids be INT[] now that we've moved to having uint64 as the type of +// StmtID and TxnKey. Issue #55284 var crdbInternalTransactionStatisticsTable = virtualSchemaTable{ comment: `finer-grained transaction statistics (in-memory, not durable; local node only). ` + `This table is wiped periodically (by default, at least every two hours)`, @@ -952,7 +956,7 @@ CREATE TABLE crdb_internal.node_transaction_statistics ( } stmtIDsDatum := tree.NewDArray(types.String) for _, stmtID := range s.statementIDs { - if err := stmtIDsDatum.Append(tree.NewDString(string(stmtID))); err != nil { + if err := stmtIDsDatum.Append(tree.NewDString(strconv.FormatUint(uint64(stmtID), 10))); err != nil { return err } } @@ -962,7 +966,7 @@ CREATE TABLE crdb_internal.node_transaction_statistics ( err := addRow( tree.NewDInt(tree.DInt(nodeID)), tree.NewDString(appName), - tree.NewDString(string(txnKey)), + tree.NewDString(strconv.FormatUint(uint64(txnKey), 10)), stmtIDsDatum, tree.NewDInt(tree.DInt(s.mu.data.Count)), tree.NewDInt(tree.DInt(s.mu.data.MaxRetries)), diff --git a/pkg/sql/executor_statement_metrics.go b/pkg/sql/executor_statement_metrics.go index 4d7bff701ea5..85d17c2ecaa6 100644 --- a/pkg/sql/executor_statement_metrics.go +++ b/pkg/sql/executor_statement_metrics.go @@ -194,10 +194,10 @@ func (ex *connExecutor) recordStatementSummary( ex.extraTxnState.transactionStatementIDs, stmtID) } // Add the current statement's ID to the hash. We don't track queries issued - // by the internal executor, in which case the hash function may be nil, and + // by the internal executor, in which case the hash is uninitialized, and // can therefore be safely ignored. - if ex.extraTxnState.transactionStatementsHash != nil { - ex.extraTxnState.transactionStatementsHash.Write([]byte(stmtID)) + if ex.extraTxnState.transactionStatementsHash.IsInitialized() { + ex.extraTxnState.transactionStatementsHash.Add(uint64(stmtID)) } ex.extraTxnState.numRows += rowsAffected diff --git a/pkg/sql/logictest/testdata/logic_test/crdb_internal b/pkg/sql/logictest/testdata/logic_test/crdb_internal index 635c234ea4e4..13e66fc90398 100644 --- a/pkg/sql/logictest/testdata/logic_test/crdb_internal +++ b/pkg/sql/logictest/testdata/logic_test/crdb_internal @@ -668,9 +668,9 @@ SELECT * FROM t_53504 query ITTTI colnames SELECT node_id, application_name, key, statement_ids, count FROM crdb_internal.node_transaction_statistics where application_name = 'test_txn_statistics' ---- -node_id application_name key statement_ids count -1 test_txn_statistics 02c92b698a5cb4c59708e7de50bdbf53 {7d5470c38539309ff3b933fec35fefad,7d5470c38539309ff3b933fec35fefad,7d5470c38539309ff3b933fec35fefad} 1 -1 test_txn_statistics 5a29a80adae483c4cfa89f5b228682fb {94a482186de515de23eca3d1b443e033} 1 -1 test_txn_statistics 8199fcfefafda1121b286c08b21559c3 {7d5470c38539309ff3b933fec35fefad} 1 -1 test_txn_statistics cb83bd7423a1016e148d2d9a6d89427d {7d5470c38539309ff3b933fec35fefad,7d5470c38539309ff3b933fec35fefad} 2 -1 test_txn_statistics cd8858558756fb4a4917ccfface8c12b {ca672b3b015c5f7ca3b4d8488eb2f528} 1 +node_id application_name key statement_ids count +1 test_txn_statistics 4572719807723346491 {14727561584397653505,14727561584397653505} 2 +1 test_txn_statistics 6826443595058283584 {14727561584397653505,14727561584397653505,14727561584397653505} 1 +1 test_txn_statistics 7134109142904971730 {14727561584397653517} 1 +1 test_txn_statistics 7134109142904971742 {14727561584397653505} 1 +1 test_txn_statistics 10166963080898232577 {2484845987516053214} 1 diff --git a/pkg/util/hash.go b/pkg/util/hash.go index 27397b647ce7..b767deda5eb1 100644 --- a/pkg/util/hash.go +++ b/pkg/util/hash.go @@ -24,3 +24,35 @@ func CRC32(data []byte) uint32 { } return hash.Sum32() } + +// Magic FNV Base constant as suitable for a FNV-64 hash. +const fnvBase = uint64(14695981039346656037) +const fnvPrime = 1099511628211 + +// FNV64 encapsulates the hash state. +type FNV64 struct { + sum uint64 +} + +// MakeFNV64 initializes a new FNV64 hash state. +func MakeFNV64() FNV64 { + return FNV64{sum: fnvBase} +} + +// IsInitialized returns true if the hash struct was initialized, which happens +// automatically when created through MakeFNV64 above. +func (f *FNV64) IsInitialized() bool { + return f.sum != 0 +} + +// Add modifies the underlying FNV64 state by accumulating the given integer +// hash to the existing state. +func (f *FNV64) Add(c uint64) { + f.sum *= fnvPrime + f.sum ^= c +} + +// Sum returns the hash value accumulated till now. +func (f *FNV64) Sum() uint64 { + return f.sum +} From 9757d77b29ed3bc108161873b8dc6b521c791316 Mon Sep 17 00:00:00 2001 From: David Hartunian Date: Tue, 6 Oct 2020 14:32:11 -0400 Subject: [PATCH 5/5] ui: update admin-ui-components to 0.1.17 This change upgrades the `admin-ui-components` dependency. The latest version uses the modified protobuf type for statements IDs in order to render the Transactions Page Table. No user-facing changes to any functionality were made. Release note: none --- pkg/ui/package.json | 2 +- pkg/ui/yarn-vendor | 2 +- pkg/ui/yarn.lock | 18 +++++++++--------- 3 files changed, 11 insertions(+), 11 deletions(-) diff --git a/pkg/ui/package.json b/pkg/ui/package.json index e3011524c496..65a68d58c34c 100644 --- a/pkg/ui/package.json +++ b/pkg/ui/package.json @@ -13,7 +13,7 @@ "cypress:update-snapshots": "yarn cypress run --env updateSnapshots=true --spec 'cypress/integration/**/*.visual.spec.ts'" }, "dependencies": { - "@cockroachlabs/admin-ui-components": "^0.1.16", + "@cockroachlabs/admin-ui-components": "^0.1.17", "analytics-node": "^3.4.0-beta.1", "antd": "^3.25.2", "babel-polyfill": "^6.26.0", diff --git a/pkg/ui/yarn-vendor b/pkg/ui/yarn-vendor index f843b09514f3..27a9950ea92a 160000 --- a/pkg/ui/yarn-vendor +++ b/pkg/ui/yarn-vendor @@ -1 +1 @@ -Subproject commit f843b09514f3ca43cbd56d0612d36afdc226aa37 +Subproject commit 27a9950ea92a9ca8effd7aa5f568bac16e1d36c0 diff --git a/pkg/ui/yarn.lock b/pkg/ui/yarn.lock index d2247ff413b2..f0dfa235db03 100644 --- a/pkg/ui/yarn.lock +++ b/pkg/ui/yarn.lock @@ -1806,12 +1806,12 @@ lodash "^4.17.13" to-fast-properties "^2.0.0" -"@cockroachlabs/admin-ui-components@^0.1.16": - version "0.1.16" - resolved "https://registry.yarnpkg.com/@cockroachlabs/admin-ui-components/-/admin-ui-components-0.1.16.tgz#01060843a1f66a545d5dc7f9b2c3e5ef228ed930" - integrity sha512-RM3D90NoubKm+fUjnUnFnTz3w17Neyk25mpJORiUh9t8feBWiv0Q6ORQe3qQvyyO48dEahOmMdQz5dHXX1mgPQ== +"@cockroachlabs/admin-ui-components@^0.1.17": + version "0.1.17" + resolved "https://registry.yarnpkg.com/@cockroachlabs/admin-ui-components/-/admin-ui-components-0.1.17.tgz#8bd6d262a95229794ee6adcbb5403d85f284b24e" + integrity sha512-CLG6cG04K4Wp6qr00xz+pFw+mYUGN4qg8ZwjgzTOF1pQxB1lpwL1JqWn+cnwH6CZa134K3+VAs03X2T6T1/McQ== dependencies: - "@cockroachlabs/crdb-protobuf-client" "^0.0.2" + "@cockroachlabs/crdb-protobuf-client" "^0.0.3" "@cockroachlabs/icons" "^0.2.2" "@cockroachlabs/ui-components" "^0.2.8" "@popperjs/core" "^2.4.0" @@ -1823,10 +1823,10 @@ react-select "^1.2.1" reselect "^4.0.0" -"@cockroachlabs/crdb-protobuf-client@^0.0.2": - version "0.0.2" - resolved "https://registry.yarnpkg.com/@cockroachlabs/crdb-protobuf-client/-/crdb-protobuf-client-0.0.2.tgz#0dff7158b75971adfa18e7f32e90b9d88d6215a7" - integrity sha512-p/FyoVLeXTluRwPPwkRE6dU/cfTq01ej4vBNC64nxM4PDEcKOdlAt3xT+j9GD1m/f8VYdk7kksdFUOty+Ilklw== +"@cockroachlabs/crdb-protobuf-client@^0.0.3": + version "0.0.3" + resolved "https://registry.yarnpkg.com/@cockroachlabs/crdb-protobuf-client/-/crdb-protobuf-client-0.0.3.tgz#3ce8dd4953a1209f1895c713cf90595a15c54ab1" + integrity sha512-AXHWWW7hI05hj5fTdXgIIjfZrqfacQ/zsT83LoUsrnFUOeWZCa6qSF3qVonaR2h8FloRfEeFhC+27TDsi8RI0A== "@cockroachlabs/icons@^0.2.2": version "0.2.9"