Skip to content

Commit

Permalink
server: add cpu profiler
Browse files Browse the repository at this point in the history
This PR adds a cpu profiler to the server package.
The following cluster settings have been added to configure
the cpu profiler:
- server.cpu_profile.cpu_usage_combined_threshold is the baseline
value for when cpu profiles should be taken
- server.cpu_profile.interval is when the high water mark resets to
the cpu_usage_combined_threshold value
- server.cpu_profile.duration is how long a cpu profile is taken
- server.cpu_profile.enabled is whether the on/off switch of the
cpu profiler

Fixes: #75799

Release note: None
  • Loading branch information
Santamaura committed Mar 2, 2023
1 parent a3db679 commit cf97b4f
Show file tree
Hide file tree
Showing 27 changed files with 415 additions and 237 deletions.
2 changes: 1 addition & 1 deletion build/bazelutil/nogo_config.json
Original file line number Diff line number Diff line change
Expand Up @@ -739,7 +739,7 @@
"pkg/security/securitytest/securitytest.go$": "excluded until all uses of io/ioutil are replaced",
"pkg/server/dumpstore/dumpstore.go$": "excluded until all uses of io/ioutil are replaced",
"pkg/server/dumpstore/dumpstore_test.go$": "excluded until all uses of io/ioutil are replaced",
"pkg/server/heapprofiler/profilestore_test.go$": "excluded until all uses of io/ioutil are replaced",
"pkg/server/profiler/profilestore_test.go$": "excluded until all uses of io/ioutil are replaced",
"pkg/util/log/file_api.go$": "excluded until all uses of io/ioutil are replaced",
"pkg/build/bazel/bazel\\.go$": "Runfile function deprecated"
},
Expand Down
8 changes: 4 additions & 4 deletions pkg/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -273,8 +273,8 @@ ALL_TESTS = [
"//pkg/server/diagnostics:diagnostics_test",
"//pkg/server/dumpstore:dumpstore_test",
"//pkg/server/goroutinedumper:goroutinedumper_test",
"//pkg/server/heapprofiler:heapprofiler_test",
"//pkg/server/pgurl:pgurl_test",
"//pkg/server/profiler:profiler_test",
"//pkg/server/serverpb:serverpb_test",
"//pkg/server/serverrules:serverrules_test",
"//pkg/server/settingswatcher:settingswatcher_test",
Expand Down Expand Up @@ -1419,10 +1419,10 @@ GO_TARGETS = [
"//pkg/server/dumpstore:dumpstore_test",
"//pkg/server/goroutinedumper:goroutinedumper",
"//pkg/server/goroutinedumper:goroutinedumper_test",
"//pkg/server/heapprofiler:heapprofiler",
"//pkg/server/heapprofiler:heapprofiler_test",
"//pkg/server/pgurl:pgurl",
"//pkg/server/pgurl:pgurl_test",
"//pkg/server/profiler:profiler",
"//pkg/server/profiler:profiler_test",
"//pkg/server/serverpb:serverpb",
"//pkg/server/serverpb:serverpb_test",
"//pkg/server/serverrules:serverrules",
Expand Down Expand Up @@ -2778,8 +2778,8 @@ GET_X_DATA_TARGETS = [
"//pkg/server/diagnostics/diagnosticspb:get_x_data",
"//pkg/server/dumpstore:get_x_data",
"//pkg/server/goroutinedumper:get_x_data",
"//pkg/server/heapprofiler:get_x_data",
"//pkg/server/pgurl:get_x_data",
"//pkg/server/profiler:get_x_data",
"//pkg/server/serverpb:get_x_data",
"//pkg/server/serverrules:get_x_data",
"//pkg/server/settingswatcher:get_x_data",
Expand Down
5 changes: 1 addition & 4 deletions pkg/cli/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -15,7 +15,6 @@ go_library(
"connect_join.go",
"context.go",
"convert_url.go",
"cpuprofile.go",
"debug.go",
"debug_check_store.go",
"debug_job_trace.go",
Expand Down Expand Up @@ -144,10 +143,8 @@ go_library(
"//pkg/security/securitytest",
"//pkg/security/username",
"//pkg/server",
"//pkg/server/debug",
"//pkg/server/dumpstore",
"//pkg/server/heapprofiler",
"//pkg/server/pgurl",
"//pkg/server/profiler",
"//pkg/server/serverpb",
"//pkg/server/status",
"//pkg/server/status/statuspb",
Expand Down
138 changes: 0 additions & 138 deletions pkg/cli/cpuprofile.go

This file was deleted.

2 changes: 1 addition & 1 deletion pkg/cli/interactive_tests/test_log_flags.tcl
Original file line number Diff line number Diff line change
Expand Up @@ -27,7 +27,7 @@ send "$argv start-single-node --insecure --store=path=logs/mystore2 --log-dir=\r
eexpect "node starting"
interrupt
eexpect ":/# "
send "ls logs/mystore2/logs 2>/dev/null | grep -vE 'heap_profiler|goroutine_dump|inflight_trace_dump' | wc -l\r"
send "ls logs/mystore2/logs 2>/dev/null | grep -vE 'heap_profiler|goroutine_dump|inflight_trace_dump|pprof_dump' | wc -l\r"
eexpect "0"
eexpect ":/# "
end_test
Expand Down
8 changes: 3 additions & 5 deletions pkg/cli/start.go
Original file line number Diff line number Diff line change
Expand Up @@ -1343,15 +1343,13 @@ func setupAndInitializeLoggingAndProfiling(
info := build.GetInfo()
log.Ops.Infof(ctx, "%s", log.SafeManaged(info.Short()))

initTraceDir(ctx, serverCfg.InflightTraceDirName)
initCPUProfile(ctx, serverCfg.CPUProfileDirName, serverCfg.Settings)
initBlockProfile()
initMutexProfile()

// Disable Stopper task tracking as performing that call site tracking is
// moderately expensive (certainly outweighing the infrequent benefit it
// provides).
stopper = stop.NewStopper()
initTraceDir(ctx, serverCfg.InflightTraceDirName)
initBlockProfile()
initMutexProfile()
log.Event(ctx, "initialized profiles")

return stopper, nil
Expand Down
4 changes: 2 additions & 2 deletions pkg/cli/start_jemalloc.go
Original file line number Diff line number Diff line change
Expand Up @@ -53,12 +53,12 @@ import (
"fmt"
"unsafe"

"github.com/cockroachdb/cockroach/pkg/server/heapprofiler"
"github.com/cockroachdb/cockroach/pkg/server/profiler"
)

func init() {
if C.is_profiling_enabled() {
heapprofiler.SetJemallocHeapDumpFn(writeJemallocProfile)
profiler.SetJemallocHeapDumpFn(writeJemallocProfile)
}
}

Expand Down
14 changes: 7 additions & 7 deletions pkg/cli/zip.go
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/cli/clisqlexec"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness/livenesspb"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/server/heapprofiler"
"github.com/cockroachdb/cockroach/pkg/server/profiler"
"github.com/cockroachdb/cockroach/pkg/server/serverpb"
"github.com/cockroachdb/cockroach/pkg/server/status/statuspb"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode"
Expand Down Expand Up @@ -339,12 +339,12 @@ done
// TODO(knz): Remove this in v21.1.
func maybeAddProfileSuffix(name string) string {
switch {
case strings.HasPrefix(name, heapprofiler.HeapFileNamePrefix+".") && !strings.HasSuffix(name, heapprofiler.HeapFileNameSuffix):
name += heapprofiler.HeapFileNameSuffix
case strings.HasPrefix(name, heapprofiler.StatsFileNamePrefix+".") && !strings.HasSuffix(name, heapprofiler.StatsFileNameSuffix):
name += heapprofiler.StatsFileNameSuffix
case strings.HasPrefix(name, heapprofiler.JemallocFileNamePrefix+".") && !strings.HasSuffix(name, heapprofiler.JemallocFileNameSuffix):
name += heapprofiler.JemallocFileNameSuffix
case strings.HasPrefix(name, profiler.HeapFileNamePrefix+".") && !strings.HasSuffix(name, profiler.HeapFileNameSuffix):
name += profiler.HeapFileNameSuffix
case strings.HasPrefix(name, profiler.StatsFileNamePrefix+".") && !strings.HasSuffix(name, profiler.StatsFileNameSuffix):
name += profiler.StatsFileNameSuffix
case strings.HasPrefix(name, profiler.JemallocFileNamePrefix+".") && !strings.HasSuffix(name, profiler.JemallocFileNameSuffix):
name += profiler.JemallocFileNameSuffix
}
return name
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/server/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -162,8 +162,8 @@ go_library(
"//pkg/server/diagnostics",
"//pkg/server/diagnostics/diagnosticspb",
"//pkg/server/goroutinedumper",
"//pkg/server/heapprofiler",
"//pkg/server/pgurl",
"//pkg/server/profiler",
"//pkg/server/serverpb",
"//pkg/server/serverrules",
"//pkg/server/settingswatcher",
Expand Down
29 changes: 20 additions & 9 deletions pkg/server/env_sampler.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,7 @@ import (

"github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/server/goroutinedumper"
"github.com/cockroachdb/cockroach/pkg/server/heapprofiler"
"github.com/cockroachdb/cockroach/pkg/server/profiler"
"github.com/cockroachdb/cockroach/pkg/server/status"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/sql"
Expand All @@ -35,6 +35,7 @@ type sampleEnvironmentCfg struct {
minSampleInterval time.Duration
goroutineDumpDirName string
heapProfileDirName string
cpuProfileDirName string
runtime *status.RuntimeStatSampler
sessionRegistry *sql.SessionRegistry
}
Expand All @@ -47,6 +48,7 @@ func startSampleEnvironment(
stopper *stop.Stopper,
goroutineDumpDirName string,
heapProfileDirName string,
cpuProfileDirName string,
runtimeSampler *status.RuntimeStatSampler,
sessionRegistry *sql.SessionRegistry,
) error {
Expand All @@ -56,6 +58,7 @@ func startSampleEnvironment(
minSampleInterval: base.DefaultMetricsSampleInterval,
goroutineDumpDirName: goroutineDumpDirName,
heapProfileDirName: heapProfileDirName,
cpuProfileDirName: cpuProfileDirName,
runtime: runtimeSampler,
sessionRegistry: sessionRegistry,
}
Expand Down Expand Up @@ -86,10 +89,11 @@ func startSampleEnvironment(

// Initialize a heap profiler if we have an output directory
// specified.
var heapProfiler *heapprofiler.HeapProfiler
var nonGoAllocProfiler *heapprofiler.NonGoAllocProfiler
var statsProfiler *heapprofiler.StatsProfiler
var queryProfiler *heapprofiler.ActiveQueryProfiler
var heapProfiler *profiler.HeapProfiler
var nonGoAllocProfiler *profiler.NonGoAllocProfiler
var statsProfiler *profiler.StatsProfiler
var queryProfiler *profiler.ActiveQueryProfiler
var cpuProfiler *profiler.CPUProfiler
if cfg.heapProfileDirName != "" {
hasValidDumpDir := true
if err := os.MkdirAll(cfg.heapProfileDirName, 0755); err != nil {
Expand All @@ -104,22 +108,26 @@ func startSampleEnvironment(

if hasValidDumpDir {
var err error
heapProfiler, err = heapprofiler.NewHeapProfiler(ctx, cfg.heapProfileDirName, cfg.st)
heapProfiler, err = profiler.NewHeapProfiler(ctx, cfg.heapProfileDirName, cfg.st)
if err != nil {
return errors.Wrap(err, "starting heap profiler worker")
}
nonGoAllocProfiler, err = heapprofiler.NewNonGoAllocProfiler(ctx, cfg.heapProfileDirName, cfg.st)
nonGoAllocProfiler, err = profiler.NewNonGoAllocProfiler(ctx, cfg.heapProfileDirName, cfg.st)
if err != nil {
return errors.Wrap(err, "starting non-go alloc profiler worker")
}
statsProfiler, err = heapprofiler.NewStatsProfiler(ctx, cfg.heapProfileDirName, cfg.st)
statsProfiler, err = profiler.NewStatsProfiler(ctx, cfg.heapProfileDirName, cfg.st)
if err != nil {
return errors.Wrap(err, "starting memory stats collector worker")
}
queryProfiler, err = heapprofiler.NewActiveQueryProfiler(ctx, cfg.heapProfileDirName, cfg.st)
queryProfiler, err = profiler.NewActiveQueryProfiler(ctx, cfg.heapProfileDirName, cfg.st)
if err != nil {
log.Warningf(ctx, "failed to start query profiler worker: %v", err)
}
cpuProfiler, err = profiler.NewCPUProfiler(ctx, cfg.cpuProfileDirName, cfg.st)
if err != nil {
log.Warningf(ctx, "failed to start cpu profiler worker: %v", err)
}
}
}

Expand Down Expand Up @@ -188,6 +196,9 @@ func startSampleEnvironment(
if queryProfiler != nil {
queryProfiler.MaybeDumpQueries(ctx, cfg.sessionRegistry, cfg.st)
}
if cpuProfiler != nil {
cpuProfiler.MaybeTakeProfile(ctx, int64(cfg.runtime.CPUCombinedPercentNorm.Value()*100))
}
}
}
})
Expand Down
Loading

0 comments on commit cf97b4f

Please sign in to comment.