Skip to content

Commit 4beb167

Browse files
committed
sql: add index_bytes_written metric
Add new `sql.statements.index_bytes_written.count` metric that counts the number of primary and secondary index bytes modified by SQL statements. Epic: AOP-30 Release note (sql change): Added sql.statements.index_bytes_written.count metric that counts the number of primary and secondary index bytes modified by SQL statements.
1 parent cb78919 commit 4beb167

21 files changed

+172
-48
lines changed

docs/generated/metrics/metrics.yaml

Lines changed: 16 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -8763,6 +8763,22 @@ layers:
87638763
unit: BYTES
87648764
aggregation: AVG
87658765
derivative: NON_NEGATIVE_DERIVATIVE
8766+
- name: sql.statements.index_bytes_written.count
8767+
exported_name: sql_statements_index_bytes_written_count
8768+
description: Number of primary and secondary index bytes modified by SQL statements
8769+
y_axis_label: SQL Statements
8770+
type: COUNTER
8771+
unit: BYTES
8772+
aggregation: AVG
8773+
derivative: NON_NEGATIVE_DERIVATIVE
8774+
- name: sql.statements.index_bytes_written.count.internal
8775+
exported_name: sql_statements_index_bytes_written_count_internal
8776+
description: Number of primary and secondary index bytes modified by SQL statements (internal queries)
8777+
y_axis_label: SQL Internal Statements
8778+
type: COUNTER
8779+
unit: BYTES
8780+
aggregation: AVG
8781+
derivative: NON_NEGATIVE_DERIVATIVE
87668782
- name: sql.statements.index_rows_written.count
87678783
exported_name: sql_statements_index_rows_written_count
87688784
description: Number of primary and secondary index rows modified by SQL statements

pkg/roachprod/agents/opentelemetry/cockroachdb_metrics.go

