Skip to content

Commit a8cc783

Browse files
committed
storage,kv: propagate ReadCategory to ComputeStats
This commit updates ComputeStats and its various variants to accept a ReadCategory. This ensures these scans' stats are appropriately categorized. Informs #158017. Epic: none Release note: none
1 parent b927461 commit a8cc783

34 files changed

+154
-66
lines changed

docs/generated/metrics/metrics.yaml

Lines changed: 24 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -16610,6 +16610,30 @@ layers:
1661016610
unit: NANOSECONDS
1661116611
aggregation: AVG
1661216612
derivative: NON_NEGATIVE_DERIVATIVE
16613+
- name: storage.iterator.category-consistency-checker.block-load.bytes
16614+
exported_name: storage_iterator_category_consistency_checker_block_load_bytes
16615+
description: Bytes loaded by storage sstable iterators (possibly cached).
16616+
y_axis_label: Bytes
16617+
type: COUNTER
16618+
unit: BYTES
16619+
aggregation: AVG
16620+
derivative: NON_NEGATIVE_DERIVATIVE
16621+
- name: storage.iterator.category-consistency-checker.block-load.cached-bytes
16622+
exported_name: storage_iterator_category_consistency_checker_block_load_cached_bytes
16623+
description: Bytes loaded by storage sstable iterators from the block cache
16624+
y_axis_label: Bytes
16625+
type: COUNTER
16626+
unit: BYTES
16627+
aggregation: AVG
16628+
derivative: NON_NEGATIVE_DERIVATIVE
16629+
- name: storage.iterator.category-consistency-checker.block-load.latency-sum
16630+
exported_name: storage_iterator_category_consistency_checker_block_load_latency_sum
16631+
description: Cumulative latency for loading bytes not in the block cache, by storage sstable iterators
16632+
y_axis_label: Latency
16633+
type: COUNTER
16634+
unit: NANOSECONDS
16635+
aggregation: AVG
16636+
derivative: NON_NEGATIVE_DERIVATIVE
1661316637
- name: storage.iterator.category-crdb-unknown.block-load.bytes
1661416638
exported_name: storage_iterator_category_crdb_unknown_block_load_bytes
1661516639
description: Bytes loaded by storage sstable iterators (possibly cached).

