Skip to content

Commit 222df72

Browse files
craig[bot]dhartunianjbowens
committed
157940: *: standardize pprof labeling code r=pav-kv,jasonlmfong a=dhartunian This commit replaces usages of `pprof.Do` with our internal implementation that wraps the `SetProfilerLabels` utility. This enables us to have better standardization around profiler label application and easier tracking of all usages. A linter is added to enforce this usage. Epic: CRDB-55080 Part of: CRDB-55923 Release note: None 158056: storage,kv: propagate iterator ReadCategory more broadly r=tbg a=jbowens These commits expand the propagation of the ReadCategory enum across more of the iterator-constructing codepaths, hoping to fix gaps in categorization observed in #158017. 158080: ui: extend storage dashboard r=sumeerbhola a=jbowens Add a new graph for the aggregate duration of compactions. Add a new graph for the compaction scores of each level. Epic: none Release note: none Co-authored-by: David Hartunian <davidh@cockroachlabs.com> Co-authored-by: Jackson Owens <jackson@cockroachlabs.com>
4 parents 149a4b2 + b2de7cd + a8cc783 + 9f20d40 commit 222df72

Some content is hidden

Large Commits have some content hidden by default. Use the searchbox below for content that may be hidden.

63 files changed

+345
-128
lines changed

docs/generated/metrics/metrics.yaml

Lines changed: 24 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -16234,6 +16234,30 @@ layers:
1623416234
unit: NANOSECONDS
1623516235
aggregation: AVG
1623616236
derivative: NON_NEGATIVE_DERIVATIVE
16237+
- name: storage.iterator.category-consistency-checker.block-load.bytes
16238+
exported_name: storage_iterator_category_consistency_checker_block_load_bytes
16239+
description: Bytes loaded by storage sstable iterators (possibly cached).
16240+
y_axis_label: Bytes
16241+
type: COUNTER
16242+
unit: BYTES
16243+
aggregation: AVG
16244+
derivative: NON_NEGATIVE_DERIVATIVE
16245+
- name: storage.iterator.category-consistency-checker.block-load.cached-bytes
16246+
exported_name: storage_iterator_category_consistency_checker_block_load_cached_bytes
16247+
description: Bytes loaded by storage sstable iterators from the block cache
16248+
y_axis_label: Bytes
16249+
type: COUNTER
16250+
unit: BYTES
16251+
aggregation: AVG
16252+
derivative: NON_NEGATIVE_DERIVATIVE
16253+
- name: storage.iterator.category-consistency-checker.block-load.latency-sum
16254+
exported_name: storage_iterator_category_consistency_checker_block_load_latency_sum
16255+
description: Cumulative latency for loading bytes not in the block cache, by storage sstable iterators
16256+
y_axis_label: Latency
16257+
type: COUNTER
16258+
unit: NANOSECONDS
16259+
aggregation: AVG
16260+
derivative: NON_NEGATIVE_DERIVATIVE
1623716261
- name: storage.iterator.category-crdb-unknown.block-load.bytes
1623816262
exported_name: storage_iterator_category_crdb_unknown_block_load_bytes
1623916263
description: Bytes loaded by storage sstable iterators (possibly cached).

