Skip to content

Commit

Permalink
server/heapprofiler: don't conside "goIdle" memory
Browse files Browse the repository at this point in the history
Base decisions about whether to take a heap profile or not on
RSS - goIdle, not RSS alone. goIdle is memory allocated from the OS
that's not actually in use. I've seen it be up to several gigs.
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 8, 2019
1 parent 6118d1d commit 0777cca
Show file tree
Hide file tree
Showing 5 changed files with 66 additions and 30 deletions.
27 changes: 27 additions & 0 deletions pkg/base/mem_stats.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,27 @@
// Copyright 2018 The Cockroach Authors.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
// implied. See the License for the specific language governing
// permissions and limitations under the License.

package base

// MemStats represents information from the Go allocator.
// All units are bytes.
type MemStats struct {
// GoAllocated is bytes of allocated heap objects.
GoAllocated uint64
// GoIdle is space allocated from the OS but not used.
GoIdle uint64
// GoTotal is approximately GoAllocated + GoIdle (it also includes stacks and
// other things not included in GoAllocated).
GoTotal uint64
}
27 changes: 16 additions & 11 deletions pkg/server/heapprofiler/heapprofiler.go
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@ import (
"strings"
"time"

"github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/settings"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/util/log"
Expand All @@ -49,7 +50,8 @@ var (
)

type stats struct {
rssBytes int64
rssBytes uint64
goStats base.MemStats
systemMemoryBytes int64
lastProfileTime time.Time
aboveSysMemThresholdSinceLastProfile bool
Expand All @@ -58,19 +60,19 @@ type stats struct {

type heuristic struct {
name string
isTrue func(s *stats, st *cluster.Settings) (score int64, isTrue bool)
isTrue func(s *stats, st *cluster.Settings) (score uint64, isTrue bool)
}

// 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.
// fractionSystemMemoryHeuristic is true if latest (RSS - goIdle) 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 - goIdle) 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
isTrue: func(s *stats, st *cluster.Settings) (score uint64, isTrue bool) {
currentValue := s.rssBytes - s.goStats.GoIdle
if float64(currentValue)/float64(s.systemMemoryBytes) > systemMemoryThresholdFraction.Get(&st.SV) {
if s.currentTime().Sub(s.lastProfileTime) < minProfileInterval ||
s.aboveSysMemThresholdSinceLastProfile {
Expand Down Expand Up @@ -99,8 +101,11 @@ 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) {
func (o *HeapProfiler) MaybeTakeProfile(
ctx context.Context, st *cluster.Settings, rssBytes uint64, goStats base.MemStats,
) {
o.rssBytes = rssBytes
o.goStats = goStats
profileTaken := false
for _, h := range o.heuristics {
if score, isTrue := h.isTrue(o.stats, st); isTrue {
Expand Down
5 changes: 3 additions & 2 deletions pkg/server/heapprofiler/heapprofiler_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,14 +21,15 @@ import (
"testing"
"time"

"github.com/cockroachdb/cockroach/pkg/base"
"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
rss uint64
}

func testHelper(
Expand Down Expand Up @@ -67,7 +68,7 @@ func testHelper(
ctx := context.TODO()
for _, r := range rssValues {
currentTime = baseTime.Add(time.Second * r.secs)
hp.MaybeTakeProfile(ctx, st, r.rss)
hp.MaybeTakeProfile(ctx, st, r.rss, base.MemStats{})
}
assert.Equal(t, numProfiles, len(expectedScores))
}
Expand Down
4 changes: 3 additions & 1 deletion pkg/server/server.go
Original file line number Diff line number Diff line change
Expand Up @@ -1947,7 +1947,9 @@ func (s *Server) startSampleEnvironment(frequency time.Duration) {
goroutineDumper.MaybeDump(ctx, s.ClusterSettings(), s.runtime.Goroutines.Value())
}
if heapProfiler != nil {
heapProfiler.MaybeTakeProfile(ctx, s.ClusterSettings(), s.runtime.RSSBytes.Value())
ms := s.runtime.GetLastMemStats()
heapProfiler.MaybeTakeProfile(
ctx, s.ClusterSettings(), uint64(s.runtime.RSSBytes.Value()), ms)
}
timer.Reset(frequency)
case <-s.stopper.ShouldStop():
Expand Down
33 changes: 17 additions & 16 deletions pkg/server/status/runtime.go
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@ import (
"time"
"unsafe"

"github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/build"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/log"
Expand Down Expand Up @@ -226,12 +227,6 @@ var (
}
)

type memStats struct {
goAllocated uint64
goIdle uint64
goTotal uint64
}

// getCgoMemStats is a function that fetches stats for the C++ portion of the code.
// We will not necessarily have implementations for all builds, so check for nil first.
// Returns the following:
Expand Down Expand Up @@ -261,7 +256,7 @@ type RuntimeStatSampler struct {
net net.IOCountersStat
}

// Memory stats that are updated atomically by SampleMemStats.
// Memory stats that are updated atomically by SampleMemStats. *base.MemStats.
memStats unsafe.Pointer

initialDiskCounters diskStats
Expand Down Expand Up @@ -387,6 +382,15 @@ func NewRuntimeStatSampler(ctx context.Context, clock *hlc.Clock) *RuntimeStatSa
return rsr
}

// GetLastMemStats returns the last sample of the mem stats.
func (rsr *RuntimeStatSampler) GetLastMemStats() base.MemStats {
ms := (*base.MemStats)(atomic.LoadPointer(&rsr.memStats))
if ms == nil {
return base.MemStats{}
}
return *ms
}

// SampleEnvironment queries the runtime system for various interesting metrics,
// storing the resulting values in the set of metric gauges maintained by
// RuntimeStatSampler. This makes runtime statistics more convenient for
Expand Down Expand Up @@ -490,18 +494,15 @@ func (rsr *RuntimeStatSampler) SampleEnvironment(ctx context.Context) {
}
}

ms := (*memStats)(atomic.LoadPointer(&rsr.memStats))
if ms == nil {
ms = &memStats{}
}
ms := rsr.GetLastMemStats()

// Log summary of statistics to console.
cgoRate := float64((numCgoCall-rsr.last.cgoCall)*int64(time.Second)) / dur
log.Infof(ctx, "runtime stats: %s RSS, %d goroutines, %s/%s/%s GO alloc/idle/total, "+
"%s/%s CGO alloc/total, %.1f CGO/sec, %.1f/%.1f %%(u/s)time, %.1f %%gc (%dx), "+
"%s/%s (r/w)net",
humanize.IBytes(mem.Resident), numGoroutine,
humanize.IBytes(ms.goAllocated), humanize.IBytes(ms.goIdle), humanize.IBytes(ms.goTotal),
humanize.IBytes(ms.GoAllocated), humanize.IBytes(ms.GoIdle), humanize.IBytes(ms.GoTotal),
humanize.IBytes(uint64(cgoAllocated)), humanize.IBytes(uint64(cgoTotal)),
cgoRate, 100*uPerc, 100*sPerc, 100*gcPausePercent, gc.NumGC-rsr.last.gcCount,
humanize.IBytes(deltaNet.BytesRecv), humanize.IBytes(deltaNet.BytesSent),
Expand Down Expand Up @@ -551,10 +552,10 @@ func (rsr *RuntimeStatSampler) SampleMemStats(ctx context.Context) {

goAllocated := ms.Alloc
goTotal := ms.Sys - ms.HeapReleased
atomic.StorePointer(&rsr.memStats, unsafe.Pointer(&memStats{
goAllocated: goAllocated,
goIdle: ms.HeapIdle - ms.HeapReleased,
goTotal: goTotal,
atomic.StorePointer(&rsr.memStats, unsafe.Pointer(&base.MemStats{
GoAllocated: goAllocated,
GoIdle: ms.HeapIdle - ms.HeapReleased,
GoTotal: goTotal,
}))

rsr.GoAllocBytes.Update(int64(goAllocated))
Expand Down

0 comments on commit 0777cca

Please sign in to comment.