/
component_stats.proto
212 lines (174 loc) · 9.75 KB
/
component_stats.proto
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
// 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 = "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb";
import "gogoproto/gogo.proto";
import "util/optional/optional.proto";
// ComponentID identifies a component in a flow. There are multiple types of
// components (e.g. processors, streams); each component of a certain type has
// an integer identifier.
message ComponentID {
enum Type {
UNSET = 0;
// The component is a processor; the ID field corresponds to a processor ID
// in the plan.
PROCESSOR = 1;
// The component is a stream; the ID field corresponds to a stream ID in the
// plan.
STREAM = 2;
// The "component" is a flow (specifically, the part of a distributed plan
// that runs on a given node).
// TODO(radu): the ID field should correspond to a node ID in the plan.
FLOW = 3;
}
optional bytes flow_id = 1 [(gogoproto.nullable) = false,
(gogoproto.customname) = "FlowID",
(gogoproto.customtype) = "FlowID"];
optional Type type = 2 [(gogoproto.nullable) = false];
// Identifier of this component, within the domain of components of the same
// type.
optional int32 id = 3 [(gogoproto.nullable) = false,
(gogoproto.customname) = "ID"];
// SQLInstanceID of the node this component is associated with. For cross-node
// streams, this is the *origin* node for the stream.
optional int32 sql_instance_id = 4 [
(gogoproto.nullable) = false,
(gogoproto.customname) = "SQLInstanceID",
(gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/base.SQLInstanceID"];
// The region the component is associated with.
optional string region = 5 [
(gogoproto.nullable) = false,
(gogoproto.customname) = "Region"];
}
// 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 {
option (gogoproto.goproto_stringer) = false;
optional ComponentID component = 1 [(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];
optional FlowStats flow_stats = 8 [(gogoproto.nullable) = false];
// WARNING! If any new fields are added, corresponding code must be added in
// Union() and possibly MakeDeterminstic().
}
// InputStats contains statistics about the rows received as an input to a
// processor. Currently only used in the row execution engine.
message InputStats {
optional util.optional.Uint num_tuples = 1 [(gogoproto.nullable) = false];
// Cumulated time spent waiting for rows from the input operator.
optional util.optional.Duration wait_time = 2 [(gogoproto.nullable) = false];
}
// 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 util.optional.Duration latency = 1 [(gogoproto.nullable) = false];
// Cumulated time spent waiting to receive or transmit tuple data.
optional util.optional.Duration wait_time = 2 [(gogoproto.nullable) = false];
// Time spent deserializing network data.
optional util.optional.Duration deserialization_time = 3 [(gogoproto.nullable) = false];
optional util.optional.Uint tuples_received = 4 [(gogoproto.nullable) = false];
optional util.optional.Uint bytes_received = 5 [(gogoproto.nullable) = false];
// Number of messages received over the network.
optional util.optional.Uint messages_received = 6 [(gogoproto.nullable) = false];
}
// NetworkTxStats contains statistics for components that send row data over the
// network. Note: Tx is shorthand for Transmit.
message NetworkTxStats {
optional util.optional.Uint tuples_sent = 1 [(gogoproto.nullable) = false];
optional util.optional.Uint bytes_sent = 2 [(gogoproto.nullable) = false];
// Number of messages sent over the network.
optional util.optional.Uint messages_sent = 3 [(gogoproto.nullable) = false];
}
// KVStats contains statistics for components that perform KV operations.
message KVStats {
optional util.optional.Uint bytes_read = 1 [(gogoproto.nullable) = false];
optional util.optional.Uint kv_pairs_read = 24 [(gogoproto.customname) = "KVPairsRead", (gogoproto.nullable) = false];
optional util.optional.Uint tuples_read = 2 [(gogoproto.nullable) = false];
optional util.optional.Uint batch_requests_issued = 9 [(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 util.optional.Duration kv_time = 3 [(gogoproto.customname) = "KVTime",
(gogoproto.nullable) = false];
// ContentionTime is the cumulative time a KV request spent contending with
// other transactions. This time accounts for a portion of KVTime above.
optional util.optional.Duration contention_time = 4 [(gogoproto.nullable) = false];
// Internal iteration statistics.
// For details, see pkg/sql/opt/exec/factory.go, pkg/storage/engine.go,
// pebble.InternalIteratorStats, and pebble.RangeKeyIteratorStats.
optional util.optional.Uint num_interface_steps = 5 [(gogoproto.nullable) = false];
optional util.optional.Uint num_internal_steps = 6 [(gogoproto.nullable) = false];
optional util.optional.Uint num_interface_seeks = 7 [(gogoproto.nullable) = false];
optional util.optional.Uint num_internal_seeks = 8 [(gogoproto.nullable) = false];
optional util.optional.Uint block_bytes = 12 [(gogoproto.nullable) = false];
optional util.optional.Uint block_bytes_in_cache = 13 [(gogoproto.nullable) = false];
optional util.optional.Uint key_bytes = 14 [(gogoproto.nullable) = false];
optional util.optional.Uint value_bytes = 15 [(gogoproto.nullable) = false];
optional util.optional.Uint point_count = 16 [(gogoproto.nullable) = false];
optional util.optional.Uint points_covered_by_range_tombstones = 17 [(gogoproto.nullable) = false];
optional util.optional.Uint range_key_count = 18 [(gogoproto.nullable) = false];
optional util.optional.Uint range_key_contained_points = 19 [(gogoproto.nullable) = false];
optional util.optional.Uint range_key_skipped_points = 20 [(gogoproto.nullable) = false];
// Cumulated CPU time spent fulfilling KV requests on a SQL goroutine. This
// would not include CPU time spent on a remote node, but would include time
// spent on a Scan's goroutine. This is only used to calculate SQL CPU time
// for processors in the row-engine that perform KV requests.
// TODO(drewk): measure wall-clock KV time directly as well, and display it
// alongside total execution time for operators that perform KV work.
optional util.optional.Duration kv_cpu_time = 11 [(gogoproto.nullable) = false,
(gogoproto.customname) = "KVCPUTime"];
optional util.optional.Uint num_gets = 21 [(gogoproto.nullable) = false];
optional util.optional.Uint num_scans = 22 [(gogoproto.nullable) = false];
optional util.optional.Uint num_reverse_scans = 23 [(gogoproto.nullable) = false];
// UsedStreamer indicates whether the Streamer API was used to perform KV
// operations.
optional bool used_streamer = 25 [(gogoproto.nullable) = false];
}
// ExecStats contains statistics about the execution of a component.
message ExecStats {
// Time spent executing the component.
optional util.optional.Duration exec_time = 1 [(gogoproto.nullable) = false];
// Maximum memory allocated by the component.
optional util.optional.Uint max_allocated_mem = 2 [(gogoproto.nullable) = false];
// Maximum scratch disk allocated by the component.
optional util.optional.Uint max_allocated_disk = 3 [(gogoproto.nullable) = false];
// Amount of RUs consumed while executing the component.
optional util.optional.Uint consumed_ru = 4 [(gogoproto.nullable) = false,
(gogoproto.customname) = "ConsumedRU"];
// CPU time spent executing the component.
optional util.optional.Duration cpu_time = 5 [(gogoproto.nullable) = false,
(gogoproto.customname) = "CPUTime"];
}
// OutputStats contains statistics about the output (results) of a component.
message OutputStats {
// Number of batches produced by the component.
optional util.optional.Uint num_batches = 1 [(gogoproto.nullable) = false];
// Number of tuples produced by the component.
optional util.optional.Uint num_tuples = 2 [(gogoproto.nullable) = false];
}
// FlowStats contains flow level statistics.
message FlowStats {
optional util.optional.Uint max_mem_usage = 1 [(gogoproto.nullable) = false];
optional util.optional.Uint max_disk_usage = 2 [(gogoproto.nullable) = false];
// Amount of RUs consumed due to CPU usage while executing the flow. Currently
// only used for remote flows.
optional util.optional.Uint consumed_ru = 3 [(gogoproto.nullable) = false,
(gogoproto.customname) = "ConsumedRU"];
}