Skip to content

Commit

Permalink
server/heapprofiler: don't consider "goIdle" memory
Browse files Browse the repository at this point in the history
Before this patch we were taking a heap profile once RSS got above 85%
of system memory and then we wouldn't take another profile until we go
below it and then again above it. The problem with the RSS is that it
includes "goIdle" memory (memory allocated from the OS that's not
actually in use) ; I've seen it be up to several gigs.
With this patch, we now take a profile every time a new high-water mark
is reached. The recorded high-water mark is reset once an hour, this way
ensuring that one big measurement doesn't stop us forever from taking
another profile. It's simpler and it has a chance of catching some
further heap increases.

The rationale behind the patch is that we want profiles when the heap is
large more than when the RSS is large. I'm looking at a case where we
took a heap profile when the heap was 4.5 gigs with 2 gigs idle and then
never took one again because of how the heuristic works. And then we
OOMed when the heap was larger and the idle space was lower, but the RSS
was about the same. With this patch, we would have taken a profile at a
more interesting time.

Release note: None
  • Loading branch information
andreimatei committed May 30, 2019
1 parent a63b42d commit f53c14a
Show file tree
Hide file tree
Showing 11 changed files with 209 additions and 261 deletions.
1 change: 0 additions & 1 deletion docs/generated/settings/settings.html
Original file line number Diff line number Diff line change
Expand Up @@ -76,7 +76,6 @@
<tr><td><code>server.goroutine_dump.num_goroutines_threshold</code></td><td>integer</td><td><code>1000</code></td><td>a threshold beyond which if number of goroutines increases, then goroutine dump can be triggered</td></tr>
<tr><td><code>server.goroutine_dump.total_dump_size_limit</code></td><td>byte size</td><td><code>500 MiB</code></td><td>total size of goroutine dumps to be kept. Dumps are GC'ed in the order of creation time. The latest dump is always kept even if its size exceeds the limit.</td></tr>
<tr><td><code>server.heap_profile.max_profiles</code></td><td>integer</td><td><code>5</code></td><td>maximum number of profiles to be kept. Profiles with lower score are GC'ed, but latest profile is always kept.</td></tr>
<tr><td><code>server.heap_profile.system_memory_threshold_fraction</code></td><td>float</td><td><code>0.85</code></td><td>fraction of system memory beyond which if Rss increases, then heap profile is triggered</td></tr>
<tr><td><code>server.host_based_authentication.configuration</code></td><td>string</td><td><code></code></td><td>host-based authentication configuration to use during connection authentication</td></tr>
<tr><td><code>server.rangelog.ttl</code></td><td>duration</td><td><code>720h0m0s</code></td><td>if nonzero, range log entries older than this duration are deleted every 10m0s. Should not be lowered below 24 hours.</td></tr>
<tr><td><code>server.remote_debugging.mode</code></td><td>string</td><td><code>local</code></td><td>set to enable remote debugging, localhost-only or disable (any, local, off)</td></tr>
Expand Down
4 changes: 4 additions & 0 deletions pkg/base/constants.go
Original file line number Diff line number Diff line change
Expand Up @@ -37,4 +37,8 @@ const (
// ChunkRaftCommandThresholdBytes is the threshold in bytes at which
// to chunk or otherwise limit commands being sent to Raft.
ChunkRaftCommandThresholdBytes = 256 * 1000

// HeapProfileDir is the directory name where the heap profiler stores profiles
// when there is a potential OOM situation.
HeapProfileDir = "heap_profiler"
)
4 changes: 3 additions & 1 deletion pkg/cli/start.go
Original file line number Diff line number Diff line change
Expand Up @@ -448,7 +448,9 @@ func runStart(cmd *cobra.Command, args []string) error {
}

serverCfg.GoroutineDumpDirName = logOutputDirectory()
serverCfg.HeapProfileDirName = logOutputDirectory()