Lines changed: 6 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -2169,6 +2169,12 @@ var cockroachdbMetrics = map[string]string{
21692169
"sql_statements_active_internal": "sql.statements.active.internal",
21702170
"sql_statements_auto_retry_count": "sql.statements.auto_retry.count",
21712171
"sql_statements_auto_retry_count_internal": "sql.statements.auto_retry.count.internal",
2172+
"sql_statements_bytes_read_count": "sql.statements.bytes_read.count",
2173+
"sql_statements_bytes_read_count_internal": "sql.statements.bytes_read.count.internal",
2174+
"sql_statements_index_bytes_written_count": "sql.statements.index_bytes_written.count",
2175+
"sql_statements_index_bytes_written_count_internal": "sql.statements.index_bytes_written.count.internal",
2176+
"sql_statements_index_rows_written_count": "sql.statements.index_rows_written.count",
2177+
"sql_statements_index_rows_written_count_internal": "sql.statements.index_rows_written.count.internal",
21722178
"sql_statements_rows_read_count": "sql.statements.rows_read.count",
21732179
"sql_statements_rows_read_count_internal": "sql.statements.rows_read.count.internal",
21742180
"sql_stats_activity_update_latency": "sql.stats.activity.update.latency",

pkg/sql/conn_executor.go

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -628,6 +628,7 @@ func makeMetrics(internal bool, sv *settings.Values) Metrics {
628628
StatementRowsRead: metric.NewCounter(getMetricMeta(MetaStatementRowsRead, internal)),
629629
StatementBytesRead: metric.NewCounter(getMetricMeta(MetaStatementBytesRead, internal)),
630630
StatementIndexRowsWritten: metric.NewCounter(getMetricMeta(MetaStatementIndexRowsWritten, internal)),
631+
StatementIndexBytesWritten: metric.NewCounter(getMetricMeta(MetaStatementIndexBytesWritten, internal)),
631632
},
632633
StartedStatementCounters: makeStartedStatementCounters(internal),
633634
ExecutedStatementCounters: makeExecutedStatementCounters(internal),

pkg/sql/conn_executor_exec.go

Lines changed: 6 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -3275,10 +3275,10 @@ func (ex *connExecutor) makeExecPlan(
32753275

32763276
// topLevelQueryStats returns some basic statistics about the run of the query.
32773277
type topLevelQueryStats struct {
3278-
// bytesRead is the number of bytes read from primary and secondary indexes.
3279-
bytesRead int64
32803278
// rowsRead is the number of rows read from primary and secondary indexes.
32813279
rowsRead int64
3280+
// bytesRead is the number of bytes read from primary and secondary indexes.
3281+
bytesRead int64
32823282
// rowsWritten is the number of rows written to the primary index. It does not
32833283
// include rows written to secondary indexes.
32843284
// NB: There is an asymmetry between rowsRead and rowsWritten - rowsRead
@@ -3289,6 +3289,9 @@ type topLevelQueryStats struct {
32893289
// indexRowsWritten is the number of rows written to primary and secondary
32903290
// indexes. It is always >= rowsWritten.
32913291
indexRowsWritten int64
3292+
// indexBytesWritten is the number of bytes written to primary and secondary
3293+
// indexes.
3294+
indexBytesWritten int64
32923295
// networkEgressEstimate is an estimate for the number of bytes sent to the
32933296
// client. It is used for estimating the number of RUs consumed by a query.
32943297
networkEgressEstimate int64
@@ -3304,6 +3307,7 @@ func (s *topLevelQueryStats) add(other *topLevelQueryStats) {
33043307
s.bytesRead += other.bytesRead
33053308
s.rowsRead += other.rowsRead
33063309
s.rowsWritten += other.rowsWritten
3310+
s.indexBytesWritten += other.indexBytesWritten
33073311
s.indexRowsWritten += other.indexRowsWritten
33083312
s.networkEgressEstimate += other.networkEgressEstimate
33093313
s.clientTime += other.clientTime

pkg/sql/delete.go

Lines changed: 5 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -255,6 +255,11 @@ func (d *deleteNode) indexRowsWritten() int64 {
255255
return d.run.td.indexRowsWritten
256256
}
257257

258+
func (d *deleteNode) indexBytesWritten() int64 {
259+
// No bytes counted as written for a deletion.
260+
return 0
261+
}
262+
258263
func (d *deleteNode) returnsRowsAffected() bool {
259264
return !d.run.rowsNeeded
260265
}

pkg/sql/delete_range.go

Lines changed: 5 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -65,6 +65,11 @@ func (d *deleteRangeNode) indexRowsWritten() int64 {
6565
return d.rowsAffected()
6666
}
6767

68+
func (d *deleteRangeNode) indexBytesWritten() int64 {
69+
// No bytes counted as written for a deletion.
70+
return 0
71+
}
72+
6873
func (d *deleteRangeNode) returnsRowsAffected() bool {
6974
// DeleteRange always returns the number of rows deleted.
7075
return true

pkg/sql/delete_swap.go

Lines changed: 5 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -134,6 +134,11 @@ func (d *deleteSwapNode) indexRowsWritten() int64 {
134134
return d.run.td.indexRowsWritten
135135
}
136136

137+
func (d *deleteSwapNode) indexBytesWritten() int64 {
138+
// No bytes counted as written for a deletion.
139+
return 0
140+
}
141+
137142
func (d *deleteSwapNode) returnsRowsAffected() bool {
138143
return !d.run.rowsNeeded
139144
}

pkg/sql/distsql_running.go

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -1624,6 +1624,7 @@ func forwardInnerQueryStats(f metadataForwarder, stats topLevelQueryStats) {
16241624
meta.Metrics.RowsRead = stats.rowsRead
16251625
meta.Metrics.RowsWritten = stats.rowsWritten
16261626
meta.Metrics.IndexRowsWritten = stats.indexRowsWritten
1627+
meta.Metrics.IndexBytesWritten = stats.indexBytesWritten
16271628
// stats.networkEgressEstimate and stats.clientTime are ignored since they
16281629
// only matter at the "true" top-level query (and actually should be zero
16291630
// here anyway).
@@ -1673,6 +1674,7 @@ func (r *DistSQLReceiver) pushMeta(meta *execinfrapb.ProducerMetadata) execinfra
16731674
r.stats.rowsRead += meta.Metrics.RowsRead
16741675
r.stats.rowsWritten += meta.Metrics.RowsWritten
16751676
r.stats.indexRowsWritten += meta.Metrics.IndexRowsWritten
1677+
r.stats.indexBytesWritten += meta.Metrics.IndexBytesWritten
16761678

16771679
if sm, ok := r.scanStageEstimateMap[meta.Metrics.StageID]; ok {
16781680
sm.rowsRead += uint64(meta.Metrics.RowsRead)

pkg/sql/distsql_running_test.go

Lines changed: 49 additions & 36 deletions
Original file line numberDiff line numberDiff line change
@@ -1226,7 +1226,7 @@ func TestTopLevelQueryStats(t *testing.T) {
12261226
var testQuery atomic.Value
12271227
// The callback will send number of rows read and rows written (for each
12281228
// ProducerMetadata.Metrics object) on these channels, respectively.
1229-
rowsReadCh, rowsWrittenCh := make(chan int64), make(chan int64)
1229+
rowsReadCh, rowsWrittenCh, indexRowsWrittenCh := make(chan int64), make(chan int64), make(chan int64)
12301230
srv, sqlDB, _ := serverutils.StartServer(t, base.TestServerArgs{
12311231
Knobs: base.TestingKnobs{
12321232
SQLExecutor: &ExecutorTestingKnobs{
@@ -1238,6 +1238,7 @@ func TestTopLevelQueryStats(t *testing.T) {
12381238
if meta != nil && meta.Metrics != nil {
12391239
rowsReadCh <- meta.Metrics.RowsRead
12401240
rowsWrittenCh <- meta.Metrics.RowsWritten
1241+
indexRowsWrittenCh <- meta.Metrics.IndexRowsWritten
12411242
}
12421243
return row, batch, meta
12431244
}
@@ -1254,23 +1255,25 @@ CREATE TABLE t (k INT PRIMARY KEY, i INT, v INT, INDEX(i));
12541255
INSERT INTO t SELECT i, 1, 1 FROM generate_series(1, 10) AS g(i);
12551256
CREATE FUNCTION no_reads() RETURNS INT AS 'SELECT 1' LANGUAGE SQL;
12561257
CREATE FUNCTION reads() RETURNS INT AS 'SELECT count(*) FROM t' LANGUAGE SQL;
1257-
CREATE FUNCTION write(x INT) RETURNS INT AS 'INSERT INTO t VALUES (x); SELECT x' LANGUAGE SQL;
1258+
CREATE FUNCTION write(x INT) RETURNS INT AS 'INSERT INTO t VALUES (x, x); SELECT x' LANGUAGE SQL;
12581259
`); err != nil {
12591260
t.Fatal(err)
12601261
}
12611262

12621263
for _, tc := range []struct {
1263-
name string
1264-
query string
1265-
setup, cleanup string // optional
1266-
expRowsRead int64
1267-
expRowsWritten int64
1264+
name string
1265+
query string
1266+
setup, cleanup string // optional
1267+
expRowsRead int64
1268+
expRowsWritten int64
1269+
expIndexRowsWritten int64
12681270
}{
12691271
{
1270-
name: "simple read",
1271-
query: "SELECT k FROM t",
1272-
expRowsRead: 10,
1273-
expRowsWritten: 0,
1272+
name: "simple read",
1273+
query: "SELECT k FROM t",
1274+
expRowsRead: 10,
1275+
expRowsWritten: 0,
1276+
expIndexRowsWritten: 0,
12741277
},
12751278
{
12761279
name: "routine and index join (used to be powered by streamer)",
@@ -1279,47 +1282,54 @@ CREATE FUNCTION write(x INT) RETURNS INT AS 'INSERT INTO t VALUES (x); SELECT x'
12791282
cleanup: "RESET distsql",
12801283
// 10 rows for secondary index, 10 for index join into primary, and
12811284
// then for each row do ten-row-scan in the routine.
1282-
expRowsRead: 120,
1283-
expRowsWritten: 0,
1285+
expRowsRead: 120,
1286+
expRowsWritten: 0,
1287+
expIndexRowsWritten: 0,
12841288
},
12851289
{
1286-
name: "simple write",
1287-
query: "INSERT INTO t SELECT generate_series(11, 42)",
1288-
expRowsRead: 0,
1289-
expRowsWritten: 32,
1290+
name: "simple write",
1291+
query: "INSERT INTO t SELECT generate_series(11, 42)",
1292+
expRowsRead: 0,
1293+
expRowsWritten: 32,
1294+
expIndexRowsWritten: 64,
12901295
},
12911296
{
12921297
name: "read with apply join",
12931298
query: `SELECT (
12941299
WITH foo AS MATERIALIZED (SELECT k FROM t AS x WHERE x.k = y.k)
12951300
SELECT * FROM foo
12961301
) FROM t AS y`,
1297-
expRowsRead: 84, // scanning the table twice
1298-
expRowsWritten: 0,
1302+
expRowsRead: 84, // scanning the table twice
1303+
expRowsWritten: 0,
1304+
expIndexRowsWritten: 0,
12991305
},
13001306
{
1301-
name: "routine, no reads",
1302-
query: "SELECT no_reads()",
1303-
expRowsRead: 0,
1304-
expRowsWritten: 0,
1307+
name: "routine, no reads",
1308+
query: "SELECT no_reads()",
1309+
expRowsRead: 0,
1310+
expRowsWritten: 0,
1311+
expIndexRowsWritten: 0,
13051312
},
13061313
{
1307-
name: "routine, reads",
1308-
query: "SELECT reads()",
1309-
expRowsRead: 42,
1310-
expRowsWritten: 0,
1314+
name: "routine, reads",
1315+
query: "SELECT reads()",
1316+
expRowsRead: 42,
1317+
expRowsWritten: 0,
1318+
expIndexRowsWritten: 0,
13111319
},
13121320
{
1313-
name: "routine, write",
1314-
query: "SELECT write(43)",
1315-
expRowsRead: 0,
1316-
expRowsWritten: 1,
1321+
name: "routine, write",
1322+
query: "SELECT write(43)",
1323+
expRowsRead: 0,
1324+
expRowsWritten: 1,
1325+
expIndexRowsWritten: 2,
13171326
},
13181327
{
1319-
name: "routine, multiple reads and writes",
1320-
query: "SELECT reads(), write(44), reads(), write(45), write(46), reads()",
1321-
expRowsRead: 133, // first read is 43 rows, second is 44, third is 46
1322-
expRowsWritten: 3,
1328+
name: "routine, multiple reads and writes",
1329+
query: "SELECT reads(), write(44), reads(), write(45), write(46), reads()",
1330+
expRowsRead: 133, // first read is 43 rows, second is 44, third is 46
1331+
expRowsWritten: 3,
1332+
expIndexRowsWritten: 6,
13231333
},
13241334
} {
13251335
t.Run(tc.name, func(t *testing.T) {
@@ -1344,21 +1354,24 @@ CREATE FUNCTION write(x INT) RETURNS INT AS 'INSERT INTO t VALUES (x); SELECT x'
13441354
}()
13451355
// In the main goroutine, loop until the query is completed while
13461356
// accumulating the top-level query stats.
1347-
var rowsRead, rowsWritten int64
1357+
var rowsRead, rowsWritten, indexRowsWritten int64
13481358
LOOP:
13491359
for {
13501360
select {
13511361
case read := <-rowsReadCh:
13521362
rowsRead += read
13531363
case written := <-rowsWrittenCh:
13541364
rowsWritten += written
1365+
case written := <-indexRowsWrittenCh:
1366+
indexRowsWritten += written
13551367
case err := <-errCh:
13561368
require.NoError(t, err)
13571369
break LOOP
13581370
}
13591371
}
13601372
require.Equal(t, tc.expRowsRead, rowsRead)
13611373
require.Equal(t, tc.expRowsWritten, rowsWritten)
1374+
require.Equal(t, tc.expIndexRowsWritten, indexRowsWritten)
13621375
})
13631376
}
13641377
}

pkg/sql/exec_util.go

Lines changed: 6 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -1532,6 +1532,12 @@ var (
15321532
Measurement: "SQL Statements",
15331533
Unit: metric.Unit_COUNT,
15341534
}
1535+
MetaStatementIndexBytesWritten = metric.Metadata{
1536+
Name: "sql.statements.index_bytes_written.count",
1537+
Help: "Number of primary and secondary index bytes modified by SQL statements",
1538+
Measurement: "SQL Statements",
1539+
Unit: metric.Unit_BYTES,
1540+
}
15351541
)
15361542

15371543
func getMetricMeta(meta metric.Metadata, internal bool) metric.Metadata {

0 commit comments

Comments
 (0)