internal/trace/v2: support old trace format

Add support for traces from Go 1.11–1.19 by converting old traces to the
Go 1.22 format on the fly.

We import Gotraceui's trace parser, which is an optimized parser based
on Go 1.19's internal/trace package, and further modify it for the needs
of the conversion process.

With the optimized parser, loading old traces using the new API is twice
as fast and uses less total memory than 'go tool trace' did in older
versions.

The new parser does not, however, support traces from versions older
than 1.11.

This commit does not update cmd/trace to use the new API for old traces.

Change-Id: If9380aa515e29445ff624274d1760ee945ca4816
Reviewed-on: https://go-review.googlesource.com/c/go/+/557356
Reviewed-by: Michael Knyszek <mknyszek@google.com>
Auto-Submit: Michael Knyszek <mknyszek@google.com>
Reviewed-by: Cherry Mui <cherryyz@google.com>
LUCI-TryBot-Result: Go LUCI <golang-scoped@luci-project-accounts.iam.gserviceaccount.com>
This commit is contained in:
Dominik Honnef 2024-01-20 17:37:50 +01:00 committed by Gopher Robot
parent 508bb17edd
commit 6133c1e4e2
24 changed files with 2608 additions and 21 deletions

View file

@ -632,7 +632,10 @@ var depsRules = `
FMT, encoding/binary, internal/trace/v2/version
< internal/trace/v2/raw;
FMT, encoding/binary, internal/trace/v2/version
FMT, internal/trace/v2/event, internal/trace/v2/version, io, sort, encoding/binary
< internal/trace/v2/internal/oldtrace;
FMT, encoding/binary, internal/trace/v2/version, internal/trace/v2/internal/oldtrace
< internal/trace/v2;
regexp, internal/trace/v2, internal/trace/v2/raw, internal/txtar

View file

@ -41,7 +41,7 @@ const (
EvGoSyscallBegin // syscall enter [timestamp, P seq, stack ID]
EvGoSyscallEnd // syscall exit [timestamp]
EvGoSyscallEndBlocked // syscall exit and it blocked at some point [timestamp]
EvGoStatus // goroutine status at the start of a generation [timestamp, goroutine ID, status]
EvGoStatus // goroutine status at the start of a generation [timestamp, goroutine ID, thread ID, status]
// STW.
EvSTWBegin // STW start [timestamp, kind]
@ -66,7 +66,7 @@ const (
EvUserTaskEnd // end of a task [timestamp, internal task ID, stack ID]
EvUserRegionBegin // trace.{Start,With}Region [timestamp, internal task ID, name string ID, stack ID]
EvUserRegionEnd // trace.{End,With}Region [timestamp, internal task ID, name string ID, stack ID]
EvUserLog // trace.Log [timestamp, internal task ID, key string ID, stack, value string ID]
EvUserLog // trace.Log [timestamp, internal task ID, key string ID, value string ID, stack]
)
// EventString returns the name of a Go 1.22 event.
@ -108,7 +108,7 @@ var specs = [...]event.Spec{
},
EvCPUSample: event.Spec{
Name: "CPUSample",
Args: []string{"time", "p", "g", "m", "stack"},
Args: []string{"time", "m", "p", "g", "stack"},
// N.B. There's clearly a timestamp here, but these Events
// are special in that they don't appear in the regular
// M streams.

View file

@ -328,7 +328,7 @@ func addStacks(stackTable *dataTable[stackID, stack], pcs map[uint64]frame, b ba
// sample contained therein to the provided samples list.
func addCPUSamples(samples []cpuSample, b batch) ([]cpuSample, error) {
if !b.isCPUSamplesBatch() {
return nil, fmt.Errorf("internal error: addStrings called on non-string batch")
return nil, fmt.Errorf("internal error: addCPUSamples called on non-CPU-sample batch")
}
r := bytes.NewReader(b.data)
hdr, err := r.ReadByte() // Consume the EvCPUSamples byte.

View file

@ -0,0 +1,186 @@
// Copyright 2024 The Go Authors. All rights reserved.
// Use of this source code is governed by a BSD-style
// license that can be found in the LICENSE file.
package oldtrace
import "errors"
type orderEvent struct {
ev Event
proc *proc
}
type gStatus int
type gState struct {
seq uint64
status gStatus
}
const (
gDead gStatus = iota
gRunnable
gRunning
gWaiting
unordered = ^uint64(0)
garbage = ^uint64(0) - 1
noseq = ^uint64(0)
seqinc = ^uint64(0) - 1
)
// stateTransition returns goroutine state (sequence and status) when the event
// becomes ready for merging (init) and the goroutine state after the event (next).
func stateTransition(ev *Event) (g uint64, init, next gState) {
// Note that we have an explicit return in each case, as that produces slightly better code (tested on Go 1.19).
switch ev.Type {
case EvGoCreate:
g = ev.Args[0]
init = gState{0, gDead}
next = gState{1, gRunnable}
return
case EvGoWaiting, EvGoInSyscall:
g = ev.G
init = gState{1, gRunnable}
next = gState{2, gWaiting}
return
case EvGoStart, EvGoStartLabel:
g = ev.G
init = gState{ev.Args[1], gRunnable}
next = gState{ev.Args[1] + 1, gRunning}
return
case EvGoStartLocal:
// noseq means that this event is ready for merging as soon as
// frontier reaches it (EvGoStartLocal is emitted on the same P
// as the corresponding EvGoCreate/EvGoUnblock, and thus the latter
// is already merged).
// seqinc is a stub for cases when event increments g sequence,
// but since we don't know current seq we also don't know next seq.
g = ev.G
init = gState{noseq, gRunnable}
next = gState{seqinc, gRunning}
return
case EvGoBlock, EvGoBlockSend, EvGoBlockRecv, EvGoBlockSelect,
EvGoBlockSync, EvGoBlockCond, EvGoBlockNet, EvGoSleep,
EvGoSysBlock, EvGoBlockGC:
g = ev.G
init = gState{noseq, gRunning}
next = gState{noseq, gWaiting}
return
case EvGoSched, EvGoPreempt:
g = ev.G
init = gState{noseq, gRunning}
next = gState{noseq, gRunnable}
return
case EvGoUnblock, EvGoSysExit:
g = ev.Args[0]
init = gState{ev.Args[1], gWaiting}
next = gState{ev.Args[1] + 1, gRunnable}
return
case EvGoUnblockLocal, EvGoSysExitLocal:
g = ev.Args[0]
init = gState{noseq, gWaiting}
next = gState{seqinc, gRunnable}
return
case EvGCStart:
g = garbage
init = gState{ev.Args[0], gDead}
next = gState{ev.Args[0] + 1, gDead}
return
default:
// no ordering requirements
g = unordered
return
}
}
func transitionReady(g uint64, curr, init gState) bool {
return g == unordered || (init.seq == noseq || init.seq == curr.seq) && init.status == curr.status
}
func transition(gs map[uint64]gState, g uint64, init, next gState) error {
if g == unordered {
return nil
}
curr := gs[g]
if !transitionReady(g, curr, init) {
// See comment near the call to transition, where we're building the frontier, for details on how this could
// possibly happen.
return errors.New("encountered impossible goroutine state transition")
}
switch next.seq {
case noseq:
next.seq = curr.seq
case seqinc:
next.seq = curr.seq + 1
}
gs[g] = next
return nil
}
type orderEventList []orderEvent
func (l *orderEventList) Less(i, j int) bool {
return (*l)[i].ev.Ts < (*l)[j].ev.Ts
}
type eventList []Event
func (l *eventList) Len() int {
return len(*l)
}
func (l *eventList) Less(i, j int) bool {
return (*l)[i].Ts < (*l)[j].Ts
}
func (l *eventList) Swap(i, j int) {
(*l)[i], (*l)[j] = (*l)[j], (*l)[i]
}
func (h *orderEventList) Push(x orderEvent) {
*h = append(*h, x)
heapUp(h, len(*h)-1)
}
func (h *orderEventList) Pop() orderEvent {
n := len(*h) - 1
(*h)[0], (*h)[n] = (*h)[n], (*h)[0]
heapDown(h, 0, n)
x := (*h)[len(*h)-1]
*h = (*h)[:len(*h)-1]
return x
}
func heapUp(h *orderEventList, j int) {
for {
i := (j - 1) / 2 // parent
if i == j || !h.Less(j, i) {
break
}
(*h)[i], (*h)[j] = (*h)[j], (*h)[i]
j = i
}
}
func heapDown(h *orderEventList, i0, n int) bool {
i := i0
for {
j1 := 2*i + 1
if j1 >= n || j1 < 0 { // j1 < 0 after int overflow
break
}
j := j1 // left child
if j2 := j1 + 1; j2 < n && h.Less(j2, j1) {
j = j2 // = 2*i + 2 // right child
}
if !h.Less(j, i) {
break
}
(*h)[i], (*h)[j] = (*h)[j], (*h)[i]
i = j
}
return i > i0
}

File diff suppressed because it is too large Load diff

View file

@ -0,0 +1,151 @@
// Copyright 2015 The Go Authors. All rights reserved.
// Use of this source code is governed by a BSD-style
// license that can be found in the LICENSE file.
package oldtrace
import (
"bytes"
"internal/trace/v2/version"
"os"
"path/filepath"
"strings"
"testing"
)
func TestCorruptedInputs(t *testing.T) {
// These inputs crashed parser previously.
tests := []string{
"gotrace\x00\x020",
"gotrace\x00Q00\x020",
"gotrace\x00T00\x020",
"gotrace\x00\xc3\x0200",
"go 1.5 trace\x00\x00\x00\x00\x020",
"go 1.5 trace\x00\x00\x00\x00Q00\x020",
"go 1.5 trace\x00\x00\x00\x00T00\x020",
"go 1.5 trace\x00\x00\x00\x00\xc3\x0200",
}
for _, data := range tests {
res, err := Parse(strings.NewReader(data), 5)
if err == nil || res.Events.Len() != 0 || res.Stacks != nil {
t.Fatalf("no error on input: %q", data)
}
}
}
func TestParseCanned(t *testing.T) {
files, err := os.ReadDir("./testdata")
if err != nil {
t.Fatalf("failed to read ./testdata: %v", err)
}
for _, f := range files {
info, err := f.Info()
if err != nil {
t.Fatal(err)
}
if testing.Short() && info.Size() > 10000 {
continue
}
name := filepath.Join("./testdata", f.Name())
data, err := os.ReadFile(name)
if err != nil {
t.Fatal(err)
}
r := bytes.NewReader(data)
v, err := version.ReadHeader(r)
if err != nil {
t.Errorf("failed to parse good trace %s: %s", f.Name(), err)
}
trace, err := Parse(r, v)
switch {
case strings.HasSuffix(f.Name(), "_good"):
if err != nil {
t.Errorf("failed to parse good trace %v: %v", f.Name(), err)
}
checkTrace(t, int(v), trace)
case strings.HasSuffix(f.Name(), "_unordered"):
if err != ErrTimeOrder {
t.Errorf("unordered trace is not detected %v: %v", f.Name(), err)
}
default:
t.Errorf("unknown input file suffix: %v", f.Name())
}
}
}
// checkTrace walks over a good trace and makes a bunch of additional checks
// that may not cause the parser to outright fail.
func checkTrace(t *testing.T, ver int, res Trace) {
for i := 0; i < res.Events.Len(); i++ {
ev := res.Events.Ptr(i)
if ver >= 21 {
if ev.Type == EvSTWStart && res.Strings[ev.Args[0]] == "unknown" {
t.Errorf("found unknown STW event; update stwReasonStrings?")
}
}
}
}
func TestBuckets(t *testing.T) {
var evs Events
const N = eventsBucketSize*3 + 123
for i := 0; i < N; i++ {
evs.append(Event{Ts: Timestamp(i)})
}
if n := len(evs.buckets); n != 4 {
t.Fatalf("got %d buckets, want %d", n, 4)
}
if n := evs.Len(); n != N {
t.Fatalf("got %d events, want %d", n, N)
}
var n int
evs.All()(func(ev *Event) bool {
n++
return true
})
if n != N {
t.Fatalf("iterated over %d events, expected %d", n, N)
}
const consume = eventsBucketSize + 50
for i := 0; i < consume; i++ {
if _, ok := evs.Pop(); !ok {
t.Fatalf("iteration failed after %d events", i)
}
}
if evs.buckets[0] != nil {
t.Fatalf("expected first bucket to have been dropped")
}
for i, b := range evs.buckets[1:] {
if b == nil {
t.Fatalf("expected bucket %d to be non-nil", i+1)
}
}
if n := evs.Len(); n != N-consume {
t.Fatalf("got %d remaining elements, expected %d", n, N-consume)
}
ev := evs.Ptr(0)
if ev.Ts != consume {
t.Fatalf("got event %d, expected %d", int(ev.Ts), consume)
}
for {
_, ok := evs.Pop()
if !ok {
break
}
}
for i, b := range evs.buckets {
if b != nil {
t.Fatalf("expected bucket %d to be nil", i)
}
}
}

Binary file not shown.

Binary file not shown.

Binary file not shown.

Binary file not shown.

Binary file not shown.

View file

@ -0,0 +1,567 @@
// Copyright 2024 The Go Authors. All rights reserved.
// Use of this source code is governed by a BSD-style
// license that can be found in the LICENSE file.
// This file implements conversion from old (Go 1.11Go 1.21) traces to the Go
// 1.22 format.
//
// Most events have direct equivalents in 1.22, at worst requiring arguments to
// be reordered. Some events, such as GoWaiting need to look ahead for follow-up
// events to determine the correct translation. GoSyscall, which is an
// instantaneous event, gets turned into a 1 ns long pair of
// GoSyscallStart+GoSyscallEnd, unless we observe a GoSysBlock, in which case we
// emit a GoSyscallStart+GoSyscallEndBlocked pair with the correct duration
// (i.e. starting at the original GoSyscall).
//
// The resulting trace treats the old trace as a single, large generation,
// sharing a single evTable for all events.
//
// We use a new (compared to what was used for 'go tool trace' in earlier
// versions of Go) parser for old traces that is optimized for speed, low memory
// usage, and minimal GC pressure. It allocates events in batches so that even
// though we have to load the entire trace into memory, the conversion process
// shouldn't result in a doubling of memory usage, even if all converted events
// are kept alive, as we free batches once we're done with them.
//
// The conversion process is lossless.
package trace
import (
"errors"
"fmt"
"internal/trace/v2/event"
"internal/trace/v2/event/go122"
"internal/trace/v2/internal/oldtrace"
"io"
)
type oldTraceConverter struct {
trace oldtrace.Trace
evt *evTable
preInit bool
createdPreInit map[GoID]struct{}
events oldtrace.Events
extra []Event
extraArr [3]Event
tasks map[TaskID]taskState
seenProcs map[ProcID]struct{}
lastTs Time
procMs map[ProcID]ThreadID
lastStwReason uint64
inlineToStringID []uint64
builtinToStringID []uint64
}
const (
// Block reasons
sForever = iota
sPreempted
sGosched
sSleep
sChanSend
sChanRecv
sNetwork
sSync
sSyncCond
sSelect
sEmpty
sMarkAssistWait
// STW kinds
sSTWUnknown
sSTWGCMarkTermination
sSTWGCSweepTermination
sSTWWriteHeapDump
sSTWGoroutineProfile
sSTWGoroutineProfileCleanup
sSTWAllGoroutinesStackTrace
sSTWReadMemStats
sSTWAllThreadsSyscall
sSTWGOMAXPROCS
sSTWStartTrace
sSTWStopTrace
sSTWCountPagesInUse
sSTWReadMetricsSlow
sSTWReadMemStatsSlow
sSTWPageCachePagesLeaked
sSTWResetDebugLog
sLast
)
func (it *oldTraceConverter) init(pr oldtrace.Trace) error {
it.trace = pr
it.preInit = true
it.createdPreInit = make(map[GoID]struct{})
it.evt = &evTable{pcs: make(map[uint64]frame)}
it.events = pr.Events
it.extra = it.extraArr[:0]
it.tasks = make(map[TaskID]taskState)
it.seenProcs = make(map[ProcID]struct{})
it.procMs = make(map[ProcID]ThreadID)
it.lastTs = -1
evt := it.evt
// Convert from oldtracer's Strings map to our dataTable.
var max uint64
for id, s := range pr.Strings {
evt.strings.insert(stringID(id), s)
if id > max {
max = id
}
}
pr.Strings = nil
// Add all strings used for UserLog. In the old trace format, these were
// stored inline and didn't have IDs. We generate IDs for them.
if max+uint64(len(pr.InlineStrings)) < max {
return errors.New("trace contains too many strings")
}
var addErr error
add := func(id stringID, s string) {
if err := evt.strings.insert(id, s); err != nil && addErr == nil {
addErr = err
}
}
for id, s := range pr.InlineStrings {
nid := max + 1 + uint64(id)
it.inlineToStringID = append(it.inlineToStringID, nid)
add(stringID(nid), s)
}
max += uint64(len(pr.InlineStrings))
pr.InlineStrings = nil
// Add strings that the converter emits explicitly.
if max+uint64(sLast) < max {
return errors.New("trace contains too many strings")
}
it.builtinToStringID = make([]uint64, sLast)
addBuiltin := func(c int, s string) {
nid := max + 1 + uint64(c)
it.builtinToStringID[c] = nid
add(stringID(nid), s)
}
addBuiltin(sForever, "forever")
addBuiltin(sPreempted, "preempted")
addBuiltin(sGosched, "runtime.Gosched")
addBuiltin(sSleep, "sleep")
addBuiltin(sChanSend, "chan send")
addBuiltin(sChanRecv, "chan receive")
addBuiltin(sNetwork, "network")
addBuiltin(sSync, "sync")
addBuiltin(sSyncCond, "sync.(*Cond).Wait")
addBuiltin(sSelect, "select")
addBuiltin(sEmpty, "")
addBuiltin(sMarkAssistWait, "GC mark assist wait for work")
addBuiltin(sSTWUnknown, "")
addBuiltin(sSTWGCMarkTermination, "GC mark termination")
addBuiltin(sSTWGCSweepTermination, "GC sweep termination")
addBuiltin(sSTWWriteHeapDump, "write heap dump")
addBuiltin(sSTWGoroutineProfile, "goroutine profile")
addBuiltin(sSTWGoroutineProfileCleanup, "goroutine profile cleanup")
addBuiltin(sSTWAllGoroutinesStackTrace, "all goroutine stack trace")
addBuiltin(sSTWReadMemStats, "read mem stats")
addBuiltin(sSTWAllThreadsSyscall, "AllThreadsSyscall")
addBuiltin(sSTWGOMAXPROCS, "GOMAXPROCS")
addBuiltin(sSTWStartTrace, "start trace")
addBuiltin(sSTWStopTrace, "stop trace")
addBuiltin(sSTWCountPagesInUse, "CountPagesInUse (test)")
addBuiltin(sSTWReadMetricsSlow, "ReadMetricsSlow (test)")
addBuiltin(sSTWReadMemStatsSlow, "ReadMemStatsSlow (test)")
addBuiltin(sSTWPageCachePagesLeaked, "PageCachePagesLeaked (test)")
addBuiltin(sSTWResetDebugLog, "ResetDebugLog (test)")
if addErr != nil {
// This should be impossible but let's be safe.
return fmt.Errorf("couldn't add strings: %w", addErr)
}
it.evt.strings.compactify()
// Convert stacks.
for id, stk := range pr.Stacks {
evt.stacks.insert(stackID(id), stack{pcs: stk})
}
// OPT(dh): if we could share the frame type between this package and
// oldtrace we wouldn't have to copy the map.
for pc, f := range pr.PCs {
evt.pcs[pc] = frame{
pc: pc,
funcID: stringID(f.Fn),
fileID: stringID(f.File),
line: uint64(f.Line),
}
}
pr.Stacks = nil
pr.PCs = nil
evt.stacks.compactify()
return nil
}
// next returns the next event, io.EOF if there are no more events, or a
// descriptive error for invalid events.
func (it *oldTraceConverter) next() (Event, error) {
if len(it.extra) > 0 {
ev := it.extra[0]
it.extra = it.extra[1:]
if len(it.extra) == 0 {
it.extra = it.extraArr[:0]
}
// Two events aren't allowed to fall on the same timestamp in the new API,
// but this may happen when we produce EvGoStatus events
if ev.base.time <= it.lastTs {
ev.base.time = it.lastTs + 1
}
it.lastTs = ev.base.time
return ev, nil
}
oev, ok := it.events.Pop()
if !ok {
return Event{}, io.EOF
}
ev, err := it.convertEvent(oev)
if err == errSkip {
return it.next()
} else if err != nil {
return Event{}, err
}
// Two events aren't allowed to fall on the same timestamp in the new API,
// but this may happen when we produce EvGoStatus events
if ev.base.time <= it.lastTs {
ev.base.time = it.lastTs + 1
}
it.lastTs = ev.base.time
return ev, nil
}
var errSkip = errors.New("skip event")
// convertEvent converts an event from the old trace format to zero or more
// events in the new format. Most events translate 1 to 1. Some events don't
// result in an event right away, in which case convertEvent returns errSkip.
// Some events result in more than one new event; in this case, convertEvent
// returns the first event and stores additional events in it.extra. When
// encountering events that oldtrace shouldn't be able to emit, ocnvertEvent
// returns a descriptive error.
func (it *oldTraceConverter) convertEvent(ev *oldtrace.Event) (OUT Event, ERR error) {
var mappedType event.Type
mappedArgs := ev.Args
switch ev.Type {
case oldtrace.EvGomaxprocs:
mappedType = go122.EvProcsChange
if it.preInit {
// The first EvGomaxprocs signals the end of trace initialization. At this point we've seen
// all goroutines that already existed at trace begin.
it.preInit = false
for gid := range it.createdPreInit {
// These are goroutines that already existed when tracing started but for which we
// received neither GoWaiting, GoInSyscall, or GoStart. These are goroutines that are in
// the states _Gidle or _Grunnable.
it.extra = append(it.extra, Event{
ctx: schedCtx{
// G: GoID(gid),
G: NoGoroutine,
P: NoProc,
M: NoThread,
},
table: it.evt,
base: baseEvent{
typ: go122.EvGoStatus,
time: Time(ev.Ts),
args: [4]uint64{uint64(gid), ^uint64(0), uint64(go122.GoRunnable)},
},
})
}
it.createdPreInit = nil
return Event{}, errSkip
}
case oldtrace.EvProcStart:
it.procMs[ProcID(ev.P)] = ThreadID(ev.Args[0])
if _, ok := it.seenProcs[ProcID(ev.P)]; ok {
mappedType = go122.EvProcStart
mappedArgs = [4]uint64{uint64(ev.P)}
} else {
it.seenProcs[ProcID(ev.P)] = struct{}{}
mappedType = go122.EvProcStatus
mappedArgs = [4]uint64{uint64(ev.P), uint64(go122.ProcRunning)}
}
case oldtrace.EvProcStop:
if _, ok := it.seenProcs[ProcID(ev.P)]; ok {
mappedType = go122.EvProcStop
mappedArgs = [4]uint64{uint64(ev.P)}
} else {
it.seenProcs[ProcID(ev.P)] = struct{}{}
mappedType = go122.EvProcStatus
mappedArgs = [4]uint64{uint64(ev.P), uint64(go122.ProcIdle)}
}
case oldtrace.EvGCStart:
mappedType = go122.EvGCBegin
case oldtrace.EvGCDone:
mappedType = go122.EvGCEnd
case oldtrace.EvSTWStart:
sid := it.builtinToStringID[sSTWUnknown+it.trace.STWReason(ev.Args[0])]
it.lastStwReason = sid
mappedType = go122.EvSTWBegin
mappedArgs = [4]uint64{uint64(sid)}
case oldtrace.EvSTWDone:
mappedType = go122.EvSTWEnd
mappedArgs = [4]uint64{it.lastStwReason}
case oldtrace.EvGCSweepStart:
mappedType = go122.EvGCSweepBegin
case oldtrace.EvGCSweepDone:
mappedType = go122.EvGCSweepEnd
case oldtrace.EvGoCreate:
if it.preInit {
it.createdPreInit[GoID(ev.Args[0])] = struct{}{}
return Event{}, errSkip
}
mappedType = go122.EvGoCreate
case oldtrace.EvGoStart:
if it.preInit {
mappedType = go122.EvGoStatus
mappedArgs = [4]uint64{ev.Args[0], ^uint64(0), uint64(go122.GoRunning)}
delete(it.createdPreInit, GoID(ev.Args[0]))
} else {
mappedType = go122.EvGoStart
}
case oldtrace.EvGoStartLabel:
it.extra = []Event{{
ctx: schedCtx{
G: GoID(ev.G),
P: ProcID(ev.P),
M: it.procMs[ProcID(ev.P)],
},
table: it.evt,
base: baseEvent{
typ: go122.EvGoLabel,
time: Time(ev.Ts),
args: [4]uint64{ev.Args[2]},
},
}}
return Event{
ctx: schedCtx{
G: GoID(ev.G),
P: ProcID(ev.P),
M: it.procMs[ProcID(ev.P)],
},
table: it.evt,
base: baseEvent{
typ: go122.EvGoStart,
time: Time(ev.Ts),
args: ev.Args,
},
}, nil
case oldtrace.EvGoEnd:
mappedType = go122.EvGoDestroy
case oldtrace.EvGoStop:
mappedType = go122.EvGoBlock
mappedArgs = [4]uint64{uint64(it.builtinToStringID[sForever]), uint64(ev.StkID)}
case oldtrace.EvGoSched:
mappedType = go122.EvGoStop
mappedArgs = [4]uint64{uint64(it.builtinToStringID[sGosched]), uint64(ev.StkID)}
case oldtrace.EvGoPreempt:
mappedType = go122.EvGoStop
mappedArgs = [4]uint64{uint64(it.builtinToStringID[sPreempted]), uint64(ev.StkID)}
case oldtrace.EvGoSleep:
mappedType = go122.EvGoBlock
mappedArgs = [4]uint64{uint64(it.builtinToStringID[sSleep]), uint64(ev.StkID)}
case oldtrace.EvGoBlock:
mappedType = go122.EvGoBlock
mappedArgs = [4]uint64{uint64(it.builtinToStringID[sEmpty]), uint64(ev.StkID)}
case oldtrace.EvGoUnblock:
mappedType = go122.EvGoUnblock
case oldtrace.EvGoBlockSend:
mappedType = go122.EvGoBlock
mappedArgs = [4]uint64{uint64(it.builtinToStringID[sChanSend]), uint64(ev.StkID)}
case oldtrace.EvGoBlockRecv:
mappedType = go122.EvGoBlock
mappedArgs = [4]uint64{uint64(it.builtinToStringID[sChanRecv]), uint64(ev.StkID)}
case oldtrace.EvGoBlockSelect:
mappedType = go122.EvGoBlock
mappedArgs = [4]uint64{uint64(it.builtinToStringID[sSelect]), uint64(ev.StkID)}
case oldtrace.EvGoBlockSync:
mappedType = go122.EvGoBlock
mappedArgs = [4]uint64{uint64(it.builtinToStringID[sSync]), uint64(ev.StkID)}
case oldtrace.EvGoBlockCond:
mappedType = go122.EvGoBlock
mappedArgs = [4]uint64{uint64(it.builtinToStringID[sSyncCond]), uint64(ev.StkID)}
case oldtrace.EvGoBlockNet:
mappedType = go122.EvGoBlock
mappedArgs = [4]uint64{uint64(it.builtinToStringID[sNetwork]), uint64(ev.StkID)}
case oldtrace.EvGoBlockGC:
mappedType = go122.EvGoBlock
mappedArgs = [4]uint64{uint64(it.builtinToStringID[sMarkAssistWait]), uint64(ev.StkID)}
case oldtrace.EvGoSysCall:
// Look for the next event for the same G to determine if the syscall
// blocked.
blocked := false
it.events.All()(func(nev *oldtrace.Event) bool {
if nev.G != ev.G {
return true
}
// After an EvGoSysCall, the next event on the same G will either be
// EvGoSysBlock to denote a blocking syscall, or some other event
// (or the end of the trace) if the syscall didn't block.
if nev.Type == oldtrace.EvGoSysBlock {
blocked = true
}
return false
})
if blocked {
mappedType = go122.EvGoSyscallBegin
mappedArgs = [4]uint64{1: uint64(ev.StkID)}
} else {
// Convert the old instantaneous syscall event to a pair of syscall
// begin and syscall end and give it the shortest possible duration,
// 1ns.
out1 := Event{
ctx: schedCtx{
G: GoID(ev.G),
P: ProcID(ev.P),
M: it.procMs[ProcID(ev.P)],
},
table: it.evt,
base: baseEvent{
typ: go122.EvGoSyscallBegin,
time: Time(ev.Ts),
args: [4]uint64{1: uint64(ev.StkID)},
},
}
out2 := Event{
ctx: out1.ctx,
table: it.evt,
base: baseEvent{
typ: go122.EvGoSyscallEnd,
time: Time(ev.Ts + 1),
args: [4]uint64{},
},
}
it.extra = append(it.extra, out2)
return out1, nil
}
case oldtrace.EvGoSysExit:
mappedType = go122.EvGoSyscallEndBlocked
case oldtrace.EvGoSysBlock:
return Event{}, errSkip
case oldtrace.EvGoWaiting:
mappedType = go122.EvGoStatus
mappedArgs = [4]uint64{ev.Args[0], ^uint64(0), uint64(go122.GoWaiting)}
delete(it.createdPreInit, GoID(ev.Args[0]))
case oldtrace.EvGoInSyscall:
mappedType = go122.EvGoStatus
// In the new tracer, GoStatus with GoSyscall knows what thread the
// syscall is on. In the old tracer, EvGoInSyscall doesn't contain that
// information and all we can do here is specify NoThread.
mappedArgs = [4]uint64{ev.Args[0], ^uint64(0), uint64(go122.GoSyscall)}
delete(it.createdPreInit, GoID(ev.Args[0]))
case oldtrace.EvHeapAlloc:
mappedType = go122.EvHeapAlloc
case oldtrace.EvHeapGoal:
mappedType = go122.EvHeapGoal
case oldtrace.EvGCMarkAssistStart:
mappedType = go122.EvGCMarkAssistBegin
case oldtrace.EvGCMarkAssistDone:
mappedType = go122.EvGCMarkAssistEnd
case oldtrace.EvUserTaskCreate:
mappedType = go122.EvUserTaskBegin
parent := ev.Args[1]
if parent == 0 {
parent = uint64(NoTask)
}
mappedArgs = [4]uint64{ev.Args[0], parent, ev.Args[2], uint64(ev.StkID)}
name, _ := it.evt.strings.get(stringID(ev.Args[2]))
it.tasks[TaskID(ev.Args[0])] = taskState{name: name, parentID: TaskID(ev.Args[1])}
case oldtrace.EvUserTaskEnd:
mappedType = go122.EvUserTaskEnd
// Event.Task expects the parent and name to be smuggled in extra args
// and as extra strings.
ts, ok := it.tasks[TaskID(ev.Args[0])]
if ok {
delete(it.tasks, TaskID(ev.Args[0]))
mappedArgs = [4]uint64{
ev.Args[0],
ev.Args[1],
uint64(ts.parentID),
uint64(it.evt.addExtraString(ts.name)),
}
} else {
mappedArgs = [4]uint64{ev.Args[0], ev.Args[1], uint64(NoTask), uint64(it.evt.addExtraString(""))}
}
case oldtrace.EvUserRegion:
switch ev.Args[1] {
case 0: // start
mappedType = go122.EvUserRegionBegin
case 1: // end
mappedType = go122.EvUserRegionEnd
}
mappedArgs = [4]uint64{ev.Args[0], ev.Args[2], uint64(ev.StkID)}
case oldtrace.EvUserLog:
mappedType = go122.EvUserLog
mappedArgs = [4]uint64{ev.Args[0], ev.Args[1], it.inlineToStringID[ev.Args[3]], uint64(ev.StkID)}
case oldtrace.EvCPUSample:
mappedType = go122.EvCPUSample
// When emitted by the Go 1.22 tracer, CPU samples have 5 arguments:
// timestamp, M, P, G, stack. However, after they get turned into Event,
// they have the arguments stack, M, P, G.
//
// In Go 1.21, CPU samples did not have Ms.
mappedArgs = [4]uint64{uint64(ev.StkID), ^uint64(0), uint64(ev.P), ev.G}
default:
return Event{}, fmt.Errorf("unexpected event type %v", ev.Type)
}
if oldtrace.EventDescriptions[ev.Type].Stack {
if stackIDs := go122.Specs()[mappedType].StackIDs; len(stackIDs) > 0 {
mappedArgs[stackIDs[0]-1] = uint64(ev.StkID)
}
}
m := NoThread
if ev.P != -1 && ev.Type != oldtrace.EvCPUSample {
if t, ok := it.procMs[ProcID(ev.P)]; ok {
m = ThreadID(t)
}
}
if ev.Type == oldtrace.EvProcStop {
delete(it.procMs, ProcID(ev.P))
}
g := GoID(ev.G)
if g == 0 {
g = NoGoroutine
}
out := Event{
ctx: schedCtx{
G: GoID(g),
P: ProcID(ev.P),
M: m,
},
table: it.evt,
base: baseEvent{
typ: mappedType,
time: Time(ev.Ts),
args: mappedArgs,
},
}
return out, nil
}
// convertOldFormat takes a fully loaded trace in the old trace format and
// returns an iterator over events in the new format.
func convertOldFormat(pr oldtrace.Trace) *oldTraceConverter {
it := &oldTraceConverter{}
it.init(pr)
return it
}

View file

@ -0,0 +1,89 @@
// Copyright 2024 The Go Authors. All rights reserved.
// Use of this source code is governed by a BSD-style
// license that can be found in the LICENSE file.
package trace_test
import (
"internal/trace/v2"
"internal/trace/v2/testtrace"
"io"
"os"
"path/filepath"
"testing"
)
func TestOldtrace(t *testing.T) {
traces, err := filepath.Glob("./internal/oldtrace/testdata/*_good")
if err != nil {
t.Fatalf("failed to glob for tests: %s", err)
}
var testedUserRegions bool
for _, p := range traces {
p := p
testName, err := filepath.Rel("./internal/oldtrace/testdata", p)
if err != nil {
t.Fatalf("failed to relativize testdata path: %s", err)
}
t.Run(testName, func(t *testing.T) {
f, err := os.Open(p)
if err != nil {
t.Fatalf("failed to open test %q: %s", p, err)
}
defer f.Close()
tr, err := trace.NewReader(f)
if err != nil {
t.Fatalf("failed to create reader: %s", err)
}
v := testtrace.NewValidator()
v.Go121 = true
for {
ev, err := tr.ReadEvent()
if err != nil {
if err == io.EOF {
break
}
t.Fatalf("couldn't read converted event: %s", err)
}
if err := v.Event(ev); err != nil {
t.Fatalf("converted event did not validate; event: \n%s\nerror: %s", ev, err)
}
if testName == "user_task_region_1_21_good" {
testedUserRegions = true
validRegions := map[string]struct{}{
"post-existing region": struct{}{},
"region0": struct{}{},
"region1": struct{}{},
}
// Check that we correctly convert user regions. These
// strings were generated by
// runtime/trace.TestUserTaskRegion, which is the basis for
// the user_task_region_* test cases. We only check for the
// Go 1.21 traces because earlier traces used different
// strings.
switch ev.Kind() {
case trace.EventRegionBegin, trace.EventRegionEnd:
if _, ok := validRegions[ev.Region().Type]; !ok {
t.Fatalf("converted event has unexpected region type:\n%s", ev)
}
case trace.EventTaskBegin, trace.EventTaskEnd:
if ev.Task().Type != "task0" {
t.Fatalf("converted event has unexpected task type name:\n%s", ev)
}
case trace.EventLog:
l := ev.Log()
if l.Task != 1 || l.Category != "key0" || l.Message != "0123456789abcdef" {
t.Fatalf("converted event has unexpected user log:\n%s", ev)
}
}
}
}
})
}
if !testedUserRegions {
t.Fatal("didn't see expected test case user_task_region_1_21_good")
}
}

View file

@ -12,6 +12,7 @@ import (
"strings"
"internal/trace/v2/event/go122"
"internal/trace/v2/internal/oldtrace"
"internal/trace/v2/version"
)
@ -25,6 +26,8 @@ type Reader struct {
cpuSamples []cpuSample
order ordering
emittedSync bool
go121Events *oldTraceConverter
}
// NewReader creates a new trace reader.
@ -34,20 +37,30 @@ func NewReader(r io.Reader) (*Reader, error) {
if err != nil {
return nil, err
}
if v != version.Go122 {
switch v {
case version.Go111, version.Go119, version.Go121:
tr, err := oldtrace.Parse(br, v)
if err != nil {
return nil, err
}
return &Reader{
go121Events: convertOldFormat(tr),
}, nil
case version.Go122:
return &Reader{
r: br,
order: ordering{
mStates: make(map[ThreadID]*mState),
pStates: make(map[ProcID]*pState),
gStates: make(map[GoID]*gState),
activeTasks: make(map[TaskID]taskState),
},
// Don't emit a sync event when we first go to emit events.
emittedSync: true,
}, nil
default:
return nil, fmt.Errorf("unknown or unsupported version go 1.%d", v)
}
return &Reader{
r: br,
order: ordering{
mStates: make(map[ThreadID]*mState),
pStates: make(map[ProcID]*pState),
gStates: make(map[GoID]*gState),
activeTasks: make(map[TaskID]taskState),
},
// Don't emit a sync event when we first go to emit events.
emittedSync: true,
}, nil
}
// ReadEvent reads a single event from the stream.
@ -55,6 +68,15 @@ func NewReader(r io.Reader) (*Reader, error) {
// If the stream has been exhausted, it returns an invalid
// event and io.EOF.
func (r *Reader) ReadEvent() (e Event, err error) {
if r.go121Events != nil {
ev, err := r.go121Events.next()
if err != nil {
// XXX do we have to emit an EventSync when the trace is done?
return Event{}, err
}
return ev, nil
}
// Go 1.22+ trace parsing algorithm.
//
// (1) Read in all the batches for the next generation from the stream.

View file

@ -0,0 +1,2 @@
go test fuzz v1
[]byte("go 1.11 trace\x00\x00\x00A00\x020$0")

View file

@ -21,6 +21,7 @@ type Validator struct {
ranges map[trace.ResourceID][]string
tasks map[trace.TaskID]string
seenSync bool
Go121 bool
}
type schedContext struct {
@ -160,7 +161,7 @@ func (v *Validator) Event(ev trace.Event) error {
}
// Validate sched context.
if new.Executing() {
ctx := v.getOrCreateThread(e, ev.Thread())
ctx := v.getOrCreateThread(e, ev, ev.Thread())
if ctx != nil {
if ctx.G != trace.NoGoroutine && ctx.G != id {
e.Errorf("tried to run goroutine %d when one was already executing (%d) on thread %d", id, ctx.G, ev.Thread())
@ -213,7 +214,7 @@ func (v *Validator) Event(ev trace.Event) error {
}
// Validate sched context.
if new.Executing() {
ctx := v.getOrCreateThread(e, ev.Thread())
ctx := v.getOrCreateThread(e, ev, ev.Thread())
if ctx != nil {
if ctx.P != trace.NoProc && ctx.P != id {
e.Errorf("tried to run proc %d when one was already executing (%d) on thread %d", id, ctx.P, ev.Thread())
@ -316,8 +317,25 @@ func (v *Validator) deleteRange(r trace.ResourceID, name string) {
v.ranges[r] = slices.Delete(ranges, i, i+1)
}
func (v *Validator) getOrCreateThread(e *errAccumulator, m trace.ThreadID) *schedContext {
if m == trace.NoThread {
func (v *Validator) getOrCreateThread(e *errAccumulator, ev trace.Event, m trace.ThreadID) *schedContext {
lenient := func() bool {
// Be lenient about GoUndetermined -> GoSyscall transitions if they
// originate from an old trace. These transitions lack thread
// information in trace formats older than 1.22.
if !v.Go121 {
return false
}
if ev.Kind() != trace.EventStateTransition {
return false
}
tr := ev.StateTransition()
if tr.Resource.Kind != trace.ResourceGoroutine {
return false
}
from, to := tr.Goroutine()
return from == trace.GoUndetermined && to == trace.GoSyscall
}
if m == trace.NoThread && !lenient() {
e.Errorf("must have thread, but thread ID is none")
return nil
}

View file

@ -16,11 +16,20 @@ import (
type Version uint32
const (
Go111 Version = 11
Go119 Version = 19
Go121 Version = 21
Go122 Version = 22
Current = Go122
)
var versions = map[Version][]event.Spec{
// Go 1.111.21 use a different parser and are only set here for the sake of
// Version.Valid.
Go111: nil,
Go119: nil,
Go121: nil,
Go122: go122.Specs(),
}