heapProfileDir := filepath.Join(logOutputDirectory(), base.HeapProfileDir)
serverCfg.HeapProfileDirName = heapProfileDir
// We don't care about GRPCs fairly verbose logs in most client commands,
// but when actually starting a server, we enable them.
grpcutil.SetSeverity(log.Severity_WARNING)
Expand Down
3 changes: 2 additions & 1 deletion pkg/server/config.go
Original file line number Diff line number Diff line change
Expand Up @@ -179,8 +179,9 @@ type Config struct {
GoroutineDumpDirName string

// HeapProfileDirName is the directory name for heap profiles using
// heapprofiler.
// heapprofiler. If empty, no heap profiles will be collected.
HeapProfileDirName string

// Parsed values.

// NodeAttributes is the parsed representation of Attrs.
Expand Down
160 changes: 69 additions & 91 deletions pkg/server/heapprofiler/heapprofiler.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@ import (
"io/ioutil"
"os"
"path/filepath"
"runtime"
"runtime/pprof"
"strings"
"time"
Expand All @@ -31,15 +32,11 @@ import (
"github.com/pkg/errors"
)

const minProfileInterval = time.Minute
// resetHighWaterMarkInterval specifies how often the high-water mark value will
// be reset. Immediately after it is reset, a new profile will be taken.
const resetHighWaterMarkInterval = time.Hour

var (
systemMemoryThresholdFraction = settings.RegisterFloatSetting(
"server.heap_profile.system_memory_threshold_fraction",
"fraction of system memory beyond which if Rss increases, "+
"then heap profile is triggered",
.85,
)
maxProfiles = settings.RegisterIntSetting(
"server.heap_profile.max_profiles",
"maximum number of profiles to be kept. "+
Expand All @@ -48,107 +45,89 @@ var (
)
)

type stats struct {
rssBytes int64
systemMemoryBytes int64
lastProfileTime time.Time
aboveSysMemThresholdSinceLastProfile bool
currentTime func() time.Time
type testingKnobs struct {
dontWriteProfiles bool
maybeTakeProfileHook func(willTakeProfile bool)
now func() time.Time
}

type heuristic struct {
name string
isTrue func(s *stats, st *cluster.Settings) (score int64, isTrue bool)
}
// HeapProfiler is used to take heap profiles.
//
// MaybeTakeProfile() is supposed to be called periodically. A profile is taken
// every time Go heap allocated bytes exceeds the previous high-water mark. The
// recorded high-water mark is also reset periodically, so that we take some
// profiles periodically.
// Profiles are also GCed periodically. The latest is always kept, and a couple
// of the ones with the largest heap are also kept.
type HeapProfiler struct {
dir string
st *cluster.Settings
// lastProfileTime marks the time when we took the last profile.
lastProfileTime time.Time
// highwaterMarkBytes represents the maximum heap size that we've seen since
// resetting the filed (which happens periodically).
highwaterMarkBytes uint64

// fractionSystemMemoryHeuristic is true if latest Rss is more than
// systemMemoryThresholdFraction of system memory. No new profile is
// taken if Rss has been above threshold since the last time profile was taken,
// but a new profile will be triggered if Rss has dipped below threshold since
// the last profile. score is the latest value of Rss.
// At max one profile will be taken in minProfileInterval.
var fractionSystemMemoryHeuristic = heuristic{
name: "fraction_system_memory",
isTrue: func(s *stats, st *cluster.Settings) (score int64, isTrue bool) {
currentValue := s.rssBytes
if float64(currentValue)/float64(s.systemMemoryBytes) > systemMemoryThresholdFraction.Get(&st.SV) {
if s.currentTime().Sub(s.lastProfileTime) < minProfileInterval ||
s.aboveSysMemThresholdSinceLastProfile {
return 0, false
}
s.aboveSysMemThresholdSinceLastProfile = true
return currentValue, true
}
s.aboveSysMemThresholdSinceLastProfile = false
return 0, false
},
knobs testingKnobs
}

// HeapProfiler is used to take heap profiles if an OOM situation is
// detected. It stores relevant functions and stats for heuristics to use.
type HeapProfiler struct {
*stats
heuristics []heuristic
takeHeapProfile func(ctx context.Context, dir string, prefix string, suffix string)
gcProfiles func(ctx context.Context, dir, prefix string, maxCount int64)
dir string
// NewHeapProfiler creates a HeapProfiler. dir is the directory in which
// profiles are to be stored.
func NewHeapProfiler(dir string, st *cluster.Settings) (*HeapProfiler, error) {
if dir == "" {
return nil, errors.Errorf("need to specify dir for NewHeapProfiler")
}
hp := &HeapProfiler{
dir: dir,
st: st,
}
return hp, nil
}

const memprof = "memprof."

// MaybeTakeProfile takes a heap profile if an OOM situation is detected using
// heuristics enabled in o. At max one profile is taken in a call of this
// function. This function is also responsible for updating stats in o.
func (o *HeapProfiler) MaybeTakeProfile(ctx context.Context, st *cluster.Settings, rssBytes int64) {
o.rssBytes = rssBytes
profileTaken := false
for _, h := range o.heuristics {
if score, isTrue := h.isTrue(o.stats, st); isTrue {
if !profileTaken {
prefix := memprof + h.name + "."
const format = "2006-01-02T15_04_05.999"
suffix := fmt.Sprintf("%018d_%s", score, o.currentTime().Format(format))
o.takeHeapProfile(ctx, o.dir, prefix, suffix)
o.lastProfileTime = o.currentTime()
profileTaken = true
if o.gcProfiles != nil {
o.gcProfiles(ctx, o.dir, memprof, maxProfiles.Get(&st.SV))
}
}
}
// MaybeTakeProfile takes a heap profile if the heap is big enough.
func (o *HeapProfiler) MaybeTakeProfile(ctx context.Context, ms runtime.MemStats) {
// If it's been too long since we took a profile, make sure we'll take one now.
if o.now().Sub(o.lastProfileTime) > resetHighWaterMarkInterval {
o.highwaterMarkBytes = 0
}
}

// NewHeapProfiler returns a HeapProfiler which has
// systemMemoryThresholdFraction heuristic enabled. dir is the directory in
// which profiles are stored.
func NewHeapProfiler(dir string, systemMemoryBytes int64) (*HeapProfiler, error) {
if dir == "" {
return nil, errors.New("directory to store profiles could not be determined")
curHeap := ms.HeapAlloc
takeProfile := curHeap > o.highwaterMarkBytes
if hook := o.knobs.maybeTakeProfileHook; hook != nil {
hook(takeProfile)
}
dir = filepath.Join(dir, "heap_profiler")
if err := os.MkdirAll(dir, 0755); err != nil {
return nil, err
if !takeProfile {
return
}
hp := &HeapProfiler{
stats: &stats{
systemMemoryBytes: systemMemoryBytes,
currentTime: timeutil.Now,
},
heuristics: []heuristic{fractionSystemMemoryHeuristic},
takeHeapProfile: takeHeapProfile,
gcProfiles: gcProfiles,
dir: dir,

o.highwaterMarkBytes = curHeap
o.lastProfileTime = o.now()

if o.knobs.dontWriteProfiles {
return
}
return hp, nil
const format = "2006-01-02T15_04_05.999"
filePrefix := "memprof."
fileName := fmt.Sprintf("%s%018d_%s", filePrefix, curHeap, o.now().Format(format))
path := filepath.Join(o.dir, fileName)
takeHeapProfile(ctx, path)
o.gcProfiles(ctx, o.dir, filePrefix)
}

func (o *HeapProfiler) now() time.Time {
if o.knobs.now != nil {
return o.knobs.now()
}
return timeutil.Now()
}

// gcProfiles removes least score profile matching the specified prefix when the
// number of profiles is more than maxCount. Requires that the suffix used for
// the profiles indicates score such that sorting the filenames corresponds to
// ordering the profiles from least to max score.
// Latest profile in the directory is not considered for GC.
func gcProfiles(ctx context.Context, dir, prefix string, maxCount int64) {
func (o *HeapProfiler) gcProfiles(ctx context.Context, dir, prefix string) {
maxCount := maxProfiles.Get(&o.st.SV)
files, err := ioutil.ReadDir(dir)
if err != nil {
log.Warning(ctx, err)
Expand Down Expand Up @@ -184,8 +163,7 @@ func gcProfiles(ctx context.Context, dir, prefix string, maxCount int64) {
}
}

func takeHeapProfile(ctx context.Context, dir string, prefix string, suffix string) {
path := filepath.Join(dir, prefix+suffix)
func takeHeapProfile(ctx context.Context, path string) {
// Try writing a go heap profile.
f, err := os.Create(path)
if err != nil {
Expand Down
96 changes: 33 additions & 63 deletions pkg/server/heapprofiler/heapprofiler_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,84 +16,54 @@ package heapprofiler

import (
"context"
"strconv"
"strings"
"runtime"
"testing"
"time"

"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/util/syncutil"
"github.com/stretchr/testify/assert"
)

type rssVal struct {
secs time.Duration // secs is the time at which this rss value was emitted
rss int64
}
func TestHeapProfiler(t *testing.T) {
ctx := context.Background()
type test struct {
secs int // The measurement's timestamp.
heapBytes uint64

func testHelper(
t *testing.T,
hp *HeapProfiler,
st *cluster.Settings,
rssValues []rssVal,
expectedScores []int64,
expectedPrefixes []string,
) {
baseTime := time.Time{}
numProfiles := 0
mockHeapProfile := func(
ctx context.Context, dir string, prefix string, suffix string,
) {
assert.Equal(t, prefix, expectedPrefixes[numProfiles])
score, err := strconv.ParseInt(strings.Split(suffix, "_")[0], 10, 64)
assert.Nil(t, err)
assert.Equal(t, expectedScores[numProfiles], score)
numProfiles++
expProfile bool
}
tests := []test{
{0, 30, true}, // we always take the first profile
{10, 40, true}, // new high-water mark
{20, 30, false}, // below high-water mark; no profile
{4000, 10, true}, // new hour; should trigger regardless of the usage
}
var currentTime time.Time
var timeLocker syncutil.RWMutex
now := func() time.Time {
timeLocker.RLock()
defer timeLocker.RUnlock()
return currentTime
}
hp.takeHeapProfile = mockHeapProfile
hp.currentTime = now
// set a large negative time so that first profile is triggered correctly
// since we start time from 0 in test.
// Not needed in main code as time will never be 0.
hp.lastProfileTime = time.Time{}.Add(-1000 * time.Second)

ctx := context.TODO()
for _, r := range rssValues {
currentTime = baseTime.Add(time.Second * r.secs)
hp.MaybeTakeProfile(ctx, st, r.rss)
var tookProfile bool
hp, err := NewHeapProfiler("dummy_dir", cluster.MakeTestingClusterSettings())
if err != nil {
t.Fatal(err)
}
assert.Equal(t, numProfiles, len(expectedScores))
}

func TestPercentSystemMemoryHeuristic(t *testing.T) {
rssValues := []rssVal{
{0, 30}, {20, 40}, // random small values
{30, 88}, // should trigger
{80, 89}, // should not trigger as less than 60s before last profile
{130, 10}, {140, 4}, // random small values
{150, 90}, // should trigger
{260, 92}, // should not trigger as continues above threshold
{290, 30}, // random small value
{380, 99}, // should trigger
{390, 30}, // random small value
{430, 91}, // should not trigger as less than 60s before last profile
{500, 95}, // should trigger
hp.knobs = testingKnobs{
now: now,
dontWriteProfiles: true,
maybeTakeProfileHook: func(willTakeProfile bool) {
tookProfile = willTakeProfile
},
}
expectedScores := []int64{88, 90, 99, 95}
prefix := "memprof.fraction_system_memory."
expectedPrefixes := []string{prefix, prefix, prefix, prefix}
hp := &HeapProfiler{
stats: &stats{systemMemoryBytes: 100},
heuristics: []heuristic{fractionSystemMemoryHeuristic},

for i, r := range tests {
currentTime = (time.Time{}).Add(time.Second * time.Duration(r.secs))

// Initialize enough of ms for the purposes of the HeapProfiler.
var ms runtime.MemStats
ms.HeapAlloc = r.heapBytes

hp.MaybeTakeProfile(ctx, ms)
assert.Equal(t, r.expProfile, tookProfile, i)
}
st := &cluster.Settings{}
systemMemoryThresholdFraction.Override(&st.SV, .85)
testHelper(t, hp, st, rssValues, expectedScores, expectedPrefixes)
}
Loading

0 comments on commit f53c14a

Please sign in to comment.