Commit 1a680a90 authored by Russ Cox's avatar Russ Cox

runtime/pprof: use new profile buffers for CPU profiling

This doesn't change the functionality of the current code,
but it sets us up for exporting the profiling labels into the profile.

The old code had a hash table of profile samples maintained
during the signal handler, with evictions going into a log.
The new code just logs every sample directly, leaving the
hash-based deduplication to an ordinary goroutine.

The new code also avoids storing the entire profile in two
forms in memory, an unfortunate regression introduced
when binary profile support was added. After this CL the
entire profile is only stored once in memory. We'd still like
to get back down to storing it zero times (streaming it to
the underlying io.Writer).

Change-Id: I0893a1788267c564aa1af17970d47377b2a43457
Reviewed-on: https://go-review.googlesource.com/36712
Run-TryBot: Russ Cox <rsc@golang.org>
TryBot-Result: Gobot Gobot <gobot@golang.org>
Reviewed-by: 's avatarMichael Matloob <matloob@golang.org>
parent a1261b8b
......@@ -3,118 +3,45 @@
// license that can be found in the LICENSE file.
// CPU profiling.
// Based on algorithms and data structures used in
// https://github.com/google/pprof.
//
// The main difference between this code and the google-perftools
// code is that this code is written to allow copying the profile data
// to an arbitrary io.Writer, while the google-perftools code always
// writes to an operating system file.
//
// The signal handler for the profiling clock tick adds a new stack trace
// to a hash table tracking counts for recent traces. Most clock ticks
// hit in the cache. In the event of a cache miss, an entry must be
// evicted from the hash table, copied to a log that will eventually be
// written as profile data. The google-perftools code flushed the
// log itself during the signal handler. This code cannot do that, because
// the io.Writer might block or need system calls or locks that are not
// safe to use from within the signal handler. Instead, we split the log
// into two halves and let the signal handler fill one half while a goroutine
// is writing out the other half. When the signal handler fills its half, it
// offers to swap with the goroutine. If the writer is not done with its half,
// we lose the stack trace for this clock tick (and record that loss).
// The goroutine interacts with the signal handler by calling getprofile() to
// get the next log piece to write, implicitly handing back the last log
// piece it obtained.
//
// The state of this dance between the signal handler and the goroutine
// is encoded in the Profile.handoff field. If handoff == 0, then the goroutine
// is not using either log half and is waiting (or will soon be waiting) for
// a new piece by calling notesleep(&p.wait). If the signal handler
// changes handoff from 0 to non-zero, it must call notewakeup(&p.wait)
// to wake the goroutine. The value indicates the number of entries in the
// log half being handed off. The goroutine leaves the non-zero value in
// place until it has finished processing the log half and then flips the number
// back to zero. Setting the high bit in handoff means that the profiling is over,
// and the goroutine is now in charge of flushing the data left in the hash table
// to the log and returning that data.
//
// The handoff field is manipulated using atomic operations.
// For the most part, the manipulation of handoff is orderly: if handoff == 0
// then the signal handler owns it and can change it to non-zero.
// If handoff != 0 then the goroutine owns it and can change it to zero.
// If that were the end of the story then we would not need to manipulate
// handoff using atomic operations. The operations are needed, however,
// in order to let the log closer set the high bit to indicate "EOF" safely
// in the situation when normally the goroutine "owns" handoff.
// to a log of recent traces. The log is read by a user goroutine that
// turns it into formatted profile data. If the reader does not keep up
// with the log, those writes will be recorded as a count of lost records.
// The actual profile buffer is in profbuf.go.
package runtime
import (
"runtime/internal/atomic"
"runtime/internal/sys"
"unsafe"
)
const (
numBuckets = 1 << 10
logSize = 1 << 17
assoc = 4
maxCPUProfStack = 64
)
type cpuprofEntry struct {
count uintptr
depth int
stack [maxCPUProfStack]uintptr
}
const maxCPUProfStack = 64
//go:notinheap
type cpuProfile struct {
on bool // profiling is on
wait note // goroutine waits here
count uintptr // tick count
evicts uintptr // eviction count
lost uintptr // lost ticks that need to be logged
// Active recent stack traces.
hash [numBuckets]struct {
entry [assoc]cpuprofEntry
}
// Log of traces evicted from hash.
// Signal handler has filled log[toggle][:nlog].
// Goroutine is writing log[1-toggle][:handoff].
log [2][logSize / 2]uintptr
nlog int
toggle int32
handoff uint32
// Writer state.
// Writer maintains its own toggle to avoid races
// looking at signal handler's toggle.
wtoggle uint32
wholding bool // holding & need to release a log half
flushing bool // flushing hash table - profile is over
eodSent bool // special end-of-data record sent; => flushing
lock mutex
on bool // profiling is on
log *profBuf // profile events written here
// extra holds extra stacks accumulated in addNonGo
// corresponding to profiling signals arriving on
// non-Go-created threads. Those stacks are written
// to log the next time a normal Go thread gets the
// signal handler.
// Assuming the stacks are 2 words each (we don't get
// a full traceback from those threads), plus one word
// size for framing, 100 Hz profiling would generate
// 300 words per second.
// Hopefully a normal Go thread will get the profiling
// signal at least once every few seconds.
extra [1000]uintptr
numExtra int
lostExtra uint64 // count of frames lost because extra is full
}
var (
cpuprofLock mutex
cpuprof *cpuProfile
eod = [3]uintptr{0, 1, 0}
)
func setcpuprofilerate(hz int32) {
systemstack(func() {
setcpuprofilerate_m(hz)
})
}
// lostProfileData is a no-op function used in profiles
// to mark the number of profiling stack traces that were
// discarded due to slow data writers.
func lostProfileData() {}
var cpuprof cpuProfile
// SetCPUProfileRate sets the CPU profiling rate to hz samples per second.
// If hz <= 0, SetCPUProfileRate turns off profiling.
......@@ -132,323 +59,144 @@ func SetCPUProfileRate(hz int) {
hz = 1000000
}
lock(&cpuprofLock)
lock(&cpuprof.lock)
if hz > 0 {
if cpuprof == nil {
cpuprof = (*cpuProfile)(sysAlloc(unsafe.Sizeof(cpuProfile{}), &memstats.other_sys))
if cpuprof == nil {
print("runtime: cpu profiling cannot allocate memory\n")
unlock(&cpuprofLock)
return
}
}
if cpuprof.on || cpuprof.handoff != 0 {
if cpuprof.on || cpuprof.log != nil {
print("runtime: cannot set cpu profile rate until previous profile has finished.\n")
unlock(&cpuprofLock)
unlock(&cpuprof.lock)
return
}
cpuprof.on = true
// pprof binary header format.
// https://github.com/gperftools/gperftools/blob/master/src/profiledata.cc#L119
p := &cpuprof.log[0]
p[0] = 0 // count for header
p[1] = 3 // depth for header
p[2] = 0 // version number
p[3] = uintptr(1e6 / hz) // period (microseconds)
p[4] = 0
cpuprof.nlog = 5
cpuprof.toggle = 0
cpuprof.wholding = false
cpuprof.wtoggle = 0
cpuprof.flushing = false
cpuprof.eodSent = false
noteclear(&cpuprof.wait)
cpuprof.log = newProfBuf(1, 1<<17, 1<<14)
hdr := [1]uint64{uint64(hz)}
cpuprof.log.write(nil, nanotime(), hdr[:], nil)
setcpuprofilerate(int32(hz))
} else if cpuprof != nil && cpuprof.on {
} else if cpuprof.on {
setcpuprofilerate(0)
cpuprof.on = false
// Now add is not running anymore, and getprofile owns the entire log.
// Set the high bit in cpuprof.handoff to tell getprofile.
for {
n := cpuprof.handoff
if n&0x80000000 != 0 {
print("runtime: setcpuprofile(off) twice\n")
}
if atomic.Cas(&cpuprof.handoff, n, n|0x80000000) {
if n == 0 {
// we did the transition from 0 -> nonzero so we wake getprofile
notewakeup(&cpuprof.wait)
}
break
}
}
cpuprof.addExtra()
cpuprof.log.close()
}
unlock(&cpuprofLock)
unlock(&cpuprof.lock)
}
// add adds the stack trace to the profile.
// It is called from signal handlers and other limited environments
// and cannot allocate memory or acquire locks that might be
// held at the time of the signal, nor can it use substantial amounts
// of stack. It is allowed to call evict.
// of stack.
//go:nowritebarrierrec
func (p *cpuProfile) add(pc []uintptr) {
p.addWithFlushlog(pc, p.flushlog)
}
// addWithFlushlog implements add and addNonGo.
// It is called from signal handlers and other limited environments
// and cannot allocate memory or acquire locks that might be
// held at the time of the signal, nor can it use substantial amounts
// of stack. It may be called by a signal handler with no g or m.
// It is allowed to call evict, passing the flushlog parameter.
//go:nosplit
//go:nowritebarrierrec
func (p *cpuProfile) addWithFlushlog(pc []uintptr, flushlog func() bool) {
if len(pc) > maxCPUProfStack {
pc = pc[:maxCPUProfStack]
}
// Compute hash.
h := uintptr(0)
for _, x := range pc {
h = h<<8 | (h >> (8 * (unsafe.Sizeof(h) - 1)))
h += x * 41
func (p *cpuProfile) add(gp *g, stk []uintptr) {
// Simple cas-lock to coordinate with setcpuprofilerate.
for !atomic.Cas(&prof.signalLock, 0, 1) {
osyield()
}
p.count++
// Add to entry count if already present in table.
b := &p.hash[h%numBuckets]
Assoc:
for i := range b.entry {
e := &b.entry[i]
if e.depth != len(pc) {
continue
if prof.hz != 0 { // implies cpuprof.log != nil
if p.numExtra > 0 || p.lostExtra > 0 {
p.addExtra()
}
for j := range pc {
if e.stack[j] != pc[j] {
continue Assoc
}
}
e.count++
return
hdr := [1]uint64{1}
// Note: write "knows" that the argument is &gp.labels,
// because otherwise its write barrier behavior may not
// be correct. See the long comment there before
// changing the argument here.
cpuprof.log.write(&gp.labels, nanotime(), hdr[:], stk)
}
// Evict entry with smallest count.
var e *cpuprofEntry
for i := range b.entry {
if e == nil || b.entry[i].count < e.count {
e = &b.entry[i]
}
}
if e.count > 0 {
if !p.evict(e, flushlog) {
// Could not evict entry. Record lost stack.
p.lost++
return
}
p.evicts++
}
// Reuse the newly evicted entry.
e.depth = len(pc)
e.count = 1
copy(e.stack[:], pc)
atomic.Store(&prof.signalLock, 0)
}
// evict copies the given entry's data into the log, so that
// the entry can be reused. evict is called from add, which
// is called from the profiling signal handler, so it must not
// allocate memory or block, and it may be called with no g or m.
// It is safe to call flushlog. evict returns true if the entry was
// copied to the log, false if there was no room available.
// addNonGo adds the non-Go stack trace to the profile.
// It is called from a non-Go thread, so we cannot use much stack at all,
// nor do anything that needs a g or an m.
// In particular, we can't call cpuprof.log.write.
// Instead, we copy the stack into cpuprof.extra,
// which will be drained the next time a Go thread
// gets the signal handling event.
//go:nosplit
//go:nowritebarrierrec
func (p *cpuProfile) evict(e *cpuprofEntry, flushlog func() bool) bool {
d := e.depth
nslot := d + 2
log := &p.log[p.toggle]
if p.nlog+nslot > len(log) {
if !flushlog() {
return false
}
log = &p.log[p.toggle]
func (p *cpuProfile) addNonGo(stk []uintptr) {
// Simple cas-lock to coordinate with SetCPUProfileRate.
// (Other calls to add or addNonGo should be blocked out
// by the fact that only one SIGPROF can be handled by the
// process at a time. If not, this lock will serialize those too.)
for !atomic.Cas(&prof.signalLock, 0, 1) {
osyield()
}
q := p.nlog
log[q] = e.count
q++
log[q] = uintptr(d)
q++
copy(log[q:], e.stack[:d])
q += d
p.nlog = q
e.count = 0
return true
}
// flushlog tries to flush the current log and switch to the other one.
// flushlog is called from evict, called from add, called from the signal handler,
// so it cannot allocate memory or block. It can try to swap logs with
// the writing goroutine, as explained in the comment at the top of this file.
//go:nowritebarrierrec
func (p *cpuProfile) flushlog() bool {
if !atomic.Cas(&p.handoff, 0, uint32(p.nlog)) {
return false
if cpuprof.numExtra+1+len(stk) < len(cpuprof.extra) {
i := cpuprof.numExtra
cpuprof.extra[i] = uintptr(1 + len(stk))
copy(cpuprof.extra[i+1:], stk)
cpuprof.numExtra += 1 + len(stk)
} else {
cpuprof.lostExtra++
}
notewakeup(&p.wait)
p.toggle = 1 - p.toggle
log := &p.log[p.toggle]
q := 0
if p.lost > 0 {
lostPC := funcPC(lostProfileData)
log[0] = p.lost
log[1] = 1
log[2] = lostPC
q = 3
p.lost = 0
}
p.nlog = q
return true
atomic.Store(&prof.signalLock, 0)
}
// addNonGo is like add, but runs on a non-Go thread.
// It can't do anything that might need a g or an m.
// With this entry point, we don't try to flush the log when evicting an
// old entry. Instead, we just drop the stack trace if we're out of space.
//go:nosplit
//go:nowritebarrierrec
func (p *cpuProfile) addNonGo(pc []uintptr) {
p.addWithFlushlog(pc, func() bool { return false })
}
// getprofile blocks until the next block of profiling data is available
// and returns it as a []byte. It is called from the writing goroutine.
func (p *cpuProfile) getprofile() []byte {
if p == nil {
return nil
}
if p.wholding {
// Release previous log to signal handling side.
// Loop because we are racing against SetCPUProfileRate(0).
for {
n := p.handoff
if n == 0 {
print("runtime: phase error during cpu profile handoff\n")
return nil
}
if n&0x80000000 != 0 {
p.wtoggle = 1 - p.wtoggle
p.wholding = false
p.flushing = true
goto Flush
}
if atomic.Cas(&p.handoff, n, 0) {
break
}
}
p.wtoggle = 1 - p.wtoggle
p.wholding = false
}
if p.flushing {
goto Flush
}
if !p.on && p.handoff == 0 {
return nil
}
// Wait for new log.
notetsleepg(&p.wait, -1)
noteclear(&p.wait)
switch n := p.handoff; {
case n == 0:
print("runtime: phase error during cpu profile wait\n")
return nil
case n == 0x80000000:
p.flushing = true
goto Flush
default:
n &^= 0x80000000
// Return new log to caller.
p.wholding = true
return uintptrBytes(p.log[p.wtoggle][:n])
}
// In flush mode.
// Add is no longer being called. We own the log.
// Also, p.handoff is non-zero, so flushlog will return false.
// Evict the hash table into the log and return it.
Flush:
for i := range p.hash {
b := &p.hash[i]
for j := range b.entry {
e := &b.entry[j]
if e.count > 0 && !p.evict(e, p.flushlog) {
// Filled the log. Stop the loop and return what we've got.
break Flush
}
// addExtra adds the "extra" profiling events,
// queued by addNonGo, to the profile log.
// addExtra is called either from a signal handler on a Go thread
// or from an ordinary goroutine; either way it can use stack
// and has a g. The world may be stopped, though.
func (p *cpuProfile) addExtra() {
// Copy accumulated non-Go profile events.
hdr := [1]uint64{1}
for i := 0; i < p.numExtra; {
p.log.write(nil, 0, hdr[:], p.extra[i+1:i+int(p.extra[i])])
i += int(p.extra[i])
}
p.numExtra = 0
// Report any lost events.
if p.lostExtra > 0 {
hdr := [1]uint64{p.lostExtra}
lostStk := [2]uintptr{
funcPC(_LostExternalCode) + sys.PCQuantum,
funcPC(_ExternalCode) + sys.PCQuantum,
}
cpuprof.log.write(nil, 0, hdr[:], lostStk[:])
}
// Return pending log data.
if p.nlog > 0 {
// Note that we're using toggle now, not wtoggle,
// because we're working on the log directly.
n := p.nlog
p.nlog = 0
return uintptrBytes(p.log[p.toggle][:n])
}
// Made it through the table without finding anything to log.
if !p.eodSent {
// We may not have space to append this to the partial log buf,
// so we always return a new slice for the end-of-data marker.
p.eodSent = true
return uintptrBytes(eod[:])
}
// Finally done. Clean up and return nil.
p.flushing = false
if !atomic.Cas(&p.handoff, p.handoff, 0) {
print("runtime: profile flush racing with something\n")
}
return nil
}
func uintptrBytes(p []uintptr) (ret []byte) {
pp := (*slice)(unsafe.Pointer(&p))
rp := (*slice)(unsafe.Pointer(&ret))
rp.array = pp.array
rp.len = pp.len * int(unsafe.Sizeof(p[0]))
rp.cap = rp.len
return
}
// CPUProfile returns the next chunk of binary CPU profiling stack trace data,
// blocking until data is available. If profiling is turned off and all the profile
// data accumulated while it was on has been returned, CPUProfile returns nil.
// The caller must save the returned data before calling CPUProfile again.
// CPUProfile panics.
// It formerly provided raw access to chunks of
// a pprof-format profile generated by the runtime.
// The details of generating that format have changed,
// so this functionality has been removed.
//
// Most clients should use the runtime/pprof package or
// the testing package's -test.cpuprofile flag instead of calling
// CPUProfile directly.
// Deprecated: use the runtime/pprof package,
// or the handlers in the net/http/pprof package,
// or the testing package's -test.cpuprofile flag instead.
func CPUProfile() []byte {
return cpuprof.getprofile()
panic("CPUProfile no longer available")
}
//go:linkname runtime_pprof_runtime_cyclesPerSecond runtime/pprof.runtime_cyclesPerSecond
func runtime_pprof_runtime_cyclesPerSecond() int64 {
return tickspersecond()
}
// readProfile, provided to runtime/pprof, returns the next chunk of
// binary CPU profiling stack trace data, blocking until data is available.
// If profiling is turned off and all the profile data accumulated while it was
// on has been returned, readProfile returns eof=true.
// The caller must save the returned data and tags before calling readProfile again.
//
//go:linkname runtime_pprof_readProfile runtime/pprof.readProfile
func runtime_pprof_readProfile() ([]uint64, []unsafe.Pointer, bool) {
lock(&cpuprof.lock)
log := cpuprof.log
unlock(&cpuprof.lock)
data, tags, eof := log.read(profBufBlocking)
if len(data) == 0 && eof {
lock(&cpuprof.lock)
cpuprof.log = nil
unlock(&cpuprof.lock)
}
return data, tags, eof
}
......@@ -83,6 +83,7 @@ import (
"sync"
"text/tabwriter"
"time"
"unsafe"
)
// BUG(rsc): Profiles are only as good as the kernel support used to generate them.
......@@ -696,30 +697,33 @@ func StartCPUProfile(w io.Writer) error {
return nil
}
// readProfile, provided by the runtime, returns the next chunk of
// binary CPU profiling stack trace data, blocking until data is available.
// If profiling is turned off and all the profile data accumulated while it was
// on has been returned, readProfile returns eof=true.
// The caller must save the returned data and tags before calling readProfile again.
func readProfile() (data []uint64, tags []unsafe.Pointer, eof bool)
func profileWriter(w io.Writer) {
startTime := time.Now()
// This will buffer the entire profile into buf and then
// translate it into a profile.Profile structure. This will
// create two copies of all the data in the profile in memory.
// TODO(matloob): Convert each chunk of the proto output and
// stream it out instead of converting the entire profile.
var buf bytes.Buffer
b := newProfileBuilder()
var err error
for {
data := runtime.CPUProfile()
if data == nil {
time.Sleep(100 * time.Millisecond)
data, _, eof := readProfile()
if e := b.addCPUData(data); e != nil && err == nil {
err = e
}
if eof {
break
}
buf.Write(data)
}
profile, err := translateCPUProfile(buf.Bytes(), startTime)
p := b.build()
if err != nil {
// The runtime should never produce an invalid or truncated profile.
// It drops records that can't fit into its log buffers.
panic(fmt.Errorf("could not translate binary profile to proto format: %v", err))
panic("runtime/pprof: converting profile: " + err.Error())
}
profile.Write(w)
p.Write(w)
cpu.done <- true
}
......
......@@ -15,50 +15,125 @@ import (
"internal/pprof/profile"
)
// translateCPUProfile parses binary CPU profiling stack trace data
// generated by runtime.CPUProfile() into a profile struct.
func translateCPUProfile(b []byte, startTime time.Time) (*profile.Profile, error) {
const wordSize = unsafe.Sizeof(uintptr(0))
const minRawProfile = 5 * wordSize // Need a minimum of 5 words.
if uintptr(len(b)) < minRawProfile {
return nil, fmt.Errorf("truncated profile")
}
n := int(uintptr(len(b)) / wordSize)
data := ((*[1 << 28]uintptr)(unsafe.Pointer(&b[0])))[:n:n]
period := data[3]
data = data[5:] // skip header
// lostProfileEvent is the function to which lost profiling
// events are attributed.
// (The name shows up in the pprof graphs.)
func lostProfileEvent() { lostProfileEvent() }
// funcPC returns the PC for the func value f.
func funcPC(f interface{}) uintptr {
return *(*[2]*uintptr)(unsafe.Pointer(&f))[1]
}
// profile initialization taken from pprof tool
// A profileBuilder builds a profile.Profile incrementally from a
// stream of profile samples delivered by the runtime.
// TODO(rsc,matloob): In the long term, we'd like to avoid
// storing the entire profile.Profile in memory, instead streaming
// the encoded form out to an underlying writer.
// Even so, this one copy is a step forward from Go 1.8,
// which had two full copies of the data in memory.
type profileBuilder struct {
p *profile.Profile
start time.Time
havePeriod bool
locs map[uint64]*profile.Location
samples map[sampleKey]*profile.Sample
}
// A sampleKey is the key for the map from stack to profile.Sample.
// It is an unbounded array of profile.Location, broken into
// fixed-size chunks. The chunks are chained by the next field,
// which is an interface{} holding a sampleKey so that the default
// Go equality will consider the whole array contents.
// (In contrast, if next were *sampleKey or the interface{} held a
// *sampleKey, equality would only look at the pointer, not the values
// in the next sampleKey in the chain.)
// This is a bit of a hack, but it has the right effect and is expedient.
// At some point we will want to do a better job, so that lookups
// of large stacks need not allocate just to build a key.
type sampleKey struct {
loc [8]*profile.Location
i int
next interface{}
}
// newProfileBuilder returns a new profileBuilder.
// CPU profiling data obtained from the runtime can be added
// by calling b.addCPUData, and then the eventual profile
// can be obtained by calling b.finish.
func newProfileBuilder() *profileBuilder {
start := time.Now()
p := &profile.Profile{
Period: int64(period) * 1000,
PeriodType: &profile.ValueType{Type: "cpu", Unit: "nanoseconds"},
SampleType: []*profile.ValueType{
{Type: "samples", Unit: "count"},
{Type: "cpu", Unit: "nanoseconds"},
},
TimeNanos: int64(startTime.UnixNano()),
DurationNanos: time.Since(startTime).Nanoseconds(),
TimeNanos: int64(start.UnixNano()),
}
return &profileBuilder{
p: p,
start: start,
locs: make(map[uint64]*profile.Location),
samples: make(map[sampleKey]*profile.Sample),
}
}
// addCPUData adds the CPU profiling data to the profile.
// The data must be a whole number of records,
// as delivered by the runtime.
func (b *profileBuilder) addCPUData(data []uint64) error {
p := b.p
if !b.havePeriod {
// first record is period
if len(data) < 3 {
return fmt.Errorf("truncated profile")
}
if data[0] != 3 || data[2] == 0 {
return fmt.Errorf("malformed profile")
}
period := int64(data[2])
p.Period = period * 1000
data = data[3:]
b.havePeriod = true
}
// Parse CPU samples from the profile.
locs := make(map[uint64]*profile.Location)
for len(b) > 0 {
if len(data) < 2 || uintptr(len(data)) < 2+data[1] {
return nil, fmt.Errorf("truncated profile")
// Each sample is 3+n uint64s:
// data[0] = 3+n
// data[1] = time stamp (ignored)
// data[2] = count
// data[3:3+n] = stack
// If the count is 0 and the stack has length 1,
// that's an overflow record inserted by the runtime
// to indicate that stack[0] samples were lost.
// Otherwise the count is usually 1,
// but in a few special cases like lost non-Go samples
// there can be larger counts.
// Because many samples with the same stack arrive,
// we want to deduplicate immediately, which we do
// using the b.samples map.
for len(data) > 0 {
if len(data) < 3 || data[0] > uint64(len(data)) {
return fmt.Errorf("truncated profile")
}
count := data[0]
nstk := data[1]
if uintptr(len(data)) < 2+nstk {
return nil, fmt.Errorf("truncated profile")
if data[0] < 3 {
return fmt.Errorf("malformed profile")
}
stk := data[2 : 2+nstk]
data = data[2+nstk:]
count := data[2]
stk := data[3:data[0]]
data = data[data[0]:]
if count == 0 && nstk == 1 && stk[0] == 0 {
// end of data marker
break
if count == 0 && len(stk) == 1 {
// overflow record
count = uint64(stk[0])
stk = []uint64{
uint64(funcPC(lostProfileEvent)),
}
}
sloc := make([]*profile.Location, len(stk))
skey := sampleKey{}
for i, addr := range stk {
addr := uint64(addr)
// Addresses from stack traces point to the next instruction after
......@@ -67,40 +142,57 @@ func translateCPUProfile(b []byte, startTime time.Time) (*profile.Profile, error
if i > 0 {
addr--
}
loc := locs[addr]
loc := b.locs[addr]
if loc == nil {
loc = &profile.Location{
ID: uint64(len(p.Location) + 1),
Address: addr,
}
locs[addr] = loc
b.locs[addr] = loc
p.Location = append(p.Location, loc)
}
sloc[i] = loc
if skey.i == len(skey.loc) {
skey = sampleKey{next: skey}
}
skey.loc[skey.i] = loc
skey.i++
}
p.Sample = append(p.Sample, &profile.Sample{
Value: []int64{int64(count), int64(count) * int64(p.Period)},
Location: sloc,
})
s := b.samples[skey]
if s == nil {
s = &profile.Sample{
Value: []int64{0, 0},
Location: sloc,
}
b.samples[skey] = s
p.Sample = append(p.Sample, s)
}
s.Value[0] += int64(count)
s.Value[1] += int64(count) * int64(p.Period)
}
return nil
}
// build completes and returns the constructed profile.
func (b *profileBuilder) build() *profile.Profile {
b.p.DurationNanos = time.Since(b.start).Nanoseconds()
if runtime.GOOS == "linux" {
if err := addMappings(p); err != nil {
return nil, err
}
addMappings(b.p)
}
symbolize(p)
return p, nil
symbolize(b.p)
return b.p
}
func addMappings(p *profile.Profile) error {
// addMappings adds information from /proc/self/maps
// to the profile if possible.
func addMappings(p *profile.Profile) {
// Parse memory map from /proc/self/maps
f, err := os.Open("/proc/self/maps")
if err != nil {
return err
return
}
defer f.Close()
return p.ParseMemoryMap(f)
p.ParseMemoryMap(f)
f.Close()
}
type function interface {
......
......@@ -6,80 +6,50 @@ package pprof
import (
"bytes"
"fmt"
"encoding/json"
"internal/pprof/profile"
"io/ioutil"
"reflect"
"runtime"
"testing"
"time"
"unsafe"
)
// Helper function to initialize empty cpu profile with sampling period provided.
func createEmptyProfileWithPeriod(t *testing.T, periodMs uint64) bytes.Buffer {
// Mock the sample header produced by cpu profiler. Write a sample
// period of 2000 microseconds, followed by no samples.
buf := new(bytes.Buffer)
// Profile header is as follows:
// The first, third and fifth words are 0. The second word is 3.
// The fourth word is the period.
// EOD marker:
// The sixth word -- count is initialized to 0 above.
// The code below sets the seventh word -- nstk to 1
// The eighth word -- addr is initialized to 0 above.
words := []int{0, 3, 0, int(periodMs), 0, 0, 1, 0}
n := int(unsafe.Sizeof(0)) * len(words)
data := ((*[1 << 29]byte)(unsafe.Pointer(&words[0])))[:n:n]
if _, err := buf.Write(data); err != nil {
t.Fatalf("createEmptyProfileWithPeriod failed: %v", err)
}
return *buf
// translateCPUProfile parses binary CPU profiling stack trace data
// generated by runtime.CPUProfile() into a profile struct.
// This is only used for testing. Real conversions stream the
// data into the profileBuilder as it becomes available.
func translateCPUProfile(data []uint64) (*profile.Profile, error) {
b := newProfileBuilder()
if err := b.addCPUData(data); err != nil {
return nil, err
}
return b.build(), nil
}
// Helper function to initialize cpu profile with two sample values.
func createProfileWithTwoSamples(t *testing.T, periodMs uintptr, count1 uintptr, count2 uintptr,
address1 uintptr, address2 uintptr) bytes.Buffer {
// Mock the sample header produced by cpu profiler. Write a sample
// period of 2000 microseconds, followed by no samples.
buf := new(bytes.Buffer)
words := []uintptr{0, 3, 0, uintptr(periodMs), 0, uintptr(count1), 2,
uintptr(address1), uintptr(address1 + 2),
uintptr(count2), 2, uintptr(address2), uintptr(address2 + 2),
0, 1, 0}
for _, n := range words {
var err error
switch unsafe.Sizeof(int(0)) {
case 8:
_, err = buf.Write((*[8]byte)(unsafe.Pointer(&n))[:8:8])
case 4:
_, err = buf.Write((*[4]byte)(unsafe.Pointer(&n))[:4:4])
}
if err != nil {
t.Fatalf("createProfileWithTwoSamples failed: %v", err)
}
}
return *buf
// fmtJSON returns a pretty-printed JSON form for x.
// It works reasonbly well for printing protocol-buffer
// data structures like profile.Profile.
func fmtJSON(x interface{}) string {
js, _ := json.MarshalIndent(x, "", "\t")
return string(js)
}
// Tests translateCPUProfile parses correct sampling period in an otherwise empty cpu profile.
func TestTranlateCPUProfileSamplingPeriod(t *testing.T) {
func TestConvertCPUProfileEmpty(t *testing.T) {
// A test server with mock cpu profile data.
var buf bytes.Buffer
startTime := time.Now()
b := createEmptyProfileWithPeriod(t, 2000)
p, err := translateCPUProfile(b.Bytes(), startTime)
b := []uint64{3, 0, 2000} // empty profile with 2000ms sample period
p, err := translateCPUProfile(b)
if err != nil {
t.Fatalf("translate failed: %v", err)
t.Fatalf("translateCPUProfile: %v", err)
}
if err := p.Write(&buf); err != nil {
t.Fatalf("write failed: %v", err)
t.Fatalf("writing profile: %v", err)
}
p, err = profile.Parse(&buf)
if err != nil {
t.Fatalf("Could not parse Profile profile: %v", err)
t.Fatalf("profile.Parse: %v", err)
}
// Expected PeriodType and SampleType.
......@@ -94,79 +64,89 @@ func TestTranlateCPUProfileSamplingPeriod(t *testing.T) {
}
}
func getSampleAsString(sample []*profile.Sample) string {
var str string
for _, x := range sample {
for _, y := range x.Location {
if y.Mapping != nil {
str += fmt.Sprintf("Mapping:%v\n", *y.Mapping)
}
str += fmt.Sprintf("Location:%v\n", y)
func f1() { f1() }
func f2() { f2() }
// testPCs returns two PCs and two corresponding memory mappings
// to use in test profiles.
func testPCs(t *testing.T) (addr1, addr2 uint64, map1, map2 *profile.Mapping) {
if runtime.GOOS == "linux" {
// Figure out two addresses from /proc/self/maps.
mmap, err := ioutil.ReadFile("/proc/self/maps")
if err != nil {
t.Fatal(err)
}
str += fmt.Sprintf("Sample:%v\n", *x)
}
return str
mprof := &profile.Profile{}
if err = mprof.ParseMemoryMap(bytes.NewReader(mmap)); err != nil {
t.Fatalf("parsing /proc/self/maps: %v", err)
}
if len(mprof.Mapping) < 2 {
// It is possible for a binary to only have 1 executable
// region of memory.
t.Skipf("need 2 or more mappings, got %v", len(mprof.Mapping))
}
addr1 = mprof.Mapping[0].Start
map1 = mprof.Mapping[0]
addr2 = mprof.Mapping[1].Start
map2 = mprof.Mapping[1]
} else {
addr1 = uint64(funcPC(f1))
addr2 = uint64(funcPC(f2))
}
return
}
// Tests translateCPUProfile parses a cpu profile with sample values present.
func TestTranslateCPUProfileWithSamples(t *testing.T) {
if runtime.GOOS != "linux" {
t.Skip("test requires a system with /proc/self/maps")
}
// Figure out two addresses from /proc/self/maps.
mmap, err := ioutil.ReadFile("/proc/self/maps")
if err != nil {
t.Fatal("Cannot read /proc/self/maps")
}
rd := bytes.NewReader(mmap)
mprof := &profile.Profile{}
if err = mprof.ParseMemoryMap(rd); err != nil {
t.Fatalf("Cannot parse /proc/self/maps")
}
if len(mprof.Mapping) < 2 {
// It is possible for a binary to only have 1 executable
// region of memory.
t.Skipf("need 2 or more mappings, got %v", len(mprof.Mapping))
}
address1 := mprof.Mapping[0].Start
address2 := mprof.Mapping[1].Start
// A test server with mock cpu profile data.
startTime := time.Now()
b := createProfileWithTwoSamples(t, 2000, 20, 40, uintptr(address1), uintptr(address2))
p, err := translateCPUProfile(b.Bytes(), startTime)
func TestConvertCPUProfile(t *testing.T) {
addr1, addr2, map1, map2 := testPCs(t)
b := []uint64{
3, 0, 2000, // periodMs = 2000
5, 0, 10, uint64(addr1), uint64(addr1 + 2), // 10 samples in addr1
5, 0, 40, uint64(addr2), uint64(addr2 + 2), // 40 samples in addr2
5, 0, 10, uint64(addr1), uint64(addr1 + 2), // 10 samples in addr1
}
p, err := translateCPUProfile(b)
if err != nil {
t.Fatalf("Could not parse Profile profile: %v", err)
t.Fatalf("translating profile: %v", err)
}
// Expected PeriodType, SampleType and Sample.
expectedPeriodType := &profile.ValueType{Type: "cpu", Unit: "nanoseconds"}
expectedSampleType := []*profile.ValueType{
period := int64(2000 * 1000)
periodType := &profile.ValueType{Type: "cpu", Unit: "nanoseconds"}
sampleType := []*profile.ValueType{
{Type: "samples", Unit: "count"},
{Type: "cpu", Unit: "nanoseconds"},
}
expectedSample := []*profile.Sample{
samples := []*profile.Sample{
{Value: []int64{20, 20 * 2000 * 1000}, Location: []*profile.Location{
{ID: 1, Mapping: mprof.Mapping[0], Address: address1},
{ID: 2, Mapping: mprof.Mapping[0], Address: address1 + 1},
{ID: 1, Mapping: map1, Address: addr1},
{ID: 2, Mapping: map1, Address: addr1 + 1},
}},
{Value: []int64{40, 40 * 2000 * 1000}, Location: []*profile.Location{
{ID: 3, Mapping: mprof.Mapping[1], Address: address2},
{ID: 4, Mapping: mprof.Mapping[1], Address: address2 + 1},
{ID: 3, Mapping: map2, Address: addr2},
{ID: 4, Mapping: map2, Address: addr2 + 1},
}},
}
if p.Period != 2000*1000 {
t.Fatalf("Sampling periods do not match")
checkProfile(t, p, period, periodType, sampleType, samples)
}
func checkProfile(t *testing.T, p *profile.Profile, period int64, periodType *profile.ValueType, sampleType []*profile.ValueType, samples []*profile.Sample) {
if p.Period != period {
t.Fatalf("p.Period = %d, want %d", p.Period, period)
}
if !reflect.DeepEqual(p.PeriodType, periodType) {
t.Fatalf("p.PeriodType = %v\nwant = %v", fmtJSON(p.PeriodType), fmtJSON(periodType))
}
if !reflect.DeepEqual(p.PeriodType, expectedPeriodType) {
t.Fatalf("Period types do not match")
if !reflect.DeepEqual(p.SampleType, sampleType) {
t.Fatalf("p.SampleType = %v\nwant = %v", fmtJSON(p.SampleType), fmtJSON(sampleType))
}
if !reflect.DeepEqual(p.SampleType, expectedSampleType) {
t.Fatalf("Sample types do not match")
// Clear line info since it is not in the expected samples.
// If we used f1 and f2 above, then the samples will have line info.
for _, s := range p.Sample {
for _, l := range s.Location {
l.Line = nil
}
}
if !reflect.DeepEqual(p.Sample, expectedSample) {
t.Fatalf("Samples do not match: Expected: %v, Got:%v", getSampleAsString(expectedSample),
getSampleAsString(p.Sample))
if !reflect.DeepEqual(p.Sample, samples) {
t.Fatalf("p.Sample = %v\nwant = %v", fmtJSON(p.Sample), fmtJSON(samples))
}
}
......@@ -179,7 +159,7 @@ type fakeFunc struct {
func (f *fakeFunc) Name() string {
return f.name
}
func (f *fakeFunc) FileLine(_ uintptr) (string, int) {
func (f *fakeFunc) FileLine(uintptr) (string, int) {
return f.file, f.lineno
}
......
......@@ -7,98 +7,54 @@ package pprof
import (
"bytes"
"internal/pprof/profile"
"io/ioutil"
"reflect"
"runtime"
"testing"
"time"
)
// TestSampledHeapAllocProfile tests encoding of a memory profile from
// runtime.MemProfileRecord data.
func TestSampledHeapAllocProfile(t *testing.T) {
if runtime.GOOS != "linux" {
t.Skip("Test requires a system with /proc/self/maps")
}
// Figure out two addresses from /proc/self/maps.
mmap, err := ioutil.ReadFile("/proc/self/maps")
if err != nil {
t.Fatal("Cannot read /proc/self/maps")
}
rd := bytes.NewReader(mmap)
mprof := &profile.Profile{}
if err = mprof.ParseMemoryMap(rd); err != nil {
t.Fatalf("Cannot parse /proc/self/maps")
}
if len(mprof.Mapping) < 2 {
// It is possible for a binary to only have 1 executable
// region of memory.
t.Skipf("need 2 or more mappings, got %v", len(mprof.Mapping))
}
address1 := mprof.Mapping[0].Start
address2 := mprof.Mapping[1].Start
func TestConvertMemProfile(t *testing.T) {
addr1, addr2, map1, map2 := testPCs(t)
var buf bytes.Buffer
a1, a2 := uintptr(addr1), uintptr(addr2)
rate := int64(512 * 1024)
rec := []runtime.MemProfileRecord{
{AllocBytes: 4096, FreeBytes: 1024, AllocObjects: 4, FreeObjects: 1, Stack0: [32]uintptr{a1, a2}},
{AllocBytes: 512 * 1024, FreeBytes: 0, AllocObjects: 1, FreeObjects: 0, Stack0: [32]uintptr{a2 + 1, a2 + 2}},
{AllocBytes: 512 * 1024, FreeBytes: 512 * 1024, AllocObjects: 1, FreeObjects: 1, Stack0: [32]uintptr{a1 + 1, a1 + 2, a2 + 3}},
}
rec, rate := testMemRecords(address1, address2)
p := encodeMemProfile(rec, rate, time.Now())
if err := p.Write(&buf); err != nil {
t.Fatalf("Failed to write profile: %v", err)
t.Fatalf("writing profile: %v", err)
}
p, err = profile.Parse(&buf)
p, err := profile.Parse(&buf)
if err != nil {
t.Fatalf("Could not parse Profile profile: %v", err)
t.Fatalf("profile.Parse: %v", err)
}
// Expected PeriodType, SampleType and Sample.
expectedPeriodType := &profile.ValueType{Type: "space", Unit: "bytes"}
expectedSampleType := []*profile.ValueType{
periodType := &profile.ValueType{Type: "space", Unit: "bytes"}
sampleType := []*profile.ValueType{
{Type: "alloc_objects", Unit: "count"},
{Type: "alloc_space", Unit: "bytes"},
{Type: "inuse_objects", Unit: "count"},
{Type: "inuse_space", Unit: "bytes"},
}
// Expected samples, with values unsampled according to the profiling rate.
expectedSample := []*profile.Sample{
samples := []*profile.Sample{
{Value: []int64{2050, 2099200, 1537, 1574400}, Location: []*profile.Location{
{ID: 1, Mapping: mprof.Mapping[0], Address: address1},
{ID: 2, Mapping: mprof.Mapping[1], Address: address2},
{ID: 1, Mapping: map1, Address: addr1},
{ID: 2, Mapping: map2, Address: addr2},
}},
{Value: []int64{1, 829411, 1, 829411}, Location: []*profile.Location{
{ID: 3, Mapping: mprof.Mapping[1], Address: address2 + 1},
{ID: 4, Mapping: mprof.Mapping[1], Address: address2 + 2},
{ID: 3, Mapping: map2, Address: addr2 + 1},
{ID: 4, Mapping: map2, Address: addr2 + 2},
}},
{Value: []int64{1, 829411, 0, 0}, Location: []*profile.Location{
{ID: 5, Mapping: mprof.Mapping[0], Address: address1 + 1},
{ID: 6, Mapping: mprof.Mapping[0], Address: address1 + 2},
{ID: 7, Mapping: mprof.Mapping[1], Address: address2 + 3},
{ID: 5, Mapping: map1, Address: addr1 + 1},
{ID: 6, Mapping: map1, Address: addr1 + 2},
{ID: 7, Mapping: map2, Address: addr2 + 3},
}},
}
if p.Period != 512*1024 {
t.Fatalf("Sampling periods do not match")
}
if !reflect.DeepEqual(p.PeriodType, expectedPeriodType) {
t.Fatalf("Period types do not match")
}
if !reflect.DeepEqual(p.SampleType, expectedSampleType) {
t.Fatalf("Sample types do not match")
}
if !reflect.DeepEqual(p.Sample, expectedSample) {
t.Fatalf("Samples do not match: Expected: %v, Got:%v", getSampleAsString(expectedSample),
getSampleAsString(p.Sample))
}
}
func testMemRecords(a1, a2 uint64) ([]runtime.MemProfileRecord, int64) {
addr1, addr2 := uintptr(a1), uintptr(a2)
rate := int64(512 * 1024)
rec := []runtime.MemProfileRecord{
{AllocBytes: 4096, FreeBytes: 1024, AllocObjects: 4, FreeObjects: 1, Stack0: [32]uintptr{addr1, addr2}},
{AllocBytes: 512 * 1024, FreeBytes: 0, AllocObjects: 1, FreeObjects: 0, Stack0: [32]uintptr{addr2 + 1, addr2 + 2}},
{AllocBytes: 512 * 1024, FreeBytes: 512 * 1024, AllocObjects: 1, FreeObjects: 1, Stack0: [32]uintptr{addr1 + 1, addr1 + 2, addr2 + 3}},
}
return rec, rate
checkProfile(t, p, rate, periodType, sampleType, samples)
}
......@@ -3137,13 +3137,14 @@ func mcount() int32 {
}
var prof struct {
lock uint32
hz int32
signalLock uint32
hz int32
}
func _System() { _System() }
func _ExternalCode() { _ExternalCode() }
func _GC() { _GC() }
func _System() { _System() }
func _ExternalCode() { _ExternalCode() }
func _LostExternalCode() { _LostExternalCode() }
func _GC() { _GC() }
// Called if we receive a SIGPROF signal.
// Called by the signal handler, may run during STW.
......@@ -3279,14 +3280,7 @@ func sigprof(pc, sp, lr uintptr, gp *g, mp *m) {
}
if prof.hz != 0 {
// Simple cas-lock to coordinate with setcpuprofilerate.
for !atomic.Cas(&prof.lock, 0, 1) {
osyield()
}
if prof.hz != 0 {
cpuprof.add(stk[:n])
}
atomic.Store(&prof.lock, 0)
cpuprof.add(gp, stk[:n])
}
getg().m.mallocing--
}
......@@ -3309,15 +3303,7 @@ func sigprofNonGo() {
for n < len(sigprofCallers) && sigprofCallers[n] != 0 {
n++
}
// Simple cas-lock to coordinate with setcpuprofilerate.
for !atomic.Cas(&prof.lock, 0, 1) {
osyield()
}
if prof.hz != 0 {
cpuprof.addNonGo(sigprofCallers[:n])
}
atomic.Store(&prof.lock, 0)
cpuprof.addNonGo(sigprofCallers[:n])
}
atomic.Store(&sigprofCallersUse, 0)
......@@ -3330,19 +3316,11 @@ func sigprofNonGo() {
//go:nowritebarrierrec
func sigprofNonGoPC(pc uintptr) {
if prof.hz != 0 {
pc := []uintptr{
stk := []uintptr{
pc,
funcPC(_ExternalCode) + sys.PCQuantum,
}
// Simple cas-lock to coordinate with setcpuprofilerate.
for !atomic.Cas(&prof.lock, 0, 1) {
osyield()
}
if prof.hz != 0 {
cpuprof.addNonGo(pc)
}
atomic.Store(&prof.lock, 0)
cpuprof.addNonGo(stk)
}
}
......@@ -3370,8 +3348,9 @@ func setsSP(pc uintptr) bool {
return false
}
// Arrange to call fn with a traceback hz times a second.
func setcpuprofilerate_m(hz int32) {
// setcpuprofilerate sets the CPU profiling rate to hz times per second.
// If hz <= 0, setcpuprofilerate turns off CPU profiling.
func setcpuprofilerate(hz int32) {
// Force sane arguments.
if hz < 0 {
hz = 0
......@@ -3387,14 +3366,14 @@ func setcpuprofilerate_m(hz int32) {
// it would deadlock.
setThreadCPUProfiler(0)
for !atomic.Cas(&prof.lock, 0, 1) {
for !atomic.Cas(&prof.signalLock, 0, 1) {
osyield()
}
if prof.hz != hz {
setProcessCPUProfiler(hz)
prof.hz = hz
}
atomic.Store(&prof.lock, 0)
atomic.Store(&prof.signalLock, 0)
lock(&sched.lock)
sched.profilehz = hz
......
Markdown is supported
0% or
You are about to add 0 people to the discussion. Proceed with caution.
Finish editing this message first!
Please register or to comment