diff --git a/packages/evm/evmutil/tx.go b/packages/evm/evmutil/tx.go new file mode 100644 index 0000000000..c84745842d --- /dev/null +++ b/packages/evm/evmutil/tx.go @@ -0,0 +1,19 @@ +package evmutil + +import ( + "slices" + + "github.com/ethereum/go-ethereum/common" + "github.com/ethereum/go-ethereum/core/types" +) + +func IsFakeTransaction(tx *types.Transaction) bool { + sender, err := GetSender(tx) + + // the error will fire when the transaction is invalid. This is most of the time a fake evm tx we use for internal calls, therefore it's fine to assume both. + if slices.Equal(sender.Bytes(), common.Address{}.Bytes()) || err != nil { + return true + } + + return false +} diff --git a/packages/evm/jsonrpc/evmchain.go b/packages/evm/jsonrpc/evmchain.go index eebd4d2471..a15ef78a15 100644 --- a/packages/evm/jsonrpc/evmchain.go +++ b/packages/evm/jsonrpc/evmchain.go @@ -10,7 +10,6 @@ import ( "math" "math/big" "path" - "slices" "github.com/ethereum/go-ethereum" "github.com/ethereum/go-ethereum/common" @@ -666,17 +665,6 @@ func (e *EVMChain) iscRequestsInBlock(evmBlockNumber uint64) (*blocklog.BlockInf return blocklog.GetRequestsInBlock(blocklogStatePartition, iscBlockIndex) } -func (e *EVMChain) isFakeTransaction(tx *types.Transaction) bool { - sender, err := evmutil.GetSender(tx) - - // the error will fire when the transaction is invalid. This is most of the time a fake evm tx we use for internal calls, therefore it's fine to assume both. - if slices.Equal(sender.Bytes(), common.Address{}.Bytes()) || err != nil { - return true - } - - return false -} - // traceTransaction allows the tracing of a single EVM transaction. // "Fake" transactions that are emitted e.g. for L1 deposits return some mocked trace. func (e *EVMChain) traceTransaction( @@ -699,12 +687,12 @@ func (e *EVMChain) traceTransaction( BlockNumber: new(big.Int).SetUint64(blockNumber), TxIndex: int(txIndex), TxHash: tx.Hash(), - }, config.TracerConfig) + }, config.TracerConfig, false, nil) if err != nil { return nil, err } - if e.isFakeTransaction(tx) { + if evmutil.IsFakeTransaction(tx) { return tracer.TraceFakeTx(tx) } @@ -729,32 +717,36 @@ func (e *EVMChain) debugTraceBlock(config *tracers.TraceConfig, block *types.Blo return nil, err } - blockTxs, err := e.txsByBlockNumber(new(big.Int).SetUint64(block.NumberU64())) + tracerType := "callTracer" + if config.Tracer != nil { + tracerType = *config.Tracer + } + + blockNumber := uint64(iscBlock.BlockIndex()) + + blockTxs := block.Transactions() + + tracer, err := newTracer(tracerType, &tracers.Context{ + BlockHash: block.Hash(), + BlockNumber: new(big.Int).SetUint64(blockNumber), + }, config.TracerConfig, true, blockTxs) if err != nil { return nil, err } - results := make([]TxTraceResult, 0) - for i, tx := range blockTxs { - result, err := e.traceTransaction( - config, - iscBlock, - iscRequestsInBlock, - tx, - uint64(i), - block.Hash(), - ) - - // Transactions which failed tracing will be omitted, so the rest of the block can be returned - if err == nil { - results = append(results, TxTraceResult{ - TxHash: tx.Hash(), - Result: result, - }) - } + err = e.backend.EVMTrace( + iscBlock.PreviousAliasOutput, + iscBlock.Timestamp, + iscRequestsInBlock, + nil, + &blockNumber, + tracer.Tracer, + ) + if err != nil { + return nil, err } - return results, nil + return tracer.GetResult() } func (e *EVMChain) TraceTransaction(txHash common.Hash, config *tracers.TraceConfig) (any, error) { diff --git a/packages/evm/jsonrpc/jsonrpctest/jsonrpc_test.go b/packages/evm/jsonrpc/jsonrpctest/jsonrpc_test.go index 1cf1bf2ea7..20f3aef73f 100644 --- a/packages/evm/jsonrpc/jsonrpctest/jsonrpc_test.go +++ b/packages/evm/jsonrpc/jsonrpctest/jsonrpc_test.go @@ -4,9 +4,12 @@ package jsonrpctest import ( + "bytes" "context" + "crypto/ecdsa" "encoding/hex" "encoding/json" + "fmt" "math/big" "slices" "strings" @@ -624,20 +627,136 @@ func TestRPCTraceEVMDeposit(t *testing.T) { require.NoError(t, err) require.EqualValues(t, types.ReceiptStatusSuccessful, rc.Status) - trace, err := env.traceTransactionWithCallTracer(tx.Hash()) + t.Run("callTracer_tx", func(t *testing.T) { + var trace jsonrpc.CallFrame + trace, err = env.traceTransactionWithCallTracer(tx.Hash()) + require.NoError(t, err) + require.Equal(t, evmAddr.String(), trace.To.String()) + require.Equal(t, hexutil.EncodeUint64(isc.NewAssetsBaseTokens(1000).BaseTokens*1e12), trace.Value.String()) + }) + + t.Run("prestateTracer_tx", func(t *testing.T) { + var prestate jsonrpc.PrestateAccountMap + prestate, err = env.traceTransactionWithPrestate(tx.Hash()) + require.NoError(t, err) + require.Empty(t, prestate) + }) + + t.Run("prestateTracerDiff_tx", func(t *testing.T) { + var prestateDiff jsonrpc.PrestateDiffResult + prestateDiff, err = env.traceTransactionWithPrestateDiff(tx.Hash()) + require.NoError(t, err) + require.Empty(t, prestateDiff.Pre) + require.Empty(t, prestateDiff.Post) + }) + + t.Run("callTracer_block", func(t *testing.T) { + callTracer := "callTracer" + var res1 json.RawMessage + // we have to use the raw client, because the normal client does not support debug methods + err = env.RawClient.CallContext( + context.Background(), + &res1, + "debug_traceBlockByNumber", + hexutil.Uint64(env.BlockNumber()).String(), + tracers.TraceConfig{Tracer: &callTracer}, + ) + require.NoError(t, err) + + traces := make([]jsonrpc.TxTraceResult, 0) + err = json.Unmarshal(res1, &traces) + require.NoError(t, err) + require.Len(t, traces, 1) + require.Equal(t, tx.Hash(), traces[0].TxHash) + + cs := jsonrpc.CallFrame{} + err = json.Unmarshal(traces[0].Result, &cs) + require.NoError(t, err) + require.Equal(t, evmAddr.String(), cs.To.String()) + require.Equal(t, hexutil.EncodeUint64(isc.NewAssetsBaseTokens(1000).BaseTokens*1e12), cs.Value.String()) + }) + + t.Run("prestateTracer_block", func(t *testing.T) { + tracer := "prestateTracer" + var res1 json.RawMessage + // we have to use the raw client, because the normal client does not support debug methods + err = env.RawClient.CallContext( + context.Background(), + &res1, + "debug_traceBlockByNumber", + hexutil.Uint64(env.BlockNumber()).String(), + tracers.TraceConfig{Tracer: &tracer}, + ) + require.NoError(t, err) + + traces := make([]jsonrpc.TxTraceResult, 0) + err = json.Unmarshal(res1, &traces) + require.NoError(t, err) + require.Len(t, traces, 1) + require.Equal(t, tx.Hash(), traces[0].TxHash) + + prestate := jsonrpc.PrestateAccountMap{} + err = json.Unmarshal(traces[0].Result, &prestate) + require.NoError(t, err) + require.Empty(t, prestate) + }) +} + +func addNRequests(n int, env *soloTestEnv, creator *ecdsa.PrivateKey, creatorAddress common.Address, contractABI abi.ABI, contractAddress common.Address) { + rqs := make([]isc.Request, 0, n) + for i := 0; i < n; i++ { + tx1 := types.MustSignNewTx(creator, types.NewEIP155Signer(big.NewInt(int64(env.ChainID))), + &types.LegacyTx{ + Nonce: env.NonceAt(creatorAddress) + uint64(i), + To: &contractAddress, + Value: big.NewInt(123), + Gas: 100000, + GasPrice: big.NewInt(10000000000), + Data: lo.Must(contractABI.Pack("sendTo", common.Address{0x1}, big.NewInt(2))), + }) + + req1 := lo.Must(isc.NewEVMOffLedgerTxRequest(env.soloChain.ChainID, tx1)) + rqs = append(rqs, req1) + } + + env.soloChain.WaitForRequestsMark() + env.soloChain.Env.AddRequestsToMempool(env.soloChain, rqs) +} + +// TestRPCTraceBlockForLargeN requires a large number of requests to be added to the mempool, for that set solo.MaxRequestsInBlock to a large value (>500) +func TestRPCTraceBlockForLargeN(t *testing.T) { + t.Skip("skipping because it requires solo parameters to be set") + + n := 400 + env := newSoloTestEnv(t) + creator, creatorAddress := env.soloChain.NewEthereumAccountWithL2Funds() + contractABI, err := abi.JSON(strings.NewReader(evmtest.ISCTestContractABI)) require.NoError(t, err) + _, _, contractAddress := env.DeployEVMContract(creator, contractABI, evmtest.ISCTestContractBytecode) + + addNRequests(n, env, creator, creatorAddress, contractABI, contractAddress) - require.Equal(t, evmAddr.String(), trace.To.String()) - require.Equal(t, hexutil.EncodeUint64(isc.NewAssetsBaseTokens(1000).BaseTokens*1e12), trace.Value.String()) + require.True(t, env.soloChain.WaitForRequestsThrough(n, 5*time.Minute)) - prestate, err := env.traceTransactionWithPrestate(tx.Hash()) + bi := env.soloChain.GetLatestBlockInfo() + require.EqualValues(t, n, bi.NumSuccessfulRequests) + + callTracer := "callTracer" + var res1 json.RawMessage + // we have to use the raw client, because the normal client does not support debug methods + err = env.RawClient.CallContext( + context.Background(), + &res1, + "debug_traceBlockByNumber", + hexutil.Uint64(env.BlockNumber()).String(), + tracers.TraceConfig{Tracer: &callTracer}, + ) require.NoError(t, err) - require.Empty(t, prestate) - prestateDiff, err := env.traceTransactionWithPrestateDiff(tx.Hash()) + var prettyJSON bytes.Buffer + err = json.Indent(&prettyJSON, res1, "", " ") require.NoError(t, err) - require.Empty(t, prestateDiff.Pre) - require.Empty(t, prestateDiff.Post) + fmt.Println(prettyJSON.String()) } func TestRPCTraceBlock(t *testing.T) { diff --git a/packages/evm/jsonrpc/tracer.go b/packages/evm/jsonrpc/tracer.go index 38a478f9c3..84156944bc 100644 --- a/packages/evm/jsonrpc/tracer.go +++ b/packages/evm/jsonrpc/tracer.go @@ -6,6 +6,8 @@ import ( "github.com/ethereum/go-ethereum/core/types" "github.com/ethereum/go-ethereum/eth/tracers" + + "github.com/iotaledger/wasp/packages/evm/evmutil" ) type Tracer struct { @@ -13,7 +15,7 @@ type Tracer struct { TraceFakeTx func(tx *types.Transaction) (json.RawMessage, error) } -type tracerFactory func(*tracers.Context, json.RawMessage) (*Tracer, error) +type tracerFactory func(traceCtx *tracers.Context, cfg json.RawMessage, traceBlock bool, initValue any) (*Tracer, error) var allTracers = map[string]tracerFactory{} @@ -21,10 +23,53 @@ func registerTracer(tracerType string, fn tracerFactory) { allTracers[tracerType] = fn } -func newTracer(tracerType string, ctx *tracers.Context, cfg json.RawMessage) (*Tracer, error) { +func newTracer(tracerType string, ctx *tracers.Context, cfg json.RawMessage, traceBlock bool, initValue any) (*Tracer, error) { fn := allTracers[tracerType] if fn == nil { return nil, fmt.Errorf("unsupported tracer type: %s", tracerType) } - return fn(ctx, cfg) + return fn(ctx, cfg, traceBlock, initValue) +} + +func GetTraceResults( + blockTxs []*types.Transaction, + traceBlock bool, + getFakeTxTrace func(tx *types.Transaction) (json.RawMessage, error), + getTxTrace func(tx *types.Transaction) (json.RawMessage, error), + getSingleTxTrace func() (json.RawMessage, error), + reason error, +) (json.RawMessage, error) { + var traceResult []byte + var err error + if traceBlock { + results := make([]TxTraceResult, 0, len(blockTxs)) + var jsResult json.RawMessage + for _, tx := range blockTxs { + if evmutil.IsFakeTransaction(tx) { + jsResult, err = getFakeTxTrace(tx) + if err != nil { + return nil, err + } + } else { + jsResult, err = getTxTrace(tx) + if err != nil { + return nil, err + } + } + + results = append(results, TxTraceResult{TxHash: tx.Hash(), Result: jsResult}) + } + + traceResult, err = json.Marshal(results) + if err != nil { + return nil, err + } + } else { + traceResult, err = getSingleTxTrace() + if err != nil { + return nil, err + } + } + + return traceResult, reason } diff --git a/packages/evm/jsonrpc/tracer_call.go b/packages/evm/jsonrpc/tracer_call.go index 54b7e53b7a..758b5aeb1e 100644 --- a/packages/evm/jsonrpc/tracer_call.go +++ b/packages/evm/jsonrpc/tracer_call.go @@ -6,6 +6,7 @@ package jsonrpc import ( "encoding/json" "errors" + "fmt" "math/big" "strings" "sync/atomic" @@ -119,12 +120,15 @@ type TxTraceResult struct { } type callTracer struct { - callstack []CallFrame - config callTracerConfig - gasLimit uint64 - depth int - interrupt atomic.Bool // Atomic flag to signal execution interruption - reason error // Textual reason for the interruption + txToStack map[common.Hash][]CallFrame + config callTracerConfig + gasLimit uint64 + depth int + interrupt atomic.Bool // Atomic flag to signal execution interruption + reason error // Textual reason for the interruption + currentTx common.Hash + traceBlock bool + blockTxs []*types.Transaction } type callTracerConfig struct { @@ -134,8 +138,21 @@ type callTracerConfig struct { // newCallTracer returns a native go tracer which tracks // call frames of a tx, and implements vm.EVMLogger. -func newCallTracer(ctx *tracers.Context, cfg json.RawMessage) (*Tracer, error) { - t, err := newCallTracerObject(ctx, cfg) +func newCallTracer(ctx *tracers.Context, cfg json.RawMessage, traceBlock bool, initValue any) (*Tracer, error) { + var fakeTxs types.Transactions + + if initValue == nil && traceBlock { + return nil, fmt.Errorf("initValue with block transactions is required for block tracing") + } + + if initValue != nil { + var ok bool + fakeTxs, ok = initValue.(types.Transactions) + if !ok { + return nil, fmt.Errorf("invalid init value type for calltracer: %T", initValue) + } + } + t, err := newCallTracerObject(ctx, cfg, traceBlock, fakeTxs) if err != nil { return nil, err } @@ -155,7 +172,7 @@ func newCallTracer(ctx *tracers.Context, cfg json.RawMessage) (*Tracer, error) { }, nil } -func newCallTracerObject(_ *tracers.Context, cfg json.RawMessage) (*callTracer, error) { +func newCallTracerObject(_ *tracers.Context, cfg json.RawMessage, traceBlock bool, blockTxs []*types.Transaction) (*callTracer, error) { var config callTracerConfig if cfg != nil { if err := json.Unmarshal(cfg, &config); err != nil { @@ -164,7 +181,7 @@ func newCallTracerObject(_ *tracers.Context, cfg json.RawMessage) (*callTracer, } // First callframe contains tx context info // and is populated on start and end. - return &callTracer{callstack: make([]CallFrame, 0, 1), config: config}, nil + return &callTracer{txToStack: make(map[common.Hash][]CallFrame), currentTx: common.Hash{}, config: config, traceBlock: traceBlock, blockTxs: blockTxs}, nil } // OnEnter is called when EVM enters a new scope (via call, create or selfdestruct). @@ -190,7 +207,7 @@ func (t *callTracer) OnEnter(depth int, typ byte, from common.Address, to common if depth == 0 { call.Gas = hexutil.Uint64(t.gasLimit) } - t.callstack = append(t.callstack, call) + t.txToStack[t.currentTx] = append(t.txToStack[t.currentTx], call) } // OnExit is called when EVM exits a scope, even if the scope didn't @@ -206,30 +223,32 @@ func (t *callTracer) OnExit(depth int, output []byte, gasUsed uint64, err error, return } - size := len(t.callstack) + size := len(t.txToStack[t.currentTx]) if size <= 1 { return } // Pop call. - call := t.callstack[size-1] - t.callstack = t.callstack[:size-1] + call := t.txToStack[t.currentTx][size-1] + t.txToStack[t.currentTx] = t.txToStack[t.currentTx][:size-1] size-- call.GasUsed = hexutil.Uint64(gasUsed) call.processOutput(output, err, reverted) // Nest call into parent. - t.callstack[size-1].Calls = append(t.callstack[size-1].Calls, call) + t.txToStack[t.currentTx][size-1].Calls = append(t.txToStack[t.currentTx][size-1].Calls, call) } func (t *callTracer) captureEnd(output []byte, _ uint64, err error, reverted bool) { - if len(t.callstack) != 1 { + if len(t.txToStack[t.currentTx]) != 1 { return } - t.callstack[0].processOutput(output, err, reverted) + t.txToStack[t.currentTx][0].processOutput(output, err, reverted) } func (t *callTracer) OnTxStart(env *tracing.VMContext, tx *types.Transaction, from common.Address) { t.gasLimit = tx.Gas() + t.currentTx = tx.Hash() + t.txToStack[t.currentTx] = make([]CallFrame, 0, 1) } func (t *callTracer) OnTxEnd(receipt *types.Receipt, err error) { @@ -237,10 +256,10 @@ func (t *callTracer) OnTxEnd(receipt *types.Receipt, err error) { if err != nil { return } - t.callstack[0].GasUsed = hexutil.Uint64(receipt.GasUsed) + t.txToStack[t.currentTx][0].GasUsed = hexutil.Uint64(receipt.GasUsed) if t.config.WithLog { // Logs are not emitted when the call fails - clearFailedLogs(&t.callstack[0], false) + clearFailedLogs(&t.txToStack[t.currentTx][0], false) } } @@ -261,9 +280,9 @@ func (t *callTracer) OnLog(log *types.Log) { Address: log.Address, Topics: log.Topics, Data: log.Data, - Position: hexutil.Uint(len(t.callstack[len(t.callstack)-1].Calls)), + Position: hexutil.Uint(len(t.txToStack[t.currentTx][len(t.txToStack[t.currentTx])-1].Calls)), } - t.callstack[len(t.callstack)-1].Logs = append(t.callstack[len(t.callstack)-1].Logs, l) + t.txToStack[t.currentTx][len(t.txToStack[t.currentTx])-1].Logs = append(t.txToStack[t.currentTx][len(t.txToStack[t.currentTx])-1].Logs, l) } var ErrIncorrectTopLevelCalls = errors.New("incorrect number of top-level calls") @@ -271,16 +290,21 @@ var ErrIncorrectTopLevelCalls = errors.New("incorrect number of top-level calls" // GetResult returns the json-encoded nested list of call traces, and any // error arising from the encoding or forceful termination (via `Stop`). func (t *callTracer) GetResult() (json.RawMessage, error) { - if len(t.callstack) != 1 { - return nil, ErrIncorrectTopLevelCalls - } - - res, err := json.Marshal(t.callstack[0]) - if err != nil { - return nil, err - } - - return res, t.reason + return GetTraceResults( + t.blockTxs, + t.traceBlock, + t.TraceFakeTx, + func(tx *types.Transaction) (json.RawMessage, error) { + stack, ok := t.txToStack[tx.Hash()] + if !ok { + return nil, fmt.Errorf("no call stack for tx %s", tx.Hash().Hex()) + } + return json.Marshal(stack[0]) + }, + func() (json.RawMessage, error) { + return json.Marshal(t.txToStack[t.currentTx][0]) + }, + t.reason) } // Stop terminates execution of the tracer at the first opportune moment. diff --git a/packages/evm/jsonrpc/tracer_prestate.go b/packages/evm/jsonrpc/tracer_prestate.go index beb443b4a7..8966913d74 100644 --- a/packages/evm/jsonrpc/tracer_prestate.go +++ b/packages/evm/jsonrpc/tracer_prestate.go @@ -6,6 +6,7 @@ package jsonrpc import ( "bytes" "encoding/json" + "fmt" "sync/atomic" "github.com/ethereum/go-ethereum/common" @@ -41,33 +42,52 @@ func (a *PrestateAccount) exists() bool { return a.Nonce > 0 || len(a.Code) > 0 || len(a.Storage) > 0 || (a.Balance != nil && a.Balance.ToInt().Sign() != 0) } +type PrestateTxValue struct { + Pre PrestateAccountMap `json:"pre"` + Post PrestateAccountMap `json:"post"` + created map[common.Address]bool + deleted map[common.Address]bool + to common.Address +} + type prestateTracer struct { - env *tracing.VMContext - pre PrestateAccountMap - post PrestateAccountMap - to common.Address - config prestateTracerConfig - interrupt atomic.Bool // Atomic flag to signal execution interruption - reason error // Textual reason for the interruption - created map[common.Address]bool - deleted map[common.Address]bool + env *tracing.VMContext + currentTxHash common.Hash + states map[common.Hash]*PrestateTxValue // key is the tx hash, value is the state diff + config prestateTracerConfig + interrupt atomic.Bool // Atomic flag to signal execution interruption + reason error // Textual reason for the interruption + traceBlock bool + blockTxs types.Transactions } type prestateTracerConfig struct { DiffMode bool `json:"diffMode"` // If true, this tracer will return state modifications } -func newPrestateTracer(ctx *tracers.Context, cfg json.RawMessage) (*Tracer, error) { +func newPrestateTracer(ctx *tracers.Context, cfg json.RawMessage, traceBlock bool, initValue any) (*Tracer, error) { + var blockTxs types.Transactions + + if initValue == nil && traceBlock { + return nil, fmt.Errorf("initValue with block transactions is required for block tracing") + } + + if initValue != nil { + var ok bool + blockTxs, ok = initValue.(types.Transactions) + if !ok { + return nil, fmt.Errorf("invalid init value type for prestateTracer: %T", initValue) + } + } var config prestateTracerConfig if err := json.Unmarshal(cfg, &config); err != nil { return nil, err } t := &prestateTracer{ - pre: PrestateAccountMap{}, - post: PrestateAccountMap{}, - config: config, - created: make(map[common.Address]bool), - deleted: make(map[common.Address]bool), + config: config, + traceBlock: traceBlock, + states: make(map[common.Hash]*PrestateTxValue), + blockTxs: blockTxs, } return &Tracer{ Tracer: &tracers.Tracer{ @@ -101,21 +121,21 @@ func (t *prestateTracer) OnOpcode(pc uint64, opcode byte, gas, cost uint64, scop switch { case stackLen >= 1 && (op == vm.SLOAD || op == vm.SSTORE): slot := common.Hash(stackData[stackLen-1].Bytes32()) - t.lookupStorage(caller, slot) + t.lookupStorage(t.currentTxHash, caller, slot) case stackLen >= 1 && (op == vm.EXTCODECOPY || op == vm.EXTCODEHASH || op == vm.EXTCODESIZE || op == vm.BALANCE || op == vm.SELFDESTRUCT): addr := common.Address(stackData[stackLen-1].Bytes20()) - t.lookupAccount(addr) + t.lookupAccount(t.currentTxHash, addr) if op == vm.SELFDESTRUCT { - t.deleted[caller] = true + t.states[t.currentTxHash].deleted[caller] = true } case stackLen >= 5 && (op == vm.DELEGATECALL || op == vm.CALL || op == vm.STATICCALL || op == vm.CALLCODE): addr := common.Address(stackData[stackLen-2].Bytes20()) - t.lookupAccount(addr) + t.lookupAccount(t.currentTxHash, addr) case op == vm.CREATE: nonce := t.env.StateDB.GetNonce(caller) addr := crypto.CreateAddress(caller, nonce) - t.lookupAccount(addr) - t.created[addr] = true + t.lookupAccount(t.currentTxHash, addr) + t.states[t.currentTxHash].created[addr] = true case stackLen >= 4 && op == vm.CREATE2: offset := stackData[stackLen-2] size := stackData[stackLen-3] @@ -127,23 +147,35 @@ func (t *prestateTracer) OnOpcode(pc uint64, opcode byte, gas, cost uint64, scop inithash := crypto.Keccak256(init) salt := stackData[stackLen-4] addr := crypto.CreateAddress2(caller, salt.Bytes32(), inithash) - t.lookupAccount(addr) - t.created[addr] = true + t.lookupAccount(t.currentTxHash, addr) + t.states[t.currentTxHash].created[addr] = true } } func (t *prestateTracer) OnTxStart(env *tracing.VMContext, tx *types.Transaction, from common.Address) { t.env = env + t.currentTxHash = tx.Hash() + + t.states[tx.Hash()] = &PrestateTxValue{ + Pre: make(PrestateAccountMap), + Post: make(PrestateAccountMap), + created: make(map[common.Address]bool), + deleted: make(map[common.Address]bool), + } + + txState := t.states[tx.Hash()] + if tx.To() == nil { - t.to = crypto.CreateAddress(from, env.StateDB.GetNonce(from)) - t.created[t.to] = true + createdAddr := crypto.CreateAddress(from, env.StateDB.GetNonce(from)) + txState.to = createdAddr + txState.created[createdAddr] = true } else { - t.to = *tx.To() + txState.to = *tx.To() } - t.lookupAccount(from) - t.lookupAccount(t.to) - t.lookupAccount(env.Coinbase) + t.lookupAccount(tx.Hash(), from) + t.lookupAccount(tx.Hash(), txState.to) + t.lookupAccount(tx.Hash(), env.Coinbase) } func (t *prestateTracer) OnTxEnd(receipt *types.Receipt, err error) { @@ -154,10 +186,10 @@ func (t *prestateTracer) OnTxEnd(receipt *types.Receipt, err error) { t.processDiffState() } // the new created contracts' prestate were empty, so delete them - for a := range t.created { + for a := range t.states[t.currentTxHash].created { // the created contract maybe exists in statedb before the creating tx - if s := t.pre[a]; s != nil && s.empty { - delete(t.pre, a) + if s := t.states[t.currentTxHash].Pre[a]; s != nil && s.empty { + delete(t.states[t.currentTxHash].Pre, a) } } } @@ -165,17 +197,22 @@ func (t *prestateTracer) OnTxEnd(receipt *types.Receipt, err error) { // GetResult returns the json-encoded nested list of call traces, and any // error arising from the encoding or forceful termination (via `Stop`). func (t *prestateTracer) GetResult() (json.RawMessage, error) { - var res []byte - var err error - if t.config.DiffMode { - res, err = json.Marshal(PrestateDiffResult{t.post, t.pre}) - } else { - res, err = json.Marshal(t.pre) - } - if err != nil { - return nil, err - } - return json.RawMessage(res), t.reason + return GetTraceResults( + t.blockTxs, + t.traceBlock, + t.TraceFakeTx, + func(tx *types.Transaction) (json.RawMessage, error) { + txState := t.states[tx.Hash()] + if t.config.DiffMode { + return json.Marshal(PrestateDiffResult{txState.Post, txState.Pre}) + } + return json.Marshal(txState.Pre) + }, func() (json.RawMessage, error) { + if t.config.DiffMode { + return json.Marshal(PrestateDiffResult{t.states[t.currentTxHash].Post, t.states[t.currentTxHash].Pre}) + } + return json.Marshal(t.states[t.currentTxHash].Pre) + }, t.reason) } // Stop terminates execution of the tracer at the first opportune moment. @@ -185,9 +222,10 @@ func (t *prestateTracer) Stop(err error) { } func (t *prestateTracer) processDiffState() { - for addr, state := range t.pre { + txState := t.states[t.currentTxHash] + for addr, state := range txState.Pre { // The deleted account's state is pruned from `post` but kept in `pre` - if _, ok := t.deleted[addr]; ok { + if _, ok := txState.deleted[addr]; ok { continue } modified := false @@ -196,15 +234,15 @@ func (t *prestateTracer) processDiffState() { newNonce := t.env.StateDB.GetNonce(addr) newCode := t.env.StateDB.GetCode(addr) - if newBalance.Cmp(t.pre[addr].Balance.ToInt()) != 0 { + if newBalance.Cmp(txState.Pre[addr].Balance.ToInt()) != 0 { modified = true postAccount.Balance = (*hexutil.Big)(newBalance) } - if newNonce != t.pre[addr].Nonce { + if newNonce != txState.Pre[addr].Nonce { modified = true postAccount.Nonce = newNonce } - if !bytes.Equal(newCode, t.pre[addr].Code) { + if !bytes.Equal(newCode, txState.Pre[addr].Code) { modified = true postAccount.Code = newCode } @@ -212,13 +250,13 @@ func (t *prestateTracer) processDiffState() { for key, val := range state.Storage { // don't include the empty slot if val == (common.Hash{}) { - delete(t.pre[addr].Storage, key) + delete(txState.Pre[addr].Storage, key) } newVal := t.env.StateDB.GetState(addr, key) if val == newVal { // Omit unchanged slots - delete(t.pre[addr].Storage, key) + delete(txState.Pre[addr].Storage, key) } else { modified = true if newVal != (common.Hash{}) { @@ -228,18 +266,18 @@ func (t *prestateTracer) processDiffState() { } if modified { - t.post[addr] = postAccount + txState.Post[addr] = postAccount } else { // if state is not modified, then no need to include into the pre state - delete(t.pre, addr) + delete(txState.Pre, addr) } } } // lookupAccount fetches details of an account and adds it to the prestate // if it doesn't exist there. -func (t *prestateTracer) lookupAccount(addr common.Address) { - if _, ok := t.pre[addr]; ok { +func (t *prestateTracer) lookupAccount(tx common.Hash, addr common.Address) { + if _, ok := t.states[tx].Pre[addr]; ok { return } @@ -252,17 +290,18 @@ func (t *prestateTracer) lookupAccount(addr common.Address) { if !acc.exists() { acc.empty = true } - t.pre[addr] = acc + + t.states[tx].Pre[addr] = acc } // lookupStorage fetches the requested storage slot and adds // it to the prestate of the given contract. It assumes `lookupAccount` // has been performed on the contract before. -func (t *prestateTracer) lookupStorage(addr common.Address, key common.Hash) { - if _, ok := t.pre[addr].Storage[key]; ok { +func (t *prestateTracer) lookupStorage(tx common.Hash, addr common.Address, key common.Hash) { + if _, ok := t.states[tx].Pre[addr].Storage[key]; ok { return } - t.pre[addr].Storage[key] = t.env.StateDB.GetState(addr, key) + t.states[tx].Pre[addr].Storage[key] = t.env.StateDB.GetState(addr, key) } func (t *prestateTracer) TraceFakeTx(tx *types.Transaction) (res json.RawMessage, err error) { diff --git a/packages/vm/core/evm/evmimpl/internal.go b/packages/vm/core/evm/evmimpl/internal.go index 7c92b7f3e0..c0a4bd47f6 100644 --- a/packages/vm/core/evm/evmimpl/internal.go +++ b/packages/vm/core/evm/evmimpl/internal.go @@ -37,7 +37,7 @@ func getTracer(ctx isc.Sandbox) *tracing.Hooks { return nil } - // if block number is set and the tx is null, we're tracing the whole block + // if block number is set and the TxIndex is null, we're tracing the whole block if tracer.TxIndex == nil && tracer.BlockNumber != nil { return tracer.Tracer.Hooks }