diff --git a/pkg/sql/colexec/BUILD.bazel b/pkg/sql/colexec/BUILD.bazel index 74aa3ff1962e..ff48f4de4d99 100644 --- a/pkg/sql/colexec/BUILD.bazel +++ b/pkg/sql/colexec/BUILD.bazel @@ -72,7 +72,6 @@ go_library( "//pkg/sql/colmem", "//pkg/sql/execinfra", "//pkg/sql/execinfrapb", - "//pkg/sql/execstats/execstatspb", "//pkg/sql/parser", "//pkg/sql/rowenc", "//pkg/sql/sem/tree", diff --git a/pkg/sql/colexec/materializer.go b/pkg/sql/colexec/materializer.go index 5a73fca2eb6c..2cefb02e309a 100644 --- a/pkg/sql/colexec/materializer.go +++ b/pkg/sql/colexec/materializer.go @@ -20,7 +20,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" - "github.com/cockroachdb/cockroach/pkg/sql/execstats/execstatspb" "github.com/cockroachdb/cockroach/pkg/sql/rowenc" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/errors" @@ -170,7 +169,7 @@ func NewMaterializer( output execinfra.RowReceiver, metadataSourcesQueue []execinfrapb.MetadataSource, toClose []colexecbase.Closer, - execStatsForTrace func() *execstatspb.ComponentStats, + execStatsForTrace func() *execinfrapb.ComponentStats, cancelFlow func() context.CancelFunc, ) (*Materializer, error) { vecIdxsToConvert := make([]int, len(typs)) diff --git a/pkg/sql/colexec/stats.go b/pkg/sql/colexec/stats.go index 5dbe3eeea6cf..0995ecf3a81c 100644 --- a/pkg/sql/colexec/stats.go +++ b/pkg/sql/colexec/stats.go @@ -20,7 +20,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" - "github.com/cockroachdb/cockroach/pkg/sql/execstats/execstatspb" "github.com/cockroachdb/cockroach/pkg/util/mon" "github.com/cockroachdb/cockroach/pkg/util/timeutil" "github.com/cockroachdb/cockroach/pkg/util/tracing" @@ -166,10 +165,10 @@ type vectorizedStatsCollectorImpl struct { } // finish returns the collected stats. -func (vsc *vectorizedStatsCollectorImpl) finish() *execstatspb.ComponentStats { +func (vsc *vectorizedStatsCollectorImpl) finish() *execinfrapb.ComponentStats { numBatches, numTuples, time := vsc.batchInfoCollector.finish() - s := &execstatspb.ComponentStats{ComponentID: vsc.operatorID} + s := &execinfrapb.ComponentStats{ComponentID: vsc.operatorID} for _, memMon := range vsc.memMonitors { s.Exec.MaxAllocatedMem.Add(memMon.MaximumBytes()) @@ -250,10 +249,10 @@ type networkVectorizedStatsCollectorImpl struct { } // finish returns the collected stats. -func (nvsc *networkVectorizedStatsCollectorImpl) finish() *execstatspb.ComponentStats { +func (nvsc *networkVectorizedStatsCollectorImpl) finish() *execinfrapb.ComponentStats { numBatches, numTuples, time := nvsc.batchInfoCollector.finish() - s := &execstatspb.ComponentStats{ComponentID: nvsc.operatorID} + s := &execinfrapb.ComponentStats{ComponentID: nvsc.operatorID} s.NetRx.Latency = nvsc.latency s.NetRx.WaitTime = time @@ -283,7 +282,7 @@ func createStatsSpan( opName string, flowID string, idTagKey string, - stats *execstatspb.ComponentStats, + stats *execinfrapb.ComponentStats, ) { // We're creating a new span for every component setting the appropriate // tag so that it is displayed correctly on the flow diagram. diff --git a/pkg/sql/colflow/BUILD.bazel b/pkg/sql/colflow/BUILD.bazel index 09c6820c4c1f..2e82a2962b7a 100644 --- a/pkg/sql/colflow/BUILD.bazel +++ b/pkg/sql/colflow/BUILD.bazel @@ -20,7 +20,6 @@ go_library( "//pkg/sql/colmem", "//pkg/sql/execinfra", "//pkg/sql/execinfrapb", - "//pkg/sql/execstats/execstatspb", "//pkg/sql/flowinfra", "//pkg/sql/pgwire/pgcode", "//pkg/sql/pgwire/pgerror", diff --git a/pkg/sql/colflow/vectorized_flow.go b/pkg/sql/colflow/vectorized_flow.go index e5411da25fa0..6caa6816e460 100644 --- a/pkg/sql/colflow/vectorized_flow.go +++ b/pkg/sql/colflow/vectorized_flow.go @@ -35,7 +35,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" - "github.com/cockroachdb/cockroach/pkg/sql/execstats/execstatspb" "github.com/cockroachdb/cockroach/pkg/sql/flowinfra" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" @@ -993,12 +992,12 @@ func (s *vectorizedFlowCreator) setupOutput( s.leaves = append(s.leaves, outbox) case execinfrapb.StreamEndpointSpec_SYNC_RESPONSE: // Make the materializer, which will write to the given receiver. - var outputStatsToTrace func() *execstatspb.ComponentStats + var outputStatsToTrace func() *execinfrapb.ComponentStats if s.recordingStats { // Make a copy given that vectorizedStatsCollectorsQueue is reset and // appended to. vscq := append([]colexec.VectorizedStatsCollector(nil), s.vectorizedStatsCollectorsQueue...) - outputStatsToTrace = func() *execstatspb.ComponentStats { + outputStatsToTrace = func() *execinfrapb.ComponentStats { // TODO(radu): this is a sketchy way to use this infrastructure. We // aren't actually returning any stats, but we are creating and closing // child spans with stats. diff --git a/pkg/sql/execinfra/BUILD.bazel b/pkg/sql/execinfra/BUILD.bazel index 510128f8df22..0484cdb05f79 100644 --- a/pkg/sql/execinfra/BUILD.bazel +++ b/pkg/sql/execinfra/BUILD.bazel @@ -41,7 +41,6 @@ go_library( "//pkg/sql/catalog/descs", "//pkg/sql/catalog/hydratedtables", "//pkg/sql/execinfrapb", - "//pkg/sql/execstats/execstatspb", "//pkg/sql/rowenc", "//pkg/sql/sem/tree", "//pkg/sql/sqlliveness", diff --git a/pkg/sql/execinfra/processorsbase.go b/pkg/sql/execinfra/processorsbase.go index 79d44c94d93b..d332794cb83d 100644 --- a/pkg/sql/execinfra/processorsbase.go +++ b/pkg/sql/execinfra/processorsbase.go @@ -16,7 +16,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" - "github.com/cockroachdb/cockroach/pkg/sql/execstats/execstatspb" "github.com/cockroachdb/cockroach/pkg/sql/rowenc" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" @@ -342,9 +341,9 @@ func (h *ProcOutputHelper) consumerClosed() { } // Stats returns output statistics. -func (h *ProcOutputHelper) Stats() execstatspb.OutputStats { - return execstatspb.OutputStats{ - NumTuples: execstatspb.MakeIntValue(h.rowIdx), +func (h *ProcOutputHelper) Stats() execinfrapb.OutputStats { + return execinfrapb.OutputStats{ + NumTuples: execinfrapb.MakeIntValue(h.rowIdx), } } @@ -497,7 +496,7 @@ type ProcessorBase struct { // MakeDeterministic might get called on the returned stats. // // Can return nil. - ExecStatsForTrace func() *execstatspb.ComponentStats + ExecStatsForTrace func() *execinfrapb.ComponentStats // trailingMetaCallback, if set, will be called by moveToTrailingMeta(). The // callback is expected to close all inputs, do other cleanup on the processor diff --git a/pkg/sql/execinfrapb/BUILD.bazel b/pkg/sql/execinfrapb/BUILD.bazel index 049995cdb056..c4d886d5f244 100644 --- a/pkg/sql/execinfrapb/BUILD.bazel +++ b/pkg/sql/execinfrapb/BUILD.bazel @@ -1,14 +1,40 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +go_library( + name = "execstatspb", + srcs = [ + "component_stats.go", + "component_stats.pb.go", + "int_value.go", + ], + importpath = "github.com/cockroachdb/cockroach/pkg/sql/execstats/execstatspb", + visibility = ["//visibility:public"], + deps = [ + "//vendor/github.com/dustin/go-humanize", + "//vendor/github.com/gogo/protobuf/proto", + "//vendor/github.com/gogo/protobuf/types", + ], +) + +go_test( + name = "execstatspb_test", + srcs = ["int_value_test.go"], + embed = [":execstatspb"], + deps = ["//vendor/github.com/stretchr/testify/require"], +) + go_library( name = "execinfrapb", srcs = [ "api.go", "api.pb.go", + "component_stats.go", + "component_stats.pb.go", "data.go", "data.pb.go", "expr.go", "flow_diagram.go", + "int_value.go", "processors.go", "processors.pb.go", "processors_base.pb.go", @@ -71,6 +97,7 @@ go_test( srcs = [ "expr_test.go", "flow_diagram_test.go", + "int_value_test.go", ], embed = [":execinfrapb"], deps = [ @@ -80,5 +107,6 @@ go_test( "//pkg/sql/sem/tree", "//pkg/sql/types", "//pkg/util/leaktest", + "//vendor/github.com/stretchr/testify/require", ], ) diff --git a/pkg/sql/execstats/execstatspb/component_stats.go b/pkg/sql/execinfrapb/component_stats.go similarity index 99% rename from pkg/sql/execstats/execstatspb/component_stats.go rename to pkg/sql/execinfrapb/component_stats.go index d8ede65c9a01..c476fdcdaae5 100644 --- a/pkg/sql/execstats/execstatspb/component_stats.go +++ b/pkg/sql/execinfrapb/component_stats.go @@ -8,12 +8,12 @@ // by the Apache License, Version 2.0, included in the file // licenses/APL.txt. -package execstatspb +package execinfrapb import ( - fmt "fmt" + "fmt" "strings" - time "time" + "time" "github.com/dustin/go-humanize" ) diff --git a/pkg/sql/execstats/execstatspb/component_stats.pb.go b/pkg/sql/execinfrapb/component_stats.pb.go similarity index 80% rename from pkg/sql/execstats/execstatspb/component_stats.pb.go rename to pkg/sql/execinfrapb/component_stats.pb.go index 54d02135ff04..73c90056cdac 100644 --- a/pkg/sql/execstats/execstatspb/component_stats.pb.go +++ b/pkg/sql/execinfrapb/component_stats.pb.go @@ -1,7 +1,7 @@ // Code generated by protoc-gen-gogo. DO NOT EDIT. -// source: sql/execstats/execstatspb/component_stats.proto +// source: sql/execinfrapb/component_stats.proto -package execstatspb +package execinfrapb import proto "github.com/gogo/protobuf/proto" import fmt "fmt" @@ -32,21 +32,21 @@ const _ = proto.GoGoProtoPackageIsVersion2 // please upgrade the proto package // Depending on the component, not all statistics apply. For all fields, the zero // value indicates that the particular stat is not available. type ComponentStats struct { - ComponentID int32 `protobuf:"varint,1,opt,name=component_id,json=componentId,proto3" json:"component_id,omitempty"` - NetRx NetworkRxStats `protobuf:"bytes,2,opt,name=net_rx,json=netRx,proto3" json:"net_rx"` - NetTx NetworkTxStats `protobuf:"bytes,3,opt,name=net_tx,json=netTx,proto3" json:"net_tx"` - KV KVStats `protobuf:"bytes,4,opt,name=kv,proto3" json:"kv"` - Exec ExecStats `protobuf:"bytes,5,opt,name=exec,proto3" json:"exec"` - Output OutputStats `protobuf:"bytes,6,opt,name=output,proto3" json:"output"` + ComponentID int32 `protobuf:"varint,1,opt,name=component_id,json=componentId" json:"component_id"` + NetRx NetworkRxStats `protobuf:"bytes,2,opt,name=net_rx,json=netRx" json:"net_rx"` + NetTx NetworkTxStats `protobuf:"bytes,3,opt,name=net_tx,json=netTx" json:"net_tx"` + KV KVStats `protobuf:"bytes,4,opt,name=kv" json:"kv"` + Exec ExecStats `protobuf:"bytes,5,opt,name=exec" json:"exec"` + Output OutputStats `protobuf:"bytes,6,opt,name=output" json:"output"` // Stats for the inputs of an operator (only in the row execution engine). - Inputs []InputStats `protobuf:"bytes,7,rep,name=inputs,proto3" json:"inputs"` + Inputs []InputStats `protobuf:"bytes,7,rep,name=inputs" json:"inputs"` } func (m *ComponentStats) Reset() { *m = ComponentStats{} } func (m *ComponentStats) String() string { return proto.CompactTextString(m) } func (*ComponentStats) ProtoMessage() {} func (*ComponentStats) Descriptor() ([]byte, []int) { - return fileDescriptor_component_stats_49d4098c21480260, []int{0} + return fileDescriptor_component_stats_b0b131113a063b54, []int{0} } func (m *ComponentStats) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -74,16 +74,16 @@ var xxx_messageInfo_ComponentStats proto.InternalMessageInfo // InputStats contains statistics about the rows received as an input to a // processor. Currently only used in the row execution engine. type InputStats struct { - NumTuples IntValue `protobuf:"varint,1,opt,name=num_tuples,json=numTuples,proto3,customtype=IntValue" json:"num_tuples"` + NumTuples IntValue `protobuf:"varint,1,opt,name=num_tuples,json=numTuples,customtype=IntValue" json:"num_tuples"` // Cumulated time spent waiting for rows from the input operator. - WaitTime time.Duration `protobuf:"bytes,2,opt,name=wait_time,json=waitTime,proto3,stdduration" json:"wait_time"` + WaitTime time.Duration `protobuf:"bytes,2,opt,name=wait_time,json=waitTime,stdduration" json:"wait_time"` } func (m *InputStats) Reset() { *m = InputStats{} } func (m *InputStats) String() string { return proto.CompactTextString(m) } func (*InputStats) ProtoMessage() {} func (*InputStats) Descriptor() ([]byte, []int) { - return fileDescriptor_component_stats_49d4098c21480260, []int{1} + return fileDescriptor_component_stats_b0b131113a063b54, []int{1} } func (m *InputStats) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -112,20 +112,20 @@ var xxx_messageInfo_InputStats proto.InternalMessageInfo // over the network. Note: Rx is shorthand for Receive. type NetworkRxStats struct { // Observed network latency (round-trip time between the two nodes). - Latency time.Duration `protobuf:"bytes,1,opt,name=latency,proto3,stdduration" json:"latency"` + Latency time.Duration `protobuf:"bytes,1,opt,name=latency,stdduration" json:"latency"` // Cumulated time spent waiting to receive or transmit tuple data. - WaitTime time.Duration `protobuf:"bytes,2,opt,name=wait_time,json=waitTime,proto3,stdduration" json:"wait_time"` + WaitTime time.Duration `protobuf:"bytes,2,opt,name=wait_time,json=waitTime,stdduration" json:"wait_time"` // Time spent deserializing network data. - DeserializationTime time.Duration `protobuf:"bytes,3,opt,name=deserialization_time,json=deserializationTime,proto3,stdduration" json:"deserialization_time"` - TuplesReceived IntValue `protobuf:"varint,4,opt,name=tuples_received,json=tuplesReceived,proto3,customtype=IntValue" json:"tuples_received"` - BytesReceived IntValue `protobuf:"varint,5,opt,name=bytes_received,json=bytesReceived,proto3,customtype=IntValue" json:"bytes_received"` + DeserializationTime time.Duration `protobuf:"bytes,3,opt,name=deserialization_time,json=deserializationTime,stdduration" json:"deserialization_time"` + TuplesReceived IntValue `protobuf:"varint,4,opt,name=tuples_received,json=tuplesReceived,customtype=IntValue" json:"tuples_received"` + BytesReceived IntValue `protobuf:"varint,5,opt,name=bytes_received,json=bytesReceived,customtype=IntValue" json:"bytes_received"` } func (m *NetworkRxStats) Reset() { *m = NetworkRxStats{} } func (m *NetworkRxStats) String() string { return proto.CompactTextString(m) } func (*NetworkRxStats) ProtoMessage() {} func (*NetworkRxStats) Descriptor() ([]byte, []int) { - return fileDescriptor_component_stats_49d4098c21480260, []int{2} + return fileDescriptor_component_stats_b0b131113a063b54, []int{2} } func (m *NetworkRxStats) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -153,15 +153,15 @@ var xxx_messageInfo_NetworkRxStats proto.InternalMessageInfo // NetworkTxStats contains statistics for components that send row data over the // network. Note: Tx is shorthand for Transmit. type NetworkTxStats struct { - TuplesSent IntValue `protobuf:"varint,1,opt,name=tuples_sent,json=tuplesSent,proto3,customtype=IntValue" json:"tuples_sent"` - BytesSent IntValue `protobuf:"varint,2,opt,name=bytes_sent,json=bytesSent,proto3,customtype=IntValue" json:"bytes_sent"` + TuplesSent IntValue `protobuf:"varint,1,opt,name=tuples_sent,json=tuplesSent,customtype=IntValue" json:"tuples_sent"` + BytesSent IntValue `protobuf:"varint,2,opt,name=bytes_sent,json=bytesSent,customtype=IntValue" json:"bytes_sent"` } func (m *NetworkTxStats) Reset() { *m = NetworkTxStats{} } func (m *NetworkTxStats) String() string { return proto.CompactTextString(m) } func (*NetworkTxStats) ProtoMessage() {} func (*NetworkTxStats) Descriptor() ([]byte, []int) { - return fileDescriptor_component_stats_49d4098c21480260, []int{3} + return fileDescriptor_component_stats_b0b131113a063b54, []int{3} } func (m *NetworkTxStats) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -188,18 +188,18 @@ var xxx_messageInfo_NetworkTxStats proto.InternalMessageInfo // KVStats contains statistics for components that perform KV operations. type KVStats struct { - BytesRead IntValue `protobuf:"varint,1,opt,name=bytes_read,json=bytesRead,proto3,customtype=IntValue" json:"bytes_read"` - TuplesRead IntValue `protobuf:"varint,2,opt,name=tuples_read,json=tuplesRead,proto3,customtype=IntValue" json:"tuples_read"` + BytesRead IntValue `protobuf:"varint,1,opt,name=bytes_read,json=bytesRead,customtype=IntValue" json:"bytes_read"` + TuplesRead IntValue `protobuf:"varint,2,opt,name=tuples_read,json=tuplesRead,customtype=IntValue" json:"tuples_read"` // Cumulated time spent waiting for a KV request. This includes disk IO time // and potentially network time (if any of the keys are not local). - KVTime time.Duration `protobuf:"bytes,3,opt,name=kv_time,json=kvTime,proto3,stdduration" json:"kv_time"` + KVTime time.Duration `protobuf:"bytes,3,opt,name=kv_time,json=kvTime,stdduration" json:"kv_time"` } func (m *KVStats) Reset() { *m = KVStats{} } func (m *KVStats) String() string { return proto.CompactTextString(m) } func (*KVStats) ProtoMessage() {} func (*KVStats) Descriptor() ([]byte, []int) { - return fileDescriptor_component_stats_49d4098c21480260, []int{4} + return fileDescriptor_component_stats_b0b131113a063b54, []int{4} } func (m *KVStats) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -227,18 +227,18 @@ var xxx_messageInfo_KVStats proto.InternalMessageInfo // ExecStats contains statistics about the execution of an components. type ExecStats struct { // Time spent executing the component. - ExecTime time.Duration `protobuf:"bytes,1,opt,name=exec_time,json=execTime,proto3,stdduration" json:"exec_time"` + ExecTime time.Duration `protobuf:"bytes,1,opt,name=exec_time,json=execTime,stdduration" json:"exec_time"` // Maximum memory allocated by the component. - MaxAllocatedMem IntValue `protobuf:"varint,2,opt,name=max_allocated_mem,json=maxAllocatedMem,proto3,customtype=IntValue" json:"max_allocated_mem"` + MaxAllocatedMem IntValue `protobuf:"varint,2,opt,name=max_allocated_mem,json=maxAllocatedMem,customtype=IntValue" json:"max_allocated_mem"` // Maximum scratch disk allocated by the component. - MaxAllocatedDisk IntValue `protobuf:"varint,3,opt,name=max_allocated_disk,json=maxAllocatedDisk,proto3,customtype=IntValue" json:"max_allocated_disk"` + MaxAllocatedDisk IntValue `protobuf:"varint,3,opt,name=max_allocated_disk,json=maxAllocatedDisk,customtype=IntValue" json:"max_allocated_disk"` } func (m *ExecStats) Reset() { *m = ExecStats{} } func (m *ExecStats) String() string { return proto.CompactTextString(m) } func (*ExecStats) ProtoMessage() {} func (*ExecStats) Descriptor() ([]byte, []int) { - return fileDescriptor_component_stats_49d4098c21480260, []int{5} + return fileDescriptor_component_stats_b0b131113a063b54, []int{5} } func (m *ExecStats) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -266,16 +266,16 @@ var xxx_messageInfo_ExecStats proto.InternalMessageInfo // OutputStats contains statistics about the output (results) of an component. type OutputStats struct { // Number of batches produced by the component. - NumBatches IntValue `protobuf:"varint,1,opt,name=num_batches,json=numBatches,proto3,customtype=IntValue" json:"num_batches"` + NumBatches IntValue `protobuf:"varint,1,opt,name=num_batches,json=numBatches,customtype=IntValue" json:"num_batches"` // Number of tuples produced by the component. - NumTuples IntValue `protobuf:"varint,2,opt,name=num_tuples,json=numTuples,proto3,customtype=IntValue" json:"num_tuples"` + NumTuples IntValue `protobuf:"varint,2,opt,name=num_tuples,json=numTuples,customtype=IntValue" json:"num_tuples"` } func (m *OutputStats) Reset() { *m = OutputStats{} } func (m *OutputStats) String() string { return proto.CompactTextString(m) } func (*OutputStats) ProtoMessage() {} func (*OutputStats) Descriptor() ([]byte, []int) { - return fileDescriptor_component_stats_49d4098c21480260, []int{6} + return fileDescriptor_component_stats_b0b131113a063b54, []int{6} } func (m *OutputStats) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -301,13 +301,13 @@ func (m *OutputStats) XXX_DiscardUnknown() { var xxx_messageInfo_OutputStats proto.InternalMessageInfo func init() { - proto.RegisterType((*ComponentStats)(nil), "cockroach.sql.execstats.execstatspb.ComponentStats") - proto.RegisterType((*InputStats)(nil), "cockroach.sql.execstats.execstatspb.InputStats") - proto.RegisterType((*NetworkRxStats)(nil), "cockroach.sql.execstats.execstatspb.NetworkRxStats") - proto.RegisterType((*NetworkTxStats)(nil), "cockroach.sql.execstats.execstatspb.NetworkTxStats") - proto.RegisterType((*KVStats)(nil), "cockroach.sql.execstats.execstatspb.KVStats") - proto.RegisterType((*ExecStats)(nil), "cockroach.sql.execstats.execstatspb.ExecStats") - proto.RegisterType((*OutputStats)(nil), "cockroach.sql.execstats.execstatspb.OutputStats") + proto.RegisterType((*ComponentStats)(nil), "cockroach.sql.distsqlrun.ComponentStats") + proto.RegisterType((*InputStats)(nil), "cockroach.sql.distsqlrun.InputStats") + proto.RegisterType((*NetworkRxStats)(nil), "cockroach.sql.distsqlrun.NetworkRxStats") + proto.RegisterType((*NetworkTxStats)(nil), "cockroach.sql.distsqlrun.NetworkTxStats") + proto.RegisterType((*KVStats)(nil), "cockroach.sql.distsqlrun.KVStats") + proto.RegisterType((*ExecStats)(nil), "cockroach.sql.distsqlrun.ExecStats") + proto.RegisterType((*OutputStats)(nil), "cockroach.sql.distsqlrun.OutputStats") } func (m *ComponentStats) Marshal() (dAtA []byte, err error) { size := m.Size() @@ -324,11 +324,9 @@ func (m *ComponentStats) MarshalTo(dAtA []byte) (int, error) { _ = i var l int _ = l - if m.ComponentID != 0 { - dAtA[i] = 0x8 - i++ - i = encodeVarintComponentStats(dAtA, i, uint64(m.ComponentID)) - } + dAtA[i] = 0x8 + i++ + i = encodeVarintComponentStats(dAtA, i, uint64(m.ComponentID)) dAtA[i] = 0x12 i++ i = encodeVarintComponentStats(dAtA, i, uint64(m.NetRx.Size())) @@ -399,11 +397,9 @@ func (m *InputStats) MarshalTo(dAtA []byte) (int, error) { _ = i var l int _ = l - if m.NumTuples != 0 { - dAtA[i] = 0x8 - i++ - i = encodeVarintComponentStats(dAtA, i, uint64(m.NumTuples)) - } + dAtA[i] = 0x8 + i++ + i = encodeVarintComponentStats(dAtA, i, uint64(m.NumTuples)) dAtA[i] = 0x12 i++ i = encodeVarintComponentStats(dAtA, i, uint64(github_com_gogo_protobuf_types.SizeOfStdDuration(m.WaitTime))) @@ -454,16 +450,12 @@ func (m *NetworkRxStats) MarshalTo(dAtA []byte) (int, error) { return 0, err } i += n9 - if m.TuplesReceived != 0 { - dAtA[i] = 0x20 - i++ - i = encodeVarintComponentStats(dAtA, i, uint64(m.TuplesReceived)) - } - if m.BytesReceived != 0 { - dAtA[i] = 0x28 - i++ - i = encodeVarintComponentStats(dAtA, i, uint64(m.BytesReceived)) - } + dAtA[i] = 0x20 + i++ + i = encodeVarintComponentStats(dAtA, i, uint64(m.TuplesReceived)) + dAtA[i] = 0x28 + i++ + i = encodeVarintComponentStats(dAtA, i, uint64(m.BytesReceived)) return i, nil } @@ -482,16 +474,12 @@ func (m *NetworkTxStats) MarshalTo(dAtA []byte) (int, error) { _ = i var l int _ = l - if m.TuplesSent != 0 { - dAtA[i] = 0x8 - i++ - i = encodeVarintComponentStats(dAtA, i, uint64(m.TuplesSent)) - } - if m.BytesSent != 0 { - dAtA[i] = 0x10 - i++ - i = encodeVarintComponentStats(dAtA, i, uint64(m.BytesSent)) - } + dAtA[i] = 0x8 + i++ + i = encodeVarintComponentStats(dAtA, i, uint64(m.TuplesSent)) + dAtA[i] = 0x10 + i++ + i = encodeVarintComponentStats(dAtA, i, uint64(m.BytesSent)) return i, nil } @@ -510,16 +498,12 @@ func (m *KVStats) MarshalTo(dAtA []byte) (int, error) { _ = i var l int _ = l - if m.BytesRead != 0 { - dAtA[i] = 0x8 - i++ - i = encodeVarintComponentStats(dAtA, i, uint64(m.BytesRead)) - } - if m.TuplesRead != 0 { - dAtA[i] = 0x10 - i++ - i = encodeVarintComponentStats(dAtA, i, uint64(m.TuplesRead)) - } + dAtA[i] = 0x8 + i++ + i = encodeVarintComponentStats(dAtA, i, uint64(m.BytesRead)) + dAtA[i] = 0x10 + i++ + i = encodeVarintComponentStats(dAtA, i, uint64(m.TuplesRead)) dAtA[i] = 0x1a i++ i = encodeVarintComponentStats(dAtA, i, uint64(github_com_gogo_protobuf_types.SizeOfStdDuration(m.KVTime))) @@ -554,16 +538,12 @@ func (m *ExecStats) MarshalTo(dAtA []byte) (int, error) { return 0, err } i += n11 - if m.MaxAllocatedMem != 0 { - dAtA[i] = 0x10 - i++ - i = encodeVarintComponentStats(dAtA, i, uint64(m.MaxAllocatedMem)) - } - if m.MaxAllocatedDisk != 0 { - dAtA[i] = 0x18 - i++ - i = encodeVarintComponentStats(dAtA, i, uint64(m.MaxAllocatedDisk)) - } + dAtA[i] = 0x10 + i++ + i = encodeVarintComponentStats(dAtA, i, uint64(m.MaxAllocatedMem)) + dAtA[i] = 0x18 + i++ + i = encodeVarintComponentStats(dAtA, i, uint64(m.MaxAllocatedDisk)) return i, nil } @@ -582,16 +562,12 @@ func (m *OutputStats) MarshalTo(dAtA []byte) (int, error) { _ = i var l int _ = l - if m.NumBatches != 0 { - dAtA[i] = 0x8 - i++ - i = encodeVarintComponentStats(dAtA, i, uint64(m.NumBatches)) - } - if m.NumTuples != 0 { - dAtA[i] = 0x10 - i++ - i = encodeVarintComponentStats(dAtA, i, uint64(m.NumTuples)) - } + dAtA[i] = 0x8 + i++ + i = encodeVarintComponentStats(dAtA, i, uint64(m.NumBatches)) + dAtA[i] = 0x10 + i++ + i = encodeVarintComponentStats(dAtA, i, uint64(m.NumTuples)) return i, nil } @@ -610,9 +586,7 @@ func (m *ComponentStats) Size() (n int) { } var l int _ = l - if m.ComponentID != 0 { - n += 1 + sovComponentStats(uint64(m.ComponentID)) - } + n += 1 + sovComponentStats(uint64(m.ComponentID)) l = m.NetRx.Size() n += 1 + l + sovComponentStats(uint64(l)) l = m.NetTx.Size() @@ -638,9 +612,7 @@ func (m *InputStats) Size() (n int) { } var l int _ = l - if m.NumTuples != 0 { - n += 1 + sovComponentStats(uint64(m.NumTuples)) - } + n += 1 + sovComponentStats(uint64(m.NumTuples)) l = github_com_gogo_protobuf_types.SizeOfStdDuration(m.WaitTime) n += 1 + l + sovComponentStats(uint64(l)) return n @@ -658,12 +630,8 @@ func (m *NetworkRxStats) Size() (n int) { n += 1 + l + sovComponentStats(uint64(l)) l = github_com_gogo_protobuf_types.SizeOfStdDuration(m.DeserializationTime) n += 1 + l + sovComponentStats(uint64(l)) - if m.TuplesReceived != 0 { - n += 1 + sovComponentStats(uint64(m.TuplesReceived)) - } - if m.BytesReceived != 0 { - n += 1 + sovComponentStats(uint64(m.BytesReceived)) - } + n += 1 + sovComponentStats(uint64(m.TuplesReceived)) + n += 1 + sovComponentStats(uint64(m.BytesReceived)) return n } @@ -673,12 +641,8 @@ func (m *NetworkTxStats) Size() (n int) { } var l int _ = l - if m.TuplesSent != 0 { - n += 1 + sovComponentStats(uint64(m.TuplesSent)) - } - if m.BytesSent != 0 { - n += 1 + sovComponentStats(uint64(m.BytesSent)) - } + n += 1 + sovComponentStats(uint64(m.TuplesSent)) + n += 1 + sovComponentStats(uint64(m.BytesSent)) return n } @@ -688,12 +652,8 @@ func (m *KVStats) Size() (n int) { } var l int _ = l - if m.BytesRead != 0 { - n += 1 + sovComponentStats(uint64(m.BytesRead)) - } - if m.TuplesRead != 0 { - n += 1 + sovComponentStats(uint64(m.TuplesRead)) - } + n += 1 + sovComponentStats(uint64(m.BytesRead)) + n += 1 + sovComponentStats(uint64(m.TuplesRead)) l = github_com_gogo_protobuf_types.SizeOfStdDuration(m.KVTime) n += 1 + l + sovComponentStats(uint64(l)) return n @@ -707,12 +667,8 @@ func (m *ExecStats) Size() (n int) { _ = l l = github_com_gogo_protobuf_types.SizeOfStdDuration(m.ExecTime) n += 1 + l + sovComponentStats(uint64(l)) - if m.MaxAllocatedMem != 0 { - n += 1 + sovComponentStats(uint64(m.MaxAllocatedMem)) - } - if m.MaxAllocatedDisk != 0 { - n += 1 + sovComponentStats(uint64(m.MaxAllocatedDisk)) - } + n += 1 + sovComponentStats(uint64(m.MaxAllocatedMem)) + n += 1 + sovComponentStats(uint64(m.MaxAllocatedDisk)) return n } @@ -722,12 +678,8 @@ func (m *OutputStats) Size() (n int) { } var l int _ = l - if m.NumBatches != 0 { - n += 1 + sovComponentStats(uint64(m.NumBatches)) - } - if m.NumTuples != 0 { - n += 1 + sovComponentStats(uint64(m.NumTuples)) - } + n += 1 + sovComponentStats(uint64(m.NumBatches)) + n += 1 + sovComponentStats(uint64(m.NumTuples)) return n } @@ -1789,54 +1741,54 @@ var ( ) func init() { - proto.RegisterFile("sql/execstats/execstatspb/component_stats.proto", fileDescriptor_component_stats_49d4098c21480260) + proto.RegisterFile("sql/execinfrapb/component_stats.proto", fileDescriptor_component_stats_b0b131113a063b54) } -var fileDescriptor_component_stats_49d4098c21480260 = []byte{ - // 717 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xa4, 0x94, 0xcf, 0x6e, 0xd3, 0x4a, - 0x14, 0xc6, 0x63, 0x37, 0x71, 0xd2, 0xf1, 0xbd, 0x69, 0xef, 0xdc, 0x2e, 0x4c, 0x17, 0x4e, 0x15, - 0x36, 0x5d, 0x80, 0x0d, 0xed, 0x02, 0x21, 0x01, 0x82, 0x50, 0x50, 0xa3, 0xaa, 0x05, 0xb9, 0x51, - 0x16, 0x6c, 0xac, 0x89, 0x3d, 0xa4, 0x96, 0xff, 0x4c, 0x6a, 0x8f, 0x53, 0x97, 0x0d, 0xaf, 0xc0, - 0x12, 0x89, 0xe7, 0xe0, 0x1d, 0x2a, 0xb1, 0xe9, 0xb2, 0x42, 0x22, 0x40, 0xfa, 0x22, 0x68, 0xc6, - 0x4e, 0xe2, 0x96, 0xa6, 0xb8, 0x62, 0xe7, 0xc4, 0xdf, 0xf7, 0x9b, 0xe3, 0x39, 0xdf, 0x39, 0x40, - 0x8f, 0x0e, 0x3d, 0x1d, 0x27, 0xd8, 0x8a, 0x28, 0xa2, 0xd1, 0xec, 0x69, 0xd0, 0xd3, 0x2d, 0xe2, - 0x0f, 0x48, 0x80, 0x03, 0x6a, 0xf2, 0x7f, 0xb4, 0x41, 0x48, 0x28, 0x81, 0xb7, 0x2d, 0x62, 0xb9, - 0x21, 0x41, 0xd6, 0x81, 0x16, 0x1d, 0x7a, 0xda, 0xd4, 0xa0, 0xe5, 0xac, 0xab, 0x2b, 0x7d, 0xd2, - 0x27, 0x5c, 0xaf, 0xb3, 0xa7, 0xd4, 0xba, 0xaa, 0xf6, 0x09, 0xe9, 0x7b, 0x58, 0xe7, 0xbf, 0x7a, - 0xf1, 0x5b, 0xdd, 0x8e, 0x43, 0x44, 0x1d, 0x12, 0xa4, 0xef, 0x9b, 0x9f, 0xca, 0xa0, 0xfe, 0x7c, - 0x72, 0xe8, 0x3e, 0x43, 0xc1, 0x0d, 0xf0, 0xcf, 0xac, 0x0c, 0xc7, 0x56, 0x84, 0x35, 0x61, 0xbd, - 0xd2, 0x5a, 0x1a, 0x8f, 0x1a, 0xf2, 0x54, 0xd9, 0xde, 0x32, 0xe4, 0xa9, 0xa8, 0x6d, 0xc3, 0xd7, - 0x40, 0x0a, 0x30, 0x35, 0xc3, 0x44, 0x11, 0xd7, 0x84, 0x75, 0x79, 0x63, 0x53, 0x2b, 0x50, 0xb2, - 0xb6, 0x87, 0xe9, 0x11, 0x09, 0x5d, 0x23, 0xe1, 0x07, 0xb7, 0xca, 0x27, 0xa3, 0x46, 0xc9, 0xa8, - 0x04, 0x98, 0x1a, 0xc9, 0x84, 0x48, 0x13, 0x65, 0xe1, 0xe6, 0xc4, 0xce, 0x6f, 0xc4, 0x4e, 0x02, - 0xb7, 0x81, 0xe8, 0x0e, 0x95, 0x32, 0xa7, 0xdd, 0x29, 0x44, 0xdb, 0xe9, 0xa6, 0x18, 0xc0, 0x30, - 0xe3, 0x51, 0x43, 0xdc, 0xe9, 0x1a, 0xa2, 0x3b, 0x84, 0xdb, 0xa0, 0xcc, 0x64, 0x4a, 0x85, 0xb3, - 0xb4, 0x42, 0xac, 0x17, 0x09, 0xb6, 0xf2, 0x45, 0x71, 0x02, 0xdc, 0x03, 0x12, 0x89, 0xe9, 0x20, - 0xa6, 0x8a, 0xc4, 0x59, 0xf7, 0x0a, 0xb1, 0x5e, 0x71, 0x4b, 0x9e, 0x96, 0x51, 0xe0, 0x2e, 0x90, - 0x9c, 0x60, 0x10, 0xd3, 0x48, 0xa9, 0xae, 0x2d, 0xac, 0xcb, 0x1b, 0x7a, 0x21, 0x5e, 0x3b, 0xb8, - 0x8c, 0x4b, 0x21, 0xcd, 0xf7, 0x00, 0xcc, 0xde, 0x41, 0x1d, 0x80, 0x20, 0xf6, 0x4d, 0x1a, 0x0f, - 0x3c, 0x1c, 0xf1, 0x58, 0x94, 0x5b, 0xcb, 0x4c, 0xff, 0x75, 0xd4, 0xa8, 0xb5, 0x03, 0xda, 0x45, - 0x5e, 0x8c, 0x8d, 0xc5, 0x20, 0xf6, 0x3b, 0x5c, 0x02, 0x9f, 0x82, 0xc5, 0x23, 0xe4, 0x50, 0x93, - 0x3a, 0x3e, 0xce, 0x82, 0x71, 0x4b, 0x4b, 0x03, 0xa9, 0x4d, 0x02, 0xa9, 0x6d, 0x65, 0x81, 0x6c, - 0xd5, 0x18, 0xea, 0xe3, 0xf7, 0x86, 0x60, 0xd4, 0x98, 0xab, 0xe3, 0xf8, 0xb8, 0xf9, 0x4d, 0x04, - 0xf5, 0x8b, 0x29, 0x81, 0x8f, 0x41, 0xd5, 0x43, 0x14, 0x07, 0xd6, 0x31, 0x2f, 0xa1, 0x20, 0x72, - 0xe2, 0xf9, 0xfb, 0x9a, 0x60, 0x17, 0xac, 0xd8, 0x38, 0xc2, 0xa1, 0x83, 0x3c, 0xe7, 0x1d, 0x97, - 0xa5, 0xb0, 0x85, 0xe2, 0xb0, 0xff, 0x2f, 0x01, 0x38, 0xf7, 0x21, 0x58, 0x4a, 0xaf, 0xd6, 0x0c, - 0xb1, 0x85, 0x9d, 0x21, 0xb6, 0x79, 0x58, 0xaf, 0xba, 0xe3, 0x7a, 0x2a, 0x34, 0x32, 0x1d, 0x7c, - 0x00, 0xea, 0xbd, 0x63, 0x9a, 0x77, 0x56, 0xe6, 0x38, 0xff, 0xe5, 0xba, 0x89, 0xb1, 0x49, 0xa7, - 0xd7, 0x9b, 0x8d, 0x0c, 0xbc, 0x0f, 0xe4, 0xac, 0x8a, 0x08, 0x07, 0x74, 0x6e, 0x97, 0x41, 0x2a, - 0xda, 0xc7, 0x01, 0x65, 0xb9, 0x48, 0x4f, 0xe7, 0x0e, 0x71, 0x5e, 0x2e, 0xb8, 0x86, 0x19, 0x9a, - 0x9f, 0x05, 0x50, 0xcd, 0x66, 0x6b, 0x66, 0x0e, 0x31, 0xb2, 0xe7, 0x87, 0x2a, 0x2b, 0x1b, 0xd9, - 0xb9, 0x02, 0xb9, 0x43, 0xbc, 0xbe, 0x40, 0x6e, 0x79, 0x09, 0xaa, 0xee, 0xb0, 0x60, 0x93, 0x60, - 0x36, 0xeb, 0xd2, 0x4e, 0x97, 0x75, 0x85, 0xb7, 0x4b, 0x72, 0x87, 0x3c, 0x8d, 0x5f, 0x04, 0xb0, - 0x38, 0x9d, 0x63, 0x96, 0x24, 0x36, 0x44, 0x29, 0xf7, 0x06, 0x51, 0xac, 0x31, 0x17, 0xef, 0xf8, - 0x23, 0xf0, 0x9f, 0x8f, 0x12, 0x13, 0x79, 0x1e, 0xb1, 0x10, 0xc5, 0xb6, 0xe9, 0x63, 0x7f, 0xee, - 0x07, 0x2d, 0xf9, 0x28, 0x79, 0x36, 0x51, 0xee, 0x62, 0x1f, 0x3e, 0x01, 0xf0, 0xa2, 0xdb, 0x76, - 0x22, 0x97, 0x7f, 0xe0, 0x55, 0xf6, 0xe5, 0xbc, 0x7d, 0xcb, 0x89, 0xdc, 0xe6, 0x21, 0x90, 0x73, - 0x8b, 0x84, 0xdd, 0x2b, 0x9b, 0xee, 0x1e, 0xa2, 0xd6, 0xc1, 0x35, 0xe3, 0xcd, 0x56, 0x40, 0x2b, - 0xd5, 0x5c, 0x5a, 0x08, 0xe2, 0x1f, 0x17, 0x42, 0xeb, 0xee, 0xc9, 0x4f, 0xb5, 0x74, 0x32, 0x56, - 0x85, 0xd3, 0xb1, 0x2a, 0x9c, 0x8d, 0x55, 0xe1, 0xc7, 0x58, 0x15, 0x3e, 0x9c, 0xab, 0xa5, 0xd3, - 0x73, 0xb5, 0x74, 0x76, 0xae, 0x96, 0xde, 0xc8, 0xb9, 0xbd, 0xd4, 0x93, 0xf8, 0x35, 0x6e, 0xfe, - 0x0a, 0x00, 0x00, 0xff, 0xff, 0x9a, 0x7a, 0xc2, 0xcc, 0x31, 0x07, 0x00, 0x00, +var fileDescriptor_component_stats_b0b131113a063b54 = []byte{ + // 712 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xa4, 0x95, 0xcd, 0x6e, 0xd3, 0x40, + 0x14, 0x85, 0x63, 0xe7, 0xb7, 0x13, 0x48, 0xcb, 0xb4, 0x0b, 0xd3, 0x85, 0x53, 0x02, 0x95, 0xb2, + 0xc1, 0x11, 0x5d, 0x80, 0x90, 0x28, 0x82, 0xb4, 0x45, 0x8a, 0x2a, 0x40, 0x72, 0xa3, 0x2c, 0xd8, + 0x58, 0x13, 0x7b, 0x9a, 0x5a, 0xb6, 0x67, 0x12, 0x7b, 0x9c, 0xba, 0x6c, 0x78, 0x05, 0x96, 0xbc, + 0x08, 0xef, 0x50, 0x89, 0x4d, 0x97, 0x15, 0x52, 0x0b, 0xa4, 0x2f, 0x82, 0x66, 0xec, 0xa4, 0x6e, + 0xa9, 0x4b, 0x10, 0xbb, 0x56, 0x3e, 0xe7, 0xbb, 0x77, 0xe6, 0x9e, 0x3b, 0x01, 0xeb, 0xc1, 0xc8, + 0x6d, 0xe1, 0x08, 0x9b, 0x36, 0xd9, 0xf7, 0xd1, 0xb0, 0xdf, 0x32, 0xa9, 0x37, 0xa4, 0x04, 0x13, + 0x66, 0x04, 0x0c, 0xb1, 0x40, 0x1b, 0xfa, 0x94, 0x51, 0xa8, 0x98, 0xd4, 0x74, 0x7c, 0x8a, 0xcc, + 0x03, 0x2d, 0x18, 0xb9, 0x9a, 0x65, 0x07, 0x2c, 0x18, 0xb9, 0x7e, 0x48, 0x56, 0x57, 0x06, 0x74, + 0x40, 0x85, 0xa8, 0xc5, 0xff, 0x8a, 0xf5, 0xab, 0xea, 0x80, 0xd2, 0x81, 0x8b, 0x5b, 0xe2, 0xbf, + 0x7e, 0xb8, 0xdf, 0xb2, 0x42, 0x1f, 0x31, 0x9b, 0x92, 0xf8, 0x7b, 0xe3, 0x2c, 0x0f, 0x6a, 0x5b, + 0xd3, 0x4a, 0x7b, 0xbc, 0x10, 0x7c, 0x0a, 0xee, 0x5c, 0xd6, 0xb6, 0x2d, 0x45, 0x5a, 0x93, 0x9a, + 0xc5, 0xf6, 0xf2, 0xf1, 0x79, 0x3d, 0x37, 0x39, 0xaf, 0x57, 0x67, 0xea, 0xce, 0xb6, 0x5e, 0x9d, + 0x09, 0x3b, 0x16, 0xdc, 0x01, 0x25, 0x82, 0x99, 0xe1, 0x47, 0x8a, 0xbc, 0x26, 0x35, 0xab, 0x1b, + 0x4d, 0x2d, 0xab, 0x57, 0xed, 0x1d, 0x66, 0x87, 0xd4, 0x77, 0xf4, 0x48, 0x54, 0x6c, 0x17, 0x38, + 0x5b, 0x2f, 0x12, 0xcc, 0xf4, 0x68, 0x8a, 0x61, 0x91, 0x92, 0x9f, 0x13, 0xd3, 0xfd, 0x03, 0xd3, + 0x8d, 0xe0, 0x26, 0x90, 0x9d, 0xb1, 0x52, 0x10, 0x88, 0x07, 0xd9, 0x88, 0xdd, 0x5e, 0xec, 0x05, + 0xc9, 0xf1, 0xe4, 0xdd, 0x9e, 0x2e, 0x3b, 0x63, 0xb8, 0x09, 0x0a, 0x7c, 0x18, 0x4a, 0x51, 0x00, + 0x1e, 0x66, 0x03, 0x76, 0x22, 0x6c, 0xa6, 0xcb, 0x0b, 0x1b, 0xdc, 0x02, 0x25, 0x1a, 0xb2, 0x61, + 0xc8, 0x94, 0x92, 0x00, 0xac, 0x67, 0x03, 0xde, 0x0b, 0x5d, 0x1a, 0x91, 0x58, 0x61, 0x1b, 0x94, + 0x6c, 0x32, 0x0c, 0x59, 0xa0, 0x94, 0xd7, 0xf2, 0xcd, 0xea, 0xc6, 0xa3, 0x6c, 0x48, 0x87, 0x5c, + 0x67, 0xc4, 0xce, 0xc6, 0x27, 0x00, 0x2e, 0xbf, 0xc1, 0x16, 0x00, 0x24, 0xf4, 0x0c, 0x16, 0x0e, + 0x5d, 0x1c, 0x88, 0xc1, 0x16, 0xda, 0x4b, 0x5c, 0xff, 0xfd, 0xbc, 0x5e, 0xe9, 0x10, 0xd6, 0x43, + 0x6e, 0x88, 0xf5, 0x05, 0x12, 0x7a, 0x5d, 0x21, 0x81, 0xaf, 0xc0, 0xc2, 0x21, 0xb2, 0x99, 0xc1, + 0x6c, 0x0f, 0x27, 0x63, 0xbd, 0xaf, 0xc5, 0x91, 0xd2, 0xa6, 0x91, 0xd2, 0xb6, 0x93, 0x48, 0xb5, + 0x2b, 0x1c, 0xf5, 0xe5, 0x47, 0x5d, 0xd2, 0x2b, 0xdc, 0xd5, 0xb5, 0x3d, 0xdc, 0x38, 0x93, 0x41, + 0xed, 0xea, 0xb8, 0xe1, 0x26, 0x28, 0xbb, 0x88, 0x61, 0x62, 0x1e, 0x89, 0x16, 0xe6, 0x44, 0x4e, + 0x3d, 0xff, 0xdf, 0x13, 0xec, 0x81, 0x15, 0x0b, 0x07, 0xd8, 0xb7, 0x91, 0x6b, 0x7f, 0x14, 0xb2, + 0x18, 0x96, 0x9f, 0x1f, 0xb6, 0x7c, 0x0d, 0x20, 0xb8, 0xcf, 0xc1, 0x62, 0x7c, 0xb5, 0x86, 0x8f, + 0x4d, 0x6c, 0x8f, 0xb1, 0x25, 0x02, 0x78, 0xd3, 0x1d, 0xd7, 0x62, 0xa1, 0x9e, 0xe8, 0xe0, 0x33, + 0x50, 0xeb, 0x1f, 0xb1, 0xb4, 0xb3, 0x98, 0xe1, 0xbc, 0x2b, 0x74, 0x53, 0x63, 0x83, 0xcd, 0xae, + 0x37, 0x59, 0x03, 0xf8, 0x04, 0x54, 0x93, 0x2e, 0x02, 0x4c, 0x58, 0xe6, 0x94, 0x41, 0x2c, 0xda, + 0xc3, 0x84, 0xf1, 0x5c, 0xc4, 0xd5, 0x85, 0x43, 0xce, 0xca, 0x85, 0xd0, 0x70, 0x43, 0xe3, 0xab, + 0x04, 0xca, 0xc9, 0xea, 0x5c, 0x9a, 0x7d, 0x8c, 0xac, 0xec, 0x50, 0x25, 0x6d, 0x23, 0x2b, 0xd5, + 0xa0, 0x70, 0xc8, 0xb7, 0x37, 0x28, 0x2c, 0x6f, 0x40, 0xd9, 0x19, 0xcf, 0x39, 0x24, 0x98, 0xac, + 0x72, 0x69, 0xb7, 0xc7, 0xa7, 0x22, 0xc6, 0x55, 0x72, 0xc6, 0x22, 0x8d, 0xdf, 0x24, 0xb0, 0x30, + 0xdb, 0x58, 0x9e, 0x24, 0xbe, 0xad, 0x31, 0xf7, 0x1f, 0xa2, 0x58, 0xe1, 0x2e, 0x31, 0xf1, 0x17, + 0xe0, 0x9e, 0x87, 0x22, 0x03, 0xb9, 0x2e, 0x35, 0x11, 0xc3, 0x96, 0xe1, 0x61, 0x2f, 0xf3, 0x40, + 0x8b, 0x1e, 0x8a, 0x5e, 0x4f, 0x95, 0x6f, 0xb1, 0x07, 0x5f, 0x02, 0x78, 0xd5, 0x6d, 0xd9, 0x81, + 0x23, 0x0e, 0x78, 0x93, 0x7d, 0x29, 0x6d, 0xdf, 0xb6, 0x03, 0xa7, 0x31, 0x02, 0xd5, 0xd4, 0xeb, + 0xc1, 0xef, 0x95, 0x6f, 0x77, 0x1f, 0x31, 0xf3, 0xe0, 0x96, 0xf5, 0xe6, 0x4f, 0x40, 0x3b, 0xd6, + 0x5c, 0x7b, 0x10, 0xe4, 0xbf, 0x3e, 0x08, 0xed, 0xc7, 0xc7, 0xbf, 0xd4, 0xdc, 0xf1, 0x44, 0x95, + 0x4e, 0x26, 0xaa, 0x74, 0x3a, 0x51, 0xa5, 0x9f, 0x13, 0x55, 0xfa, 0x7c, 0xa1, 0xe6, 0x4e, 0x2e, + 0xd4, 0xdc, 0xe9, 0x85, 0x9a, 0xfb, 0x50, 0x4d, 0xfd, 0x88, 0xfd, 0x0e, 0x00, 0x00, 0xff, 0xff, + 0x67, 0xf3, 0x03, 0x14, 0xd6, 0x06, 0x00, 0x00, } diff --git a/pkg/sql/execinfrapb/component_stats.proto b/pkg/sql/execinfrapb/component_stats.proto new file mode 100644 index 000000000000..987c0124ffd0 --- /dev/null +++ b/pkg/sql/execinfrapb/component_stats.proto @@ -0,0 +1,115 @@ +// Copyright 2020 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +syntax = "proto2"; +package cockroach.sql.distsqlrun; +option go_package = "execinfrapb"; + +import "gogoproto/gogo.proto"; +import "google/protobuf/duration.proto"; + +// ComponentStats contains statistics for an execution component. A component is +// an arbitrary unit in the execution infrastructure; it can correspond to an +// operator or a stream. +// +// Depending on the component, not all statistics apply. For all fields, the zero +// value indicates that the particular stat is not available. +message ComponentStats { + optional int32 component_id = 1 [(gogoproto.customname) = "ComponentID", + (gogoproto.nullable) = false]; + + optional NetworkRxStats net_rx = 2 [(gogoproto.nullable) = false]; + optional NetworkTxStats net_tx = 3 [(gogoproto.nullable) = false]; + optional KVStats kv = 4 [(gogoproto.customname) = "KV", (gogoproto.nullable) = false]; + optional ExecStats exec = 5 [(gogoproto.nullable) = false]; + optional OutputStats output = 6 [(gogoproto.nullable) = false]; + + // Stats for the inputs of an operator (only in the row execution engine). + repeated InputStats inputs = 7 [(gogoproto.nullable) = false]; +} + +// InputStats contains statistics about the rows received as an input to a +// processor. Currently only used in the row execution engine. +message InputStats { + optional uint64 num_tuples = 1 [(gogoproto.customtype) = "IntValue", + (gogoproto.nullable) = false]; + // Cumulated time spent waiting for rows from the input operator. + optional google.protobuf.Duration wait_time = 2 [(gogoproto.nullable) = false, + (gogoproto.stdduration) = true]; +} + +// NetworkRxStats contains statistics for components that receive row data +// over the network. Note: Rx is shorthand for Receive. +message NetworkRxStats { + // Observed network latency (round-trip time between the two nodes). + optional google.protobuf.Duration latency = 1 [(gogoproto.nullable) = false, + (gogoproto.stdduration) = true]; + + // Cumulated time spent waiting to receive or transmit tuple data. + optional google.protobuf.Duration wait_time = 2 [(gogoproto.nullable) = false, + (gogoproto.stdduration) = true]; + + // Time spent deserializing network data. + optional google.protobuf.Duration deserialization_time = 3 [(gogoproto.nullable) = false, + (gogoproto.stdduration) = true]; + + optional uint64 tuples_received = 4 [(gogoproto.customtype) = "IntValue", + (gogoproto.nullable) = false]; + optional uint64 bytes_received = 5 [(gogoproto.customtype) = "IntValue", + (gogoproto.nullable) = false]; +} + +// NetworkTxStats contains statistics for components that send row data over the +// network. Note: Tx is shorthand for Transmit. +message NetworkTxStats { + optional uint64 tuples_sent = 1 [(gogoproto.customtype) = "IntValue", + (gogoproto.nullable) = false]; + optional uint64 bytes_sent = 2 [(gogoproto.customtype) = "IntValue", + (gogoproto.nullable) = false]; +} + +// KVStats contains statistics for components that perform KV operations. +message KVStats { + optional uint64 bytes_read = 1 [(gogoproto.customtype) = "IntValue", + (gogoproto.nullable) = false]; + optional uint64 tuples_read = 2 [(gogoproto.customtype) = "IntValue", + (gogoproto.nullable) = false]; + + // Cumulated time spent waiting for a KV request. This includes disk IO time + // and potentially network time (if any of the keys are not local). + optional google.protobuf.Duration kv_time = 3 [(gogoproto.customname) = "KVTime", + (gogoproto.nullable) = false, + (gogoproto.stdduration) = true]; +} + +// ExecStats contains statistics about the execution of an components. +message ExecStats { + // Time spent executing the component. + optional google.protobuf.Duration exec_time = 1 [(gogoproto.nullable) = false, + (gogoproto.stdduration) = true]; + // Maximum memory allocated by the component. + optional uint64 max_allocated_mem = 2 [(gogoproto.customtype) = "IntValue", + (gogoproto.nullable) = false]; + + // Maximum scratch disk allocated by the component. + optional uint64 max_allocated_disk = 3 [(gogoproto.customtype) = "IntValue", + (gogoproto.nullable) = false]; +} + +// OutputStats contains statistics about the output (results) of an component. +message OutputStats { + // Number of batches produced by the component. + optional uint64 num_batches = 1 [(gogoproto.customtype) = "IntValue", + (gogoproto.nullable) = false]; + + // Number of tuples produced by the component. + optional uint64 num_tuples = 2 [(gogoproto.customtype) = "IntValue", + (gogoproto.nullable) = false]; +} diff --git a/pkg/sql/execstats/execstatspb/int_value.go b/pkg/sql/execinfrapb/int_value.go similarity index 97% rename from pkg/sql/execstats/execstatspb/int_value.go rename to pkg/sql/execinfrapb/int_value.go index d8e1a4a680b3..50b9b54869f1 100644 --- a/pkg/sql/execstats/execstatspb/int_value.go +++ b/pkg/sql/execinfrapb/int_value.go @@ -8,9 +8,9 @@ // by the Apache License, Version 2.0, included in the file // licenses/APL.txt. -package execstatspb +package execinfrapb -import fmt "fmt" +import "fmt" // IntValue stores an optional unsigned integer value, used in the ComponentStats // message. diff --git a/pkg/sql/execstats/execstatspb/int_value_test.go b/pkg/sql/execinfrapb/int_value_test.go similarity index 98% rename from pkg/sql/execstats/execstatspb/int_value_test.go rename to pkg/sql/execinfrapb/int_value_test.go index b857c8851f05..bc298b21a15f 100644 --- a/pkg/sql/execstats/execstatspb/int_value_test.go +++ b/pkg/sql/execinfrapb/int_value_test.go @@ -8,7 +8,7 @@ // by the Apache License, Version 2.0, included in the file // licenses/APL.txt. -package execstatspb +package execinfrapb import ( "testing" diff --git a/pkg/sql/execstats/BUILD.bazel b/pkg/sql/execstats/BUILD.bazel index 776c3410197c..588b5093a1a7 100644 --- a/pkg/sql/execstats/BUILD.bazel +++ b/pkg/sql/execstats/BUILD.bazel @@ -8,7 +8,6 @@ go_library( deps = [ "//pkg/roachpb", "//pkg/sql/execinfrapb", - "//pkg/sql/execstats/execstatspb", "//pkg/util/tracing/tracingpb", "//vendor/github.com/cockroachdb/errors", "//vendor/github.com/gogo/protobuf/types", diff --git a/pkg/sql/execstats/execstatspb/BUILD.bazel b/pkg/sql/execstats/execstatspb/BUILD.bazel deleted file mode 100644 index a6caa1f241c0..000000000000 --- a/pkg/sql/execstats/execstatspb/BUILD.bazel +++ /dev/null @@ -1,24 +0,0 @@ -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") - -go_library( - name = "execstatspb", - srcs = [ - "component_stats.go", - "component_stats.pb.go", - "int_value.go", - ], - importpath = "github.com/cockroachdb/cockroach/pkg/sql/execstats/execstatspb", - visibility = ["//visibility:public"], - deps = [ - "//vendor/github.com/dustin/go-humanize", - "//vendor/github.com/gogo/protobuf/proto", - "//vendor/github.com/gogo/protobuf/types", - ], -) - -go_test( - name = "execstatspb_test", - srcs = ["int_value_test.go"], - embed = [":execstatspb"], - deps = ["//vendor/github.com/stretchr/testify/require"], -) diff --git a/pkg/sql/execstats/execstatspb/component_stats.proto b/pkg/sql/execstats/execstatspb/component_stats.proto deleted file mode 100644 index 7db11e840a79..000000000000 --- a/pkg/sql/execstats/execstatspb/component_stats.proto +++ /dev/null @@ -1,103 +0,0 @@ -// Copyright 2020 The Cockroach Authors. -// -// Use of this software is governed by the Business Source License -// included in the file licenses/BSL.txt. -// -// As of the Change Date specified in that file, in accordance with -// the Business Source License, use of this software will be governed -// by the Apache License, Version 2.0, included in the file -// licenses/APL.txt. - -syntax = "proto3"; -package cockroach.sql.execstats.execstatspb; -option go_package = "execstatspb"; - -import "gogoproto/gogo.proto"; -import "google/protobuf/duration.proto"; - -// ComponentStats contains statistics for an execution component. A component is -// an arbitrary unit in the execution infrastructure; it can correspond to an -// operator or a stream. -// -// Depending on the component, not all statistics apply. For all fields, the zero -// value indicates that the particular stat is not available. -message ComponentStats { - int32 component_id = 1 [(gogoproto.customname) = "ComponentID"]; - - NetworkRxStats net_rx = 2 [(gogoproto.nullable) = false]; - NetworkTxStats net_tx = 3 [(gogoproto.nullable) = false]; - KVStats kv = 4 [(gogoproto.customname) = "KV", (gogoproto.nullable) = false]; - ExecStats exec = 5 [(gogoproto.nullable) = false]; - OutputStats output = 6 [(gogoproto.nullable) = false]; - - // Stats for the inputs of an operator (only in the row execution engine). - repeated InputStats inputs = 7 [(gogoproto.nullable) = false]; -} - -// InputStats contains statistics about the rows received as an input to a -// processor. Currently only used in the row execution engine. -message InputStats { - uint64 num_tuples = 1 [(gogoproto.customtype) = "IntValue", (gogoproto.nullable) = false]; - // Cumulated time spent waiting for rows from the input operator. - google.protobuf.Duration wait_time = 2 [(gogoproto.nullable) = false, - (gogoproto.stdduration) = true]; -} - -// NetworkRxStats contains statistics for components that receive row data -// over the network. Note: Rx is shorthand for Receive. -message NetworkRxStats { - // Observed network latency (round-trip time between the two nodes). - google.protobuf.Duration latency = 1 [(gogoproto.nullable) = false, - (gogoproto.stdduration) = true]; - - // Cumulated time spent waiting to receive or transmit tuple data. - google.protobuf.Duration wait_time = 2 [(gogoproto.nullable) = false, - (gogoproto.stdduration) = true]; - - // Time spent deserializing network data. - google.protobuf.Duration deserialization_time = 3 [(gogoproto.nullable) = false, - (gogoproto.stdduration) = true]; - - uint64 tuples_received = 4 [(gogoproto.customtype) = "IntValue", (gogoproto.nullable) = false]; - uint64 bytes_received = 5 [(gogoproto.customtype) = "IntValue", (gogoproto.nullable) = false]; -} - -// NetworkTxStats contains statistics for components that send row data over the -// network. Note: Tx is shorthand for Transmit. -message NetworkTxStats { - uint64 tuples_sent = 1 [(gogoproto.customtype) = "IntValue", (gogoproto.nullable) = false]; - uint64 bytes_sent = 2 [(gogoproto.customtype) = "IntValue", (gogoproto.nullable) = false]; -} - -// KVStats contains statistics for components that perform KV operations. -message KVStats { - uint64 bytes_read = 1 [(gogoproto.customtype) = "IntValue", (gogoproto.nullable) = false]; - uint64 tuples_read = 2 [(gogoproto.customtype) = "IntValue", (gogoproto.nullable) = false]; - - // Cumulated time spent waiting for a KV request. This includes disk IO time - // and potentially network time (if any of the keys are not local). - google.protobuf.Duration kv_time = 3 [(gogoproto.customname) = "KVTime", - (gogoproto.nullable) = false, - (gogoproto.stdduration) = true]; -} - -// ExecStats contains statistics about the execution of an components. -message ExecStats { - // Time spent executing the component. - google.protobuf.Duration exec_time = 1 [(gogoproto.nullable) = false, - (gogoproto.stdduration) = true]; - // Maximum memory allocated by the component. - uint64 max_allocated_mem = 2 [(gogoproto.customtype) = "IntValue", (gogoproto.nullable) = false]; - - // Maximum scratch disk allocated by the component. - uint64 max_allocated_disk = 3 [(gogoproto.customtype) = "IntValue", (gogoproto.nullable) = false]; -} - -// OutputStats contains statistics about the output (results) of an component. -message OutputStats { - // Number of batches produced by the component. - uint64 num_batches = 1 [(gogoproto.customtype) = "IntValue", (gogoproto.nullable) = false]; - - // Number of tuples produced by the component. - uint64 num_tuples = 2 [(gogoproto.customtype) = "IntValue", (gogoproto.nullable) = false]; -} diff --git a/pkg/sql/execstats/traceanalyzer.go b/pkg/sql/execstats/traceanalyzer.go index 4e874ffa6ff7..b5b4ca9b4e81 100644 --- a/pkg/sql/execstats/traceanalyzer.go +++ b/pkg/sql/execstats/traceanalyzer.go @@ -15,7 +15,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" - "github.com/cockroachdb/cockroach/pkg/sql/execstats/execstatspb" "github.com/cockroachdb/cockroach/pkg/util/tracing/tracingpb" "github.com/cockroachdb/errors" "github.com/gogo/protobuf/types" @@ -128,7 +127,7 @@ func (a *TraceAnalyzer) AddTrace(trace []tracingpb.RecordedSpan) error { } func getNetworkBytesFromDistSQLSpanStats(dss execinfrapb.DistSQLSpanStats) (int64, error) { - v, ok := dss.(*execstatspb.ComponentStats) + v, ok := dss.(*execinfrapb.ComponentStats) if !ok { return 0, errors.Errorf("could not get network bytes from %T", dss) } diff --git a/pkg/sql/flowinfra/BUILD.bazel b/pkg/sql/flowinfra/BUILD.bazel index 9e689eeae5ec..1e097412f530 100644 --- a/pkg/sql/flowinfra/BUILD.bazel +++ b/pkg/sql/flowinfra/BUILD.bazel @@ -24,7 +24,6 @@ go_library( "//pkg/sql/catalog/descpb", "//pkg/sql/execinfra", "//pkg/sql/execinfrapb", - "//pkg/sql/execstats/execstatspb", "//pkg/sql/mutations", "//pkg/sql/pgwire/pgcode", "//pkg/sql/pgwire/pgerror", diff --git a/pkg/sql/flowinfra/outbox.go b/pkg/sql/flowinfra/outbox.go index 2405df7a6a58..fe7704a339dd 100644 --- a/pkg/sql/flowinfra/outbox.go +++ b/pkg/sql/flowinfra/outbox.go @@ -19,7 +19,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" - "github.com/cockroachdb/cockroach/pkg/sql/execstats/execstatspb" "github.com/cockroachdb/cockroach/pkg/sql/rowenc" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/contextutil" @@ -64,7 +63,7 @@ type Outbox struct { err error statsCollectionEnabled bool - stats execstatspb.ComponentStats + stats execinfrapb.ComponentStats } var _ execinfra.RowReceiver = &Outbox{} diff --git a/pkg/sql/rowexec/BUILD.bazel b/pkg/sql/rowexec/BUILD.bazel index 556d28af780e..01b0d9996be8 100644 --- a/pkg/sql/rowexec/BUILD.bazel +++ b/pkg/sql/rowexec/BUILD.bazel @@ -55,7 +55,6 @@ go_library( "//pkg/sql/catalog/tabledesc", "//pkg/sql/execinfra", "//pkg/sql/execinfrapb", - "//pkg/sql/execstats/execstatspb", "//pkg/sql/flowinfra", "//pkg/sql/opt/invertedexpr", "//pkg/sql/opt/invertedidx", @@ -144,7 +143,6 @@ go_test( "//pkg/sql/catalog/tabledesc", "//pkg/sql/execinfra", "//pkg/sql/execinfrapb", - "//pkg/sql/execstats/execstatspb", "//pkg/sql/flowinfra", "//pkg/sql/opt/invertedexpr", "//pkg/sql/rowcontainer", diff --git a/pkg/sql/rowexec/aggregator.go b/pkg/sql/rowexec/aggregator.go index 7dc852470560..ebf43dc6adde 100644 --- a/pkg/sql/rowexec/aggregator.go +++ b/pkg/sql/rowexec/aggregator.go @@ -16,7 +16,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" - "github.com/cockroachdb/cockroach/pkg/sql/execstats/execstatspb" "github.com/cockroachdb/cockroach/pkg/sql/rowenc" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" @@ -153,15 +152,15 @@ func (ag *aggregatorBase) init( } // execStatsForTrace implements ProcessorBase.ExecStatsForTrace. -func (ag *aggregatorBase) execStatsForTrace() *execstatspb.ComponentStats { +func (ag *aggregatorBase) execStatsForTrace() *execinfrapb.ComponentStats { is, ok := getInputStats(ag.input) if !ok { return nil } - return &execstatspb.ComponentStats{ - Inputs: []execstatspb.InputStats{is}, - Exec: execstatspb.ExecStats{ - MaxAllocatedMem: execstatspb.MakeIntValue(uint64(ag.MemMonitor.MaximumBytes())), + return &execinfrapb.ComponentStats{ + Inputs: []execinfrapb.InputStats{is}, + Exec: execinfrapb.ExecStats{ + MaxAllocatedMem: execinfrapb.MakeIntValue(uint64(ag.MemMonitor.MaximumBytes())), }, Output: ag.Out.Stats(), } diff --git a/pkg/sql/rowexec/countrows.go b/pkg/sql/rowexec/countrows.go index 03e6bb134267..c53b6fe8276f 100644 --- a/pkg/sql/rowexec/countrows.go +++ b/pkg/sql/rowexec/countrows.go @@ -15,7 +15,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" - "github.com/cockroachdb/cockroach/pkg/sql/execstats/execstatspb" "github.com/cockroachdb/cockroach/pkg/sql/rowenc" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" @@ -107,13 +106,13 @@ func (ag *countAggregator) ConsumerClosed() { } // execStatsForTrace implements ProcessorBase.ExecStatsForTrace. -func (ag *countAggregator) execStatsForTrace() *execstatspb.ComponentStats { +func (ag *countAggregator) execStatsForTrace() *execinfrapb.ComponentStats { is, ok := getInputStats(ag.input) if !ok { return nil } - return &execstatspb.ComponentStats{ - Inputs: []execstatspb.InputStats{is}, + return &execinfrapb.ComponentStats{ + Inputs: []execinfrapb.InputStats{is}, Output: ag.Out.Stats(), } } diff --git a/pkg/sql/rowexec/distinct.go b/pkg/sql/rowexec/distinct.go index 8a72f5830043..ee9302501bd1 100644 --- a/pkg/sql/rowexec/distinct.go +++ b/pkg/sql/rowexec/distinct.go @@ -15,7 +15,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" - "github.com/cockroachdb/cockroach/pkg/sql/execstats/execstatspb" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" "github.com/cockroachdb/cockroach/pkg/sql/rowenc" @@ -345,15 +344,15 @@ func (d *distinct) ConsumerClosed() { } // execStatsForTrace implements ProcessorBase.ExecStatsForTrace. -func (d *distinct) execStatsForTrace() *execstatspb.ComponentStats { +func (d *distinct) execStatsForTrace() *execinfrapb.ComponentStats { is, ok := getInputStats(d.input) if !ok { return nil } - return &execstatspb.ComponentStats{ - Inputs: []execstatspb.InputStats{is}, - Exec: execstatspb.ExecStats{ - MaxAllocatedMem: execstatspb.MakeIntValue(uint64(d.MemMonitor.MaximumBytes())), + return &execinfrapb.ComponentStats{ + Inputs: []execinfrapb.InputStats{is}, + Exec: execinfrapb.ExecStats{ + MaxAllocatedMem: execinfrapb.MakeIntValue(uint64(d.MemMonitor.MaximumBytes())), }, Output: d.Out.Stats(), } diff --git a/pkg/sql/rowexec/hashjoiner.go b/pkg/sql/rowexec/hashjoiner.go index dee8b5e85a77..763c61d56f21 100644 --- a/pkg/sql/rowexec/hashjoiner.go +++ b/pkg/sql/rowexec/hashjoiner.go @@ -16,7 +16,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" - "github.com/cockroachdb/cockroach/pkg/sql/execstats/execstatspb" "github.com/cockroachdb/cockroach/pkg/sql/rowcontainer" "github.com/cockroachdb/cockroach/pkg/sql/rowenc" "github.com/cockroachdb/cockroach/pkg/util/cancelchecker" @@ -556,7 +555,7 @@ func (h *hashJoiner) shouldEmitUnmatched( } // execStatsForTrace implements ProcessorBase.ExecStatsForTrace. -func (h *hashJoiner) execStatsForTrace() *execstatspb.ComponentStats { +func (h *hashJoiner) execStatsForTrace() *execinfrapb.ComponentStats { lis, ok := getInputStats(h.leftSource) if !ok { return nil @@ -565,11 +564,11 @@ func (h *hashJoiner) execStatsForTrace() *execstatspb.ComponentStats { if !ok { return nil } - return &execstatspb.ComponentStats{ - Inputs: []execstatspb.InputStats{lis, ris}, - Exec: execstatspb.ExecStats{ - MaxAllocatedMem: execstatspb.MakeIntValue(uint64(h.MemMonitor.MaximumBytes())), - MaxAllocatedDisk: execstatspb.MakeIntValue(uint64(h.diskMonitor.MaximumBytes())), + return &execinfrapb.ComponentStats{ + Inputs: []execinfrapb.InputStats{lis, ris}, + Exec: execinfrapb.ExecStats{ + MaxAllocatedMem: execinfrapb.MakeIntValue(uint64(h.MemMonitor.MaximumBytes())), + MaxAllocatedDisk: execinfrapb.MakeIntValue(uint64(h.diskMonitor.MaximumBytes())), }, Output: h.Out.Stats(), } diff --git a/pkg/sql/rowexec/inverted_filterer.go b/pkg/sql/rowexec/inverted_filterer.go index ac4235946269..ce89b69bcdcb 100644 --- a/pkg/sql/rowexec/inverted_filterer.go +++ b/pkg/sql/rowexec/inverted_filterer.go @@ -15,7 +15,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" - "github.com/cockroachdb/cockroach/pkg/sql/execstats/execstatspb" "github.com/cockroachdb/cockroach/pkg/sql/opt/invertedexpr" "github.com/cockroachdb/cockroach/pkg/sql/opt/invertedidx" "github.com/cockroachdb/cockroach/pkg/sql/rowcontainer" @@ -288,16 +287,16 @@ func (ifr *invertedFilterer) close() { // execStatsForTrace outputs the collected invertedFilterer stats to the // trace. Will fail silently if the invertedFilterer is not collecting stats. -func (ifr *invertedFilterer) execStatsForTrace() *execstatspb.ComponentStats { +func (ifr *invertedFilterer) execStatsForTrace() *execinfrapb.ComponentStats { is, ok := getInputStats(ifr.input) if !ok { return nil } - return &execstatspb.ComponentStats{ - Inputs: []execstatspb.InputStats{is}, - Exec: execstatspb.ExecStats{ - MaxAllocatedMem: execstatspb.MakeIntValue(uint64(ifr.MemMonitor.MaximumBytes())), - MaxAllocatedDisk: execstatspb.MakeIntValue(uint64(ifr.diskMonitor.MaximumBytes())), + return &execinfrapb.ComponentStats{ + Inputs: []execinfrapb.InputStats{is}, + Exec: execinfrapb.ExecStats{ + MaxAllocatedMem: execinfrapb.MakeIntValue(uint64(ifr.MemMonitor.MaximumBytes())), + MaxAllocatedDisk: execinfrapb.MakeIntValue(uint64(ifr.diskMonitor.MaximumBytes())), }, Output: ifr.Out.Stats(), } diff --git a/pkg/sql/rowexec/inverted_joiner.go b/pkg/sql/rowexec/inverted_joiner.go index 792dbfc47e85..0d299f269b55 100644 --- a/pkg/sql/rowexec/inverted_joiner.go +++ b/pkg/sql/rowexec/inverted_joiner.go @@ -18,7 +18,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" - "github.com/cockroachdb/cockroach/pkg/sql/execstats/execstatspb" "github.com/cockroachdb/cockroach/pkg/sql/opt/invertedexpr" "github.com/cockroachdb/cockroach/pkg/sql/opt/invertedidx" "github.com/cockroachdb/cockroach/pkg/sql/row" @@ -668,7 +667,7 @@ func (ij *invertedJoiner) close() { } // execStatsForTrace implements ProcessorBase.ExecStatsForTrace. -func (ij *invertedJoiner) execStatsForTrace() *execstatspb.ComponentStats { +func (ij *invertedJoiner) execStatsForTrace() *execinfrapb.ComponentStats { is, ok := getInputStats(ij.input) if !ok { return nil @@ -677,15 +676,15 @@ func (ij *invertedJoiner) execStatsForTrace() *execstatspb.ComponentStats { if !ok { return nil } - return &execstatspb.ComponentStats{ - Inputs: []execstatspb.InputStats{is}, - KV: execstatspb.KVStats{ + return &execinfrapb.ComponentStats{ + Inputs: []execinfrapb.InputStats{is}, + KV: execinfrapb.KVStats{ TuplesRead: fis.NumTuples, KVTime: fis.WaitTime, }, - Exec: execstatspb.ExecStats{ - MaxAllocatedMem: execstatspb.MakeIntValue(uint64(ij.MemMonitor.MaximumBytes())), - MaxAllocatedDisk: execstatspb.MakeIntValue(uint64(ij.diskMonitor.MaximumBytes())), + Exec: execinfrapb.ExecStats{ + MaxAllocatedMem: execinfrapb.MakeIntValue(uint64(ij.MemMonitor.MaximumBytes())), + MaxAllocatedDisk: execinfrapb.MakeIntValue(uint64(ij.diskMonitor.MaximumBytes())), }, Output: ij.Out.Stats(), } diff --git a/pkg/sql/rowexec/joinreader.go b/pkg/sql/rowexec/joinreader.go index 5c7a8f2cf99f..1b3630f6bf8e 100644 --- a/pkg/sql/rowexec/joinreader.go +++ b/pkg/sql/rowexec/joinreader.go @@ -21,7 +21,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" - "github.com/cockroachdb/cockroach/pkg/sql/execstats/execstatspb" "github.com/cockroachdb/cockroach/pkg/sql/row" "github.com/cockroachdb/cockroach/pkg/sql/rowcontainer" "github.com/cockroachdb/cockroach/pkg/sql/rowenc" @@ -675,7 +674,7 @@ func (jr *joinReader) close() { // execStatsForTrace outputs the collected joinReader stats to the trace. Will // fail silently if the joinReader is not collecting stats. -func (jr *joinReader) execStatsForTrace() *execstatspb.ComponentStats { +func (jr *joinReader) execStatsForTrace() *execinfrapb.ComponentStats { is, ok := getInputStats(jr.input) if !ok { return nil @@ -686,9 +685,9 @@ func (jr *joinReader) execStatsForTrace() *execstatspb.ComponentStats { } // TODO(asubiotto): Add memory and disk usage to EXPLAIN ANALYZE. - return &execstatspb.ComponentStats{ - Inputs: []execstatspb.InputStats{is}, - KV: execstatspb.KVStats{ + return &execinfrapb.ComponentStats{ + Inputs: []execinfrapb.InputStats{is}, + KV: execinfrapb.KVStats{ TuplesRead: fis.NumTuples, KVTime: fis.WaitTime, }, diff --git a/pkg/sql/rowexec/mergejoiner.go b/pkg/sql/rowexec/mergejoiner.go index 65fa1d14c75f..4147a1977b5c 100644 --- a/pkg/sql/rowexec/mergejoiner.go +++ b/pkg/sql/rowexec/mergejoiner.go @@ -16,7 +16,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" - "github.com/cockroachdb/cockroach/pkg/sql/execstats/execstatspb" "github.com/cockroachdb/cockroach/pkg/sql/rowenc" "github.com/cockroachdb/cockroach/pkg/util" "github.com/cockroachdb/cockroach/pkg/util/cancelchecker" @@ -274,7 +273,7 @@ func (m *mergeJoiner) ConsumerClosed() { } // execStatsForTrace implements ProcessorBase.ExecStatsForTrace. -func (m *mergeJoiner) execStatsForTrace() *execstatspb.ComponentStats { +func (m *mergeJoiner) execStatsForTrace() *execinfrapb.ComponentStats { lis, ok := getInputStats(m.leftSource) if !ok { return nil @@ -283,10 +282,10 @@ func (m *mergeJoiner) execStatsForTrace() *execstatspb.ComponentStats { if !ok { return nil } - return &execstatspb.ComponentStats{ - Inputs: []execstatspb.InputStats{lis, ris}, - Exec: execstatspb.ExecStats{ - MaxAllocatedMem: execstatspb.MakeIntValue(uint64(m.MemMonitor.MaximumBytes())), + return &execinfrapb.ComponentStats{ + Inputs: []execinfrapb.InputStats{lis, ris}, + Exec: execinfrapb.ExecStats{ + MaxAllocatedMem: execinfrapb.MakeIntValue(uint64(m.MemMonitor.MaximumBytes())), }, Output: m.Out.Stats(), } diff --git a/pkg/sql/rowexec/ordinality.go b/pkg/sql/rowexec/ordinality.go index 527f2140ea4b..ff064c380ef4 100644 --- a/pkg/sql/rowexec/ordinality.go +++ b/pkg/sql/rowexec/ordinality.go @@ -15,7 +15,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" - "github.com/cockroachdb/cockroach/pkg/sql/execstats/execstatspb" "github.com/cockroachdb/cockroach/pkg/sql/rowenc" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" @@ -117,13 +116,13 @@ func (o *ordinalityProcessor) ConsumerClosed() { // execStatsForTrace outputs the collected distinct stats to the trace. Will // fail silently if the Distinct processor is not collecting stats. -func (o *ordinalityProcessor) execStatsForTrace() *execstatspb.ComponentStats { +func (o *ordinalityProcessor) execStatsForTrace() *execinfrapb.ComponentStats { is, ok := getInputStats(o.input) if !ok { return nil } - return &execstatspb.ComponentStats{ - Inputs: []execstatspb.InputStats{is}, + return &execinfrapb.ComponentStats{ + Inputs: []execinfrapb.InputStats{is}, Output: o.Out.Stats(), } } diff --git a/pkg/sql/rowexec/sorter.go b/pkg/sql/rowexec/sorter.go index 8d468be4a43b..7971ddc8cc8c 100644 --- a/pkg/sql/rowexec/sorter.go +++ b/pkg/sql/rowexec/sorter.go @@ -18,7 +18,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" - "github.com/cockroachdb/cockroach/pkg/sql/execstats/execstatspb" "github.com/cockroachdb/cockroach/pkg/sql/rowcontainer" "github.com/cockroachdb/cockroach/pkg/sql/rowenc" "github.com/cockroachdb/cockroach/pkg/util/mon" @@ -127,16 +126,16 @@ func (s *sorterBase) close() { } // execStatsForTrace implements ProcessorBase.ExecStatsForTrace. -func (s *sorterBase) execStatsForTrace() *execstatspb.ComponentStats { +func (s *sorterBase) execStatsForTrace() *execinfrapb.ComponentStats { is, ok := getInputStats(s.input) if !ok { return nil } - return &execstatspb.ComponentStats{ - Inputs: []execstatspb.InputStats{is}, - Exec: execstatspb.ExecStats{ - MaxAllocatedMem: execstatspb.MakeIntValue(uint64(s.MemMonitor.MaximumBytes())), - MaxAllocatedDisk: execstatspb.MakeIntValue(uint64(s.diskMonitor.MaximumBytes())), + return &execinfrapb.ComponentStats{ + Inputs: []execinfrapb.InputStats{is}, + Exec: execinfrapb.ExecStats{ + MaxAllocatedMem: execinfrapb.MakeIntValue(uint64(s.MemMonitor.MaximumBytes())), + MaxAllocatedDisk: execinfrapb.MakeIntValue(uint64(s.diskMonitor.MaximumBytes())), }, Output: s.Out.Stats(), } diff --git a/pkg/sql/rowexec/stats.go b/pkg/sql/rowexec/stats.go index 8f17511d630f..e300121373b9 100644 --- a/pkg/sql/rowexec/stats.go +++ b/pkg/sql/rowexec/stats.go @@ -20,7 +20,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" - "github.com/cockroachdb/cockroach/pkg/sql/execstats/execstatspb" "github.com/cockroachdb/cockroach/pkg/sql/row" "github.com/cockroachdb/cockroach/pkg/sql/rowenc" "github.com/cockroachdb/cockroach/pkg/util/hlc" @@ -31,7 +30,7 @@ import ( // inputStatCollector wraps an execinfra.RowSource and collects stats from it. type inputStatCollector struct { execinfra.RowSource - stats execstatspb.InputStats + stats execinfrapb.InputStats } var _ execinfra.RowSource = &inputStatCollector{} @@ -79,7 +78,7 @@ func (isc *inputStatCollector) Next() (rowenc.EncDatumRow, *execinfrapb.Producer type rowFetcherStatCollector struct { *row.Fetcher // stats contains the collected stats. - stats execstatspb.InputStats + stats execinfrapb.InputStats startScanStallTime time.Duration } @@ -142,10 +141,10 @@ func (c *rowFetcherStatCollector) StartInconsistentScan( // getInputStats is a utility function to check whether the given input is // collecting stats, returning true and the stats if so. If false is returned, // the input is not collecting stats. -func getInputStats(input execinfra.RowSource) (execstatspb.InputStats, bool) { +func getInputStats(input execinfra.RowSource) (execinfrapb.InputStats, bool) { isc, ok := input.(*inputStatCollector) if !ok { - return execstatspb.InputStats{}, false + return execinfrapb.InputStats{}, false } return isc.stats, true } @@ -153,10 +152,10 @@ func getInputStats(input execinfra.RowSource) (execstatspb.InputStats, bool) { // getFetcherInputStats is a utility function to check whether the given input // is collecting row fetcher stats, returning true and the stats if so. If // false is returned, the input is not collecting row fetcher stats. -func getFetcherInputStats(f rowFetcher) (execstatspb.InputStats, bool) { +func getFetcherInputStats(f rowFetcher) (execinfrapb.InputStats, bool) { rfsc, ok := f.(*rowFetcherStatCollector) if !ok { - return execstatspb.InputStats{}, false + return execinfrapb.InputStats{}, false } // Add row fetcher start scan stall time to Next() stall time. rfsc.stats.WaitTime += rfsc.startScanStallTime diff --git a/pkg/sql/rowexec/tablereader.go b/pkg/sql/rowexec/tablereader.go index 347125675582..9ec071497b02 100644 --- a/pkg/sql/rowexec/tablereader.go +++ b/pkg/sql/rowexec/tablereader.go @@ -21,7 +21,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" - "github.com/cockroachdb/cockroach/pkg/sql/execstats/execstatspb" "github.com/cockroachdb/cockroach/pkg/sql/row" "github.com/cockroachdb/cockroach/pkg/sql/rowenc" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -272,15 +271,15 @@ func (tr *tableReader) ConsumerClosed() { } // execStatsForTrace implements ProcessorBase.ExecStatsForTrace. -func (tr *tableReader) execStatsForTrace() *execstatspb.ComponentStats { +func (tr *tableReader) execStatsForTrace() *execinfrapb.ComponentStats { is, ok := getFetcherInputStats(tr.fetcher) if !ok { return nil } - return &execstatspb.ComponentStats{ - KV: execstatspb.KVStats{ + return &execinfrapb.ComponentStats{ + KV: execinfrapb.KVStats{ TuplesRead: is.NumTuples, - BytesRead: execstatspb.MakeIntValue(uint64(tr.GetBytesRead())), + BytesRead: execinfrapb.MakeIntValue(uint64(tr.GetBytesRead())), KVTime: is.WaitTime, }, Output: tr.Out.Stats(), diff --git a/pkg/sql/rowexec/tablereader_test.go b/pkg/sql/rowexec/tablereader_test.go index b4dd65f640ed..bc8efef32249 100644 --- a/pkg/sql/rowexec/tablereader_test.go +++ b/pkg/sql/rowexec/tablereader_test.go @@ -25,7 +25,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkv" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" - "github.com/cockroachdb/cockroach/pkg/sql/execstats/execstatspb" "github.com/cockroachdb/cockroach/pkg/sql/rowenc" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/testutils" @@ -439,7 +438,7 @@ func TestLimitScans(t *testing.T) { for _, span := range spans { if span.Operation == tableReaderProcName { // Verify that stat collection lines up with results. - stats := execstatspb.ComponentStats{} + stats := execinfrapb.ComponentStats{} if err := types.UnmarshalAny(span.Stats, &stats); err != nil { t.Fatal(err) } diff --git a/pkg/sql/rowexec/windower.go b/pkg/sql/rowexec/windower.go index 3570c0ae4ad8..89e1dcb5eb97 100644 --- a/pkg/sql/rowexec/windower.go +++ b/pkg/sql/rowexec/windower.go @@ -16,7 +16,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" - "github.com/cockroachdb/cockroach/pkg/sql/execstats/execstatspb" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" "github.com/cockroachdb/cockroach/pkg/sql/rowcontainer" @@ -839,16 +838,16 @@ func CreateWindowerSpecFunc(funcStr string) (execinfrapb.WindowerSpec_Func, erro } // execStatsForTrace implements ProcessorBase.ExecStatsForTrace. -func (w *windower) execStatsForTrace() *execstatspb.ComponentStats { +func (w *windower) execStatsForTrace() *execinfrapb.ComponentStats { is, ok := getInputStats(w.input) if !ok { return nil } - return &execstatspb.ComponentStats{ - Inputs: []execstatspb.InputStats{is}, - Exec: execstatspb.ExecStats{ - MaxAllocatedMem: execstatspb.MakeIntValue(uint64(w.MemMonitor.MaximumBytes())), - MaxAllocatedDisk: execstatspb.MakeIntValue(uint64(w.diskMonitor.MaximumBytes())), + return &execinfrapb.ComponentStats{ + Inputs: []execinfrapb.InputStats{is}, + Exec: execinfrapb.ExecStats{ + MaxAllocatedMem: execinfrapb.MakeIntValue(uint64(w.MemMonitor.MaximumBytes())), + MaxAllocatedDisk: execinfrapb.MakeIntValue(uint64(w.diskMonitor.MaximumBytes())), }, Output: w.Out.Stats(), } diff --git a/pkg/sql/rowflow/BUILD.bazel b/pkg/sql/rowflow/BUILD.bazel index 8b80f995a595..d94d4c815c21 100644 --- a/pkg/sql/rowflow/BUILD.bazel +++ b/pkg/sql/rowflow/BUILD.bazel @@ -14,7 +14,6 @@ go_library( "//pkg/sql/catalog/colinfo", "//pkg/sql/execinfra", "//pkg/sql/execinfrapb", - "//pkg/sql/execstats/execstatspb", "//pkg/sql/flowinfra", "//pkg/sql/rowcontainer", "//pkg/sql/rowenc", diff --git a/pkg/sql/rowflow/routers.go b/pkg/sql/rowflow/routers.go index 7933770b35ab..8d8f87921c97 100644 --- a/pkg/sql/rowflow/routers.go +++ b/pkg/sql/rowflow/routers.go @@ -24,7 +24,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" - "github.com/cockroachdb/cockroach/pkg/sql/execstats/execstatspb" "github.com/cockroachdb/cockroach/pkg/sql/flowinfra" "github.com/cockroachdb/cockroach/pkg/sql/rowcontainer" "github.com/cockroachdb/cockroach/pkg/sql/rowenc" @@ -102,7 +101,7 @@ type routerOutput struct { // TODO(radu): add padding of size sys.CacheLineSize to ensure there is no // false-sharing? - stats execstatspb.ComponentStats + stats execinfrapb.ComponentStats // memoryMonitor and diskMonitor are mu.rowContainer's monitors. memoryMonitor, diskMonitor *mon.BytesMonitor @@ -312,7 +311,7 @@ func (rb *routerBase) Start(ctx context.Context, wg *sync.WaitGroup, ctxCancel c if rb.statsCollectionEnabled { ctx, span = execinfra.ProcessorSpan(ctx, "router output") span.SetTag(execinfrapb.StreamIDTagKey, ro.streamID) - ro.stats.Inputs = make([]execstatspb.InputStats, 1) + ro.stats.Inputs = make([]execinfrapb.InputStats, 1) } drain := false