delve/pkg/proc/target_exec.go
Alessandro Arzilli abad6bb97e
proc: use .closureptr for stepping through range-over-func statements (#3763)
* proc: use .closureptr for stepping through range-over-func statements

Uses special variables .closureptr and #yieldN to correctly identify
the parent frame of a range-over-func body closure call.

Updates #3733

* fix
2024-07-11 10:26:38 -07:00

1660 lines
49 KiB
Go

package proc
import (
"bytes"
"debug/dwarf"
"errors"
"fmt"
"go/ast"
"go/constant"
"go/token"
"path/filepath"
"strings"
"golang.org/x/arch/ppc64/ppc64asm"
"github.com/go-delve/delve/pkg/astutil"
"github.com/go-delve/delve/pkg/dwarf/godwarf"
"github.com/go-delve/delve/pkg/dwarf/reader"
"github.com/go-delve/delve/pkg/logflags"
)
const maxSkipAutogeneratedWrappers = 5 // maximum recursion depth for skipAutogeneratedWrappers
// ErrNoSourceForPC is returned when the given address
// does not correspond with a source file location.
type ErrNoSourceForPC struct {
pc uint64
}
func (err *ErrNoSourceForPC) Error() string {
return fmt.Sprintf("no source for PC %#x", err.pc)
}
// Next resumes the processes in the group, continuing the selected target
// until the next source line.
func (grp *TargetGroup) Next() (err error) {
if _, err := grp.Valid(); err != nil {
return err
}
if grp.HasSteppingBreakpoints() {
return errors.New("next while nexting")
}
if err = next(grp.Selected, false, false); err != nil {
grp.Selected.ClearSteppingBreakpoints()
return
}
return grp.Continue()
}
// Continue continues execution of the debugged
// processes. It will continue until it hits a breakpoint
// or is otherwise stopped.
func (grp *TargetGroup) Continue() error {
if grp.numValid() == 0 {
_, err := grp.targets[0].Valid()
return err
}
for _, dbp := range grp.targets {
if isvalid, _ := dbp.Valid(); !isvalid {
continue
}
for _, thread := range dbp.ThreadList() {
thread.Common().CallReturn = false
thread.Common().returnValues = nil
}
dbp.Breakpoints().WatchOutOfScope = nil
dbp.clearHardcodedBreakpoints()
}
grp.cctx.CheckAndClearManualStopRequest()
defer func() {
// Make sure we clear internal breakpoints if we simultaneously receive a
// manual stop request and hit a breakpoint.
if grp.cctx.CheckAndClearManualStopRequest() {
grp.finishManualStop()
}
}()
for {
if grp.cctx.CheckAndClearManualStopRequest() {
grp.finishManualStop()
return nil
}
for _, dbp := range grp.targets {
dbp.ClearCaches()
}
logflags.DebuggerLogger().Debugf("ContinueOnce")
trapthread, stopReason, contOnceErr := grp.procgrp.ContinueOnce(grp.cctx)
var traptgt *Target
if trapthread != nil {
traptgt = grp.TargetForThread(trapthread.ThreadID())
if traptgt == nil {
return fmt.Errorf("could not find target for thread %d", trapthread.ThreadID())
}
} else {
traptgt = grp.targets[0]
}
traptgt.StopReason = stopReason
it := ValidTargets{Group: grp}
for it.Next() {
// Both selectedGoroutine and current thread are stale here, since we can
// only set their definitive value *after* evaluating breakpoint
// conditions here we give them temporary non-stale values.
it.selectedGoroutine = nil
curthread := it.currentThread
for _, thread := range it.ThreadList() {
if thread.Breakpoint().Breakpoint != nil {
it.currentThread = thread
thread.Breakpoint().Breakpoint.checkCondition(it.Target, thread, thread.Breakpoint())
}
}
it.currentThread = curthread
// Clear watchpoints that have gone out of scope
for _, watchpoint := range it.Breakpoints().WatchOutOfScope {
err := it.ClearBreakpoint(watchpoint.Addr)
if err != nil {
logflags.DebuggerLogger().Errorf("could not clear out-of-scope watchpoint: %v", err)
}
delete(it.Breakpoints().Logical, watchpoint.LogicalID())
}
}
if contOnceErr != nil {
// Attempt to refresh status of current thread/current goroutine, see
// Issue #2078.
// Errors are ignored because depending on why ContinueOnce failed this
// might very well not work.
_ = grp.setCurrentThreads(traptgt, trapthread)
if pe, ok := contOnceErr.(ErrProcessExited); ok {
traptgt.exitStatus = pe.Status
}
return contOnceErr
}
if stopReason == StopLaunched {
it.Reset()
for it.Next() {
it.Target.ClearSteppingBreakpoints()
}
}
var callInjectionDone bool
var callErr error
var hcbpErr error
it.Reset()
for it.Next() {
dbp := it.Target
threads := dbp.ThreadList()
if logflags.Debugger() {
log := logflags.DebuggerLogger()
log.Debugf("callInjection protocol on:")
for _, th := range threads {
regs, _ := th.Registers()
log.Debugf("\t%d PC=%#x", th.ThreadID(), regs.PC())
}
}
callInjectionDoneThis, callErrThis := callInjectionProtocol(dbp, threads)
callInjectionDone = callInjectionDone || callInjectionDoneThis
if callInjectionDoneThis {
dbp.StopReason = StopCallReturned
}
if callErrThis != nil && callErr == nil {
callErr = callErrThis
}
hcbpErrThis := dbp.handleHardcodedBreakpoints(grp, trapthread, threads)
if hcbpErrThis != nil && hcbpErr == nil {
hcbpErr = hcbpErrThis
}
}
// callErr and hcbpErr check delayed until after pickCurrentThread, which
// must always happen, otherwise the debugger could be left in an
// inconsistent state.
it = ValidTargets{Group: grp}
for it.Next() {
var th Thread = nil
if it.Target == traptgt {
th = trapthread
}
err := pickCurrentThread(it.Target, th)
if err != nil {
return err
}
}
grp.pickCurrentTarget(traptgt)
dbp := grp.Selected
if callErr != nil {
return callErr
}
if hcbpErr != nil {
return hcbpErr
}
curthread := dbp.CurrentThread()
curbp := curthread.Breakpoint()
switch {
case curbp.Active && curbp.Stepping:
if curbp.SteppingInto {
// See description of proc.(*Process).next for the meaning of StepBreakpoints
if err := conditionErrors(grp); err != nil {
return err
}
if grp.GetDirection() == Backward {
if err := dbp.ClearSteppingBreakpoints(); err != nil {
return err
}
return grp.StepInstruction(false)
}
} else {
curthread.Common().returnValues = curbp.Breakpoint.returnInfo.Collect(dbp, curthread)
if err := dbp.ClearSteppingBreakpoints(); err != nil {
return err
}
dbp.StopReason = StopNextFinished
return conditionErrors(grp)
}
case curbp.Active:
onNextGoroutine, err := onNextGoroutine(dbp, curthread, dbp.Breakpoints())
if err != nil {
return err
}
if onNextGoroutine &&
(!isTraceOrTraceReturn(curbp.Breakpoint) || grp.KeepSteppingBreakpoints&TracepointKeepsSteppingBreakpoints == 0) {
err := dbp.ClearSteppingBreakpoints()
if err != nil {
return err
}
}
if curbp.LogicalID() == unrecoveredPanicID {
dbp.ClearSteppingBreakpoints()
}
if curbp.LogicalID() != hardcodedBreakpointID {
dbp.StopReason = StopBreakpoint
}
if curbp.Breakpoint.WatchType != 0 {
dbp.StopReason = StopWatchpoint
}
return conditionErrors(grp)
case stopReason == StopLaunched:
return nil
default:
// not a manual stop, not on runtime.Breakpoint, not on a breakpoint, just repeat
}
if callInjectionDone {
// a call injection was finished, don't let a breakpoint with a failed
// condition or a step breakpoint shadow this.
return conditionErrors(grp)
}
}
}
func (grp *TargetGroup) finishManualStop() {
for _, dbp := range grp.targets {
if isvalid, _ := dbp.Valid(); !isvalid {
continue
}
dbp.StopReason = StopManual
dbp.clearHardcodedBreakpoints()
if grp.KeepSteppingBreakpoints&HaltKeepsSteppingBreakpoints == 0 {
dbp.ClearSteppingBreakpoints()
}
}
}
// setCurrentThreads switches traptgt to trapthread, then for each target in
// the group if its current thread exists it refreshes the current
// goroutine, otherwise it switches it to a randomly selected thread.
func (grp *TargetGroup) setCurrentThreads(traptgt *Target, trapthread Thread) error {
var err error
if traptgt != nil && trapthread != nil {
err = traptgt.SwitchThread(trapthread.ThreadID())
}
for _, tgt := range grp.targets {
if isvalid, _ := tgt.Valid(); !isvalid {
continue
}
if _, ok := tgt.FindThread(tgt.currentThread.ThreadID()); ok {
tgt.selectedGoroutine, _ = GetG(tgt.currentThread)
} else {
threads := tgt.ThreadList()
if len(threads) > 0 {
err1 := tgt.SwitchThread(threads[0].ThreadID())
if err1 != nil && err == nil {
err = err1
}
}
}
}
return err
}
func isTraceOrTraceReturn(bp *Breakpoint) bool {
if bp.Logical == nil {
return false
}
return bp.Logical.Tracepoint || bp.Logical.TraceReturn
}
func conditionErrors(grp *TargetGroup) error {
var condErr error
for _, dbp := range grp.targets {
if isvalid, _ := dbp.Valid(); !isvalid {
continue
}
for _, th := range dbp.ThreadList() {
if bp := th.Breakpoint(); bp.Breakpoint != nil && bp.CondError != nil {
if condErr == nil {
condErr = bp.CondError
} else {
return errors.New("multiple errors evaluating conditions")
}
}
}
}
return condErr
}
// pick a new dbp.currentThread, with the following priority:
//
// - a thread with an active stepping breakpoint
// - a thread with an active breakpoint, prioritizing trapthread
// - trapthread if it is not nil
// - the previous current thread if it still exists
// - a randomly selected thread
func pickCurrentThread(dbp *Target, trapthread Thread) error {
threads := dbp.ThreadList()
for _, th := range threads {
if bp := th.Breakpoint(); bp.Active && bp.Stepping {
return dbp.SwitchThread(th.ThreadID())
}
}
if trapthread != nil {
if bp := trapthread.Breakpoint(); bp.Active {
return dbp.SwitchThread(trapthread.ThreadID())
}
}
for _, th := range threads {
if bp := th.Breakpoint(); bp.Active {
return dbp.SwitchThread(th.ThreadID())
}
}
if trapthread != nil {
return dbp.SwitchThread(trapthread.ThreadID())
}
if _, ok := dbp.FindThread(dbp.currentThread.ThreadID()); ok {
dbp.selectedGoroutine, _ = GetG(dbp.currentThread)
return nil
}
if len(threads) > 0 {
return dbp.SwitchThread(threads[0].ThreadID())
}
return nil
}
// pickCurrentTarget picks a new current target, with the following property:
//
// - a target with an active stepping breakpoint
// - a target with StopReason == StopCallReturned
// - a target with an active breakpoint, prioritizing traptgt
// - traptgt
func (grp *TargetGroup) pickCurrentTarget(traptgt *Target) {
if len(grp.targets) == 1 {
grp.Selected = grp.targets[0]
return
}
for _, dbp := range grp.targets {
if isvalid, _ := dbp.Valid(); !isvalid {
continue
}
bp := dbp.currentThread.Breakpoint()
if bp.Active && bp.Stepping {
grp.Selected = dbp
return
}
}
for _, dbp := range grp.targets {
if isvalid, _ := dbp.Valid(); !isvalid {
continue
}
if dbp.StopReason == StopCallReturned {
grp.Selected = dbp
return
}
}
if traptgt.currentThread.Breakpoint().Active {
grp.Selected = traptgt
return
}
for _, dbp := range grp.targets {
if isvalid, _ := dbp.Valid(); !isvalid {
continue
}
bp := dbp.currentThread.Breakpoint()
if bp.Active {
grp.Selected = dbp
return
}
}
grp.Selected = traptgt
}
func disassembleCurrentInstruction(p Process, thread Thread, off int64) ([]AsmInstruction, error) {
regs, err := thread.Registers()
if err != nil {
return nil, err
}
pc := regs.PC() + uint64(off)
return disassemble(p.Memory(), regs, p.Breakpoints(), p.BinInfo(), pc, pc+uint64(p.BinInfo().Arch.MaxInstructionLength()), true)
}
// stepInstructionOut repeatedly calls StepInstruction until the current
// function is neither fnname1 or fnname2.
// This function is used to step out of runtime.Breakpoint as well as
// runtime.debugCallV1.
func stepInstructionOut(grp *TargetGroup, dbp *Target, curthread Thread, fnname1, fnname2 string) error {
defer dbp.ClearCaches()
for {
if err := grp.procgrp.StepInstruction(curthread.ThreadID()); err != nil {
return err
}
loc, err := curthread.Location()
var locFnName string
if loc.Fn != nil && !loc.Fn.cu.image.Stripped() {
locFnName = loc.Fn.Name
// Calls to runtime.Breakpoint are inlined in some versions of Go when
// inlining is enabled. Here we attempt to resolve any inlining.
dwarfTree, _ := loc.Fn.cu.image.getDwarfTree(loc.Fn.offset)
if dwarfTree != nil {
inlstack := reader.InlineStack(dwarfTree, loc.PC)
if len(inlstack) > 0 {
if locFnName2, ok := inlstack[0].Val(dwarf.AttrName).(string); ok {
locFnName = locFnName2
}
}
}
}
if err != nil || loc.Fn == nil || (locFnName != fnname1 && locFnName != fnname2) {
g, _ := GetG(curthread)
selg := dbp.SelectedGoroutine()
if g != nil && selg != nil && g.ID == selg.ID {
selg.CurrentLoc = *loc
}
return curthread.SetCurrentBreakpoint(true)
}
}
}
// Step resumes the processes in the group, continuing the selected target
// until the next source line. Will step into functions.
func (grp *TargetGroup) Step() (err error) {
if _, err := grp.Valid(); err != nil {
return err
}
if grp.HasSteppingBreakpoints() {
return errors.New("next while nexting")
}
if err = next(grp.Selected, true, false); err != nil {
_ = grp.Selected.ClearSteppingBreakpoints()
return err
}
if bpstate := grp.Selected.CurrentThread().Breakpoint(); bpstate.Breakpoint != nil && bpstate.Active && bpstate.SteppingInto && grp.GetDirection() == Backward {
grp.Selected.ClearSteppingBreakpoints()
return grp.StepInstruction(false)
}
return grp.Continue()
}
// sameGoroutineCondition returns an expression that evaluates to true when
// the current goroutine is g.
func sameGoroutineCondition(bi *BinaryInfo, g *G, threadID int) ast.Expr {
if g == nil {
if len(bi.Images[0].compileUnits) == 0 {
// It's unclear what the right behavior is here. We are probably
// debugging a process without debug info, this means we can't properly
// create a same goroutine condition (we don't have a description for the
// runtime.g type). If we don't set the condition then 'next' (and step,
// stepout) will work for single-threaded programs (in limited
// circumstances) but fail in presence of any concurrency.
// If we set a thread ID condition even single threaded programs can fail
// due to goroutine migration, but sometimes it will work even with
// concurrency.
return nil
}
return astutil.Eql(astutil.PkgVar("runtime", "threadid"), astutil.Int(int64(threadID)))
}
return astutil.Eql(astutil.Sel(astutil.PkgVar("runtime", "curg"), "goid"), astutil.Int(g.ID))
}
func frameoffCondition(frame *Stackframe) ast.Expr {
return astutil.Eql(astutil.PkgVar("runtime", "frameoff"), astutil.Int(frame.FrameOffset()))
}
// StepOut resumes the processes in the group, continuing the selected target
// until the current goroutine exits the function currently being
// executed or a deferred function is executed
func (grp *TargetGroup) StepOut() error {
backward := grp.GetDirection() == Backward
if _, err := grp.Valid(); err != nil {
return err
}
if grp.HasSteppingBreakpoints() {
return errors.New("next while nexting")
}
dbp := grp.Selected
selg := dbp.SelectedGoroutine()
curthread := dbp.CurrentThread()
topframe, retframe, err := topframe(dbp, selg, curthread)
if err != nil {
return err
}
rangeFrames, err := rangeFuncStackTrace(dbp, selg)
if err != nil {
return err
}
if rangeFrames != nil {
// There are range-over-func body closures skip all of them to the
// function containing them and its caller function.
topframe, retframe = rangeFrames[len(rangeFrames)-2], rangeFrames[len(rangeFrames)-1]
}
success := false
defer func() {
if !success {
dbp.ClearSteppingBreakpoints()
}
}()
if topframe.Inlined {
if err := next(dbp, false, true); err != nil {
return err
}
success = true
return grp.Continue()
}
sameGCond := sameGoroutineCondition(dbp.BinInfo(), selg, curthread.ThreadID())
if backward {
if err := stepOutReverse(dbp, topframe, retframe, sameGCond); err != nil {
return err
}
success = true
return grp.Continue()
}
deferpc, err := setDeferBreakpoint(dbp, nil, topframe, sameGCond, false)
if err != nil {
return err
}
if topframe.Ret == 0 && deferpc == 0 {
return errors.New("nothing to stepout to")
}
if topframe.Ret != 0 {
topframe, retframe := skipAutogeneratedWrappersOut(grp.Selected, selg, curthread, &topframe, &retframe)
retFrameCond := astutil.And(sameGCond, frameoffCondition(retframe))
bp, err := allowDuplicateBreakpoint(dbp.SetBreakpoint(0, retframe.Current.PC, NextBreakpoint, retFrameCond))
if err != nil {
return err
}
if bp != nil {
configureReturnBreakpoint(dbp.BinInfo(), bp, topframe, retFrameCond)
}
}
if bp := curthread.Breakpoint(); bp.Breakpoint == nil {
curthread.SetCurrentBreakpoint(false)
}
success = true
return grp.Continue()
}
// StepInstruction will continue the current thread for exactly
// one instruction. This method affects only the thread
// associated with the selected goroutine. All other
// threads will remain stopped.
func (grp *TargetGroup) StepInstruction(skipCalls bool) (err error) {
dbp := grp.Selected
thread := dbp.CurrentThread()
g := dbp.SelectedGoroutine()
if g != nil {
if g.Thread == nil {
// Step called on parked goroutine
if _, err := dbp.SetBreakpoint(0, g.PC, NextBreakpoint,
sameGoroutineCondition(dbp.BinInfo(), dbp.SelectedGoroutine(), thread.ThreadID())); err != nil {
return err
}
return grp.Continue()
}
thread = g.Thread
}
dbp.ClearCaches()
if ok, err := dbp.Valid(); !ok {
return err
}
var isCall bool
instr, err := disassembleCurrentInstruction(dbp, thread, 0)
if err != nil {
return err
}
isCall = len(instr) > 0 && instr[0].IsCall()
err = grp.procgrp.StepInstruction(thread.ThreadID())
if err != nil {
return err
}
thread.Breakpoint().Clear()
err = thread.SetCurrentBreakpoint(false)
if err != nil {
return err
}
if tg, _ := GetG(thread); tg != nil {
dbp.selectedGoroutine = tg
}
dbp.StopReason = StopNextFinished
if skipCalls && isCall {
return grp.StepOut()
}
return nil
}
// Set breakpoints at every line, and the return address. Also look for
// a deferred function and set a breakpoint there too.
// If stepInto is true it will also set breakpoints inside all
// functions called on the current source line, for non-absolute CALLs
// a breakpoint of kind StepBreakpoint is set on the CALL instruction,
// Continue will take care of setting a breakpoint to the destination
// once the CALL is reached.
//
// Regardless of stepInto the following breakpoints will be set:
// - a breakpoint on the first deferred function with NextDeferBreakpoint
// kind, the list of all the addresses to deferreturn calls in this function
// and condition checking that we remain on the same goroutine
// - a breakpoint on each line of the function, with a condition checking
// that we stay on the same stack frame and goroutine.
// - a breakpoint on the return address of the function, with a condition
// checking that we move to the previous stack frame and stay on the same
// goroutine.
//
// The breakpoint on the return address is *not* set if the current frame is
// an inlined call. For inlined calls topframe.Current.Fn is the function
// where the inlining happened and the second set of breakpoints will also
// cover the "return address".
//
// If inlinedStepOut is true this function implements the StepOut operation
// for an inlined function call. Everything works the same as normal except
// when removing instructions belonging to inlined calls we also remove all
// instructions belonging to the current inlined call.
func next(dbp *Target, stepInto, inlinedStepOut bool) error {
backward := dbp.recman.GetDirection() == Backward
selg := dbp.SelectedGoroutine()
curthread := dbp.CurrentThread()
bi := dbp.BinInfo()
topframe, retframe, err := topframe(dbp, selg, curthread)
if err != nil {
return err
}
if topframe.Current.Fn == nil {
return &ErrNoSourceForPC{topframe.Current.PC}
}
if backward && retframe.Current.Fn == nil {
return &ErrNoSourceForPC{retframe.Current.PC}
}
// sanity check
if inlinedStepOut && !topframe.Inlined {
panic("next called with inlinedStepOut but topframe was not inlined")
}
rangeFrames, err := rangeFuncStackTrace(dbp, selg)
if err != nil {
return err
}
success := false
defer func() {
if !success {
dbp.ClearSteppingBreakpoints()
}
}()
ext := filepath.Ext(topframe.Current.File)
csource := ext != ".go" && ext != ".s"
var regs Registers
if selg != nil && selg.Thread != nil {
regs, err = selg.Thread.Registers()
if err != nil {
return err
}
}
sameGCond := sameGoroutineCondition(bi, selg, curthread.ThreadID())
firstPCAfterPrologue, err := FirstPCAfterPrologue(dbp, topframe.Current.Fn, false)
if err != nil {
return err
}
if backward {
if firstPCAfterPrologue == topframe.Current.PC {
// We don't want to step into the prologue so we just execute a reverse step out instead
if err := stepOutReverse(dbp, topframe, retframe, sameGCond); err != nil {
return err
}
success = true
return nil
}
topframe.Ret, err = findCallInstrForRet(dbp, dbp.Memory(), topframe.Ret, retframe.Current.Fn)
if err != nil {
return err
}
}
text, err := disassemble(dbp.Memory(), regs, dbp.Breakpoints(), bi, topframe.Current.Fn.Entry, topframe.Current.Fn.End, false)
if err != nil && stepInto {
return err
}
sameFrameCond := astutil.And(sameGCond, frameoffCondition(&topframe))
if stepInto && !backward {
err := setStepIntoBreakpoints(dbp, topframe.Current.Fn, text, topframe, sameGCond)
if err != nil {
return err
}
}
if !backward && !topframe.Current.Fn.cu.image.Stripped() {
fr := topframe
if len(rangeFrames) != 0 && !stepInto {
fr = rangeFrames[len(rangeFrames)-2]
}
_, err = setDeferBreakpoint(dbp, text, fr, sameGCond, stepInto)
if err != nil {
return err
}
}
// Add breakpoints on all the lines in the current function
pcs, err := topframe.Current.Fn.AllPCs(topframe.Current.File, topframe.Current.Line)
if err != nil {
return err
}
if backward {
// Ensure that pcs contains firstPCAfterPrologue when reverse stepping.
found := false
for _, pc := range pcs {
if pc == firstPCAfterPrologue {
found = true
break
}
}
if !found {
pcs = append(pcs, firstPCAfterPrologue)
}
}
if !stepInto {
// Removing any PC range belonging to an inlined call
frame := topframe
if inlinedStepOut {
frame = retframe
}
pcs, err = removeInlinedCalls(pcs, &frame, bi)
if err != nil {
return err
}
}
if !csource {
var covered bool
for i := range pcs {
if topframe.Current.Fn.Entry <= pcs[i] && pcs[i] < topframe.Current.Fn.End {
covered = true
break
}
}
if !covered {
fn := bi.PCToFunc(topframe.Ret)
if selg != nil && fn != nil && fn.Name == "runtime.goexit" {
return nil
}
}
}
for _, pc := range pcs {
if !stepInto && topframe.Call.Fn.extra(bi).rangeParent != nil {
if pc < firstPCAfterPrologue {
continue
}
}
if _, err := allowDuplicateBreakpoint(dbp.SetBreakpoint(0, pc, NextBreakpoint, sameFrameCond)); err != nil {
return err
}
}
if stepInto && backward {
err := setStepIntoBreakpointsReverse(dbp, text, topframe, sameGCond)
if err != nil {
return err
}
}
// Stepping into range-over-func-bodies
if !stepInto && !inlinedStepOut {
rangeParent := topframe.Call.Fn.extra(bi).rangeParent
if rangeParent == nil {
rangeParent = topframe.Call.Fn
}
rpoff := topframe.FrameOffset()
if len(rangeFrames) > 0 {
rpoff = rangeFrames[len(rangeFrames)-2].FrameOffset()
}
rpc := astutil.And(sameGCond, astutil.Eql(astutil.PkgVar("runtime", "rangeParentOffset"), astutil.Int(rpoff)))
for _, fn := range rangeParent.extra(bi).rangeBodies {
if fn.Entry != 0 {
pc, err := FirstPCAfterPrologue(dbp, fn, false)
if err != nil {
return err
}
if _, err := allowDuplicateBreakpoint(dbp.SetBreakpoint(0, pc, NextBreakpoint, rpc)); err != nil {
return err
}
}
}
}
// Set step-out breakpoints for range-over-func body closures
if !stepInto && selg != nil && topframe.Current.Fn.extra(bi).rangeParent != nil && len(rangeFrames) > 0 {
for _, fr := range rangeFrames[:len(rangeFrames)-1] {
retframecond := astutil.And(sameGCond, frameoffCondition(&fr))
if !fr.hasInlines {
dbp.SetBreakpoint(0, fr.Current.PC, NextBreakpoint, retframecond)
} else {
// fr.Current.PC does not belong to fr.Call.Fn, because there are inlined calls, therefore set a breakpoint on every statement of fr.Call.Fn
pcs, err := fr.Current.Fn.AllPCs("", 0)
if err != nil {
return err
}
pcs, err = removeInlinedCalls(pcs, &fr, bi)
if err != nil {
return err
}
for _, pc := range pcs {
dbp.SetBreakpoint(0, pc, NextBreakpoint, retframecond)
}
}
}
topframe, retframe = rangeFrames[len(rangeFrames)-2], rangeFrames[len(rangeFrames)-1]
}
// Step-out breakpoint
if !topframe.Inlined {
topframe, retframe := skipAutogeneratedWrappersOut(dbp, selg, curthread, &topframe, &retframe)
retFrameCond := astutil.And(sameGCond, frameoffCondition(retframe))
// Add a breakpoint on the return address for the current frame.
// For inlined functions there is no need to do this, the set of PCs
// returned by the AllPCsBetween call above already cover all instructions
// of the containing function.
bp, _ := dbp.SetBreakpoint(0, retframe.Current.PC, NextBreakpoint, retFrameCond)
// Return address could be wrong, if we are unable to set a breakpoint
// there it's ok.
if bp != nil {
configureReturnBreakpoint(bi, bp, topframe, retFrameCond)
}
}
if bp := curthread.Breakpoint(); bp.Breakpoint == nil {
curthread.SetCurrentBreakpoint(false)
}
success = true
return nil
}
func setStepIntoBreakpoints(dbp *Target, curfn *Function, text []AsmInstruction, topframe Stackframe, sameGCond ast.Expr) error {
gostmt := false
for _, instr := range text {
if instr.Loc.File != topframe.Current.File || instr.Loc.Line != topframe.Current.Line || !instr.IsCall() {
continue
}
if instr.DestLoc != nil {
if err := setStepIntoBreakpoint(dbp, curfn, []AsmInstruction{instr}, sameGCond); err != nil {
return err
}
if curfn != nil && curfn.Name != "runtime." && instr.DestLoc.Fn != nil && instr.DestLoc.Fn.Name == "runtime.newproc" {
// The current statement is a go statement, i.e. "go somecall()"
// We are excluding this check inside the runtime package because
// functions in the runtime package can call runtime.newproc directly.
gostmt = true
}
} else {
// Non-absolute call instruction, set a StepBreakpoint here
bp, err := allowDuplicateBreakpoint(dbp.SetBreakpoint(0, instr.Loc.PC, StepBreakpoint, sameGCond))
if err != nil {
return err
}
breaklet := bp.Breaklets[len(bp.Breaklets)-1]
breaklet.callback = stepIntoCallback
}
}
if gostmt {
setStepIntoNewProcBreakpoint(dbp, sameGCond)
}
return nil
}
// stepIntoCallback is a callback called when a StepBreakpoint is hit, it
// disassembles the current instruction to figure out its destination and
// sets a breakpoint on it.
func stepIntoCallback(curthread Thread, p *Target) (bool, error) {
if p.recman.GetDirection() != Forward {
// This should never happen, step into breakpoints with callbacks are only
// set when moving forward and direction changes are forbidden while
// breakpoints are set.
return true, nil
}
text, err := disassembleCurrentInstruction(p, curthread, 0)
if err != nil {
return false, err
}
var fn *Function
if loc, _ := curthread.Location(); loc != nil {
fn = loc.Fn
}
g, _ := GetG(curthread)
// here we either set a breakpoint into the destination of the CALL
// instruction or we determined that the called function is hidden,
// either way we need to resume execution
if err = setStepIntoBreakpoint(p, fn, text, sameGoroutineCondition(p.BinInfo(), g, curthread.ThreadID())); err != nil {
return false, err
}
return false, nil
}
func setStepIntoBreakpointsReverse(dbp *Target, text []AsmInstruction, topframe Stackframe, sameGCond ast.Expr) error {
bpmap := dbp.Breakpoints()
// Set a breakpoint after every CALL instruction
for i, instr := range text {
if instr.Loc.File != topframe.Current.File || !instr.IsCall() || instr.DestLoc == nil || instr.DestLoc.Fn == nil {
continue
}
if instr.DestLoc.Fn.privateRuntime() {
continue
}
if nextIdx := i + 1; nextIdx < len(text) {
_, ok := bpmap.M[text[nextIdx].Loc.PC]
if !ok {
if _, err := allowDuplicateBreakpoint(dbp.SetBreakpoint(0, text[nextIdx].Loc.PC, StepBreakpoint, sameGCond)); err != nil {
return err
}
}
}
}
return nil
}
func FindDeferReturnCalls(text []AsmInstruction) []uint64 {
const deferreturn = "runtime.deferreturn"
deferreturns := []uint64{}
// Find all runtime.deferreturn locations in the function
// See documentation of Breakpoint.DeferCond for why this is necessary
for _, instr := range text {
if instr.IsCall() && instr.DestLoc != nil && instr.DestLoc.Fn != nil && instr.DestLoc.Fn.Name == deferreturn {
deferreturns = append(deferreturns, instr.Loc.PC)
}
}
return deferreturns
}
// Removes instructions belonging to inlined calls of topframe from pcs.
// Inlined calls that belong to range-over-func bodies are not removed.
func removeInlinedCalls(pcs []uint64, topframe *Stackframe, bi *BinaryInfo) ([]uint64, error) {
// TODO(derekparker) it should be possible to still use some internal
// runtime information to do this.
if topframe.Call.Fn == nil || topframe.Call.Fn.cu.image.Stripped() {
return pcs, nil
}
topframeRangeParentName := topframe.Call.Fn.Name
if topframe.Call.Fn.extra(bi).rangeParent != nil {
topframeRangeParentName = topframe.Call.Fn.extra(bi).rangeParent.Name
}
dwarfTree, err := topframe.Current.Fn.cu.image.getDwarfTree(topframe.Current.Fn.offset)
if err != nil {
return pcs, err
}
color := make([]removePC, len(pcs))
removeInlinedCallsColor(topframe, topframeRangeParentName, pcs, color, dwarfTree)
out := make([]uint64, 0, len(pcs))
for i := range pcs {
if color[i] != removePCRemove {
out = append(out, pcs[i])
}
}
return out, nil
}
type removePC uint8
const (
removePCUnknown removePC = iota
removePCRemove
removePCKeep
)
// removeInlinedCallsColor sets color[i] to removePCRemove or removePCKeep
// depending on whether pcs[i] should be removed by removeInlinedCalls.
// This determination is made by checking, for each PC, what is the topmost
// inlined call.
func removeInlinedCallsColor(topframe *Stackframe, topframeRangeParentName string, pcs []uint64, color []removePC, e *godwarf.Tree) {
switch e.Tag {
case dwarf.TagSubprogram, dwarf.TagInlinedSubroutine, dwarf.TagLexDwarfBlock:
// ok
default:
return
}
for _, child := range e.Children {
removeInlinedCallsColor(topframe, topframeRangeParentName, pcs, color, child)
}
switch e.Tag {
case dwarf.TagInlinedSubroutine:
c := removePCRemove
if e.Offset == topframe.Call.Fn.offset {
c = removePCKeep
} else {
fnname, _ := e.Val(dwarf.AttrName).(string)
ridx := rangeParentName(fnname)
var rpn string
if ridx == -1 {
rpn = fnname
} else {
rpn = fnname[:ridx]
}
if rpn == topframeRangeParentName {
c = removePCKeep
}
}
for _, rng := range e.Ranges {
colorPCsBetween(pcs, color, c, rng[0], rng[1])
}
}
}
// colorPCsBetween sets color[i] to c if start <= pcs[i] < end
func colorPCsBetween(pcs []uint64, color []removePC, c removePC, start, end uint64) {
for i, pc := range pcs {
if color[i] == removePCUnknown && pc >= start && pc < end {
color[i] = c
}
}
}
func setStepIntoBreakpoint(dbp *Target, curfn *Function, text []AsmInstruction, cond ast.Expr) error {
if len(text) == 0 {
return nil
}
// If the current function is already a runtime function then
// setStepIntoBreakpoint is allowed to step into unexported runtime
// functions.
stepIntoUnexportedRuntime := curfn != nil && strings.HasPrefix(curfn.Name, "runtime.")
instr := text[0]
if instr.DestLoc == nil {
// Call destination couldn't be resolved because this was not the
// current instruction, therefore the step-into breakpoint can not be set.
return nil
}
pc := instr.DestLoc.PC
fn := instr.DestLoc.Fn
if dbp.BinInfo().Arch.Name == "ppc64le" && instr.Inst.OpcodeEquals(uint64(ppc64asm.BCLRL)) {
regs, err := dbp.CurrentThread().Registers()
if err != nil {
return err
}
lr := regs.LR()
fn = dbp.BinInfo().PCToFunc(lr)
}
// Skip unexported runtime functions
if !stepIntoUnexportedRuntime && fn != nil && fn.privateRuntime() {
return nil
}
//TODO(aarzilli): if we want to let users hide functions
// or entire packages from being stepped into with 'step'
// those extra checks should be done here.
// Skip InhibitStepInto functions for different arch.
if dbp.BinInfo().Arch.inhibitStepInto(dbp.BinInfo(), pc) {
return nil
}
fn, pc = skipAutogeneratedWrappersIn(dbp, fn, pc, false)
// We want to skip the function prologue but we should only do it if the
// destination address of the CALL instruction is the entry point of the
// function.
// Calls to runtime.duffzero and duffcopy inserted by the compiler can
// sometimes point inside the body of those functions, well after the
// prologue.
if fn != nil && fn.Entry == pc {
pc, _ = FirstPCAfterPrologue(dbp, fn, false)
}
// Set a breakpoint after the function's prologue
if _, err := allowDuplicateBreakpoint(dbp.SetBreakpoint(0, pc, NextBreakpoint, cond)); err != nil {
return err
}
return nil
}
// setStepIntoNewProcBreakpoint sets a temporary breakpoint on
// runtime.newproc that, when hit, clears all temporary breakpoints and sets
// a new temporary breakpoint on the starting function for the new
// goroutine.
func setStepIntoNewProcBreakpoint(p *Target, sameGCond ast.Expr) {
const (
runtimeNewprocFunc1 = "runtime.newproc.func1"
runtimeRunqput = "runtime.runqput"
)
rnf := p.BinInfo().LookupFunc()[runtimeNewprocFunc1]
if len(rnf) != 1 {
logflags.DebuggerLogger().Error("could not find " + runtimeNewprocFunc1)
return
}
text, err := Disassemble(p.Memory(), nil, p.Breakpoints(), p.BinInfo(), rnf[0].Entry, rnf[0].End)
if err != nil {
logflags.DebuggerLogger().Errorf("could not disassemble "+runtimeNewprocFunc1+": %v", err)
return
}
callfile, callline := "", 0
for _, instr := range text {
if instr.Kind == CallInstruction && instr.DestLoc != nil && instr.DestLoc.Fn != nil && instr.DestLoc.Fn.Name == runtimeRunqput {
callfile = instr.Loc.File
callline = instr.Loc.Line
break
}
}
if callfile == "" {
logflags.DebuggerLogger().Error("could not find " + runtimeRunqput + " call in " + runtimeNewprocFunc1)
return
}
var pc uint64
for _, pcstmt := range rnf[0].cu.lineInfo.LineToPCs(callfile, callline) {
if pcstmt.Stmt {
pc = pcstmt.PC
break
}
}
if pc == 0 {
logflags.DebuggerLogger().Errorf("could not set newproc breakpoint: location not found for " + runtimeRunqput + " call")
return
}
bp, err := p.SetBreakpoint(0, pc, StepIntoNewProcBreakpoint, sameGCond)
if err != nil {
logflags.DebuggerLogger().Errorf("could not set StepIntoNewProcBreakpoint: %v", err)
return
}
blet := bp.Breaklets[len(bp.Breaklets)-1]
blet.callback = func(th Thread, p *Target) (bool, error) {
// Clear temp breakpoints that exist and set a new one for goroutine
// newg.goid on the go statement's target
scope, err := ThreadScope(p, th)
if err != nil {
return false, err
}
v, err := scope.EvalExpression("newg.goid", loadSingleValue)
if err != nil {
return false, err
}
if v.Unreadable != nil {
return false, v.Unreadable
}
newGGoID, _ := constant.Int64Val(v.Value)
v, err = scope.EvalExpression("newg.startpc", loadSingleValue)
if err != nil {
return false, err
}
if v.Unreadable != nil {
return false, v.Unreadable
}
startpc, _ := constant.Int64Val(v.Value)
// Temp breakpoints must be cleared because the current goroutine could
// hit one of them before the new goroutine manages to start.
err = p.ClearSteppingBreakpoints()
if err != nil {
return false, err
}
newGCond := astutil.Eql(astutil.Sel(astutil.PkgVar("runtime", "curg"), "goid"), astutil.Int(newGGoID))
// We don't want to use startpc directly because it will be an
// autogenerated wrapper on some versions of Go. Addditionally, once we
// have the correct function we must also skip to prologue.
startfn := p.BinInfo().PCToFunc(uint64(startpc))
if startfn2, _ := skipAutogeneratedWrappersIn(p, startfn, uint64(startpc), true); startfn2 != nil {
startfn = startfn2
}
if startpc2, err := FirstPCAfterPrologue(p, startfn, false); err == nil {
startpc = int64(startpc2)
}
// The new breakpoint must have 'NextBreakpoint' kind because we want to
// stop on it.
_, err = p.SetBreakpoint(0, uint64(startpc), NextBreakpoint, newGCond)
return false, err // we don't want to stop at this breakpoint if there is no error
}
}
func allowDuplicateBreakpoint(bp *Breakpoint, err error) (*Breakpoint, error) {
if err != nil {
//lint:ignore S1020 this is clearer
if _, isexists := err.(BreakpointExistsError); isexists {
return bp, nil
}
}
return bp, err
}
func isAutogenerated(loc Location) bool {
return (loc.File == "<autogenerated>" && loc.Line == 1) || (loc.Fn != nil && loc.Fn.trampoline)
}
func isAutogeneratedOrDeferReturn(loc Location) bool {
return isAutogenerated(loc) || (loc.Fn != nil && loc.Fn.Name == "runtime.deferreturn")
}
// skipAutogeneratedWrappersIn skips autogenerated wrappers when setting a
// step-into breakpoint.
// If alwaysSkipFirst is set the first function is always skipped if it is
// autogenerated, even if it isn't a wrapper for the function it is calling.
// See genwrapper in: $GOROOT/src/cmd/compile/internal/gc/subr.go
func skipAutogeneratedWrappersIn(p Process, startfn *Function, startpc uint64, alwaysSkipFirst bool) (*Function, uint64) {
if startfn == nil {
return nil, startpc
}
fn := startfn
for count := 0; count < maxSkipAutogeneratedWrappers; count++ {
if !fn.cu.isgo {
// can't exit Go
return startfn, startpc
}
text, err := Disassemble(p.Memory(), nil, p.Breakpoints(), p.BinInfo(), fn.Entry, fn.End)
if err != nil {
break
}
if len(text) == 0 {
break
}
if !isAutogenerated(text[0].Loc) {
return fn, fn.Entry
}
tgtfns := []*Function{}
// collect all functions called by the current destination function
for _, instr := range text {
switch {
case instr.IsCall():
if instr.DestLoc == nil {
return startfn, startpc
}
if p.BinInfo().Arch.inhibitStepInto(p.BinInfo(), instr.DestLoc.PC) {
// ignored
continue
}
if instr.DestLoc.Fn == nil {
return startfn, startpc
}
// calls to non private runtime functions
if !instr.DestLoc.Fn.privateRuntime() {
tgtfns = append(tgtfns, instr.DestLoc.Fn)
}
case instr.IsJmp():
// unconditional jumps to a different function that isn't a private runtime function
if instr.DestLoc != nil && instr.DestLoc.Fn != fn && !instr.DestLoc.Fn.privateRuntime() {
tgtfns = append(tgtfns, instr.DestLoc.Fn)
}
}
}
if len(tgtfns) != 1 {
// too many or not enough function calls
break
}
tgtfn := tgtfns[0]
if alwaysSkipFirst {
alwaysSkipFirst = false
startfn, startpc = tgtfn, tgtfn.Entry
} else if strings.TrimSuffix(tgtfn.BaseName(), "-fm") != strings.TrimSuffix(fn.BaseName(), "-fm") {
return startfn, startpc
}
fn = tgtfn
}
return startfn, startpc
}
// skipAutogeneratedWrappersOut skip autogenerated wrappers when setting a
// step out breakpoint.
// See genwrapper in: $GOROOT/src/cmd/compile/internal/gc/subr.go
// It also skips runtime.deferreturn frames (which are only ever on the stack on Go 1.18 or later)
func skipAutogeneratedWrappersOut(tgt *Target, g *G, thread Thread, startTopframe, startRetframe *Stackframe) (topframe, retframe *Stackframe) {
topframe, retframe = startTopframe, startRetframe
if startTopframe.Ret == 0 {
return
}
if !isAutogeneratedOrDeferReturn(startRetframe.Current) {
return
}
retfn := thread.BinInfo().PCToFunc(startTopframe.Ret)
if retfn == nil {
return
}
if !retfn.cu.isgo {
return
}
var err error
var frames []Stackframe
if g == nil {
frames, err = ThreadStacktrace(tgt, thread, maxSkipAutogeneratedWrappers)
} else {
frames, err = GoroutineStacktrace(tgt, g, maxSkipAutogeneratedWrappers, 0)
}
if err != nil {
return
}
bi := thread.BinInfo()
for i := 1; i < len(frames); i++ {
frame := frames[i]
if frame.Current.Fn == nil {
return
}
file, line := bi.EntryLineForFunc(frame.Current.Fn)
if !isAutogeneratedOrDeferReturn(Location{File: file, Line: line, Fn: frame.Current.Fn}) {
return &frames[i-1], &frames[i]
}
}
return
}
// setDeferBreakpoint is a helper function used by next and StepOut to set a
// breakpoint on the first deferred function.
func setDeferBreakpoint(p *Target, text []AsmInstruction, topframe Stackframe, sameGCond ast.Expr, stepInto bool) (uint64, error) {
// Set breakpoint on the most recently deferred function (if any)
var deferpc uint64
if topframe.TopmostDefer != nil && topframe.TopmostDefer.DwrapPC != 0 {
_, _, deferfn := topframe.TopmostDefer.DeferredFunc(p)
if deferfn != nil {
var err error
deferpc, err = FirstPCAfterPrologue(p, deferfn, false)
if err != nil {
return 0, err
}
}
}
if deferpc != 0 && deferpc != topframe.Current.PC {
bp, err := allowDuplicateBreakpoint(p.SetBreakpoint(0, deferpc, NextDeferBreakpoint, sameGCond))
if err != nil {
return 0, err
}
if bp != nil && stepInto {
// If DeferReturns is set then the breakpoint will also be triggered when
// called from runtime.deferreturn. We only do this for the step command,
// not for next or stepout.
for _, breaklet := range bp.Breaklets {
if breaklet.Kind == NextDeferBreakpoint {
breaklet.DeferReturns = FindDeferReturnCalls(text)
break
}
}
}
}
return deferpc, nil
}
// findCallInstrForRet returns the PC address of the CALL instruction
// immediately preceding the instruction at ret.
func findCallInstrForRet(p Process, mem MemoryReadWriter, ret uint64, fn *Function) (uint64, error) {
text, err := disassemble(mem, nil, p.Breakpoints(), p.BinInfo(), fn.Entry, fn.End, false)
if err != nil {
return 0, err
}
var prevInstr AsmInstruction
for _, instr := range text {
if instr.Loc.PC == ret {
return prevInstr.Loc.PC, nil
}
prevInstr = instr
}
return 0, fmt.Errorf("could not find CALL instruction for address %#x in %s", ret, fn.Name)
}
// stepOutReverse sets a breakpoint on the CALL instruction that created the current frame, this is either:
// - the CALL instruction immediately preceding the return address of the
// current frame
// - the return address of the current frame if the current frame was
// created by a runtime.deferreturn run
// - the return address of the runtime.gopanic frame if the current frame
// was created by a panic
//
// This function is used to implement reversed StepOut
func stepOutReverse(p *Target, topframe, retframe Stackframe, sameGCond ast.Expr) error {
curthread := p.CurrentThread()
selg := p.SelectedGoroutine()
if selg != nil && selg.Thread != nil {
curthread = selg.Thread
}
callerText, err := disassemble(p.Memory(), nil, p.Breakpoints(), p.BinInfo(), retframe.Current.Fn.Entry, retframe.Current.Fn.End, false)
if err != nil {
return err
}
deferReturns := FindDeferReturnCalls(callerText)
var frames []Stackframe
if selg == nil {
frames, err = ThreadStacktrace(p, curthread, 3)
} else {
frames, err = GoroutineStacktrace(p, selg, 3, 0)
}
if err != nil {
return err
}
var callpc uint64
if ok, panicFrame := isPanicCall(frames); ok {
if len(frames) < panicFrame+2 || frames[panicFrame+1].Current.Fn == nil {
if panicFrame < len(frames) {
return &ErrNoSourceForPC{frames[panicFrame].Current.PC}
} else {
return &ErrNoSourceForPC{frames[0].Current.PC}
}
}
callpc, err = findCallInstrForRet(p, p.Memory(), frames[panicFrame].Ret, frames[panicFrame+1].Current.Fn)
if err != nil {
return err
}
} else {
callpc, err = findCallInstrForRet(p, p.Memory(), topframe.Ret, retframe.Current.Fn)
if err != nil {
return err
}
// check if the call instruction to this frame is a call to runtime.deferreturn
if len(frames) > 0 {
frames[0].Ret = callpc
}
if ok, pc := isDeferReturnCall(frames, deferReturns); ok && pc != 0 {
callpc = pc
}
}
_, err = allowDuplicateBreakpoint(p.SetBreakpoint(0, callpc, NextBreakpoint, sameGCond))
return err
}
// onNextGoroutine returns true if this thread is on the goroutine requested by the current 'next' command
func onNextGoroutine(tgt *Target, thread Thread, breakpoints *BreakpointMap) (bool, error) {
var breaklet *Breaklet
breakletSearch:
for i := range breakpoints.M {
for _, blet := range breakpoints.M[i].Breaklets {
if blet.Kind&steppingMask != 0 && blet.Cond != nil {
breaklet = blet
break breakletSearch
}
}
}
if breaklet == nil {
return false, nil
}
// Internal breakpoint conditions can take multiple different forms:
// Step into breakpoints:
// runtime.curg.goid == X
// Next or StepOut breakpoints:
// runtime.curg.goid == X && runtime.frameoff == Y
// Breakpoints that can be hit either by stepping on a line in the same
// function or by returning from the function:
// runtime.curg.goid == X && (runtime.frameoff == Y || runtime.frameoff == Z)
// Here we are only interested in testing the runtime.curg.goid clause.
w := onNextGoroutineWalker{tgt: tgt, thread: thread}
ast.Walk(&w, breaklet.Cond)
return w.ret, w.err
}
type onNextGoroutineWalker struct {
tgt *Target
thread Thread
ret bool
err error
}
func (w *onNextGoroutineWalker) Visit(n ast.Node) ast.Visitor {
if binx, isbin := n.(*ast.BinaryExpr); isbin && binx.Op == token.EQL {
x := exprToString(binx.X)
if x == "runtime.curg.goid" || x == "runtime.threadid" {
w.ret, w.err = evalBreakpointCondition(w.tgt, w.thread, n.(ast.Expr))
return nil
}
}
return w
}
func (t *Target) clearHardcodedBreakpoints() {
threads := t.ThreadList()
for _, thread := range threads {
if thread.Breakpoint().Breakpoint != nil && thread.Breakpoint().LogicalID() == hardcodedBreakpointID {
thread.Breakpoint().Active = false
thread.Breakpoint().Breakpoint = nil
}
}
}
// handleHardcodedBreakpoints looks for threads stopped at a hardcoded
// breakpoint (i.e. a breakpoint instruction, like INT 3, hardcoded in the
// program's text) and sets a fake breakpoint on them with logical id
// hardcodedBreakpointID.
// It checks trapthread and all threads that have SoftExc returning true.
func (t *Target) handleHardcodedBreakpoints(grp *TargetGroup, trapthread Thread, threads []Thread) error {
mem := t.Memory()
arch := t.BinInfo().Arch
recorded, _ := t.recman.Recorded()
isHardcodedBreakpoint := func(thread Thread, pc uint64) uint64 {
for _, bpinstr := range [][]byte{arch.BreakpointInstruction(), arch.AltBreakpointInstruction()} {
if bpinstr == nil {
continue
}
buf := make([]byte, len(bpinstr))
pc2 := pc
if arch.BreakInstrMovesPC() {
pc2 -= uint64(len(bpinstr))
}
_, _ = mem.ReadMemory(buf, pc2)
if bytes.Equal(buf, bpinstr) {
return uint64(len(bpinstr))
}
}
return 0
}
stepOverBreak := func(thread Thread, pc uint64) {
if arch.BreakInstrMovesPC() {
return
}
if recorded {
return
}
if bpsize := isHardcodedBreakpoint(thread, pc); bpsize > 0 {
setPC(thread, pc+bpsize)
}
}
setHardcodedBreakpoint := func(thread Thread, loc *Location) {
bpstate := thread.Breakpoint()
hcbp := &Breakpoint{}
bpstate.Active = true
bpstate.Breakpoint = hcbp
hcbp.FunctionName = loc.Fn.Name
hcbp.File = loc.File
hcbp.Line = loc.Line
hcbp.Addr = loc.PC
hcbp.Logical = &LogicalBreakpoint{}
hcbp.Logical.Name = HardcodedBreakpoint
hcbp.Breaklets = []*Breaklet{{Kind: UserBreakpoint, LogicalID: hardcodedBreakpointID}}
t.StopReason = StopHardcodedBreakpoint
}
for _, thread := range threads {
if thread.Breakpoint().Breakpoint != nil {
continue
}
if (thread.ThreadID() != trapthread.ThreadID()) && !thread.SoftExc() {
continue
}
if (thread.ThreadID() == trapthread.ThreadID()) && grp.cctx.GetManualStopRequested() {
continue
}
loc, err := thread.Location()
if err != nil || loc.Fn == nil {
continue
}
g, _ := GetG(thread)
switch {
case loc.Fn.Name == "runtime.breakpoint":
if recorded, _ := t.recman.Recorded(); recorded {
setHardcodedBreakpoint(thread, loc)
continue
}
stepOverBreak(thread, loc.PC)
// In linux-arm64, PtraceSingleStep seems cannot step over BRK instruction
// (linux-arm64 feature or kernel bug maybe).
if !arch.BreakInstrMovesPC() {
setPC(thread, loc.PC+uint64(arch.BreakpointSize()))
}
// Single-step current thread until we exit runtime.breakpoint and
// runtime.Breakpoint.
// On go < 1.8 it was sufficient to single-step twice on go1.8 a change
// to the compiler requires 4 steps.
if err := stepInstructionOut(grp, t, thread, "runtime.breakpoint", "runtime.Breakpoint"); err != nil {
return err
}
setHardcodedBreakpoint(thread, loc)
case g == nil || t.fncallForG[g.ID] == nil:
// Check that PC is inside a function (not the entry point) and the
// preceding instruction is a hardcoded breakpoint.
// We explicitly check for entry points of functions because the space
// between functions is usually filled with hardcoded breakpoints.
if (loc.Fn == nil || loc.Fn.Entry != loc.PC) && isHardcodedBreakpoint(thread, loc.PC) > 0 {
stepOverBreak(thread, loc.PC)
setHardcodedBreakpoint(thread, loc)
}
}
}
return nil
}