pkg/cli/debug_check_store.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -125,7 +125,7 @@ func worker(ctx context.Context, in checkInput) checkResult {
125125
res.err = err
126126
return res
127127
}
128-
ms, err := rditer.ComputeStatsForRange(ctx, desc, eng, claimedMS.LastUpdateNanos)
128+
ms, err := rditer.ComputeStatsForRange(ctx, desc, eng, fs.UnknownReadCategory, claimedMS.LastUpdateNanos)
129129
if err != nil {
130130
res.err = err
131131
return res

pkg/kv/kvserver/batcheval/cmd_clear_range.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -189,7 +189,7 @@ func computeStatsDelta(
189189
// If we can't use the fast stats path, or race test is enabled, compute stats
190190
// across the key span to be cleared.
191191
if !entireRange || util.RaceEnabled {
192-
computed, err := storage.ComputeStats(ctx, reader, from, to, delta.LastUpdateNanos)
192+
computed, err := storage.ComputeStats(ctx, reader, fs.BatchEvalReadCategory, from, to, delta.LastUpdateNanos)
193193
if err != nil {
194194
return enginepb.MVCCStats{}, err
195195
}

pkg/kv/kvserver/batcheval/cmd_delete_range_test.go

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -20,6 +20,7 @@ import (
2020
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
2121
"github.com/cockroachdb/cockroach/pkg/storage"
2222
"github.com/cockroachdb/cockroach/pkg/storage/enginepb"
23+
"github.com/cockroachdb/cockroach/pkg/storage/fs"
2324
"github.com/cockroachdb/cockroach/pkg/storage/mvccencoding"
2425
"github.com/cockroachdb/cockroach/pkg/util/hlc"
2526
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
@@ -411,7 +412,7 @@ func computeStats(
411412
if len(to) == 0 {
412413
to = keys.MaxKey
413414
}
414-
ms, err := storage.ComputeStats(context.Background(), reader, from, to, nowNanos)
415+
ms, err := storage.ComputeStats(t.Context(), reader, fs.UnknownReadCategory, from, to, nowNanos)
415416
require.NoError(t, err)
416417
return ms
417418
}

pkg/kv/kvserver/batcheval/cmd_end_transaction.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -1292,7 +1292,7 @@ func makeScanStatsFn(
12921292
computeStatsFn = rditer.ComputeStatsForRangeExcludingUser
12931293
}
12941294
return func() (enginepb.MVCCStats, error) {
1295-
sideMS, err := computeStatsFn(ctx, sideDesc, reader, ts.WallTime)
1295+
sideMS, err := computeStatsFn(ctx, sideDesc, reader, fs.BatchEvalReadCategory, ts.WallTime)
12961296
if err != nil {
12971297
return enginepb.MVCCStats{}, errors.Wrapf(err,
12981298
"unable to compute stats for %s range after split", sideName)

pkg/kv/kvserver/batcheval/cmd_recompute_stats.go

Lines changed: 3 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -18,6 +18,7 @@ import (
1818
"github.com/cockroachdb/cockroach/pkg/roachpb"
1919
"github.com/cockroachdb/cockroach/pkg/storage"
2020
"github.com/cockroachdb/cockroach/pkg/storage/enginepb"
21+
"github.com/cockroachdb/cockroach/pkg/storage/fs"
2122
"github.com/cockroachdb/cockroach/pkg/util/uuid"
2223
"github.com/cockroachdb/errors"
2324
)
@@ -75,7 +76,8 @@ func RecomputeStats(
7576

7677
args = nil // avoid accidental use below
7778

78-
actualMS, err := rditer.ComputeStatsForRange(ctx, desc, reader, cArgs.Header.Timestamp.WallTime)
79+
actualMS, err := rditer.ComputeStatsForRange(ctx, desc, reader,
80+
fs.BatchEvalReadCategory, cArgs.Header.Timestamp.WallTime)
7981
if err != nil {
8082
return result.Result{}, err
8183
}

pkg/kv/kvserver/batcheval/cmd_revert_range_test.go

Lines changed: 4 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -119,7 +119,8 @@ func TestCmdRevertRange(t *testing.T) {
119119
Stats: stats,
120120
}
121121
cArgs.EvalCtx = evalCtx.EvalContext()
122-
afterStats, err := storage.ComputeStats(ctx, eng, keys.LocalMax, keys.MaxKey, 0)
122+
afterStats, err := storage.ComputeStats(ctx, eng, fs.BatchEvalReadCategory,
123+
keys.LocalMax, keys.MaxKey, 0)
123124
require.NoError(t, err)
124125
for _, tc := range []struct {
125126
name string
@@ -174,7 +175,8 @@ func TestCmdRevertRange(t *testing.T) {
174175
}
175176
evalStats := afterStats
176177
evalStats.Add(*cArgs.Stats)
177-
realStats, err := storage.ComputeStats(ctx, batch, keys.LocalMax, keys.MaxKey, evalStats.LastUpdateNanos)
178+
realStats, err := storage.ComputeStats(ctx, batch, fs.BatchEvalReadCategory,
179+
keys.LocalMax, keys.MaxKey, evalStats.LastUpdateNanos)
178180
require.NoError(t, err)
179181
require.Equal(t, realStats, evalStats)
180182
})

pkg/kv/kvserver/batcheval/cmd_subsume.go

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -231,7 +231,8 @@ func Subsume(
231231
// rather than introducing additional synchronization complexity.
232232
ridPrefix := keys.MakeRangeIDReplicatedPrefix(desc.RangeID)
233233
reply.RangeIDLocalMVCCStats, err = storage.ComputeStats(
234-
ctx, readWriter, ridPrefix, ridPrefix.PrefixEnd(), 0 /* nowNanos */)
234+
ctx, readWriter, fs.BatchEvalReadCategory,
235+
ridPrefix, ridPrefix.PrefixEnd(), 0 /* nowNanos */)
235236
if err != nil {
236237
return result.Result{}, err
237238
}

pkg/kv/kvserver/batcheval/cmd_truncate_log.go

Lines changed: 3 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -17,6 +17,7 @@ import (
1717
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/spanset"
1818
"github.com/cockroachdb/cockroach/pkg/roachpb"
1919
"github.com/cockroachdb/cockroach/pkg/storage"
20+
"github.com/cockroachdb/cockroach/pkg/storage/fs"
2021
"github.com/cockroachdb/cockroach/pkg/util/log"
2122
"github.com/cockroachdb/errors"
2223
)
@@ -128,7 +129,8 @@ func TruncateLog(
128129
// are not tracked in the raft log delta. The delta will be adjusted below
129130
// raft.
130131
// We can pass zero as nowNanos because we're only interested in SysBytes.
131-
ms, err := storage.ComputeStats(ctx, logReader, start, end, 0 /* nowNanos */)
132+
ms, err := storage.ComputeStats(ctx, logReader, fs.ReplicationReadCategory,
133+
start, end, 0 /* nowNanos */)
132134
if err != nil {
133135
return result.Result{}, errors.Wrap(err, "while computing stats of Raft log freed by truncation")
134136
}

pkg/kv/kvserver/batcheval/cmd_truncate_log_test.go

Lines changed: 3 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -15,6 +15,7 @@ import (
1515
"github.com/cockroachdb/cockroach/pkg/roachpb"
1616
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
1717
"github.com/cockroachdb/cockroach/pkg/storage"
18+
"github.com/cockroachdb/cockroach/pkg/storage/fs"
1819
"github.com/cockroachdb/cockroach/pkg/util/hlc"
1920
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
2021
"github.com/cockroachdb/cockroach/pkg/util/log"
@@ -123,13 +124,13 @@ func TestTruncateLog(t *testing.T) {
123124
// Verify the stats match what's actually in the log engine.
124125
start := keys.RaftLogKey(rangeID, compactedIndex+1)
125126
end := keys.RaftLogKey(rangeID, compactedIndex+8)
126-
expectedStats, err := storage.ComputeStats(ctx, logEng, start, end, 0)
127+
expectedStats, err := storage.ComputeStats(ctx, logEng, fs.ReplicationReadCategory, start, end, 0)
127128
require.NoError(t, err)
128129
assert.Equal(t, -expectedStats.SysBytes, res.Replicated.RaftLogDelta,
129130
"RaftLogDelta should match stats computed from log engine")
130131

131132
// The state machine engine's stats should be zero.
132-
zeroStats, err := storage.ComputeStats(ctx, stateEng, start, end, 0)
133+
zeroStats, err := storage.ComputeStats(ctx, stateEng, fs.UnknownReadCategory, start, end, 0)
133134
require.NoError(t, err)
134135
require.Zero(t, zeroStats.SysBytes)
135136
}

0 commit comments

Comments
 (0)