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. A cpu profile
is taken based on a qps threshold and an interval for when the
last profile was taken.

Fixes: #75799

Release note: None
  • Loading branch information
Santamaura committed Jan 23, 2023
1 parent b861696 commit 41da6e9
Show file tree
Hide file tree
Showing 11 changed files with 299 additions and 14 deletions.
4 changes: 4 additions & 0 deletions pkg/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -251,6 +251,7 @@ ALL_TESTS = [
"//pkg/security/username:username_disallowed_imports_test",
"//pkg/security/username:username_test",
"//pkg/security:security_test",
"//pkg/server/cpuprofiler:cpuprofiler_test",
"//pkg/server/debug/goroutineui:goroutineui_test",
"//pkg/server/debug/pprofui:pprofui_test",
"//pkg/server/debug:debug_test",
Expand Down Expand Up @@ -1344,6 +1345,8 @@ GO_TARGETS = [
"//pkg/security/username:username_test",
"//pkg/security:security",
"//pkg/security:security_test",
"//pkg/server/cpuprofiler:cpuprofiler",
"//pkg/server/cpuprofiler:cpuprofiler_test",
"//pkg/server/debug/goroutineui:goroutineui",
"//pkg/server/debug/goroutineui:goroutineui_test",
"//pkg/server/debug/pprofui:pprofui",
Expand Down Expand Up @@ -2672,6 +2675,7 @@ GET_X_DATA_TARGETS = [
"//pkg/security/sessionrevival:get_x_data",
"//pkg/security/username:get_x_data",
"//pkg/server:get_x_data",
"//pkg/server/cpuprofiler:get_x_data",
"//pkg/server/debug:get_x_data",
"//pkg/server/debug/goroutineui:get_x_data",
"//pkg/server/debug/pprofui:get_x_data",
Expand Down
1 change: 1 addition & 0 deletions pkg/cli/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -142,6 +142,7 @@ go_library(
"//pkg/security/securitytest",
"//pkg/security/username",
"//pkg/server",
"//pkg/server/cpuprofiler",
"//pkg/server/debug",
"//pkg/server/dumpstore",
"//pkg/server/heapprofiler",
Expand Down
18 changes: 4 additions & 14 deletions pkg/cli/cpuprofile.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,26 +18,16 @@ import (
"strings"
"time"

"github.com/cockroachdb/cockroach/pkg/server/cpuprofiler"
"github.com/cockroachdb/cockroach/pkg/server/debug"
"github.com/cockroachdb/cockroach/pkg/server/dumpstore"
"github.com/cockroachdb/cockroach/pkg/settings"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/util/envutil"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/log/logcrash"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
)

var maxCombinedCPUProfFileSize = settings.RegisterByteSizeSetting(
settings.TenantWritable,
"server.cpu_profile.total_dump_size_limit",
"maximum combined disk size of preserved CPU profiles",
128<<20, // 128MiB
)

const cpuProfTimeFormat = "2006-01-02T15_04_05.000"
const cpuProfFileNamePrefix = "cpuprof."

type cpuProfiler struct{}

// PreFilter is part of the dumpstore.Dumper interface.
Expand All @@ -57,7 +47,7 @@ func (s cpuProfiler) PreFilter(

// CheckOwnsFile is part of the dumpstore.Dumper interface.
func (s cpuProfiler) CheckOwnsFile(_ context.Context, fi os.FileInfo) bool {
return strings.HasPrefix(fi.Name(), cpuProfFileNamePrefix)
return strings.HasPrefix(fi.Name(), cpuprofiler.CpuProfFileNamePrefix)
}

func initCPUProfile(ctx context.Context, dir string, st *cluster.Settings) {
Expand Down Expand Up @@ -85,7 +75,7 @@ func initCPUProfile(ctx context.Context, dir string, st *cluster.Settings) {
cpuProfileInterval = min
}

profilestore := dumpstore.NewStore(dir, maxCombinedCPUProfFileSize, st)
profilestore := dumpstore.NewStore(dir, cpuprofiler.MaxCombinedCPUProfFileSize, st)
profiler := dumpstore.Dumper(cpuProfiler{})

// TODO(knz,tbg): The caller of initCPUProfile() also defines a stopper;
Expand Down Expand Up @@ -122,7 +112,7 @@ func initCPUProfile(ctx context.Context, dir string, st *cluster.Settings) {
pprof.StopCPUProfile()

now := timeutil.Now()
name := cpuProfFileNamePrefix + now.Format(cpuProfTimeFormat)
name := cpuprofiler.CpuProfFileNamePrefix + now.Format(cpuprofiler.CpuProfTimeFormat)
path := profilestore.GetFullPath(name)
if err := os.WriteFile(path, buf.Bytes(), 0644); err != nil {
return err
Expand Down
1 change: 1 addition & 0 deletions pkg/server/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -134,6 +134,7 @@ go_library(
"//pkg/security/password",
"//pkg/security/securityassets",
"//pkg/security/username",
"//pkg/server/cpuprofiler",
"//pkg/server/debug",
"//pkg/server/diagnostics",
"//pkg/server/diagnostics/diagnosticspb",
Expand Down
34 changes: 34 additions & 0 deletions pkg/server/cpuprofiler/BUILD.bazel
Original file line number Diff line number Diff line change
@@ -0,0 +1,34 @@
load("//build/bazelutil/unused_checker:unused.bzl", "get_x_data")
load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test")

go_library(
name = "cpuprofiler",
srcs = ["cpuprofiler.go"],
importpath = "github.com/cockroachdb/cockroach/pkg/server/cpuprofiler",
visibility = ["//visibility:public"],
deps = [
"//pkg/server/debug",
"//pkg/server/dumpstore",
"//pkg/settings",
"//pkg/settings/cluster",
"//pkg/util/log",
"//pkg/util/timeutil",
"@com_github_cockroachdb_errors//:errors",
],
)

go_test(
name = "cpuprofiler_test",
srcs = ["cpuprofiler_test.go"],
args = ["-test.timeout=295s"],
embed = [":cpuprofiler"],
deps = [
"//pkg/clusterversion",
"//pkg/server/dumpstore",
"//pkg/settings/cluster",
"//pkg/util/timeutil",
"@com_github_stretchr_testify//assert",
],
)

get_x_data(name = "get_x_data")
158 changes: 158 additions & 0 deletions pkg/server/cpuprofiler/cpuprofiler.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,158 @@
// Copyright 2023 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.

package cpuprofiler

import (
"bytes"
"context"
"os"
"runtime"
"runtime/pprof"
"strings"
"time"

"github.com/cockroachdb/cockroach/pkg/server/debug"
"github.com/cockroachdb/cockroach/pkg/server/dumpstore"
"github.com/cockroachdb/cockroach/pkg/settings"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
"github.com/cockroachdb/errors"
)

var MaxCombinedCPUProfFileSize = settings.RegisterByteSizeSetting(
settings.TenantWritable,
"server.cpu_profile.total_dump_size_limit",
"maximum combined disk size of preserved CPU profiles",
128<<20, // 128MiB
)

var qpsThreshold = settings.RegisterIntSetting(
settings.TenantWritable,
"server.cpu_profile.qps_threshold",
"a threshold beyond which if the qps is above, "+
"then a cpu profile can be triggered",
100,
)

var cpuProfileInterval = settings.RegisterDurationSetting(
settings.TenantWritable,
"server.cpu_profile.interval",
"rate at which cpu profiles can be taken if above qps threshold",
1*time.Minute, settings.PositiveDuration,
)

const CpuProfTimeFormat = "2006-01-02T15_04_05.000"
const CpuProfFileNamePrefix = "cpuprof."

type testingKnobs struct {
dontWrite bool
}

type CpuProfiler struct {
lastProfileTime time.Time
currentTime func() time.Time
store *dumpstore.DumpStore
st *cluster.Settings
knobs testingKnobs
}

// NewCpuProfiler creates a new CpuProfiler. dir indicates the directory which
// dumps are stored.
func NewCpuProfiler(ctx context.Context, dir string, st *cluster.Settings) (*CpuProfiler, error) {
if dir == "" {
return nil, errors.New("directory to store dumps could not be determined")
}

log.Infof(ctx, "writing cpu profile dumps to %s", log.SafeManaged(dir))

cp := &CpuProfiler{
currentTime: timeutil.Now,
store: dumpstore.NewStore(dir, MaxCombinedCPUProfFileSize, st),
st: st,
knobs: testingKnobs{},
}
return cp, nil
}

// PreFilter is part of the dumpstore.Dumper interface.
func (cp *CpuProfiler) PreFilter(
ctx context.Context, files []os.FileInfo, _ func(fileName string) error,
) (preserved map[int]bool, _ error) {
preserved = make(map[int]bool)
// Always keep at least the last profile.
for i := len(files) - 1; i >= 0; i-- {
if cp.CheckOwnsFile(ctx, files[i]) {
preserved[i] = true
break
}
}
return
}

// CheckOwnsFile is part of the dumpstore.Dumper interface.
func (cp *CpuProfiler) CheckOwnsFile(_ context.Context, fi os.FileInfo) bool {
return strings.HasPrefix(fi.Name(), CpuProfFileNamePrefix)
}

// MaybeTakeProfile takes a cpu profile and writes to a file provided the
// threshold is met and the time between the last profile is greater than the
// cpuProfileInterval setting.
func (cp *CpuProfiler) MaybeTakeProfile(
ctx context.Context, readCt int64, writeCt int64, uptime int64,
) {
// Should this be the metric used to determine threshold?
// Exit early if threshold is not met.
qps := (readCt + writeCt) / uptime
if qpsThreshold.Get(&cp.st.SV) > qps {
return
}

// Exit early if a profile has already been taken within the interval period.
lastProfileTimeDiff := timeutil.Since(cp.lastProfileTime)
if cpuProfileInterval.Get(&cp.st.SV).Seconds() > lastProfileTimeDiff.Seconds() {
return
}

if err := debug.CPUProfileDo(cp.st, cluster.CPUProfileDefault, func() error {
var buf bytes.Buffer
var now time.Time
// Only create the profile if not testing.
if !cp.knobs.dontWrite {
// Set profiling rate to 10hz samples/s to reduce sample rate and limit cpu overhead.
runtime.SetCPUProfileRate(10)
// Start the new profile.
if err := pprof.StartCPUProfile(&buf); err != nil {
return err
}
defer pprof.StopCPUProfile()
now = cp.currentTime()
name := CpuProfFileNamePrefix + now.Format(CpuProfTimeFormat)
path := cp.store.GetFullPath(name)
f, err := os.Create(path)
if err != nil {
log.Errorf(ctx, "error creating cpu profile %s: %v", path, err)
return err
}
defer f.Close()
if _, err := f.Write(buf.Bytes()); err != nil {
return err
}
cp.store.GC(ctx, now, cp)
}
cp.lastProfileTime = now
return nil
}); err != nil {
// Only log the errors, since errors can occur due to cpu profiles being taken
// elsewhere.
log.Infof(ctx, "error during CPU profile: %s", err)
}
}
80 changes: 80 additions & 0 deletions pkg/server/cpuprofiler/cpuprofiler_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,80 @@
// Copyright 2023 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.

package cpuprofiler

import (
"context"
"testing"
"time"

"github.com/cockroachdb/cockroach/pkg/clusterversion"
"github.com/cockroachdb/cockroach/pkg/server/dumpstore"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
"github.com/stretchr/testify/assert"
)

var cpuProfilerDirName = "cockroach-data/logs/pprof_dump"

func TestCPUProfiler(t *testing.T) {
ctx := context.Background()
s := &cluster.Settings{}
sv := &s.SV
s.Version = clusterversion.MakeVersionHandle(sv)
sv.Init(ctx, s.Version)
pastTime := time.Date(2023, 1, 1, 1, 1, 1, 1, time.UTC)
cases := []struct {
name string
params []int64
lastProfileTime time.Time
currentTime func() time.Time
expectNewProfile bool
}{
{
name: "take profile",
params: []int64{100, 100, 1},
lastProfileTime: pastTime,
currentTime: timeutil.Now,
expectNewProfile: true,
},
{
name: "no profile due to low qps",
params: []int64{1, 1, 1},
lastProfileTime: pastTime,
currentTime: timeutil.Now,
expectNewProfile: false,
},
{
name: "no profile due to last profile being within the time interval",
params: []int64{100, 100, 1},
lastProfileTime: timeutil.Now(),
currentTime: timeutil.Now,
expectNewProfile: false,
},
}
for _, c := range cases {
t.Run(c.name, func(t *testing.T) {
profiler := CpuProfiler{
lastProfileTime: c.lastProfileTime,
currentTime: c.currentTime,
store: dumpstore.NewStore(cpuProfilerDirName, nil, nil),
st: s,
knobs: testingKnobs{dontWrite: true},
}
profiler.MaybeTakeProfile(ctx, c.params[0], c.params[1], c.params[2])
if c.expectNewProfile {
assert.NotEqual(t, c.lastProfileTime, profiler.lastProfileTime)
} else {
assert.Equal(t, c.lastProfileTime, profiler.lastProfileTime)
}
})
}
}
Loading

0 comments on commit 41da6e9

Please sign in to comment.