pkg/cli/debug.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -497,7 +497,7 @@ func runDebugRangeData(cmd *cobra.Command, args []string) error {
497497
defer snapshot.Close()
498498

499499
var results int
500-
return rditer.IterateReplicaKeySpans(cmd.Context(), &desc, snapshot, rditer.SelectOpts{
500+
return rditer.IterateReplicaKeySpans(cmd.Context(), &desc, snapshot, fs.UnknownReadCategory, rditer.SelectOpts{
501501
Ranged: rditer.SelectRangedOptions{
502502
SystemKeys: true,
503503
LockTable: true,

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/crosscluster/logical/BUILD.bazel

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -101,6 +101,7 @@ go_library(
101101
"//pkg/util/log/logcrash",
102102
"//pkg/util/metamorphic",
103103
"//pkg/util/metric",
104+
"//pkg/util/pprofutil",
104105
"//pkg/util/protoutil",
105106
"//pkg/util/randutil",
106107
"//pkg/util/retry",

pkg/crosscluster/logical/logical_replication_writer_processor.go

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -10,7 +10,6 @@ import (
1010
"fmt"
1111
"hash/fnv"
1212
"regexp"
13-
"runtime/pprof"
1413
"slices"
1514
"strings"
1615
"time"
@@ -45,6 +44,7 @@ import (
4544
"github.com/cockroachdb/cockroach/pkg/util/ctxgroup"
4645
"github.com/cockroachdb/cockroach/pkg/util/log"
4746
"github.com/cockroachdb/cockroach/pkg/util/log/logcrash"
47+
"github.com/cockroachdb/cockroach/pkg/util/pprofutil"
4848
"github.com/cockroachdb/cockroach/pkg/util/protoutil"
4949
"github.com/cockroachdb/cockroach/pkg/util/span"
5050
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
@@ -349,12 +349,12 @@ func (lrw *logicalReplicationWriterProcessor) Start(ctx context.Context) {
349349
})
350350
lrw.workerGroup.GoCtx(func(ctx context.Context) error {
351351
defer close(lrw.checkpointCh)
352-
pprof.Do(ctx, pprof.Labels("proc", fmt.Sprintf("%d", lrw.ProcessorID)), func(ctx context.Context) {
352+
pprofutil.Do(ctx, func(ctx context.Context) {
353353
if err := lrw.consumeEvents(ctx); err != nil {
354354
log.Dev.Infof(lrw.Ctx(), "consumer completed. Error: %s", err)
355355
lrw.sendError(errors.Wrap(err, "consume events"))
356356
}
357-
})
357+
}, "proc", fmt.Sprintf("%d", lrw.ProcessorID))
358358
return nil
359359
})
360360
}

pkg/crosscluster/logical/offline_initial_scan_processor.go

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -8,7 +8,6 @@ package logical
88
import (
99
"context"
1010
"fmt"
11-
"runtime/pprof"
1211
"time"
1312

1413
"github.com/cockroachdb/cockroach/pkg/backup"
@@ -31,6 +30,7 @@ import (
3130
"github.com/cockroachdb/cockroach/pkg/util/hlc"
3231
"github.com/cockroachdb/cockroach/pkg/util/log"
3332
"github.com/cockroachdb/cockroach/pkg/util/log/logcrash"
33+
"github.com/cockroachdb/cockroach/pkg/util/pprofutil"
3434
"github.com/cockroachdb/cockroach/pkg/util/protoutil"
3535
"github.com/cockroachdb/cockroach/pkg/util/span"
3636
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
@@ -225,7 +225,7 @@ func (o *offlineInitialScanProcessor) Start(ctx context.Context) {
225225
o.workerGroup.GoCtx(func(ctx context.Context) error {
226226
defer close(o.checkpointCh)
227227
defer close(o.rangeStatsCh)
228-
pprof.Do(ctx, pprof.Labels("proc", fmt.Sprintf("%d", o.ProcessorID)), func(ctx context.Context) {
228+
pprofutil.Do(ctx, func(ctx context.Context) {
229229
for event := range o.subscription.Events() {
230230
if err := o.handleEvent(ctx, event); err != nil {
231231
log.Dev.Infof(o.Ctx(), "consumer completed. Error: %s", err)
@@ -235,7 +235,7 @@ func (o *offlineInitialScanProcessor) Start(ctx context.Context) {
235235
if err := o.subscription.Err(); err != nil {
236236
o.sendError(errors.Wrap(err, "subscription"))
237237
}
238-
})
238+
}, "proc", fmt.Sprintf("%d", o.ProcessorID))
239239
return nil
240240
})
241241
}

pkg/kv/kvclient/rangefeed/BUILD.bazel

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -26,6 +26,7 @@ go_library(
2626
"//pkg/util/limit",
2727
"//pkg/util/log",
2828
"//pkg/util/mon",
29+
"//pkg/util/pprofutil",
2930
"//pkg/util/retry",
3031
"//pkg/util/span",
3132
"//pkg/util/stop",

pkg/kv/kvclient/rangefeed/rangefeed.go

Lines changed: 4 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -8,7 +8,6 @@ package rangefeed
88
import (
99
"context"
1010
"fmt"
11-
"runtime/pprof"
1211
"strings"
1312
"sync"
1413
"sync/atomic"
@@ -23,6 +22,7 @@ import (
2322
"github.com/cockroachdb/cockroach/pkg/util/ctxgroup"
2423
"github.com/cockroachdb/cockroach/pkg/util/hlc"
2524
"github.com/cockroachdb/cockroach/pkg/util/log"
25+
"github.com/cockroachdb/cockroach/pkg/util/pprofutil"
2626
"github.com/cockroachdb/cockroach/pkg/util/retry"
2727
"github.com/cockroachdb/cockroach/pkg/util/span"
2828
"github.com/cockroachdb/cockroach/pkg/util/stop"
@@ -241,9 +241,9 @@ func (f *RangeFeed) start(
241241
// pprof.Do function does exactly what we do here, but it also results in
242242
// pprof.Do function showing up in the stack traces -- so, just set and reset
243243
// labels manually.
244-
defer pprof.SetGoroutineLabels(ctx)
245-
ctx = pprof.WithLabels(ctx, pprof.Labels(append(f.extraPProfLabels, "rangefeed", f.name)...))
246-
pprof.SetGoroutineLabels(ctx)
244+
ctx, reset := pprofutil.SetProfilerLabels(ctx, append(f.extraPProfLabels, "rangefeed", f.name)...)
245+
defer reset()
246+
247247
if f.invoker != nil {
248248
_ = f.invoker(func() error {
249249
f.run(ctx, frontier, resumeFromFrontier)

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
}

0 commit comments

Comments
 (0)