proc: use stack machine to evaluate expressions (#3508)

* proc: use stack machine to evaluate expressions

This commit splits expression evaluation into two parts. The first part (in
pkg/proc/evalop/evalcompile.go) "compiles" as ast.Expr into a list of
instructions (defined in pkg/proc/evalop/ops.go) for a stack machine
(defined by `proc.(*evalStack)`).
The second part is a stack machine (implemented by `proc.(*EvalScope).eval`
and `proc.(*EvalScope).evalOne`) that has two modes of operation: in the
main mode it executes inteructions from the list (by calling `evalOne`), in
the second mode it executes the call injection protocol by calling
`funcCallStep` repeatedly until it either the protocol finishes, needs more
input from the stack machine (to set call arguments) or fails.

This approach has several benefits:

- it is now possible to remove the goroutine we use to evaluate expression
  and the channel used to communicate with the Continue loop.
- every time we resume the target to execute the call injection protocol we
  need to update several local variables to match the changed state of the
  target, this is now done at the top level of the evaluation loop instead of
  being hidden inside a recurisive evaluator
- using runtime.Pin to pin addresses returned by an injected call would
  allow us to use a more natural evaluation order for function calls, which
  would solve some bugs #3310, allow users to inspect values returned by a
  call injection #1599 and allow implementing some other features #1465. Doing
  this with the recursive evaluator, while keeping backwards compatibility
  with versions of Go that do not have runtime.Pin is very hard. However after
  this change we can simply conditionally change how compileFunctionCall works
  and add some opcodes.

* review round 1

* review round 2
This commit is contained in:
Alessandro Arzilli
2023-10-17 20:21:59 +02:00
committed by GitHub
parent 788df884e6
commit 2c700230de
11 changed files with 1880 additions and 820 deletions

View File

@ -0,0 +1,55 @@
package godwarf
import (
"fmt"
"math/bits"
"reflect"
)
// FakeSliceType synthesizes a slice type with the given field type.
func FakeSliceType(fieldType Type) Type {
return &SliceType{
StructType: StructType{
CommonType: CommonType{
ByteSize: 24,
Name: "",
},
StructName: "[]" + fieldType.Common().Name,
Kind: "struct",
Field: nil,
},
ElemType: fieldType,
}
}
// FakeBasicType synthesizes a basic type numeric type (int8, uint16,
// float32, etc)
func FakeBasicType(name string, bitSize int) Type {
byteSize := bitSize / 8
szr := bits.OnesCount64(uint64(byteSize^(byteSize-1))) - 1 // position of rightmost 1 bit, minus 1
basic := func(kind reflect.Kind) BasicType {
return BasicType{
CommonType: CommonType{
ByteSize: int64(byteSize),
Name: fmt.Sprintf("%s%d", name, bitSize),
ReflectKind: kind,
},
BitSize: int64(bitSize),
BitOffset: 0,
}
}
switch name {
case "int":
return &IntType{BasicType: basic(reflect.Int8 + reflect.Kind(szr))}
case "uint":
return &UintType{BasicType: basic(reflect.Uint8 + reflect.Kind(szr))}
case "float":
return &FloatType{BasicType: basic(reflect.Float32 + reflect.Kind(szr-2))}
case "complex":
return &ComplexType{BasicType: basic(reflect.Complex64 + reflect.Kind(szr-3))}
default:
panic("unsupported")
}
}

File diff suppressed because it is too large Load Diff

View File

@ -1,6 +1,6 @@
//go:build !go1.18
package proc
package evalop
import "go/ast"

View File

@ -1,6 +1,6 @@
//go:build go1.18
package proc
package evalop
import "go/ast"

View File

@ -0,0 +1,556 @@
package evalop
import (
"bytes"
"errors"
"fmt"
"go/ast"
"go/constant"
"go/printer"
"go/token"
"strconv"
"strings"
"github.com/go-delve/delve/pkg/dwarf/godwarf"
"github.com/go-delve/delve/pkg/dwarf/reader"
)
var (
ErrFuncCallNotAllowed = errors.New("function calls not allowed without using 'call'")
)
type compileCtx struct {
evalLookup
ops []Op
allowCalls bool
curCall int
}
type evalLookup interface {
FindTypeExpr(ast.Expr) (godwarf.Type, error)
HasLocal(string) bool
HasGlobal(string, string) bool
HasBuiltin(string) bool
LookupRegisterName(string) (int, bool)
}
// Compile compiles the expression t into a list of instructions.
func Compile(lookup evalLookup, t ast.Expr) ([]Op, error) {
ctx := &compileCtx{evalLookup: lookup, allowCalls: true}
err := ctx.compileAST(t)
if err != nil {
return nil, err
}
err = ctx.depthCheck(1)
if err != nil {
return ctx.ops, err
}
return ctx.ops, nil
}
// CompileSet compiles the expression setting lhe to rhe into a list of
// instructions.
func CompileSet(lookup evalLookup, lhe, rhe ast.Expr) ([]Op, error) {
ctx := &compileCtx{evalLookup: lookup, allowCalls: true}
err := ctx.compileAST(rhe)
if err != nil {
return nil, err
}
if isStringLiteral(rhe) {
ctx.compileAllocLiteralString()
}
err = ctx.compileAST(lhe)
if err != nil {
return nil, err
}
ctx.pushOp(&SetValue{lhe: lhe, Rhe: rhe})
err = ctx.depthCheck(0)
if err != nil {
return ctx.ops, err
}
return ctx.ops, nil
}
func (ctx *compileCtx) compileAllocLiteralString() {
ctx.pushOp(&CallInjectionAllocString{Phase: 0})
ctx.pushOp(&CallInjectionAllocString{Phase: 1})
ctx.pushOp(&CallInjectionAllocString{Phase: 2})
}
func (ctx *compileCtx) pushOp(op Op) {
ctx.ops = append(ctx.ops, op)
}
// depthCheck validates the list of instructions produced by Compile and
// CompileSet by peforming a stack depth check.
// It calculates the depth of the stack at every instruction in ctx.ops and
// checks that they have enough arguments to execute. For instructions that
// can be reached through multiple paths (because of a jump) it checks that
// all paths reach the instruction with the same stack depth.
// Finally it checks that the stack depth after all instructions have
// executed is equal to endDepth.
func (ctx *compileCtx) depthCheck(endDepth int) error {
depth := make([]int, len(ctx.ops)+1) // depth[i] is the depth of the stack before i-th instruction
for i := range depth {
depth[i] = -1
}
depth[0] = 0
var err error
checkAndSet := func(j, d int) { // sets depth[j] to d after checking that we can
if depth[j] < 0 {
depth[j] = d
}
if d != depth[j] {
err = fmt.Errorf("internal debugger error: depth check error at instruction %d: expected depth %d have %d (jump target)\n%s", j, d, depth[j], Listing(depth, ctx.ops))
}
}
for i, op := range ctx.ops {
npop, npush := op.depthCheck()
if depth[i] < npop {
return fmt.Errorf("internal debugger error: depth check error at instruction %d: expected at least %d have %d\n%s", i, npop, depth[i], Listing(depth, ctx.ops))
}
d := depth[i] - npop + npush
checkAndSet(i+1, d)
if jmp, _ := op.(*Jump); jmp != nil {
checkAndSet(jmp.Target, d)
}
if err != nil {
return err
}
}
if depth[len(ctx.ops)] != endDepth {
return fmt.Errorf("internal debugger error: depth check failed: depth at the end is not %d (got %d)\n%s", depth[len(ctx.ops)], endDepth, Listing(depth, ctx.ops))
}
return nil
}
func (ctx *compileCtx) compileAST(t ast.Expr) error {
switch node := t.(type) {
case *ast.CallExpr:
return ctx.compileTypeCastOrFuncCall(node)
case *ast.Ident:
return ctx.compileIdent(node)
case *ast.ParenExpr:
// otherwise just eval recursively
return ctx.compileAST(node.X)
case *ast.SelectorExpr: // <expression>.<identifier>
switch x := node.X.(type) {
case *ast.Ident:
switch {
case x.Name == "runtime" && node.Sel.Name == "curg":
ctx.pushOp(&PushCurg{})
case x.Name == "runtime" && node.Sel.Name == "frameoff":
ctx.pushOp(&PushFrameoff{})
case x.Name == "runtime" && node.Sel.Name == "threadid":
ctx.pushOp(&PushThreadID{})
case ctx.HasLocal(x.Name):
ctx.pushOp(&PushLocal{x.Name})
ctx.pushOp(&Select{node.Sel.Name})
case ctx.HasGlobal(x.Name, node.Sel.Name):
ctx.pushOp(&PushPackageVar{x.Name, node.Sel.Name})
default:
return ctx.compileUnary(node.X, &Select{node.Sel.Name})
}
case *ast.BasicLit: // try to accept "package/path".varname syntax for package variables
s, err := strconv.Unquote(x.Value)
if err != nil {
return err
}
if ctx.HasGlobal(s, node.Sel.Name) {
ctx.pushOp(&PushPackageVar{s, node.Sel.Name})
return nil
}
return ctx.compileUnary(node.X, &Select{node.Sel.Name})
default:
return ctx.compileUnary(node.X, &Select{node.Sel.Name})
}
case *ast.TypeAssertExpr: // <expression>.(<type>)
return ctx.compileTypeAssert(node)
case *ast.IndexExpr:
return ctx.compileBinary(node.X, node.Index, nil, &Index{node})
case *ast.SliceExpr:
if node.Slice3 {
return fmt.Errorf("3-index slice expressions not supported")
}
return ctx.compileReslice(node)
case *ast.StarExpr:
// pointer dereferencing *<expression>
return ctx.compileUnary(node.X, &PointerDeref{node})
case *ast.UnaryExpr:
// The unary operators we support are +, - and & (note that unary * is parsed as ast.StarExpr)
switch node.Op {
case token.AND:
return ctx.compileUnary(node.X, &AddrOf{node})
default:
return ctx.compileUnary(node.X, &Unary{node})
}
case *ast.BinaryExpr:
switch node.Op {
case token.INC, token.DEC, token.ARROW:
return fmt.Errorf("operator %s not supported", node.Op.String())
}
// short circuits logical operators
var sop *Jump
switch node.Op {
case token.LAND:
sop = &Jump{When: JumpIfFalse, Node: node.X}
case token.LOR:
sop = &Jump{When: JumpIfTrue, Node: node.X}
}
err := ctx.compileBinary(node.X, node.Y, sop, &Binary{node})
if err != nil {
return err
}
if sop != nil {
sop.Target = len(ctx.ops)
ctx.pushOp(&BoolToConst{})
}
case *ast.BasicLit:
ctx.pushOp(&PushConst{constant.MakeFromLiteral(node.Value, node.Kind, 0)})
default:
return fmt.Errorf("expression %T not implemented", t)
}
return nil
}
func (ctx *compileCtx) compileTypeCastOrFuncCall(node *ast.CallExpr) error {
if len(node.Args) != 1 {
// Things that have more or less than one argument are always function calls.
return ctx.compileFunctionCall(node)
}
ambiguous := func() error {
// Ambiguous, could be a function call or a type cast, if node.Fun can be
// evaluated then try to treat it as a function call, otherwise try the
// type cast.
ctx2 := &compileCtx{evalLookup: ctx.evalLookup}
err0 := ctx2.compileAST(node.Fun)
if err0 == nil {
return ctx.compileFunctionCall(node)
}
return ctx.compileTypeCast(node, err0)
}
fnnode := node.Fun
for {
fnnode = removeParen(fnnode)
n, _ := fnnode.(*ast.StarExpr)
if n == nil {
break
}
fnnode = n.X
}
switch n := fnnode.(type) {
case *ast.BasicLit:
// It can only be a ("type string")(x) type cast
return ctx.compileTypeCast(node, nil)
case *ast.ArrayType, *ast.StructType, *ast.FuncType, *ast.InterfaceType, *ast.MapType, *ast.ChanType:
return ctx.compileTypeCast(node, nil)
case *ast.SelectorExpr:
if _, isident := n.X.(*ast.Ident); isident {
if typ, _ := ctx.FindTypeExpr(n); typ != nil {
return ctx.compileTypeCast(node, nil)
}
return ambiguous()
}
return ctx.compileFunctionCall(node)
case *ast.Ident:
if ctx.HasBuiltin(n.Name) {
return ctx.compileFunctionCall(node)
}
if ctx.HasGlobal("", n.Name) || ctx.HasLocal(n.Name) {
return ctx.compileFunctionCall(node)
}
return ctx.compileTypeCast(node, fmt.Errorf("could not find symbol value for %s", n.Name))
case *ast.IndexExpr:
// Ambiguous, could be a parametric type
switch n.X.(type) {
case *ast.Ident, *ast.SelectorExpr:
// Do the type-cast first since evaluating node.Fun could be expensive.
err := ctx.compileTypeCast(node, nil)
if err == nil || err != reader.ErrTypeNotFound {
return err
}
return ctx.compileFunctionCall(node)
default:
return ctx.compileFunctionCall(node)
}
case *astIndexListExpr:
return ctx.compileTypeCast(node, nil)
default:
// All other expressions must be function calls
return ctx.compileFunctionCall(node)
}
}
func (ctx *compileCtx) compileTypeCast(node *ast.CallExpr, ambiguousErr error) error {
err := ctx.compileAST(node.Args[0])
if err != nil {
return err
}
fnnode := node.Fun
// remove all enclosing parenthesis from the type name
fnnode = removeParen(fnnode)
targetTypeStr := exprToString(removeParen(node.Fun))
styp, err := ctx.FindTypeExpr(fnnode)
if err != nil {
switch targetTypeStr {
case "[]byte", "[]uint8":
styp = godwarf.FakeSliceType(godwarf.FakeBasicType("uint", 8))
case "[]int32", "[]rune":
styp = godwarf.FakeSliceType(godwarf.FakeBasicType("int", 32))
default:
if ambiguousErr != nil && err == reader.ErrTypeNotFound {
return fmt.Errorf("could not evaluate function or type %s: %v", exprToString(node.Fun), ambiguousErr)
}
return err
}
}
ctx.pushOp(&TypeCast{DwarfType: styp, Node: node})
return nil
}
func (ctx *compileCtx) compileBuiltinCall(builtin string, args []ast.Expr) error {
for _, arg := range args {
err := ctx.compileAST(arg)
if err != nil {
return err
}
}
ctx.pushOp(&BuiltinCall{builtin, args})
return nil
}
func (ctx *compileCtx) compileIdent(node *ast.Ident) error {
switch {
case ctx.HasLocal(node.Name):
ctx.pushOp(&PushLocal{node.Name})
case ctx.HasGlobal("", node.Name):
ctx.pushOp(&PushPackageVar{"", node.Name})
case node.Name == "true" || node.Name == "false":
ctx.pushOp(&PushConst{constant.MakeBool(node.Name == "true")})
case node.Name == "nil":
ctx.pushOp(&PushNil{})
default:
found := false
if regnum, ok := ctx.LookupRegisterName(node.Name); ok {
ctx.pushOp(&PushRegister{regnum, node.Name})
found = true
}
if !found {
return fmt.Errorf("could not find symbol value for %s", node.Name)
}
}
return nil
}
func (ctx *compileCtx) compileUnary(expr ast.Expr, op Op) error {
err := ctx.compileAST(expr)
if err != nil {
return err
}
ctx.pushOp(op)
return nil
}
func (ctx *compileCtx) compileTypeAssert(node *ast.TypeAssertExpr) error {
err := ctx.compileAST(node.X)
if err != nil {
return err
}
// Accept .(data) as a type assertion that always succeeds, so that users
// can access the data field of an interface without actually having to
// type the concrete type.
if idtyp, isident := node.Type.(*ast.Ident); !isident || idtyp.Name != "data" {
typ, err := ctx.FindTypeExpr(node.Type)
if err != nil {
return err
}
ctx.pushOp(&TypeAssert{typ, node})
return nil
}
ctx.pushOp(&TypeAssert{nil, node})
return nil
}
func (ctx *compileCtx) compileBinary(a, b ast.Expr, sop *Jump, op Op) error {
err := ctx.compileAST(a)
if err != nil {
return err
}
if sop != nil {
ctx.pushOp(sop)
}
err = ctx.compileAST(b)
if err != nil {
return err
}
ctx.pushOp(op)
return nil
}
func (ctx *compileCtx) compileReslice(node *ast.SliceExpr) error {
err := ctx.compileAST(node.X)
if err != nil {
return err
}
hasHigh := false
if node.High != nil {
hasHigh = true
err = ctx.compileAST(node.High)
if err != nil {
return err
}
}
if node.Low != nil {
err = ctx.compileAST(node.Low)
if err != nil {
return err
}
} else {
ctx.pushOp(&PushConst{constant.MakeInt64(0)})
}
ctx.pushOp(&Reslice{Node: node, HasHigh: hasHigh})
return nil
}
func (ctx *compileCtx) compileFunctionCall(node *ast.CallExpr) error {
if fnnode, ok := node.Fun.(*ast.Ident); ok {
if ctx.HasBuiltin(fnnode.Name) {
return ctx.compileBuiltinCall(fnnode.Name, node.Args)
}
}
if !ctx.allowCalls {
return ErrFuncCallNotAllowed
}
id := ctx.curCall
ctx.curCall++
oldAllowCalls := ctx.allowCalls
oldOps := ctx.ops
ctx.allowCalls = false
err := ctx.compileAST(node.Fun)
ctx.allowCalls = oldAllowCalls
hasFunc := false
if err != nil {
ctx.ops = oldOps
if err != ErrFuncCallNotAllowed {
return err
}
} else {
hasFunc = true
}
ctx.pushOp(&CallInjectionStart{HasFunc: hasFunc, id: id, Node: node})
// CallInjectionStart pushes true on the stack if it needs the function argument re-evaluated
var jmpif *Jump
if hasFunc {
jmpif = &Jump{When: JumpIfFalse, Pop: true}
ctx.pushOp(jmpif)
}
ctx.pushOp(&Pop{})
err = ctx.compileAST(node.Fun)
if err != nil {
return err
}
if jmpif != nil {
jmpif.Target = len(ctx.ops)
}
ctx.pushOp(&CallInjectionSetTarget{id: id})
for i, arg := range node.Args {
err := ctx.compileAST(arg)
if err != nil {
return fmt.Errorf("error evaluating %q as argument %d in function %s: %v", exprToString(arg), i+1, exprToString(node.Fun), err)
}
if isStringLiteral(arg) {
ctx.compileAllocLiteralString()
}
ctx.pushOp(&CallInjectionCopyArg{id: id, ArgNum: i, ArgExpr: arg})
}
ctx.pushOp(&CallInjectionComplete{id: id})
return nil
}
func Listing(depth []int, ops []Op) string {
if depth == nil {
depth = make([]int, len(ops)+1)
}
buf := new(strings.Builder)
for i, op := range ops {
fmt.Fprintf(buf, " %3d (%2d->%2d) %#v\n", i, depth[i], depth[i+1], op)
}
return buf.String()
}
func isStringLiteral(expr ast.Expr) bool {
switch expr := expr.(type) {
case *ast.BasicLit:
return expr.Kind == token.STRING
case *ast.BinaryExpr:
if expr.Op == token.ADD {
return isStringLiteral(expr.X) && isStringLiteral(expr.Y)
}
case *ast.ParenExpr:
return isStringLiteral(expr.X)
}
return false
}
func removeParen(n ast.Expr) ast.Expr {
for {
p, ok := n.(*ast.ParenExpr)
if !ok {
break
}
n = p.X
}
return n
}
func exprToString(t ast.Expr) string {
var buf bytes.Buffer
printer.Fprint(&buf, token.NewFileSet(), t)
return buf.String()
}

View File

@ -0,0 +1,75 @@
package evalop
import (
"go/ast"
"go/parser"
"go/token"
"testing"
)
func assertNoError(err error, t testing.TB, s string) {
t.Helper()
if err != nil {
t.Fatalf("failed assertion %s: %s\n", s, err)
}
}
func TestEvalSwitchExhaustiveness(t *testing.T) {
// Checks that the switch statement in (*EvalScope).evalOne of
// pkg/proc/eval.go exhaustively covers all implementations of the
// evalop.Op interface.
ops := make(map[string]bool)
var fset, fset2 token.FileSet
f, err := parser.ParseFile(&fset, "ops.go", nil, 0)
assertNoError(err, t, "ParseFile")
for _, decl := range f.Decls {
decl, _ := decl.(*ast.FuncDecl)
if decl == nil {
continue
}
if decl.Name.Name != "depthCheck" {
continue
}
ops[decl.Recv.List[0].Type.(*ast.StarExpr).X.(*ast.Ident).Name] = false
}
f, err = parser.ParseFile(&fset2, "../eval.go", nil, 0)
assertNoError(err, t, "ParseFile")
for _, decl := range f.Decls {
decl, _ := decl.(*ast.FuncDecl)
if decl == nil {
continue
}
if decl.Name.Name != "executeOp" {
continue
}
ast.Inspect(decl, func(n ast.Node) bool {
sw, _ := n.(*ast.TypeSwitchStmt)
if sw == nil {
return true
}
for _, c := range sw.Body.List {
if len(c.(*ast.CaseClause).List) == 0 {
// default clause
continue
}
sel := c.(*ast.CaseClause).List[0].(*ast.StarExpr).X.(*ast.SelectorExpr)
if sel.X.(*ast.Ident).Name != "evalop" {
t.Fatalf("wrong case statement at: %v", fset2.Position(sel.Pos()))
}
ops[sel.Sel.Name] = true
}
return false
})
}
for op := range ops {
if !ops[op] {
t.Errorf("evalop.Op %s not used in evalOne", op)
}
}
}

250
pkg/proc/evalop/ops.go Normal file
View File

@ -0,0 +1,250 @@
package evalop
import (
"go/ast"
"go/constant"
"github.com/go-delve/delve/pkg/dwarf/godwarf"
)
// Op is a stack machine opcode
type Op interface {
depthCheck() (npop, npush int)
}
// PushCurg pushes the current goroutine on the stack.
type PushCurg struct {
}
func (*PushCurg) depthCheck() (npop, npush int) { return 0, 1 }
// PushFrameoff pushes the frame offset for the current frame on the stack.
type PushFrameoff struct {
}
func (*PushFrameoff) depthCheck() (npop, npush int) { return 0, 1 }
// PushThreadID pushes the ID of the current thread on the stack.
type PushThreadID struct {
}
func (*PushThreadID) depthCheck() (npop, npush int) { return 0, 1 }
// PushConst pushes a constant on the stack.
type PushConst struct {
Value constant.Value
}
func (*PushConst) depthCheck() (npop, npush int) { return 0, 1 }
// PushLocal pushes the local variable with the given name on the stack.
type PushLocal struct {
Name string
}
func (*PushLocal) depthCheck() (npop, npush int) { return 0, 1 }
// PushNil pushes an untyped nil on the stack.
type PushNil struct {
}
func (*PushNil) depthCheck() (npop, npush int) { return 0, 1 }
// PushRegister pushes the CPU register Regnum on the stack.
type PushRegister struct {
Regnum int
Regname string
}
func (*PushRegister) depthCheck() (npop, npush int) { return 0, 1 }
// PushPackageVar pushes a package variable on the stack.
type PushPackageVar struct {
PkgName, Name string // if PkgName == "" use current function's package
}
func (*PushPackageVar) depthCheck() (npop, npush int) { return 0, 1 }
// Select replaces the topmost stack variable v with v.Name.
type Select struct {
Name string
}
func (*Select) depthCheck() (npop, npush int) { return 1, 1 }
// TypeAssert replaces the topmost stack variable v with v.(DwarfType).
type TypeAssert struct {
DwarfType godwarf.Type
Node *ast.TypeAssertExpr
}
func (*TypeAssert) depthCheck() (npop, npush int) { return 1, 1 }
// PointerDeref replaces the topmost stack variable v with *v.
type PointerDeref struct {
Node *ast.StarExpr
}
func (*PointerDeref) depthCheck() (npop, npush int) { return 1, 1 }
// Unary applies the given unary operator to the topmost stack variable.
type Unary struct {
Node *ast.UnaryExpr
}
func (*Unary) depthCheck() (npop, npush int) { return 1, 1 }
// AddrOf replaces the topmost stack variable v with &v.
type AddrOf struct {
Node *ast.UnaryExpr
}
func (*AddrOf) depthCheck() (npop, npush int) { return 1, 1 }
// TypeCast replaces the topmost stack variable v with (DwarfType)(v).
type TypeCast struct {
DwarfType godwarf.Type
Node *ast.CallExpr
}
func (*TypeCast) depthCheck() (npop, npush int) { return 1, 1 }
// Reslice implements a reslice operation.
// If HasHigh is set it pops three variables, low, high and v, and pushes
// v[low:high].
// Otherwise it pops two variables, low and v, and pushes v[low:].
type Reslice struct {
HasHigh bool
Node *ast.SliceExpr
}
func (op *Reslice) depthCheck() (npop, npush int) {
if op.HasHigh {
return 3, 1
} else {
return 2, 1
}
}
// Index pops two variables, idx and v, and pushes v[idx].
type Index struct {
Node *ast.IndexExpr
}
func (*Index) depthCheck() (npop, npush int) { return 2, 1 }
// Jump looks at the topmost stack variable and if it satisifies the
// condition specified by When it jumps to the stack machine instruction at
// Target+1.
// If Pop is set the topmost stack variable is also popped.
type Jump struct {
When JumpCond
Pop bool
Target int
Node ast.Expr
}
func (jmpif *Jump) depthCheck() (npop, npush int) {
if jmpif.Pop {
return 1, 0
}
return 0, 0
}
// JumpCond specifies a condition for the Jump instruction.
type JumpCond uint8
const (
JumpIfFalse JumpCond = iota
JumpIfTrue
)
// Binary pops two variables from the stack, applies the specified binary
// operator to them and pushes the result back on the stack.
type Binary struct {
Node *ast.BinaryExpr
}
func (*Binary) depthCheck() (npop, npush int) { return 2, 1 }
// BoolToConst pops the topmost variable from the stack, which must be a
// boolean variable, and converts it to a constant.
type BoolToConst struct {
}
func (*BoolToConst) depthCheck() (npop, npush int) { return 1, 1 }
// Pop removes the topmost variable from the stack.
type Pop struct {
}
func (*Pop) depthCheck() (npop, npush int) { return 1, 0 }
// BuiltinCall pops len(Args) argument from the stack, calls the specified
// builtin on them and pushes the result back on the stack.
type BuiltinCall struct {
Name string
Args []ast.Expr
}
func (bc *BuiltinCall) depthCheck() (npop, npush int) {
return len(bc.Args), 1
}
// CallInjectionStart starts call injection by calling
// runtime.debugCallVn.
type CallInjectionStart struct {
id int // identifier for all the call injection instructions that belong to the same sequence, this only exists to make reading listings easier
HasFunc bool // target function already pushed on the stack
Node *ast.CallExpr
}
func (*CallInjectionStart) depthCheck() (npop, npush int) { return 0, 1 }
// CallInjectionSetTarget starts the call injection, after
// runtime.debugCallVn set up the stack for us, by copying the entry point
// of the function, setting the closure register and copying the receiver.
type CallInjectionSetTarget struct {
id int
}
func (*CallInjectionSetTarget) depthCheck() (npop, npush int) { return 1, 0 }
// CallInjectionCopyArg copies one argument for call injection.
type CallInjectionCopyArg struct {
id int
ArgNum int
ArgExpr ast.Expr
}
func (*CallInjectionCopyArg) depthCheck() (npop, npush int) { return 1, 0 }
// CallInjectionComplete resumes target execution so that the injected call can run.
type CallInjectionComplete struct {
id int
}
func (*CallInjectionComplete) depthCheck() (npop, npush int) { return 0, 1 }
// CallInjectionAllocString uses the call injection protocol to allocate the
// value of a string literal somewhere on the target's memory so that it can
// be assigned to a variable (or passed to a function).
// There are three phases to CallInjectionAllocString, distinguished by the
// Phase field. They must always appear in sequence in the program:
//
// CallInjectionAllocString{Phase: 0}
// CallInjectionAllocString{Phase: 1}
// CallInjectionAllocString{Phase: 2}
type CallInjectionAllocString struct {
Phase int
}
func (op *CallInjectionAllocString) depthCheck() (npop, npush int) { return 1, 1 }
// SetValue pops to variables from the stack, lhv and rhv, and sets lhv to
// rhv.
type SetValue struct {
lhe, Rhe ast.Expr
}
func (*SetValue) depthCheck() (npop, npush int) { return 2, 0 }

View File

@ -19,12 +19,13 @@ import (
"github.com/go-delve/delve/pkg/dwarf/regnum"
"github.com/go-delve/delve/pkg/goversion"
"github.com/go-delve/delve/pkg/logflags"
"github.com/go-delve/delve/pkg/proc/evalop"
)
// This file implements the function call injection introduced in go1.11.
//
// The protocol is described in $GOROOT/src/runtime/asm_amd64.s in the
// comments for function runtime·debugCallV1.
// comments for function runtime·debugCallVn.
//
// The main entry point is EvalExpressionWithCalls which will start a goroutine to
// evaluate the provided expression.
@ -37,7 +38,12 @@ import (
// hits a breakpoint in the call injection protocol.
//
// The work of setting up the function call and executing the protocol is
// done by evalFunctionCall and funcCallStep.
// done by:
//
// - evalop.CallInjectionStart
// - evalop.CallInjectionSetTarget
// - evalCallInjectionCopyArg
// - evalCallInjectionComplete
const (
debugCallFunctionNamePrefix1 = "debugCall"
@ -56,7 +62,6 @@ var (
errTooManyArguments = errors.New("too many arguments")
errNotEnoughArguments = errors.New("not enough arguments")
errNotAGoFunction = errors.New("not a Go function")
errFuncCallNotAllowed = errors.New("function calls not allowed without using 'call'")
errFuncCallNotAllowedStrAlloc = errors.New("literal string can not be allocated because function calls are not allowed without using 'call'")
)
@ -82,9 +87,16 @@ type functionCallState struct {
// panicvar is a variable used to store the value of the panic, if the
// called function panics.
panicvar *Variable
// lateCallFailure is set to true if the function call could not be
// completed after we started evaluating the arguments.
lateCallFailure bool
// undoInjection is set after evalop.CallInjectionSetTarget runs and cleared by evalCallInjectionComplete
// it contains informations on how to undo a function call injection without running it
undoInjection *undoInjection
protocolReg uint64
debugCallName string
}
type undoInjection struct {
oldpc, oldlr uint64
}
type callContext struct {
@ -250,23 +262,10 @@ func finishEvalExpressionWithCalls(t *Target, g *G, contReq continueRequest, ok
return err
}
// evalFunctionCall evaluates a function call.
// If this is a built-in function it's evaluated directly.
// Otherwise this will start the function call injection protocol and
// request that the target process resumes.
// See the comment describing the field EvalScope.callCtx for a description
// of the preconditions that make starting the function call protocol
// possible.
// See runtime.debugCallV1 in $GOROOT/src/runtime/asm_amd64.s for a
// description of the protocol.
func evalFunctionCall(scope *EvalScope, node *ast.CallExpr) (*Variable, error) {
r, err := scope.evalBuiltinCall(node)
if r != nil || err != nil {
// it was a builtin call
return r, err
}
func (scope *EvalScope) evalCallInjectionStart(op *evalop.CallInjectionStart, stack *evalStack) {
if scope.callCtx == nil {
return nil, errFuncCallNotAllowed
stack.err = evalop.ErrFuncCallNotAllowed
return
}
thread := scope.g.Thread
stacklo := scope.g.stack.lo
@ -278,63 +277,79 @@ func evalFunctionCall(scope *EvalScope, node *ast.CallExpr) (*Variable, error) {
thread = scope.callCtx.injectionThread
g2, err := GetG(thread)
if err != nil {
return nil, err
stack.err = err
return
}
stacklo = g2.stack.lo
}
if thread == nil {
return nil, errGoroutineNotRunning
stack.err = errGoroutineNotRunning
return
}
p := scope.callCtx.p
bi := scope.BinInfo
if !p.SupportsFunctionCalls() {
return nil, errFuncCallUnsupportedBackend
stack.err = errFuncCallUnsupportedBackend
return
}
dbgcallfn, dbgcallversion := debugCallFunction(bi)
if dbgcallfn == nil {
return nil, errFuncCallUnsupported
stack.err = errFuncCallUnsupported
return
}
// check that there are at least 256 bytes free on the stack
regs, err := thread.Registers()
if err != nil {
return nil, err
stack.err = err
return
}
regs, err = regs.Copy()
if err != nil {
return nil, err
stack.err = err
return
}
if regs.SP()-bi.Arch.debugCallMinStackSize <= stacklo {
return nil, errNotEnoughStack
stack.err = errNotEnoughStack
return
}
protocolReg, ok := debugCallProtocolReg(bi.Arch.Name, dbgcallversion)
if !ok {
return nil, errFuncCallUnsupported
stack.err = errFuncCallUnsupported
return
}
if bi.Arch.RegistersToDwarfRegisters(0, regs).Reg(protocolReg) == nil {
return nil, errFuncCallUnsupportedBackend
stack.err = errFuncCallUnsupportedBackend
return
}
fncall := functionCallState{
expr: node,
savedRegs: regs,
expr: op.Node,
savedRegs: regs,
protocolReg: protocolReg,
debugCallName: dbgcallfn.Name,
}
err = funcCallEvalFuncExpr(scope, &fncall, false)
if err != nil {
return nil, err
if op.HasFunc {
err = funcCallEvalFuncExpr(scope, stack, &fncall)
if err != nil {
stack.err = err
return
}
}
switch bi.Arch.Name {
case "amd64":
if err := callOP(bi, thread, regs, dbgcallfn.Entry); err != nil {
return nil, err
stack.err = err
return
}
// write the desired argument frame size at SP-(2*pointer_size) (the extra pointer is the saved PC)
if err := writePointer(bi, scope.Mem, regs.SP()-3*uint64(bi.Arch.PtrSize()), uint64(fncall.argFrameSize)); err != nil {
return nil, err
stack.err = err
return
}
case "arm64", "ppc64le":
// debugCallV2 on arm64 needs a special call sequence, callOP can not be used
@ -347,31 +362,37 @@ func evalFunctionCall(scope *EvalScope, node *ast.CallExpr) (*Variable, error) {
}
sp -= spOffset
if err := setSP(thread, sp); err != nil {
return nil, err
stack.err = err
return
}
if err := writePointer(bi, scope.Mem, sp, regs.LR()); err != nil {
return nil, err
stack.err = err
return
}
if err := setLR(thread, regs.PC()); err != nil {
return nil, err
stack.err = err
return
}
if err := writePointer(bi, scope.Mem, sp-spOffset, uint64(fncall.argFrameSize)); err != nil {
return nil, err
stack.err = err
return
}
regs, err = thread.Registers()
if err != nil {
return nil, err
stack.err = err
return
}
regs, err = regs.Copy()
if err != nil {
return nil, err
stack.err = err
return
}
fncall.savedRegs = regs
err = setPC(thread, dbgcallfn.Entry)
if err != nil {
return nil, err
stack.err = err
return
}
}
fncallLog("function call initiated %v frame size %d goroutine %d (thread %d)", fncall.fn, fncall.argFrameSize, scope.g.ID, thread.ThreadID())
@ -379,58 +400,38 @@ func evalFunctionCall(scope *EvalScope, node *ast.CallExpr) (*Variable, error) {
thread.Breakpoint().Clear() // since we moved address in PC the thread is no longer stopped at a breakpoint, leaving the breakpoint set will confuse Continue
p.fncallForG[scope.g.ID].startThreadID = thread.ThreadID()
spoff := int64(scope.Regs.Uint64Val(scope.Regs.SPRegNum)) - int64(scope.g.stack.hi)
bpoff := int64(scope.Regs.Uint64Val(scope.Regs.BPRegNum)) - int64(scope.g.stack.hi)
fboff := scope.Regs.FrameBase - int64(scope.g.stack.hi)
for {
scope.callCtx.injectionThread = nil
g := scope.callCtx.doContinue()
// Go 1.15 will move call injection execution to a different goroutine,
// but we want to keep evaluation on the original goroutine.
if g.ID == scope.g.ID {
scope.g = g
} else {
// We are in Go 1.15 and we switched to a new goroutine, the original
// goroutine is now parked and therefore does not have a thread
// associated.
scope.g.Thread = nil
scope.g.Status = Gwaiting
scope.callCtx.injectionThread = g.Thread
}
// adjust the value of registers inside scope
pcreg, bpreg, spreg := scope.Regs.Reg(scope.Regs.PCRegNum), scope.Regs.Reg(scope.Regs.BPRegNum), scope.Regs.Reg(scope.Regs.SPRegNum)
scope.Regs.ClearRegisters()
scope.Regs.AddReg(scope.Regs.PCRegNum, pcreg)
scope.Regs.AddReg(scope.Regs.BPRegNum, bpreg)
scope.Regs.AddReg(scope.Regs.SPRegNum, spreg)
scope.Regs.Reg(scope.Regs.SPRegNum).Uint64Val = uint64(spoff + int64(scope.g.stack.hi))
scope.Regs.Reg(scope.Regs.BPRegNum).Uint64Val = uint64(bpoff + int64(scope.g.stack.hi))
scope.Regs.FrameBase = fboff + int64(scope.g.stack.hi)
scope.Regs.CFA = scope.frameOffset + int64(scope.g.stack.hi)
finished := funcCallStep(scope, &fncall, g.Thread, protocolReg, dbgcallfn.Name)
if finished {
break
}
}
stack.fncallPush(&fncall)
stack.push(newConstant(constant.MakeBool(fncall.fn == nil || fncall.receiver != nil || fncall.closureAddr != 0), scope.Mem))
stack.callInjectionContinue = true
}
func funcCallFinish(scope *EvalScope, stack *evalStack) {
fncall := stack.fncallPop()
if fncall.err != nil {
return nil, fncall.err
if stack.err == nil {
stack.err = fncall.err
} else {
fncallLog("additional fncall error: %v", fncall.err)
}
return
}
if fncall.panicvar != nil {
return nil, fncallPanicErr{fncall.panicvar}
if stack.err == nil {
stack.err = fncallPanicErr{fncall.panicvar}
} else {
fncallLog("additional fncall panic: %v", fncall.panicvar)
}
return
}
switch len(fncall.retvars) {
case 0:
r := newVariable("", 0, nil, scope.BinInfo, nil)
r.loaded = true
r.Unreadable = errors.New("no return values")
return r, nil
stack.push(r)
case 1:
return fncall.retvars[0], nil
stack.push(fncall.retvars[0])
default:
// create a fake variable without address or type to return multiple values
r := newVariable("", 0, nil, scope.BinInfo, nil)
@ -439,7 +440,7 @@ func evalFunctionCall(scope *EvalScope, node *ast.CallExpr) (*Variable, error) {
for i := range fncall.retvars {
r.Children[i] = *fncall.retvars[i]
}
return r, nil
stack.push(r)
}
}
@ -503,27 +504,10 @@ func callOP(bi *BinaryInfo, thread Thread, regs Registers, callAddr uint64) erro
// funcCallEvalFuncExpr evaluates expr.Fun and returns the function that we're trying to call.
// If allowCalls is false function calls will be disabled even if scope.callCtx != nil
func funcCallEvalFuncExpr(scope *EvalScope, fncall *functionCallState, allowCalls bool) error {
func funcCallEvalFuncExpr(scope *EvalScope, stack *evalStack, fncall *functionCallState) error {
bi := scope.BinInfo
if !allowCalls {
callCtx := scope.callCtx
scope.callCtx = nil
defer func() {
scope.callCtx = callCtx
}()
}
fnvar, err := scope.evalAST(fncall.expr.Fun)
if err == errFuncCallNotAllowed {
// we can't determine the frame size because callexpr.Fun can't be
// evaluated without enabling function calls, just set up an argument
// frame for the maximum possible argument size.
fncall.argFrameSize = maxArgFrameSize
return nil
} else if err != nil {
return err
}
fnvar := stack.peek()
if fnvar.Kind != reflect.Func {
return fmt.Errorf("expression %q is not a function", exprToString(fncall.expr.Fun))
}
@ -543,6 +527,7 @@ func funcCallEvalFuncExpr(scope *EvalScope, fncall *functionCallState, allowCall
}
fncall.closureAddr = fnvar.closureAddr
var err error
fncall.argFrameSize, fncall.formalArgs, err = funcCallArgs(fncall.fn, bi, false)
if err != nil {
return err
@ -579,56 +564,7 @@ type funcCallArg struct {
isret bool
}
// funcCallEvalArgs evaluates the arguments of the function call, copying
// them into the argument frame starting at argFrameAddr.
func funcCallEvalArgs(callScope *EvalScope, fncall *functionCallState, thread Thread) error {
if callScope.g == nil {
// this should never happen
return errNoGoroutine
}
formalScope, err := GoroutineScope(callScope.target, thread)
if err != nil {
return err
}
if fncall.receiver != nil {
err := funcCallCopyOneArg(callScope, fncall, fncall.receiver, &fncall.formalArgs[0], formalScope)
if err != nil {
return err
}
fncall.formalArgs = fncall.formalArgs[1:]
}
for i := range fncall.formalArgs {
formalArg := &fncall.formalArgs[i]
actualArg, err := callScope.evalAST(fncall.expr.Args[i])
if err != nil {
if _, ispanic := err.(fncallPanicErr); ispanic {
return err
}
return fmt.Errorf("error evaluating %q as argument %s in function %s: %v", exprToString(fncall.expr.Args[i]), formalArg.name, fncall.fn.Name, err)
}
actualArg.Name = exprToString(fncall.expr.Args[i])
// evalAST can cause the current thread to change, recover it
thread = callScope.callCtx.p.CurrentThread()
formalScope, err = GoroutineScope(callScope.target, thread)
if err != nil {
return err
}
err = funcCallCopyOneArg(callScope, fncall, actualArg, formalArg, formalScope)
if err != nil {
return err
}
}
return nil
}
func funcCallCopyOneArg(scope *EvalScope, fncall *functionCallState, actualArg *Variable, formalArg *funcCallArg, formalScope *EvalScope) error {
func funcCallCopyOneArg(scope *EvalScope, fncall *functionCallState, actualArg *Variable, formalArg *funcCallArg, thread Thread) error {
if scope.callCtx.checkEscape {
//TODO(aarzilli): only apply the escapeCheck to leaking parameters.
if err := escapeCheck(actualArg, formalArg.name, scope.g.stack); err != nil {
@ -644,6 +580,11 @@ func funcCallCopyOneArg(scope *EvalScope, fncall *functionCallState, actualArg *
//TODO(aarzilli): automatic wrapping in interfaces for cases not handled
// by convertToEface.
formalScope, err := GoroutineScope(scope.target, thread)
if err != nil {
return err
}
var formalArgVar *Variable
if formalArg.dwarfEntry != nil {
var err error
@ -842,9 +783,10 @@ const (
)
// funcCallStep executes one step of the function call injection protocol.
func funcCallStep(callScope *EvalScope, fncall *functionCallState, thread Thread, protocolReg uint64, debugCallName string) bool {
func funcCallStep(callScope *EvalScope, stack *evalStack, thread Thread) bool {
p := callScope.callCtx.p
bi := p.BinInfo()
fncall := stack.fncallPeek()
regs, err := thread.Registers()
if err != nil {
@ -852,7 +794,7 @@ func funcCallStep(callScope *EvalScope, fncall *functionCallState, thread Thread
return true
}
regval := bi.Arch.RegistersToDwarfRegisters(0, regs).Uint64Val(protocolReg)
regval := bi.Arch.RegistersToDwarfRegisters(0, regs).Uint64Val(fncall.protocolReg)
if logflags.FnCall() {
loc, _ := thread.Location()
@ -869,6 +811,7 @@ func funcCallStep(callScope *EvalScope, fncall *functionCallState, thread Thread
switch regval {
case debugCallRegPrecheckFailed: // 8
stack.callInjectionContinue = true
archoff := uint64(0)
if bi.Arch.Name == "arm64" {
archoff = 8
@ -887,65 +830,6 @@ func funcCallStep(callScope *EvalScope, fncall *functionCallState, thread Thread
case debugCallRegCompleteCall: // 0
p.fncallForG[callScope.g.ID].startThreadID = 0
// evaluate arguments of the target function, copy them into its argument frame and call the function
if fncall.fn == nil || fncall.receiver != nil || fncall.closureAddr != 0 {
// if we couldn't figure out which function we are calling before
// (because the function we are calling is the return value of a call to
// another function) now we have to figure it out by recursively
// evaluating the function calls.
// This also needs to be done if the function call has a receiver
// argument or a closure address (because those addresses could be on the stack
// and have changed position between the start of the call and now).
err := funcCallEvalFuncExpr(callScope, fncall, true)
if err != nil {
fncall.err = err
fncall.lateCallFailure = true
break
}
//TODO: double check that function call size isn't too big
// funcCallEvalFuncExpr can start a function call injection itself, we
// need to recover the correct thread here.
thread = p.CurrentThread()
}
// instead of evaluating the arguments we start first by pushing the call
// on the stack, this is the opposite of what would happen normally but
// it's necessary because otherwise the GC wouldn't be able to deal with
// the argument frame.
if fncall.closureAddr != 0 {
// When calling a function pointer we must set the DX register to the
// address of the function pointer itself.
setClosureReg(thread, fncall.closureAddr)
}
cfa := regs.SP()
oldpc := regs.PC()
var oldlr uint64
if bi.Arch.Name == "arm64" || bi.Arch.Name == "ppc64le" {
oldlr = regs.LR()
}
callOP(bi, thread, regs, fncall.fn.Entry)
err = funcCallEvalArgs(callScope, fncall, thread)
thread = p.CurrentThread() // call evaluation in funcCallEvalArgs can cause the current thread to change
if err != nil {
// rolling back the call, note: this works because we called regs.Copy() above
switch bi.Arch.Name {
case "amd64":
setSP(thread, cfa)
setPC(thread, oldpc)
case "arm64", "ppc64le":
setLR(thread, oldlr)
setPC(thread, oldpc)
default:
panic("not implemented")
}
fncall.err = err
fncall.lateCallFailure = true
break
}
case debugCallRegRestoreRegisters: // 16
// runtime requests that we restore the registers (all except pc and sp),
// this is also the last step of the function call protocol.
@ -960,9 +844,8 @@ func funcCallStep(callScope *EvalScope, fncall *functionCallState, thread Thread
fncall.err = fmt.Errorf("could not restore SP: %v", err)
}
fncallLog("stepping thread %d", thread.ThreadID())
if err := stepInstructionOut(callScope.callCtx.grp, p, thread, debugCallName, debugCallName); err != nil {
fncall.err = fmt.Errorf("could not step out of %s: %v", debugCallName, err)
if err := stepInstructionOut(callScope.callCtx.grp, p, thread, fncall.debugCallName, fncall.debugCallName); err != nil {
fncall.err = fmt.Errorf("could not step out of %s: %v", fncall.debugCallName, err)
}
if bi.Arch.Name == "amd64" {
// The tail of debugCallV2 corrupts the state of RFLAGS, we must restore
@ -979,7 +862,8 @@ func funcCallStep(callScope *EvalScope, fncall *functionCallState, thread Thread
case debugCallRegReadReturn: // 1
// read return arguments from stack
if fncall.panicvar != nil || fncall.lateCallFailure {
stack.callInjectionContinue = true
if fncall.panicvar != nil || fncall.err != nil {
break
}
retScope, err := ThreadScope(p, thread)
@ -1030,6 +914,7 @@ func funcCallStep(callScope *EvalScope, fncall *functionCallState, thread Thread
case debugCallRegReadPanic: // 2
// read panic value from stack
stack.callInjectionContinue = true
archoff := uint64(0)
if bi.Arch.Name == "arm64" {
archoff = 8
@ -1046,12 +931,51 @@ func funcCallStep(callScope *EvalScope, fncall *functionCallState, thread Thread
default:
// Got an unknown protocol register value, this is probably bad but the safest thing
// possible is to ignore it and hope it didn't matter.
stack.callInjectionContinue = true
fncallLog("unknown value of protocol register %#x", regval)
}
return false
}
func (scope *EvalScope) evalCallInjectionSetTarget(op *evalop.CallInjectionSetTarget, stack *evalStack, thread Thread) {
fncall := stack.fncallPeek()
if fncall.fn == nil || fncall.receiver != nil || fncall.closureAddr != 0 {
funcCallEvalFuncExpr(scope, stack, fncall)
}
stack.pop() // target function, consumed by funcCallEvalFuncExpr either above or in evalop.CallInjectionStart
regs, err := thread.Registers()
if err != nil {
stack.err = err
return
}
if fncall.closureAddr != 0 {
// When calling a function pointer we must set the DX register to the
// address of the function pointer itself.
setClosureReg(thread, fncall.closureAddr)
}
undo := new(undoInjection)
undo.oldpc = regs.PC()
if scope.BinInfo.Arch.Name == "arm64" || scope.BinInfo.Arch.Name == "ppc64le" {
undo.oldlr = regs.LR()
}
callOP(scope.BinInfo, thread, regs, fncall.fn.Entry)
fncall.undoInjection = undo
if fncall.receiver != nil {
err := funcCallCopyOneArg(scope, fncall, fncall.receiver, &fncall.formalArgs[0], thread)
if err != nil {
stack.err = err
return
}
fncall.formalArgs = fncall.formalArgs[1:]
}
}
func readStackVariable(t *Target, thread Thread, regs Registers, off uint64, typename string, loadCfg LoadConfig) (*Variable, error) {
bi := thread.BinInfo()
scope, err := ThreadScope(t, thread)
@ -1093,47 +1017,92 @@ func fakeFunctionEntryScope(scope *EvalScope, fn *Function, cfa int64, sp uint64
return nil
}
// allocString allocates spaces for the contents of v if it needs to be allocated
func allocString(scope *EvalScope, v *Variable) error {
if v.Base != 0 || v.Len == 0 {
// already allocated
return nil
func (scope *EvalScope) allocString(phase int, stack *evalStack, curthread Thread) bool {
switch phase {
case 0:
x := stack.peek()
if !(x.Kind == reflect.String && x.Addr == 0 && (x.Flags&VariableConstant) != 0 && x.Len > 0) {
stack.opidx += 2 // skip the next two allocString phases, we don't need to do an allocation
return false
}
if scope.callCtx == nil {
// do not complain here, setValue will if no other errors happen
stack.opidx += 2
return false
}
mallocv, err := scope.findGlobal("runtime", "mallocgc")
if mallocv == nil {
stack.err = err
return false
}
stack.push(mallocv)
scope.evalCallInjectionStart(&evalop.CallInjectionStart{HasFunc: true, Node: &ast.CallExpr{
Fun: &ast.SelectorExpr{
X: &ast.Ident{Name: "runtime"},
Sel: &ast.Ident{Name: "mallocgc"},
},
Args: []ast.Expr{
&ast.BasicLit{Kind: token.INT, Value: "0"},
&ast.Ident{Name: "nil"},
&ast.Ident{Name: "false"},
},
}}, stack)
if stack.err == nil {
stack.pop() // return value of evalop.CallInjectionStart
}
return true
case 1:
fncall := stack.fncallPeek()
savedLoadCfg := scope.callCtx.retLoadCfg
scope.callCtx.retLoadCfg = loadFullValue
defer func() {
scope.callCtx.retLoadCfg = savedLoadCfg
}()
scope.evalCallInjectionSetTarget(nil, stack, curthread)
strvar := stack.peek()
stack.err = funcCallCopyOneArg(scope, fncall, newConstant(constant.MakeInt64(strvar.Len), scope.Mem), &fncall.formalArgs[0], curthread)
if stack.err != nil {
return false
}
stack.err = funcCallCopyOneArg(scope, fncall, nilVariable, &fncall.formalArgs[1], curthread)
if stack.err != nil {
return false
}
stack.err = funcCallCopyOneArg(scope, fncall, newConstant(constant.MakeBool(false), scope.Mem), &fncall.formalArgs[2], curthread)
if stack.err != nil {
return false
}
return true
case 2:
mallocv := stack.pop()
v := stack.pop()
if mallocv.Unreadable != nil {
stack.err = mallocv.Unreadable
return false
}
if mallocv.DwarfType.String() != "*void" {
stack.err = fmt.Errorf("unexpected return type for mallocgc call: %v", mallocv.DwarfType.String())
return false
}
if len(mallocv.Children) != 1 {
stack.err = errors.New("internal error, could not interpret return value of mallocgc call")
return false
}
v.Base = mallocv.Children[0].Addr
_, stack.err = scope.Mem.WriteMemory(v.Base, []byte(constant.StringVal(v.Value)))
stack.push(v)
return false
}
if scope.callCtx == nil {
return errFuncCallNotAllowedStrAlloc
}
savedLoadCfg := scope.callCtx.retLoadCfg
scope.callCtx.retLoadCfg = loadFullValue
defer func() {
scope.callCtx.retLoadCfg = savedLoadCfg
}()
mallocv, err := evalFunctionCall(scope, &ast.CallExpr{
Fun: &ast.SelectorExpr{
X: &ast.Ident{Name: "runtime"},
Sel: &ast.Ident{Name: "mallocgc"},
},
Args: []ast.Expr{
&ast.BasicLit{Kind: token.INT, Value: strconv.Itoa(int(v.Len))},
&ast.Ident{Name: "nil"},
&ast.Ident{Name: "false"},
},
})
if err != nil {
return err
}
if mallocv.Unreadable != nil {
return mallocv.Unreadable
}
if mallocv.DwarfType.String() != "*void" {
return fmt.Errorf("unexpected return type for mallocgc call: %v", mallocv.DwarfType.String())
}
if len(mallocv.Children) != 1 {
return errors.New("internal error, could not interpret return value of mallocgc call")
}
v.Base = mallocv.Children[0].Addr
_, err = scope.Mem.WriteMemory(v.Base, []byte(constant.StringVal(v.Value)))
return err
panic("unreachable")
}
func isCallInjectionStop(t *Target, thread Thread, loc *Location) bool {

View File

@ -9,6 +9,7 @@ import (
"go/constant"
"go/token"
"math"
"math/bits"
"reflect"
"sort"
"strconv"
@ -1685,6 +1686,9 @@ func (v *Variable) loadArrayValues(recurseLevel int, cfg LoadConfig) {
if count > int64(cfg.MaxArrayValues) {
count = int64(cfg.MaxArrayValues)
}
if v.Base+uint64(v.stride*count) < v.Base {
v.Unreadable = fmt.Errorf("bad array base address %#x", v.Base)
}
if v.stride < maxArrayStridePrefetch {
v.mem = cacheMemory(v.mem, v.Base, int(v.stride*count))
@ -1724,7 +1728,7 @@ func (v *Variable) readComplex(size int64) {
return
}
ftyp := fakeBasicType("float", int(fs*8))
ftyp := godwarf.FakeBasicType("float", int(fs*8))
realvar := v.newVariable("real", v.Addr, ftyp, v.mem)
imagvar := v.newVariable("imaginary", v.Addr+uint64(fs), ftyp, v.mem)
@ -2425,7 +2429,7 @@ func (v *Variable) registerVariableTypeConv(newtyp string) (*Variable, error) {
break
}
}
if n == 0 || popcnt(uint64(n)) != 1 {
if n == 0 || bits.OnesCount64(uint64(n)) != 1 {
return nil, fmt.Errorf("unknown CPU register type conversion to %q", newtyp)
}
n = n / 8
@ -2444,23 +2448,6 @@ func (v *Variable) registerVariableTypeConv(newtyp string) (*Variable, error) {
return v, nil
}
// popcnt is the number of bits set to 1 in x.
// It's the same as math/bits.OnesCount64, copied here so that we can build
// on versions of go that don't have math/bits.
func popcnt(x uint64) int {
const m0 = 0x5555555555555555 // 01010101 ...
const m1 = 0x3333333333333333 // 00110011 ...
const m2 = 0x0f0f0f0f0f0f0f0f // 00001111 ...
const m = 1<<64 - 1
x = x>>1&(m0&m) + x&(m0&m)
x = x>>2&(m1&m) + x&(m1&m)
x = (x>>4 + x) & (m2 & m)
x += x >> 8
x += x >> 16
x += x >> 32
return int(x) & (1<<7 - 1)
}
func isCgoType(bi *BinaryInfo, typ godwarf.Type) bool {
cu := bi.Images[typ.Common().Index].findCompileUnitForOffset(typ.Common().Offset)
if cu == nil {
@ -2503,7 +2490,7 @@ func (cm constantsMap) Get(typ godwarf.Type) *constantType {
sort.Sort(constantValuesByValue(ctyp.values))
for i := range ctyp.values {
ctyp.values[i].name = strings.TrimPrefix(ctyp.values[i].name, typepkg)
if popcnt(uint64(ctyp.values[i].value)) == 1 {
if bits.OnesCount64(uint64(ctyp.values[i].value)) == 1 {
ctyp.values[i].singleBit = true
}
}

View File

@ -86,7 +86,12 @@ func FuzzEvalExpression(f *testing.F) {
scope := &proc.EvalScope{Location: *fi.Loc, Regs: fi.Regs, Mem: memoryReaderWithFailingWrites{mem}, BinInfo: bi}
for _, tc := range getEvalExpressionTestCases() {
scope.EvalExpression(tc.name, pnormalLoadConfig)
_, err := scope.EvalExpression(tc.name, pnormalLoadConfig)
if err != nil {
if strings.Contains(err.Error(), "internal debugger error") {
panic(err)
}
}
}
})
}

View File

@ -180,6 +180,7 @@ func TestVariableEvaluation2(t *testing.T) {
}
func TestSetVariable(t *testing.T) {
const errorPrefix = "ERROR:"
var testcases = []struct {
name string
typ string // type of <name>
@ -202,6 +203,7 @@ func TestSetVariable(t *testing.T) {
{"s3", "[]int", `[]int len: 0, cap: 6, []`, "s4[2:5]", "[]int len: 3, cap: 3, [3,4,5]"},
{"s3", "[]int", "[]int len: 3, cap: 3, [3,4,5]", "arr1[:]", "[]int len: 4, cap: 4, [0,1,2,3]"},
{"str1", "string", `"01234567890"`, `"new value"`, errorPrefix + "literal string can not be allocated because function calls are not allowed without using 'call'"},
}
withTestProcess("testvariables2", t, func(p *proc.Target, grp *proc.TargetGroup, fixture protest.Fixture) {
@ -218,11 +220,23 @@ func TestSetVariable(t *testing.T) {
assertNoError(err, t, "EvalVariable()")
assertVariable(t, variable, varTest{tc.name, true, tc.startVal, "", tc.typ, nil})
assertNoError(setVariable(p, tc.name, tc.expr), t, fmt.Sprintf("SetVariable(%q, %q)", tc.name, tc.expr))
err = setVariable(p, tc.name, tc.expr)
if strings.HasPrefix(tc.finalVal, errorPrefix) {
experr := tc.finalVal[len(errorPrefix):]
if err == nil {
t.Fatalf("expected error %q but didn't get an error", experr)
}
if err.Error() != experr {
t.Fatalf("expected error %q got %v", experr, err)
}
} else {
assertNoError(err, t, fmt.Sprintf("SetVariable(%q, %q)", tc.name, tc.expr))
variable, err = evalVariableWithCfg(p, tc.name, pnormalLoadConfig)
assertNoError(err, t, "EvalVariable()")
assertVariable(t, variable, varTest{tc.name, true, tc.finalVal, "", tc.typ, nil})
}
variable, err = evalVariableWithCfg(p, tc.name, pnormalLoadConfig)
assertNoError(err, t, "EvalVariable()")
assertVariable(t, variable, varTest{tc.name, true, tc.finalVal, "", tc.typ, nil})
}
})
}
@ -792,11 +806,11 @@ func getEvalExpressionTestCases() []varTest {
{"bytearray[0] * bytearray[0]", false, "144", "144", "uint8", nil},
// function call / typecast errors
{"unknownthing(1, 2)", false, "", "", "", errors.New("function calls not allowed without using 'call'")},
{"(unknownthing)(1, 2)", false, "", "", "", errors.New("function calls not allowed without using 'call'")},
{"unknownthing(1, 2)", false, "", "", "", errors.New("could not find symbol value for unknownthing")},
{"(unknownthing)(1, 2)", false, "", "", "", errors.New("could not find symbol value for unknownthing")},
{"afunc(2)", false, "", "", "", errors.New("function calls not allowed without using 'call'")},
{"(afunc)(2)", false, "", "", "", errors.New("function calls not allowed without using 'call'")},
{"(*afunc)(2)", false, "", "", "", errors.New("could not evaluate function or type (*afunc): expression \"afunc\" (func()) can not be dereferenced")},
{"(*afunc)(2)", false, "", "", "", errors.New("expression \"afunc\" (func()) can not be dereferenced")},
{"unknownthing(2)", false, "", "", "", errors.New("could not evaluate function or type unknownthing: could not find symbol value for unknownthing")},
{"(*unknownthing)(2)", false, "", "", "", errors.New("could not evaluate function or type (*unknownthing): could not find symbol value for unknownthing")},
{"(*strings.Split)(2)", false, "", "", "", errors.New("could not evaluate function or type (*strings.Split): could not find symbol value for strings")},
@ -1171,7 +1185,8 @@ func TestCallFunction(t *testing.T) {
{"callpanic()", []string{`~panic:interface {}:interface {}(string) "callpanic panicked"`}, nil},
{`stringsJoin(nil, "")`, []string{`:string:""`}, nil},
{`stringsJoin(stringslice, comma)`, []string{`:string:"one,two,three"`}, nil},
{`stringsJoin(s1, comma)`, nil, errors.New(`error evaluating "s1" as argument v in function main.stringsJoin: could not find symbol value for s1`)},
{`stringsJoin(stringslice, "~~")`, []string{`:string:"one~~two~~three"`}, nil},
{`stringsJoin(s1, comma)`, nil, errors.New(`error evaluating "s1" as argument 1 in function stringsJoin: could not find symbol value for s1`)},
{`stringsJoin(intslice, comma)`, nil, errors.New("can not convert value of type []int to []string")},
{`noreturncall(2)`, nil, nil},
@ -1220,7 +1235,7 @@ func TestCallFunction(t *testing.T) {
{`onetwothree(intcallpanic(2))`, []string{`:[]int:[]int len: 3, cap: 3, [3,4,5]`}, nil},
{`onetwothree(intcallpanic(0))`, []string{`~panic:interface {}:interface {}(string) "panic requested"`}, nil},
{`onetwothree(intcallpanic(2)+1)`, []string{`:[]int:[]int len: 3, cap: 3, [4,5,6]`}, nil},
{`onetwothree(intcallpanic("not a number"))`, nil, errors.New("error evaluating \"intcallpanic(\\\"not a number\\\")\" as argument n in function main.onetwothree: can not convert \"not a number\" constant to int")},
{`onetwothree(intcallpanic("not a number"))`, nil, errors.New("can not convert \"not a number\" constant to int")},
// Variable setting tests
{`pa2 = getAStructPtr(8); pa2`, []string{`pa2:*main.astruct:*main.astruct {X: 8}`}, nil},
@ -1271,11 +1286,11 @@ func TestCallFunction(t *testing.T) {
}
var testcasesBefore114After112 = []testCaseCallFunction{
{`strings.Join(s1, comma)`, nil, errors.New(`error evaluating "s1" as argument a in function strings.Join: could not find symbol value for s1`)},
{`strings.Join(s1, comma)`, nil, errors.New(`error evaluating "s1" as argument 1 in function strings.Join: could not find symbol value for s1`)},
}
var testcases114 = []testCaseCallFunction{
{`strings.Join(s1, comma)`, nil, errors.New(`error evaluating "s1" as argument elems in function strings.Join: could not find symbol value for s1`)},
{`strings.Join(s1, comma)`, nil, errors.New(`error evaluating "s1" as argument 1 in function strings.Join: could not find symbol value for s1`)},
}
var testcases117 = []testCaseCallFunction{