-
Notifications
You must be signed in to change notification settings - Fork 3.8k
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
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
1 parent
b861696
commit 41da6e9
Showing
11 changed files
with
299 additions
and
14 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
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") |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
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) | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
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) | ||
} | ||
}) | ||
} | ||
} |
Oops, something went wrong.