cmd, core, eth/tracers: support fancier js tracing (#15516)
* cmd, core, eth/tracers: support fancier js tracing * eth, internal/web3ext: rework trace API, concurrency, chain tracing * eth/tracers: add three more JavaScript tracers * eth/tracers, vendor: swap ottovm to duktape for tracing * core, eth, internal: finalize call tracer and needed extras * eth, tests: prestate tracer, call test suite, rewinding * vendor: fix windows builds for tracer js engine * vendor: temporary duktape fix * eth/tracers: fix up 4byte and evmdis tracer * vendor: pull in latest duktape with my upstream fixes * eth: fix some review comments * eth: rename rewind to reexec to make it more obvious * core/vm: terminate tracing using defers
This commit is contained in:
251
eth/api.go
251
eth/api.go
@ -17,24 +17,19 @@
|
||||
package eth
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"compress/gzip"
|
||||
"context"
|
||||
"fmt"
|
||||
"io"
|
||||
"io/ioutil"
|
||||
"math/big"
|
||||
"os"
|
||||
"strings"
|
||||
"time"
|
||||
|
||||
"github.com/ethereum/go-ethereum/common"
|
||||
"github.com/ethereum/go-ethereum/common/hexutil"
|
||||
"github.com/ethereum/go-ethereum/core"
|
||||
"github.com/ethereum/go-ethereum/core/state"
|
||||
"github.com/ethereum/go-ethereum/core/types"
|
||||
"github.com/ethereum/go-ethereum/core/vm"
|
||||
"github.com/ethereum/go-ethereum/internal/ethapi"
|
||||
"github.com/ethereum/go-ethereum/log"
|
||||
"github.com/ethereum/go-ethereum/miner"
|
||||
"github.com/ethereum/go-ethereum/params"
|
||||
@ -43,8 +38,6 @@ import (
|
||||
"github.com/ethereum/go-ethereum/trie"
|
||||
)
|
||||
|
||||
const defaultTraceTimeout = 5 * time.Second
|
||||
|
||||
// PublicEthereumAPI provides an API to access Ethereum full node-related
|
||||
// information.
|
||||
type PublicEthereumAPI struct {
|
||||
@ -348,248 +341,6 @@ func NewPrivateDebugAPI(config *params.ChainConfig, eth *Ethereum) *PrivateDebug
|
||||
return &PrivateDebugAPI{config: config, eth: eth}
|
||||
}
|
||||
|
||||
// BlockTraceResult is the returned value when replaying a block to check for
|
||||
// consensus results and full VM trace logs for all included transactions.
|
||||
type BlockTraceResult struct {
|
||||
Validated bool `json:"validated"`
|
||||
StructLogs []ethapi.StructLogRes `json:"structLogs"`
|
||||
Error string `json:"error"`
|
||||
}
|
||||
|
||||
// TraceArgs holds extra parameters to trace functions
|
||||
type TraceArgs struct {
|
||||
*vm.LogConfig
|
||||
Tracer *string
|
||||
Timeout *string
|
||||
}
|
||||
|
||||
// TraceBlock processes the given block'api RLP but does not import the block in to
|
||||
// the chain.
|
||||
func (api *PrivateDebugAPI) TraceBlock(blockRlp []byte, config *vm.LogConfig) BlockTraceResult {
|
||||
var block types.Block
|
||||
err := rlp.Decode(bytes.NewReader(blockRlp), &block)
|
||||
if err != nil {
|
||||
return BlockTraceResult{Error: fmt.Sprintf("could not decode block: %v", err)}
|
||||
}
|
||||
|
||||
validated, logs, err := api.traceBlock(&block, config)
|
||||
return BlockTraceResult{
|
||||
Validated: validated,
|
||||
StructLogs: ethapi.FormatLogs(logs),
|
||||
Error: formatError(err),
|
||||
}
|
||||
}
|
||||
|
||||
// TraceBlockFromFile loads the block'api RLP from the given file name and attempts to
|
||||
// process it but does not import the block in to the chain.
|
||||
func (api *PrivateDebugAPI) TraceBlockFromFile(file string, config *vm.LogConfig) BlockTraceResult {
|
||||
blockRlp, err := ioutil.ReadFile(file)
|
||||
if err != nil {
|
||||
return BlockTraceResult{Error: fmt.Sprintf("could not read file: %v", err)}
|
||||
}
|
||||
return api.TraceBlock(blockRlp, config)
|
||||
}
|
||||
|
||||
// TraceBlockByNumber processes the block by canonical block number.
|
||||
func (api *PrivateDebugAPI) TraceBlockByNumber(blockNr rpc.BlockNumber, config *vm.LogConfig) BlockTraceResult {
|
||||
// Fetch the block that we aim to reprocess
|
||||
var block *types.Block
|
||||
switch blockNr {
|
||||
case rpc.PendingBlockNumber:
|
||||
// Pending block is only known by the miner
|
||||
block = api.eth.miner.PendingBlock()
|
||||
case rpc.LatestBlockNumber:
|
||||
block = api.eth.blockchain.CurrentBlock()
|
||||
default:
|
||||
block = api.eth.blockchain.GetBlockByNumber(uint64(blockNr))
|
||||
}
|
||||
|
||||
if block == nil {
|
||||
return BlockTraceResult{Error: fmt.Sprintf("block #%d not found", blockNr)}
|
||||
}
|
||||
|
||||
validated, logs, err := api.traceBlock(block, config)
|
||||
return BlockTraceResult{
|
||||
Validated: validated,
|
||||
StructLogs: ethapi.FormatLogs(logs),
|
||||
Error: formatError(err),
|
||||
}
|
||||
}
|
||||
|
||||
// TraceBlockByHash processes the block by hash.
|
||||
func (api *PrivateDebugAPI) TraceBlockByHash(hash common.Hash, config *vm.LogConfig) BlockTraceResult {
|
||||
// Fetch the block that we aim to reprocess
|
||||
block := api.eth.BlockChain().GetBlockByHash(hash)
|
||||
if block == nil {
|
||||
return BlockTraceResult{Error: fmt.Sprintf("block #%x not found", hash)}
|
||||
}
|
||||
|
||||
validated, logs, err := api.traceBlock(block, config)
|
||||
return BlockTraceResult{
|
||||
Validated: validated,
|
||||
StructLogs: ethapi.FormatLogs(logs),
|
||||
Error: formatError(err),
|
||||
}
|
||||
}
|
||||
|
||||
// traceBlock processes the given block but does not save the state.
|
||||
func (api *PrivateDebugAPI) traceBlock(block *types.Block, logConfig *vm.LogConfig) (bool, []vm.StructLog, error) {
|
||||
// Validate and reprocess the block
|
||||
var (
|
||||
blockchain = api.eth.BlockChain()
|
||||
validator = blockchain.Validator()
|
||||
processor = blockchain.Processor()
|
||||
)
|
||||
|
||||
structLogger := vm.NewStructLogger(logConfig)
|
||||
|
||||
config := vm.Config{
|
||||
Debug: true,
|
||||
Tracer: structLogger,
|
||||
}
|
||||
if err := api.eth.engine.VerifyHeader(blockchain, block.Header(), true); err != nil {
|
||||
return false, structLogger.StructLogs(), err
|
||||
}
|
||||
statedb, err := blockchain.StateAt(blockchain.GetBlock(block.ParentHash(), block.NumberU64()-1).Root())
|
||||
if err != nil {
|
||||
switch err.(type) {
|
||||
case *trie.MissingNodeError:
|
||||
return false, structLogger.StructLogs(), fmt.Errorf("required historical state unavailable")
|
||||
default:
|
||||
return false, structLogger.StructLogs(), err
|
||||
}
|
||||
}
|
||||
|
||||
receipts, _, usedGas, err := processor.Process(block, statedb, config)
|
||||
if err != nil {
|
||||
return false, structLogger.StructLogs(), err
|
||||
}
|
||||
if err := validator.ValidateState(block, blockchain.GetBlock(block.ParentHash(), block.NumberU64()-1), statedb, receipts, usedGas); err != nil {
|
||||
return false, structLogger.StructLogs(), err
|
||||
}
|
||||
return true, structLogger.StructLogs(), nil
|
||||
}
|
||||
|
||||
// formatError formats a Go error into either an empty string or the data content
|
||||
// of the error itself.
|
||||
func formatError(err error) string {
|
||||
if err == nil {
|
||||
return ""
|
||||
}
|
||||
return err.Error()
|
||||
}
|
||||
|
||||
type timeoutError struct{}
|
||||
|
||||
func (t *timeoutError) Error() string {
|
||||
return "Execution time exceeded"
|
||||
}
|
||||
|
||||
// TraceTransaction returns the structured logs created during the execution of EVM
|
||||
// and returns them as a JSON object.
|
||||
func (api *PrivateDebugAPI) TraceTransaction(ctx context.Context, txHash common.Hash, config *TraceArgs) (interface{}, error) {
|
||||
var tracer vm.Tracer
|
||||
if config != nil && config.Tracer != nil {
|
||||
timeout := defaultTraceTimeout
|
||||
if config.Timeout != nil {
|
||||
var err error
|
||||
if timeout, err = time.ParseDuration(*config.Timeout); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
}
|
||||
|
||||
var err error
|
||||
if tracer, err = ethapi.NewJavascriptTracer(*config.Tracer); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
// Handle timeouts and RPC cancellations
|
||||
deadlineCtx, cancel := context.WithTimeout(ctx, timeout)
|
||||
go func() {
|
||||
<-deadlineCtx.Done()
|
||||
tracer.(*ethapi.JavascriptTracer).Stop(&timeoutError{})
|
||||
}()
|
||||
defer cancel()
|
||||
} else if config == nil {
|
||||
tracer = vm.NewStructLogger(nil)
|
||||
} else {
|
||||
tracer = vm.NewStructLogger(config.LogConfig)
|
||||
}
|
||||
|
||||
// Retrieve the tx from the chain and the containing block
|
||||
tx, blockHash, _, txIndex := core.GetTransaction(api.eth.ChainDb(), txHash)
|
||||
if tx == nil {
|
||||
return nil, fmt.Errorf("transaction %x not found", txHash)
|
||||
}
|
||||
msg, context, statedb, err := api.computeTxEnv(blockHash, int(txIndex))
|
||||
if err != nil {
|
||||
switch err.(type) {
|
||||
case *trie.MissingNodeError:
|
||||
return nil, fmt.Errorf("required historical state unavailable")
|
||||
default:
|
||||
return nil, err
|
||||
}
|
||||
}
|
||||
|
||||
// Run the transaction with tracing enabled.
|
||||
vmenv := vm.NewEVM(context, statedb, api.config, vm.Config{Debug: true, Tracer: tracer})
|
||||
ret, gas, failed, err := core.ApplyMessage(vmenv, msg, new(core.GasPool).AddGas(tx.Gas()))
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("tracing failed: %v", err)
|
||||
}
|
||||
switch tracer := tracer.(type) {
|
||||
case *vm.StructLogger:
|
||||
return ðapi.ExecutionResult{
|
||||
Gas: gas,
|
||||
Failed: failed,
|
||||
ReturnValue: fmt.Sprintf("%x", ret),
|
||||
StructLogs: ethapi.FormatLogs(tracer.StructLogs()),
|
||||
}, nil
|
||||
case *ethapi.JavascriptTracer:
|
||||
return tracer.GetResult()
|
||||
default:
|
||||
panic(fmt.Sprintf("bad tracer type %T", tracer))
|
||||
}
|
||||
}
|
||||
|
||||
// computeTxEnv returns the execution environment of a certain transaction.
|
||||
func (api *PrivateDebugAPI) computeTxEnv(blockHash common.Hash, txIndex int) (core.Message, vm.Context, *state.StateDB, error) {
|
||||
// Create the parent state.
|
||||
block := api.eth.BlockChain().GetBlockByHash(blockHash)
|
||||
if block == nil {
|
||||
return nil, vm.Context{}, nil, fmt.Errorf("block %x not found", blockHash)
|
||||
}
|
||||
parent := api.eth.BlockChain().GetBlock(block.ParentHash(), block.NumberU64()-1)
|
||||
if parent == nil {
|
||||
return nil, vm.Context{}, nil, fmt.Errorf("block parent %x not found", block.ParentHash())
|
||||
}
|
||||
statedb, err := api.eth.BlockChain().StateAt(parent.Root())
|
||||
if err != nil {
|
||||
return nil, vm.Context{}, nil, err
|
||||
}
|
||||
txs := block.Transactions()
|
||||
|
||||
// Recompute transactions up to the target index.
|
||||
signer := types.MakeSigner(api.config, block.Number())
|
||||
for idx, tx := range txs {
|
||||
// Assemble the transaction call message
|
||||
msg, _ := tx.AsMessage(signer)
|
||||
context := core.NewEVMContext(msg, block.Header(), api.eth.BlockChain(), nil)
|
||||
if idx == txIndex {
|
||||
return msg, context, statedb, nil
|
||||
}
|
||||
|
||||
vmenv := vm.NewEVM(context, statedb, api.config, vm.Config{})
|
||||
gp := new(core.GasPool).AddGas(tx.Gas())
|
||||
_, _, _, err := core.ApplyMessage(vmenv, msg, gp)
|
||||
if err != nil {
|
||||
return nil, vm.Context{}, nil, fmt.Errorf("tx %x failed: %v", tx.Hash(), err)
|
||||
}
|
||||
statedb.DeleteSuicides()
|
||||
}
|
||||
return nil, vm.Context{}, nil, fmt.Errorf("tx index %d out of range for block %x", txIndex, blockHash)
|
||||
}
|
||||
|
||||
// Preimage is a debug API function that returns the preimage for a sha3 hash, if known.
|
||||
func (api *PrivateDebugAPI) Preimage(ctx context.Context, hash common.Hash) (hexutil.Bytes, error) {
|
||||
db := core.PreimageTable(api.eth.ChainDb())
|
||||
@ -617,7 +368,7 @@ type storageEntry struct {
|
||||
|
||||
// StorageRangeAt returns the storage at the given block height and transaction index.
|
||||
func (api *PrivateDebugAPI) StorageRangeAt(ctx context.Context, blockHash common.Hash, txIndex int, contractAddress common.Address, keyStart hexutil.Bytes, maxResult int) (StorageRangeResult, error) {
|
||||
_, _, statedb, err := api.computeTxEnv(blockHash, txIndex)
|
||||
_, _, statedb, err := api.computeTxEnv(blockHash, txIndex, 0)
|
||||
if err != nil {
|
||||
return StorageRangeResult{}, err
|
||||
}
|
||||
|
727
eth/api_tracer.go
Normal file
727
eth/api_tracer.go
Normal file
@ -0,0 +1,727 @@
|
||||
// Copyright 2017 The go-ethereum Authors
|
||||
// This file is part of the go-ethereum library.
|
||||
//
|
||||
// The go-ethereum library is free software: you can redistribute it and/or modify
|
||||
// it under the terms of the GNU Lesser General Public License as published by
|
||||
// the Free Software Foundation, either version 3 of the License, or
|
||||
// (at your option) any later version.
|
||||
//
|
||||
// The go-ethereum library is distributed in the hope that it will be useful,
|
||||
// but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||
// GNU Lesser General Public License for more details.
|
||||
//
|
||||
// You should have received a copy of the GNU Lesser General Public License
|
||||
// along with the go-ethereum library. If not, see <http://www.gnu.org/licenses/>.
|
||||
|
||||
package eth
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"context"
|
||||
"errors"
|
||||
"fmt"
|
||||
"io/ioutil"
|
||||
"runtime"
|
||||
"sync"
|
||||
"sync/atomic"
|
||||
"time"
|
||||
|
||||
"github.com/ethereum/go-ethereum/common"
|
||||
"github.com/ethereum/go-ethereum/common/hexutil"
|
||||
"github.com/ethereum/go-ethereum/core"
|
||||
"github.com/ethereum/go-ethereum/core/state"
|
||||
"github.com/ethereum/go-ethereum/core/types"
|
||||
"github.com/ethereum/go-ethereum/core/vm"
|
||||
"github.com/ethereum/go-ethereum/eth/tracers"
|
||||
"github.com/ethereum/go-ethereum/ethdb"
|
||||
"github.com/ethereum/go-ethereum/internal/ethapi"
|
||||
"github.com/ethereum/go-ethereum/log"
|
||||
"github.com/ethereum/go-ethereum/rlp"
|
||||
"github.com/ethereum/go-ethereum/rpc"
|
||||
"github.com/ethereum/go-ethereum/trie"
|
||||
)
|
||||
|
||||
const (
|
||||
// defaultTraceTimeout is the amount of time a single transaction can execute
|
||||
// by default before being forcefully aborted.
|
||||
defaultTraceTimeout = 5 * time.Second
|
||||
|
||||
// defaultTraceReexec is the number of blocks the tracer is willing to go back
|
||||
// and reexecute to produce missing historical state necessary to run a specific
|
||||
// trace.
|
||||
defaultTraceReexec = uint64(128)
|
||||
)
|
||||
|
||||
// TraceConfig holds extra parameters to trace functions.
|
||||
type TraceConfig struct {
|
||||
*vm.LogConfig
|
||||
Tracer *string
|
||||
Timeout *string
|
||||
Reexec *uint64
|
||||
}
|
||||
|
||||
// txTraceResult is the result of a single transaction trace.
|
||||
type txTraceResult struct {
|
||||
Result interface{} `json:"result,omitempty"` // Trace results produced by the tracer
|
||||
Error string `json:"error,omitempty"` // Trace failure produced by the tracer
|
||||
}
|
||||
|
||||
// blockTraceTask represents a single block trace task when an entire chain is
|
||||
// being traced.
|
||||
type blockTraceTask struct {
|
||||
statedb *state.StateDB // Intermediate state prepped for tracing
|
||||
block *types.Block // Block to trace the transactions from
|
||||
results []*txTraceResult // Trace results procudes by the task
|
||||
}
|
||||
|
||||
// blockTraceResult represets the results of tracing a single block when an entire
|
||||
// chain is being traced.
|
||||
type blockTraceResult struct {
|
||||
Block hexutil.Uint64 `json:"block"` // Block number corresponding to this trace
|
||||
Hash common.Hash `json:"hash"` // Block hash corresponding to this trace
|
||||
Traces []*txTraceResult `json:"traces"` // Trace results produced by the task
|
||||
}
|
||||
|
||||
// txTraceTask represents a single transaction trace task when an entire block
|
||||
// is being traced.
|
||||
type txTraceTask struct {
|
||||
statedb *state.StateDB // Intermediate state prepped for tracing
|
||||
index int // Transaction offset in the block
|
||||
}
|
||||
|
||||
// ephemeralDatabase is a memory wrapper around a proper database, which acts as
|
||||
// an ephemeral write layer. This construct is used by the chain tracer to write
|
||||
// state tries for intermediate blocks without serializing to disk, but at the
|
||||
// same time to allow disk fallback for reads that do no hit the memory layer.
|
||||
type ephemeralDatabase struct {
|
||||
diskdb ethdb.Database // Persistent disk database to fall back to with reads
|
||||
memdb *ethdb.MemDatabase // Ephemeral memory database for primary reads and writes
|
||||
}
|
||||
|
||||
func (db *ephemeralDatabase) Put(key []byte, value []byte) error { return db.memdb.Put(key, value) }
|
||||
func (db *ephemeralDatabase) Delete(key []byte) error { return errors.New("delete not supported") }
|
||||
func (db *ephemeralDatabase) Close() { db.memdb.Close() }
|
||||
func (db *ephemeralDatabase) NewBatch() ethdb.Batch {
|
||||
return db.memdb.NewBatch()
|
||||
}
|
||||
func (db *ephemeralDatabase) Has(key []byte) (bool, error) {
|
||||
if has, _ := db.memdb.Has(key); has {
|
||||
return has, nil
|
||||
}
|
||||
return db.diskdb.Has(key)
|
||||
}
|
||||
func (db *ephemeralDatabase) Get(key []byte) ([]byte, error) {
|
||||
if blob, _ := db.memdb.Get(key); blob != nil {
|
||||
return blob, nil
|
||||
}
|
||||
return db.diskdb.Get(key)
|
||||
}
|
||||
|
||||
// Prune does a state sync into a new memory write layer and replaces the old one.
|
||||
// This allows us to discard entries that are no longer referenced from the current
|
||||
// state.
|
||||
func (db *ephemeralDatabase) Prune(root common.Hash) {
|
||||
// Pull the still relevant state data into memory
|
||||
sync := state.NewStateSync(root, db.diskdb)
|
||||
for sync.Pending() > 0 {
|
||||
hash := sync.Missing(1)[0]
|
||||
|
||||
// Move the next trie node from the memory layer into a sync struct
|
||||
node, err := db.memdb.Get(hash[:])
|
||||
if err != nil {
|
||||
panic(err) // memdb must have the data
|
||||
}
|
||||
if _, _, err := sync.Process([]trie.SyncResult{{Hash: hash, Data: node}}); err != nil {
|
||||
panic(err) // it's not possible to fail processing a node
|
||||
}
|
||||
}
|
||||
// Discard the old memory layer and write a new one
|
||||
db.memdb, _ = ethdb.NewMemDatabaseWithCap(db.memdb.Len())
|
||||
if _, err := sync.Commit(db); err != nil {
|
||||
panic(err) // writing into a memdb cannot fail
|
||||
}
|
||||
}
|
||||
|
||||
// TraceChain returns the structured logs created during the execution of EVM
|
||||
// between two blocks (excluding start) and returns them as a JSON object.
|
||||
func (api *PrivateDebugAPI) TraceChain(ctx context.Context, start, end rpc.BlockNumber, config *TraceConfig) (*rpc.Subscription, error) {
|
||||
// Fetch the block interval that we want to trace
|
||||
var from, to *types.Block
|
||||
|
||||
switch start {
|
||||
case rpc.PendingBlockNumber:
|
||||
from = api.eth.miner.PendingBlock()
|
||||
case rpc.LatestBlockNumber:
|
||||
from = api.eth.blockchain.CurrentBlock()
|
||||
default:
|
||||
from = api.eth.blockchain.GetBlockByNumber(uint64(start))
|
||||
}
|
||||
switch end {
|
||||
case rpc.PendingBlockNumber:
|
||||
to = api.eth.miner.PendingBlock()
|
||||
case rpc.LatestBlockNumber:
|
||||
to = api.eth.blockchain.CurrentBlock()
|
||||
default:
|
||||
to = api.eth.blockchain.GetBlockByNumber(uint64(end))
|
||||
}
|
||||
// Trace the chain if we've found all our blocks
|
||||
if from == nil {
|
||||
return nil, fmt.Errorf("starting block #%d not found", start)
|
||||
}
|
||||
if to == nil {
|
||||
return nil, fmt.Errorf("end block #%d not found", end)
|
||||
}
|
||||
return api.traceChain(ctx, from, to, config)
|
||||
}
|
||||
|
||||
// traceChain configures a new tracer according to the provided configuration, and
|
||||
// executes all the transactions contained within. The return value will be one item
|
||||
// per transaction, dependent on the requestd tracer.
|
||||
func (api *PrivateDebugAPI) traceChain(ctx context.Context, start, end *types.Block, config *TraceConfig) (*rpc.Subscription, error) {
|
||||
// Tracing a chain is a **long** operation, only do with subscriptions
|
||||
notifier, supported := rpc.NotifierFromContext(ctx)
|
||||
if !supported {
|
||||
return &rpc.Subscription{}, rpc.ErrNotificationsUnsupported
|
||||
}
|
||||
sub := notifier.CreateSubscription()
|
||||
|
||||
// Ensure we have a valid starting state before doing any work
|
||||
origin := start.NumberU64()
|
||||
|
||||
memdb, _ := ethdb.NewMemDatabase()
|
||||
db := &ephemeralDatabase{
|
||||
diskdb: api.eth.ChainDb(),
|
||||
memdb: memdb,
|
||||
}
|
||||
if number := start.NumberU64(); number > 0 {
|
||||
start = api.eth.blockchain.GetBlock(start.ParentHash(), start.NumberU64()-1)
|
||||
if start == nil {
|
||||
return nil, fmt.Errorf("parent block #%d not found", number-1)
|
||||
}
|
||||
}
|
||||
statedb, err := state.New(start.Root(), state.NewDatabase(db))
|
||||
if err != nil {
|
||||
// If the starting state is missing, allow some number of blocks to be reexecuted
|
||||
reexec := defaultTraceReexec
|
||||
if config.Reexec != nil {
|
||||
reexec = *config.Reexec
|
||||
}
|
||||
// Find the most recent block that has the state available
|
||||
for i := uint64(0); i < reexec; i++ {
|
||||
start = api.eth.blockchain.GetBlock(start.ParentHash(), start.NumberU64()-1)
|
||||
if start == nil {
|
||||
break
|
||||
}
|
||||
if statedb, err = state.New(start.Root(), state.NewDatabase(db)); err == nil {
|
||||
break
|
||||
}
|
||||
}
|
||||
// If we still don't have the state available, bail out
|
||||
if err != nil {
|
||||
switch err.(type) {
|
||||
case *trie.MissingNodeError:
|
||||
return nil, errors.New("required historical state unavailable")
|
||||
default:
|
||||
return nil, err
|
||||
}
|
||||
}
|
||||
}
|
||||
// Execute all the transaction contained within the chain concurrently for each block
|
||||
blocks := int(end.NumberU64() - origin)
|
||||
|
||||
threads := runtime.NumCPU()
|
||||
if threads > blocks {
|
||||
threads = blocks
|
||||
}
|
||||
var (
|
||||
pend = new(sync.WaitGroup)
|
||||
tasks = make(chan *blockTraceTask, threads)
|
||||
results = make(chan *blockTraceTask, threads)
|
||||
)
|
||||
for th := 0; th < threads; th++ {
|
||||
pend.Add(1)
|
||||
go func() {
|
||||
defer pend.Done()
|
||||
|
||||
// Fetch and execute the next block trace tasks
|
||||
for task := range tasks {
|
||||
signer := types.MakeSigner(api.config, task.block.Number())
|
||||
|
||||
// Trace all the transactions contained within
|
||||
for i, tx := range task.block.Transactions() {
|
||||
msg, _ := tx.AsMessage(signer)
|
||||
vmctx := core.NewEVMContext(msg, task.block.Header(), api.eth.blockchain, nil)
|
||||
|
||||
res, err := api.traceTx(ctx, msg, vmctx, task.statedb, config)
|
||||
if err != nil {
|
||||
task.results[i] = &txTraceResult{Error: err.Error()}
|
||||
log.Warn("Tracing failed", "err", err)
|
||||
break
|
||||
}
|
||||
task.statedb.DeleteSuicides()
|
||||
task.results[i] = &txTraceResult{Result: res}
|
||||
}
|
||||
// Stream the result back to the user or abort on teardown
|
||||
select {
|
||||
case results <- task:
|
||||
case <-notifier.Closed():
|
||||
return
|
||||
}
|
||||
}
|
||||
}()
|
||||
}
|
||||
// Start a goroutine to feed all the blocks into the tracers
|
||||
begin := time.Now()
|
||||
complete := start.NumberU64()
|
||||
|
||||
go func() {
|
||||
var (
|
||||
logged time.Time
|
||||
number uint64
|
||||
traced uint64
|
||||
failed error
|
||||
)
|
||||
// Ensure everything is properly cleaned up on any exit path
|
||||
defer func() {
|
||||
close(tasks)
|
||||
pend.Wait()
|
||||
|
||||
switch {
|
||||
case failed != nil:
|
||||
log.Warn("Chain tracing failed", "start", start.NumberU64(), "end", end.NumberU64(), "transactions", traced, "elapsed", time.Since(begin), "err", failed)
|
||||
case number < end.NumberU64():
|
||||
log.Warn("Chain tracing aborted", "start", start.NumberU64(), "end", end.NumberU64(), "abort", number, "transactions", traced, "elapsed", time.Since(begin))
|
||||
default:
|
||||
log.Info("Chain tracing finished", "start", start.NumberU64(), "end", end.NumberU64(), "transactions", traced, "elapsed", time.Since(begin))
|
||||
}
|
||||
close(results)
|
||||
}()
|
||||
// Feed all the blocks both into the tracer, as well as fast process concurrently
|
||||
for number = start.NumberU64() + 1; number <= end.NumberU64(); number++ {
|
||||
// Stop tracing if interruption was requested
|
||||
select {
|
||||
case <-notifier.Closed():
|
||||
return
|
||||
default:
|
||||
}
|
||||
// Print progress logs if long enough time elapsed
|
||||
if time.Since(logged) > 8*time.Second {
|
||||
if number > origin {
|
||||
log.Info("Tracing chain segment", "start", origin, "end", end.NumberU64(), "current", number, "transactions", traced, "elapsed", time.Since(begin))
|
||||
} else {
|
||||
log.Info("Preparing state for chain trace", "block", number, "start", origin, "elapsed", time.Since(begin))
|
||||
}
|
||||
logged = time.Now()
|
||||
}
|
||||
// Retrieve the next block to trace
|
||||
block := api.eth.blockchain.GetBlockByNumber(number)
|
||||
if block == nil {
|
||||
failed = fmt.Errorf("block #%d not found", number)
|
||||
break
|
||||
}
|
||||
// Send the block over to the concurrent tracers (if not in the fast-forward phase)
|
||||
if number > origin {
|
||||
txs := block.Transactions()
|
||||
|
||||
select {
|
||||
case tasks <- &blockTraceTask{statedb: statedb.Copy(), block: block, results: make([]*txTraceResult, len(txs))}:
|
||||
case <-notifier.Closed():
|
||||
return
|
||||
}
|
||||
traced += uint64(len(txs))
|
||||
} else {
|
||||
atomic.StoreUint64(&complete, number)
|
||||
}
|
||||
// Generate the next state snapshot fast without tracing
|
||||
_, _, _, err := api.eth.blockchain.Processor().Process(block, statedb, vm.Config{})
|
||||
if err != nil {
|
||||
failed = err
|
||||
break
|
||||
}
|
||||
// Finalize the state so any modifications are written to the trie
|
||||
root, err := statedb.CommitTo(db, true)
|
||||
if err != nil {
|
||||
failed = err
|
||||
break
|
||||
}
|
||||
if err := statedb.Reset(root); err != nil {
|
||||
failed = err
|
||||
break
|
||||
}
|
||||
// After every N blocks, prune the database to only retain relevant data
|
||||
if (number-start.NumberU64())%4096 == 0 {
|
||||
// Wait until currently pending trace jobs finish
|
||||
for atomic.LoadUint64(&complete) != number {
|
||||
select {
|
||||
case <-time.After(100 * time.Millisecond):
|
||||
case <-notifier.Closed():
|
||||
return
|
||||
}
|
||||
}
|
||||
// No more concurrent access at this point, prune the database
|
||||
var (
|
||||
nodes = db.memdb.Len()
|
||||
start = time.Now()
|
||||
)
|
||||
db.Prune(root)
|
||||
log.Info("Pruned tracer state entries", "deleted", nodes-db.memdb.Len(), "left", db.memdb.Len(), "elapsed", time.Since(start))
|
||||
|
||||
statedb, _ = state.New(root, state.NewDatabase(db))
|
||||
}
|
||||
}
|
||||
}()
|
||||
|
||||
// Keep reading the trace results and stream the to the user
|
||||
go func() {
|
||||
var (
|
||||
done = make(map[uint64]*blockTraceResult)
|
||||
next = origin + 1
|
||||
)
|
||||
for res := range results {
|
||||
// Queue up next received result
|
||||
result := &blockTraceResult{
|
||||
Block: hexutil.Uint64(res.block.NumberU64()),
|
||||
Hash: res.block.Hash(),
|
||||
Traces: res.results,
|
||||
}
|
||||
done[uint64(result.Block)] = result
|
||||
|
||||
// Stream completed traces to the user, aborting on the first error
|
||||
for result, ok := done[next]; ok; result, ok = done[next] {
|
||||
if len(result.Traces) > 0 || next == end.NumberU64() {
|
||||
notifier.Notify(sub.ID, result)
|
||||
}
|
||||
atomic.StoreUint64(&complete, next)
|
||||
delete(done, next)
|
||||
next++
|
||||
}
|
||||
}
|
||||
}()
|
||||
return sub, nil
|
||||
}
|
||||
|
||||
// TraceBlockByNumber returns the structured logs created during the execution of
|
||||
// EVM and returns them as a JSON object.
|
||||
func (api *PrivateDebugAPI) TraceBlockByNumber(ctx context.Context, number rpc.BlockNumber, config *TraceConfig) ([]*txTraceResult, error) {
|
||||
// Fetch the block that we want to trace
|
||||
var block *types.Block
|
||||
|
||||
switch number {
|
||||
case rpc.PendingBlockNumber:
|
||||
block = api.eth.miner.PendingBlock()
|
||||
case rpc.LatestBlockNumber:
|
||||
block = api.eth.blockchain.CurrentBlock()
|
||||
default:
|
||||
block = api.eth.blockchain.GetBlockByNumber(uint64(number))
|
||||
}
|
||||
// Trace the block if it was found
|
||||
if block == nil {
|
||||
return nil, fmt.Errorf("block #%d not found", number)
|
||||
}
|
||||
return api.traceBlock(ctx, block, config)
|
||||
}
|
||||
|
||||
// TraceBlockByHash returns the structured logs created during the execution of
|
||||
// EVM and returns them as a JSON object.
|
||||
func (api *PrivateDebugAPI) TraceBlockByHash(ctx context.Context, hash common.Hash, config *TraceConfig) ([]*txTraceResult, error) {
|
||||
block := api.eth.blockchain.GetBlockByHash(hash)
|
||||
if block == nil {
|
||||
return nil, fmt.Errorf("block #%x not found", hash)
|
||||
}
|
||||
return api.traceBlock(ctx, block, config)
|
||||
}
|
||||
|
||||
// TraceBlock returns the structured logs created during the execution of EVM
|
||||
// and returns them as a JSON object.
|
||||
func (api *PrivateDebugAPI) TraceBlock(ctx context.Context, blob []byte, config *TraceConfig) ([]*txTraceResult, error) {
|
||||
block := new(types.Block)
|
||||
if err := rlp.Decode(bytes.NewReader(blob), block); err != nil {
|
||||
return nil, fmt.Errorf("could not decode block: %v", err)
|
||||
}
|
||||
return api.traceBlock(ctx, block, config)
|
||||
}
|
||||
|
||||
// TraceBlockFromFile returns the structured logs created during the execution of
|
||||
// EVM and returns them as a JSON object.
|
||||
func (api *PrivateDebugAPI) TraceBlockFromFile(ctx context.Context, file string, config *TraceConfig) ([]*txTraceResult, error) {
|
||||
blob, err := ioutil.ReadFile(file)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("could not read file: %v", err)
|
||||
}
|
||||
return api.TraceBlock(ctx, blob, config)
|
||||
}
|
||||
|
||||
// traceBlock configures a new tracer according to the provided configuration, and
|
||||
// executes all the transactions contained within. The return value will be one item
|
||||
// per transaction, dependent on the requestd tracer.
|
||||
func (api *PrivateDebugAPI) traceBlock(ctx context.Context, block *types.Block, config *TraceConfig) ([]*txTraceResult, error) {
|
||||
// Create the parent state database
|
||||
if err := api.eth.engine.VerifyHeader(api.eth.blockchain, block.Header(), true); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
parent := api.eth.blockchain.GetBlock(block.ParentHash(), block.NumberU64()-1)
|
||||
if parent == nil {
|
||||
return nil, fmt.Errorf("parent %x not found", block.ParentHash())
|
||||
}
|
||||
reexec := defaultTraceReexec
|
||||
if config.Reexec != nil {
|
||||
reexec = *config.Reexec
|
||||
}
|
||||
statedb, err := api.computeStateDB(parent, reexec)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
// Execute all the transaction contained within the block concurrently
|
||||
var (
|
||||
signer = types.MakeSigner(api.config, block.Number())
|
||||
|
||||
txs = block.Transactions()
|
||||
results = make([]*txTraceResult, len(txs))
|
||||
|
||||
pend = new(sync.WaitGroup)
|
||||
jobs = make(chan *txTraceTask, len(txs))
|
||||
)
|
||||
threads := runtime.NumCPU()
|
||||
if threads > len(txs) {
|
||||
threads = len(txs)
|
||||
}
|
||||
for th := 0; th < threads; th++ {
|
||||
pend.Add(1)
|
||||
go func() {
|
||||
defer pend.Done()
|
||||
|
||||
// Fetch and execute the next transaction trace tasks
|
||||
for task := range jobs {
|
||||
msg, _ := txs[task.index].AsMessage(signer)
|
||||
vmctx := core.NewEVMContext(msg, block.Header(), api.eth.blockchain, nil)
|
||||
|
||||
res, err := api.traceTx(ctx, msg, vmctx, task.statedb, config)
|
||||
if err != nil {
|
||||
results[task.index] = &txTraceResult{Error: err.Error()}
|
||||
continue
|
||||
}
|
||||
results[task.index] = &txTraceResult{Result: res}
|
||||
}
|
||||
}()
|
||||
}
|
||||
// Feed the transactions into the tracers and return
|
||||
var failed error
|
||||
for i, tx := range txs {
|
||||
// Send the trace task over for execution
|
||||
jobs <- &txTraceTask{statedb: statedb.Copy(), index: i}
|
||||
|
||||
// Generate the next state snapshot fast without tracing
|
||||
msg, _ := tx.AsMessage(signer)
|
||||
vmctx := core.NewEVMContext(msg, block.Header(), api.eth.blockchain, nil)
|
||||
|
||||
vmenv := vm.NewEVM(vmctx, statedb, api.config, vm.Config{})
|
||||
if _, _, _, err := core.ApplyMessage(vmenv, msg, new(core.GasPool).AddGas(msg.Gas())); err != nil {
|
||||
failed = err
|
||||
break
|
||||
}
|
||||
// Finalize the state so any modifications are written to the trie
|
||||
statedb.Finalise(true)
|
||||
}
|
||||
close(jobs)
|
||||
pend.Wait()
|
||||
|
||||
// If execution failed in between, abort
|
||||
if failed != nil {
|
||||
return nil, failed
|
||||
}
|
||||
return results, nil
|
||||
}
|
||||
|
||||
// computeStateDB retrieves the state database associated with a certain block.
|
||||
// If no state is locally available for the given block, a number of blocks are
|
||||
// attempted to be reexecuted to generate the desired state.
|
||||
func (api *PrivateDebugAPI) computeStateDB(block *types.Block, reexec uint64) (*state.StateDB, error) {
|
||||
// If we have the state fully available, use that
|
||||
statedb, err := api.eth.blockchain.StateAt(block.Root())
|
||||
if err == nil {
|
||||
return statedb, nil
|
||||
}
|
||||
// Otherwise try to reexec blocks until we find a state or reach our limit
|
||||
origin := block.NumberU64()
|
||||
|
||||
memdb, _ := ethdb.NewMemDatabase()
|
||||
db := &ephemeralDatabase{
|
||||
diskdb: api.eth.ChainDb(),
|
||||
memdb: memdb,
|
||||
}
|
||||
for i := uint64(0); i < reexec; i++ {
|
||||
block = api.eth.blockchain.GetBlock(block.ParentHash(), block.NumberU64()-1)
|
||||
if block == nil {
|
||||
break
|
||||
}
|
||||
if statedb, err = state.New(block.Root(), state.NewDatabase(db)); err == nil {
|
||||
break
|
||||
}
|
||||
}
|
||||
if err != nil {
|
||||
switch err.(type) {
|
||||
case *trie.MissingNodeError:
|
||||
return nil, errors.New("required historical state unavailable")
|
||||
default:
|
||||
return nil, err
|
||||
}
|
||||
}
|
||||
// State was available at historical point, regenerate
|
||||
var (
|
||||
start = time.Now()
|
||||
logged time.Time
|
||||
)
|
||||
for block.NumberU64() < origin {
|
||||
// Print progress logs if long enough time elapsed
|
||||
if time.Since(logged) > 8*time.Second {
|
||||
log.Info("Regenerating historical state", "block", block.NumberU64()+1, "target", origin, "elapsed", time.Since(start))
|
||||
logged = time.Now()
|
||||
}
|
||||
// Retrieve the next block to regenerate and process it
|
||||
if block = api.eth.blockchain.GetBlockByNumber(block.NumberU64() + 1); block == nil {
|
||||
return nil, fmt.Errorf("block #%d not found", block.NumberU64()+1)
|
||||
}
|
||||
_, _, _, err := api.eth.blockchain.Processor().Process(block, statedb, vm.Config{})
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
// Finalize the state so any modifications are written to the trie
|
||||
root, err := statedb.CommitTo(db, true)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if err := statedb.Reset(root); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
// After every N blocks, prune the database to only retain relevant data
|
||||
if block.NumberU64()%4096 == 0 || block.NumberU64() == origin {
|
||||
var (
|
||||
nodes = db.memdb.Len()
|
||||
begin = time.Now()
|
||||
)
|
||||
db.Prune(root)
|
||||
log.Info("Pruned tracer state entries", "deleted", nodes-db.memdb.Len(), "left", db.memdb.Len(), "elapsed", time.Since(begin))
|
||||
|
||||
statedb, _ = state.New(root, state.NewDatabase(db))
|
||||
}
|
||||
}
|
||||
log.Info("Historical state regenerated", "block", block.NumberU64(), "elapsed", time.Since(start))
|
||||
return statedb, nil
|
||||
}
|
||||
|
||||
// TraceTransaction returns the structured logs created during the execution of EVM
|
||||
// and returns them as a JSON object.
|
||||
func (api *PrivateDebugAPI) TraceTransaction(ctx context.Context, hash common.Hash, config *TraceConfig) (interface{}, error) {
|
||||
// Retrieve the transaction and assemble its EVM context
|
||||
tx, blockHash, _, index := core.GetTransaction(api.eth.ChainDb(), hash)
|
||||
if tx == nil {
|
||||
return nil, fmt.Errorf("transaction %x not found", hash)
|
||||
}
|
||||
reexec := defaultTraceReexec
|
||||
if config.Reexec != nil {
|
||||
reexec = *config.Reexec
|
||||
}
|
||||
msg, vmctx, statedb, err := api.computeTxEnv(blockHash, int(index), reexec)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
// Trace the transaction and return
|
||||
return api.traceTx(ctx, msg, vmctx, statedb, config)
|
||||
}
|
||||
|
||||
// traceTx configures a new tracer according to the provided configuration, and
|
||||
// executes the given message in the provided environment. The return value will
|
||||
// be tracer dependent.
|
||||
func (api *PrivateDebugAPI) traceTx(ctx context.Context, message core.Message, vmctx vm.Context, statedb *state.StateDB, config *TraceConfig) (interface{}, error) {
|
||||
// Assemble the structured logger or the JavaScript tracer
|
||||
var (
|
||||
tracer vm.Tracer
|
||||
err error
|
||||
)
|
||||
switch {
|
||||
case config != nil && config.Tracer != nil:
|
||||
// Define a meaningful timeout of a single transaction trace
|
||||
timeout := defaultTraceTimeout
|
||||
if config.Timeout != nil {
|
||||
if timeout, err = time.ParseDuration(*config.Timeout); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
}
|
||||
// Constuct the JavaScript tracer to execute with
|
||||
if tracer, err = tracers.New(*config.Tracer); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
// Handle timeouts and RPC cancellations
|
||||
deadlineCtx, cancel := context.WithTimeout(ctx, timeout)
|
||||
go func() {
|
||||
<-deadlineCtx.Done()
|
||||
tracer.(*tracers.Tracer).Stop(errors.New("execution timeout"))
|
||||
}()
|
||||
defer cancel()
|
||||
|
||||
case config == nil:
|
||||
tracer = vm.NewStructLogger(nil)
|
||||
|
||||
default:
|
||||
tracer = vm.NewStructLogger(config.LogConfig)
|
||||
}
|
||||
// Run the transaction with tracing enabled.
|
||||
vmenv := vm.NewEVM(vmctx, statedb, api.config, vm.Config{Debug: true, Tracer: tracer})
|
||||
|
||||
ret, gas, failed, err := core.ApplyMessage(vmenv, message, new(core.GasPool).AddGas(message.Gas()))
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("tracing failed: %v", err)
|
||||
}
|
||||
// Depending on the tracer type, format and return the output
|
||||
switch tracer := tracer.(type) {
|
||||
case *vm.StructLogger:
|
||||
return ðapi.ExecutionResult{
|
||||
Gas: gas,
|
||||
Failed: failed,
|
||||
ReturnValue: fmt.Sprintf("%x", ret),
|
||||
StructLogs: ethapi.FormatLogs(tracer.StructLogs()),
|
||||
}, nil
|
||||
|
||||
case *tracers.Tracer:
|
||||
return tracer.GetResult()
|
||||
|
||||
default:
|
||||
panic(fmt.Sprintf("bad tracer type %T", tracer))
|
||||
}
|
||||
}
|
||||
|
||||
// computeTxEnv returns the execution environment of a certain transaction.
|
||||
func (api *PrivateDebugAPI) computeTxEnv(blockHash common.Hash, txIndex int, reexec uint64) (core.Message, vm.Context, *state.StateDB, error) {
|
||||
// Create the parent state database
|
||||
block := api.eth.blockchain.GetBlockByHash(blockHash)
|
||||
if block == nil {
|
||||
return nil, vm.Context{}, nil, fmt.Errorf("block %x not found", blockHash)
|
||||
}
|
||||
parent := api.eth.blockchain.GetBlock(block.ParentHash(), block.NumberU64()-1)
|
||||
if parent == nil {
|
||||
return nil, vm.Context{}, nil, fmt.Errorf("parent %x not found", block.ParentHash())
|
||||
}
|
||||
statedb, err := api.computeStateDB(parent, reexec)
|
||||
if err != nil {
|
||||
return nil, vm.Context{}, nil, err
|
||||
}
|
||||
// Recompute transactions up to the target index.
|
||||
signer := types.MakeSigner(api.config, block.Number())
|
||||
|
||||
for idx, tx := range block.Transactions() {
|
||||
// Assemble the transaction call message and return if the requested offset
|
||||
msg, _ := tx.AsMessage(signer)
|
||||
context := core.NewEVMContext(msg, block.Header(), api.eth.blockchain, nil)
|
||||
if idx == txIndex {
|
||||
return msg, context, statedb, nil
|
||||
}
|
||||
// Not yet the searched for transaction, execute on top of the current state
|
||||
vmenv := vm.NewEVM(context, statedb, api.config, vm.Config{})
|
||||
if _, _, _, err := core.ApplyMessage(vmenv, msg, new(core.GasPool).AddGas(tx.Gas())); err != nil {
|
||||
return nil, vm.Context{}, nil, fmt.Errorf("tx %x failed: %v", tx.Hash(), err)
|
||||
}
|
||||
statedb.DeleteSuicides()
|
||||
}
|
||||
return nil, vm.Context{}, nil, fmt.Errorf("tx index %d out of range for block %x", txIndex, blockHash)
|
||||
}
|
@ -747,10 +747,11 @@ func (self *ProtocolManager) txBroadcastLoop() {
|
||||
// EthNodeInfo represents a short summary of the Ethereum sub-protocol metadata known
|
||||
// about the host peer.
|
||||
type EthNodeInfo struct {
|
||||
Network uint64 `json:"network"` // Ethereum network ID (1=Frontier, 2=Morden, Ropsten=3)
|
||||
Difficulty *big.Int `json:"difficulty"` // Total difficulty of the host's blockchain
|
||||
Genesis common.Hash `json:"genesis"` // SHA3 hash of the host's genesis block
|
||||
Head common.Hash `json:"head"` // SHA3 hash of the host's best owned block
|
||||
Network uint64 `json:"network"` // Ethereum network ID (1=Frontier, 2=Morden, Ropsten=3)
|
||||
Difficulty *big.Int `json:"difficulty"` // Total difficulty of the host's blockchain
|
||||
Genesis common.Hash `json:"genesis"` // SHA3 hash of the host's genesis block
|
||||
Config *params.ChainConfig `json:"config"` // Chain configuration for the fork rules
|
||||
Head common.Hash `json:"head"` // SHA3 hash of the host's best owned block
|
||||
}
|
||||
|
||||
// NodeInfo retrieves some protocol metadata about the running host node.
|
||||
@ -760,6 +761,7 @@ func (self *ProtocolManager) NodeInfo() *EthNodeInfo {
|
||||
Network: self.networkId,
|
||||
Difficulty: self.blockchain.GetTd(currentBlock.Hash(), currentBlock.NumberU64()),
|
||||
Genesis: self.blockchain.Genesis().Hash(),
|
||||
Config: self.blockchain.Config(),
|
||||
Head: currentBlock.Hash(),
|
||||
}
|
||||
}
|
||||
|
86
eth/tracers/internal/tracers/4byte_tracer.js
Normal file
86
eth/tracers/internal/tracers/4byte_tracer.js
Normal file
@ -0,0 +1,86 @@
|
||||
// Copyright 2017 The go-ethereum Authors
|
||||
// This file is part of the go-ethereum library.
|
||||
//
|
||||
// The go-ethereum library is free software: you can redistribute it and/or modify
|
||||
// it under the terms of the GNU Lesser General Public License as published by
|
||||
// the Free Software Foundation, either version 3 of the License, or
|
||||
// (at your option) any later version.
|
||||
//
|
||||
// The go-ethereum library is distributed in the hope that it will be useful,
|
||||
// but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||
// GNU Lesser General Public License for more details.
|
||||
//
|
||||
// You should have received a copy of the GNU Lesser General Public License
|
||||
// along with the go-ethereum library. If not, see <http://www.gnu.org/licenses/>.
|
||||
|
||||
// 4byteTracer searches for 4byte-identifiers, and collects them for post-processing.
|
||||
// It collects the methods identifiers along with the size of the supplied data, so
|
||||
// a reversed signature can be matched against the size of the data.
|
||||
//
|
||||
// Example:
|
||||
// > debug.traceTransaction( "0x214e597e35da083692f5386141e69f47e973b2c56e7a8073b1ea08fd7571e9de", {tracer: "4byteTracer"})
|
||||
// {
|
||||
// 0x27dc297e-128: 1,
|
||||
// 0x38cc4831-0: 2,
|
||||
// 0x524f3889-96: 1,
|
||||
// 0xadf59f99-288: 1,
|
||||
// 0xc281d19e-0: 1
|
||||
// }
|
||||
{
|
||||
// ids aggregates the 4byte ids found.
|
||||
ids : {},
|
||||
|
||||
// callType returns 'false' for non-calls, or the peek-index for the first param
|
||||
// after 'value', i.e. meminstart.
|
||||
callType: function(opstr){
|
||||
switch(opstr){
|
||||
case "CALL": case "CALLCODE":
|
||||
// gas, addr, val, memin, meminsz, memout, memoutsz
|
||||
return 3; // stack ptr to memin
|
||||
|
||||
case "DELEGATECALL": case "STATICCALL":
|
||||
// gas, addr, memin, meminsz, memout, memoutsz
|
||||
return 2; // stack ptr to memin
|
||||
}
|
||||
return false;
|
||||
},
|
||||
|
||||
// store save the given indentifier and datasize.
|
||||
store: function(id, size){
|
||||
var key = "" + toHex(id) + "-" + size;
|
||||
this.ids[key] = this.ids[key] + 1 || 1;
|
||||
},
|
||||
|
||||
// step is invoked for every opcode that the VM executes.
|
||||
step: function(log, db) {
|
||||
// Skip any opcodes that are not internal calls
|
||||
var ct = this.callType(log.op.toString());
|
||||
if (!ct) {
|
||||
return;
|
||||
}
|
||||
// Skip any pre-compile invocations, those are just fancy opcodes
|
||||
if (isPrecompiled(toAddress(log.stack.peek(1)))) {
|
||||
return;
|
||||
}
|
||||
// Gather internal call details
|
||||
var inSz = log.stack.peek(ct + 1).valueOf();
|
||||
if (inSz >= 4) {
|
||||
var inOff = log.stack.peek(ct).valueOf();
|
||||
this.store(log.memory.slice(inOff, inOff + 4), inSz-4);
|
||||
}
|
||||
},
|
||||
|
||||
// fault is invoked when the actual execution of an opcode fails.
|
||||
fault: function(log, db) { },
|
||||
|
||||
// result is invoked when all the opcodes have been iterated over and returns
|
||||
// the final result of the tracing.
|
||||
result: function(ctx) {
|
||||
// Save the outer calldata also
|
||||
if (ctx.input.length > 4) {
|
||||
this.store(slice(ctx.input, 0, 4), ctx.input.length-4)
|
||||
}
|
||||
return this.ids;
|
||||
},
|
||||
}
|
350
eth/tracers/internal/tracers/assets.go
Normal file
350
eth/tracers/internal/tracers/assets.go
Normal file
File diff suppressed because one or more lines are too long
246
eth/tracers/internal/tracers/call_tracer.js
Normal file
246
eth/tracers/internal/tracers/call_tracer.js
Normal file
@ -0,0 +1,246 @@
|
||||
// Copyright 2017 The go-ethereum Authors
|
||||
// This file is part of the go-ethereum library.
|
||||
//
|
||||
// The go-ethereum library is free software: you can redistribute it and/or modify
|
||||
// it under the terms of the GNU Lesser General Public License as published by
|
||||
// the Free Software Foundation, either version 3 of the License, or
|
||||
// (at your option) any later version.
|
||||
//
|
||||
// The go-ethereum library is distributed in the hope that it will be useful,
|
||||
// but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||
// GNU Lesser General Public License for more details.
|
||||
//
|
||||
// You should have received a copy of the GNU Lesser General Public License
|
||||
// along with the go-ethereum library. If not, see <http://www.gnu.org/licenses/>.
|
||||
|
||||
// callTracer is a full blown transaction tracer that extracts and reports all
|
||||
// the internal calls made by a transaction, along with any useful information.
|
||||
{
|
||||
// callstack is the current recursive call stack of the EVM execution.
|
||||
callstack: [{}],
|
||||
|
||||
// descended tracks whether we've just descended from an outer transaction into
|
||||
// an inner call.
|
||||
descended: false,
|
||||
|
||||
// step is invoked for every opcode that the VM executes.
|
||||
step: function(log, db) {
|
||||
// Capture any errors immediately
|
||||
var error = log.getError();
|
||||
if (error !== undefined) {
|
||||
this.fault(log, db);
|
||||
return;
|
||||
}
|
||||
// We only care about system opcodes, faster if we pre-check once
|
||||
var syscall = (log.op.toNumber() & 0xf0) == 0xf0;
|
||||
if (syscall) {
|
||||
var op = log.op.toString();
|
||||
}
|
||||
// If a new contract is being created, add to the call stack
|
||||
if (syscall && op == 'CREATE') {
|
||||
var inOff = log.stack.peek(1).valueOf();
|
||||
var inEnd = inOff + log.stack.peek(2).valueOf();
|
||||
|
||||
// Assemble the internal call report and store for completion
|
||||
var call = {
|
||||
type: op,
|
||||
from: toHex(log.contract.getAddress()),
|
||||
input: toHex(log.memory.slice(inOff, inEnd)),
|
||||
gasIn: log.getGas(),
|
||||
gasCost: log.getCost(),
|
||||
value: '0x' + log.stack.peek(0).toString(16)
|
||||
};
|
||||
this.callstack.push(call);
|
||||
this.descended = true
|
||||
return;
|
||||
}
|
||||
// If a contract is being self destructed, gather that as a subcall too
|
||||
if (syscall && op == 'SELFDESTRUCT') {
|
||||
var left = this.callstack.length;
|
||||
if (this.callstack[left-1].calls === undefined) {
|
||||
this.callstack[left-1].calls = [];
|
||||
}
|
||||
this.callstack[left-1].calls.push({type: op});
|
||||
return
|
||||
}
|
||||
// If a new method invocation is being done, add to the call stack
|
||||
if (syscall && (op == 'CALL' || op == 'CALLCODE' || op == 'DELEGATECALL' || op == 'STATICCALL')) {
|
||||
// Skip any pre-compile invocations, those are just fancy opcodes
|
||||
var to = toAddress(log.stack.peek(1).toString(16));
|
||||
if (isPrecompiled(to)) {
|
||||
return
|
||||
}
|
||||
var off = (op == 'DELEGATECALL' || op == 'STATICCALL' ? 0 : 1);
|
||||
|
||||
var inOff = log.stack.peek(2 + off).valueOf();
|
||||
var inEnd = inOff + log.stack.peek(3 + off).valueOf();
|
||||
|
||||
// Assemble the internal call report and store for completion
|
||||
var call = {
|
||||
type: op,
|
||||
from: toHex(log.contract.getAddress()),
|
||||
to: toHex(to),
|
||||
input: toHex(log.memory.slice(inOff, inEnd)),
|
||||
gasIn: log.getGas(),
|
||||
gasCost: log.getCost(),
|
||||
outOff: log.stack.peek(4 + off).valueOf(),
|
||||
outLen: log.stack.peek(5 + off).valueOf()
|
||||
};
|
||||
if (op != 'DELEGATECALL' && op != 'STATICCALL') {
|
||||
call.value = '0x' + log.stack.peek(2).toString(16);
|
||||
}
|
||||
this.callstack.push(call);
|
||||
this.descended = true
|
||||
return;
|
||||
}
|
||||
// If we've just descended into an inner call, retrieve it's true allowance. We
|
||||
// need to extract if from within the call as there may be funky gas dynamics
|
||||
// with regard to requested and actually given gas (2300 stipend, 63/64 rule).
|
||||
if (this.descended) {
|
||||
if (log.getDepth() >= this.callstack.length) {
|
||||
this.callstack[this.callstack.length - 1].gas = log.getGas();
|
||||
} else {
|
||||
// TODO(karalabe): The call was made to a plain account. We currently don't
|
||||
// have access to the true gas amount inside the call and so any amount will
|
||||
// mostly be wrong since it depends on a lot of input args. Skip gas for now.
|
||||
}
|
||||
this.descended = false;
|
||||
}
|
||||
// If an existing call is returning, pop off the call stack
|
||||
if (syscall && op == 'REVERT') {
|
||||
this.callstack[this.callstack.length - 1].error = "execution reverted";
|
||||
return;
|
||||
}
|
||||
if (log.getDepth() == this.callstack.length - 1) {
|
||||
// Pop off the last call and get the execution results
|
||||
var call = this.callstack.pop();
|
||||
|
||||
if (call.type == 'CREATE') {
|
||||
// If the call was a CREATE, retrieve the contract address and output code
|
||||
call.gasUsed = '0x' + bigInt(call.gasIn - call.gasCost - log.getGas()).toString(16);
|
||||
delete call.gasIn; delete call.gasCost;
|
||||
|
||||
var ret = log.stack.peek(0);
|
||||
if (!ret.equals(0)) {
|
||||
call.to = toHex(toAddress(ret.toString(16)));
|
||||
call.output = toHex(db.getCode(toAddress(ret.toString(16))));
|
||||
} else if (call.error === undefined) {
|
||||
call.error = "internal failure"; // TODO(karalabe): surface these faults somehow
|
||||
}
|
||||
} else {
|
||||
// If the call was a contract call, retrieve the gas usage and output
|
||||
if (call.gas !== undefined) {
|
||||
call.gasUsed = '0x' + bigInt(call.gasIn - call.gasCost + call.gas - log.getGas()).toString(16);
|
||||
|
||||
var ret = log.stack.peek(0);
|
||||
if (!ret.equals(0)) {
|
||||
call.output = toHex(log.memory.slice(call.outOff, call.outOff + call.outLen));
|
||||
} else if (call.error === undefined) {
|
||||
call.error = "internal failure"; // TODO(karalabe): surface these faults somehow
|
||||
}
|
||||
}
|
||||
delete call.gasIn; delete call.gasCost;
|
||||
delete call.outOff; delete call.outLen;
|
||||
}
|
||||
if (call.gas !== undefined) {
|
||||
call.gas = '0x' + bigInt(call.gas).toString(16);
|
||||
}
|
||||
// Inject the call into the previous one
|
||||
var left = this.callstack.length;
|
||||
if (this.callstack[left-1].calls === undefined) {
|
||||
this.callstack[left-1].calls = [];
|
||||
}
|
||||
this.callstack[left-1].calls.push(call);
|
||||
}
|
||||
},
|
||||
|
||||
// fault is invoked when the actual execution of an opcode fails.
|
||||
fault: function(log, db) {
|
||||
// If the topmost call already reverted, don't handle the additional fault again
|
||||
if (this.callstack[this.callstack.length - 1].error !== undefined) {
|
||||
return;
|
||||
}
|
||||
// Pop off the just failed call
|
||||
var call = this.callstack.pop();
|
||||
call.error = log.getError();
|
||||
|
||||
// Consume all available gas and clean any leftovers
|
||||
if (call.gas !== undefined) {
|
||||
call.gas = '0x' + bigInt(call.gas).toString(16);
|
||||
call.gasUsed = call.gas
|
||||
}
|
||||
delete call.gasIn; delete call.gasCost;
|
||||
delete call.outOff; delete call.outLen;
|
||||
|
||||
// Flatten the failed call into its parent
|
||||
var left = this.callstack.length;
|
||||
if (left > 0) {
|
||||
if (this.callstack[left-1].calls === undefined) {
|
||||
this.callstack[left-1].calls = [];
|
||||
}
|
||||
this.callstack[left-1].calls.push(call);
|
||||
return;
|
||||
}
|
||||
// Last call failed too, leave it in the stack
|
||||
this.callstack.push(call);
|
||||
},
|
||||
|
||||
// result is invoked when all the opcodes have been iterated over and returns
|
||||
// the final result of the tracing.
|
||||
result: function(ctx, db) {
|
||||
var result = {
|
||||
type: ctx.type,
|
||||
from: toHex(ctx.from),
|
||||
to: toHex(ctx.to),
|
||||
value: '0x' + ctx.value.toString(16),
|
||||
gas: '0x' + bigInt(ctx.gas).toString(16),
|
||||
gasUsed: '0x' + bigInt(ctx.gasUsed).toString(16),
|
||||
input: toHex(ctx.input),
|
||||
output: toHex(ctx.output),
|
||||
time: ctx.time,
|
||||
};
|
||||
if (this.callstack[0].calls !== undefined) {
|
||||
result.calls = this.callstack[0].calls;
|
||||
}
|
||||
if (this.callstack[0].error !== undefined) {
|
||||
result.error = this.callstack[0].error;
|
||||
} else if (ctx.error !== undefined) {
|
||||
result.error = ctx.error;
|
||||
}
|
||||
if (result.error !== undefined) {
|
||||
delete result.output;
|
||||
}
|
||||
return this.finalize(result);
|
||||
},
|
||||
|
||||
// finalize recreates a call object using the final desired field oder for json
|
||||
// serialization. This is a nicety feature to pass meaningfully ordered results
|
||||
// to users who don't interpret it, just display it.
|
||||
finalize: function(call) {
|
||||
var sorted = {
|
||||
type: call.type,
|
||||
from: call.from,
|
||||
to: call.to,
|
||||
value: call.value,
|
||||
gas: call.gas,
|
||||
gasUsed: call.gasUsed,
|
||||
input: call.input,
|
||||
output: call.output,
|
||||
error: call.error,
|
||||
time: call.time,
|
||||
calls: call.calls,
|
||||
}
|
||||
for (var key in sorted) {
|
||||
if (sorted[key] === undefined) {
|
||||
delete sorted[key];
|
||||
}
|
||||
}
|
||||
if (sorted.calls !== undefined) {
|
||||
for (var i=0; i<sorted.calls.length; i++) {
|
||||
sorted.calls[i] = this.finalize(sorted.calls[i]);
|
||||
}
|
||||
}
|
||||
return sorted;
|
||||
}
|
||||
}
|
93
eth/tracers/internal/tracers/evmdis_tracer.js
Normal file
93
eth/tracers/internal/tracers/evmdis_tracer.js
Normal file
@ -0,0 +1,93 @@
|
||||
// Copyright 2017 The go-ethereum Authors
|
||||
// This file is part of the go-ethereum library.
|
||||
//
|
||||
// The go-ethereum library is free software: you can redistribute it and/or modify
|
||||
// it under the terms of the GNU Lesser General Public License as published by
|
||||
// the Free Software Foundation, either version 3 of the License, or
|
||||
// (at your option) any later version.
|
||||
//
|
||||
// The go-ethereum library is distributed in the hope that it will be useful,
|
||||
// but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||
// GNU Lesser General Public License for more details.
|
||||
//
|
||||
// You should have received a copy of the GNU Lesser General Public License
|
||||
// along with the go-ethereum library. If not, see <http://www.gnu.org/licenses/>.
|
||||
|
||||
// evmdisTracer returns sufficent information from a trace to perform evmdis-style
|
||||
// disassembly.
|
||||
{
|
||||
stack: [{ops: []}],
|
||||
|
||||
npushes: {0: 0, 1: 1, 2: 1, 3: 1, 4: 1, 5: 1, 6: 1, 7: 1, 8: 1, 9: 1, 10: 1, 11: 1, 16: 1, 17: 1, 18: 1, 19: 1, 20: 1, 21: 1, 22: 1, 23: 1, 24: 1, 25: 1, 26: 1, 32: 1, 48: 1, 49: 1, 50: 1, 51: 1, 52: 1, 53: 1, 54: 1, 55: 0, 56: 1, 57: 0, 58: 1, 59: 1, 60: 0, 64: 1, 65: 1, 66: 1, 67: 1, 68: 1, 69: 1, 80: 0, 81: 1, 82: 0, 83: 0, 84: 1, 85: 0, 86: 0, 87: 0, 88: 1, 89: 1, 90: 1, 91: 0, 96: 1, 97: 1, 98: 1, 99: 1, 100: 1, 101: 1, 102: 1, 103: 1, 104: 1, 105: 1, 106: 1, 107: 1, 108: 1, 109: 1, 110: 1, 111: 1, 112: 1, 113: 1, 114: 1, 115: 1, 116: 1, 117: 1, 118: 1, 119: 1, 120: 1, 121: 1, 122: 1, 123: 1, 124: 1, 125: 1, 126: 1, 127: 1, 128: 2, 129: 3, 130: 4, 131: 5, 132: 6, 133: 7, 134: 8, 135: 9, 136: 10, 137: 11, 138: 12, 139: 13, 140: 14, 141: 15, 142: 16, 143: 17, 144: 2, 145: 3, 146: 4, 147: 5, 148: 6, 149: 7, 150: 8, 151: 9, 152: 10, 153: 11, 154: 12, 155: 13, 156: 14, 157: 15, 158: 16, 159: 17, 160: 0, 161: 0, 162: 0, 163: 0, 164: 0, 240: 1, 241: 1, 242: 1, 243: 0, 244: 0, 255: 0},
|
||||
|
||||
// result is invoked when all the opcodes have been iterated over and returns
|
||||
// the final result of the tracing.
|
||||
result: function() { return this.stack[0].ops; },
|
||||
|
||||
// fault is invoked when the actual execution of an opcode fails.
|
||||
fault: function(log, db) { },
|
||||
|
||||
// step is invoked for every opcode that the VM executes.
|
||||
step: function(log, db) {
|
||||
var frame = this.stack[this.stack.length - 1];
|
||||
|
||||
var error = log.getError();
|
||||
if (error) {
|
||||
frame["error"] = error;
|
||||
} else if (log.getDepth() == this.stack.length) {
|
||||
opinfo = {
|
||||
op: log.op.toNumber(),
|
||||
depth : log.getDepth(),
|
||||
result: [],
|
||||
};
|
||||
if (frame.ops.length > 0) {
|
||||
var prevop = frame.ops[frame.ops.length - 1];
|
||||
for(var i = 0; i < this.npushes[prevop.op]; i++)
|
||||
prevop.result.push(log.stack.peek(i).toString(16));
|
||||
}
|
||||
switch(log.op.toString()) {
|
||||
case "CALL": case "CALLCODE":
|
||||
var instart = log.stack.peek(3).valueOf();
|
||||
var insize = log.stack.peek(4).valueOf();
|
||||
opinfo["gas"] = log.stack.peek(0).valueOf();
|
||||
opinfo["to"] = log.stack.peek(1).toString(16);
|
||||
opinfo["value"] = log.stack.peek(2).toString();
|
||||
opinfo["input"] = log.memory.slice(instart, instart + insize);
|
||||
opinfo["error"] = null;
|
||||
opinfo["return"] = null;
|
||||
opinfo["ops"] = [];
|
||||
this.stack.push(opinfo);
|
||||
break;
|
||||
case "DELEGATECALL": case "STATICCALL":
|
||||
var instart = log.stack.peek(2).valueOf();
|
||||
var insize = log.stack.peek(3).valueOf();
|
||||
opinfo["op"] = log.op.toString();
|
||||
opinfo["gas"] = log.stack.peek(0).valueOf();
|
||||
opinfo["to"] = log.stack.peek(1).toString(16);
|
||||
opinfo["input"] = log.memory.slice(instart, instart + insize);
|
||||
opinfo["error"] = null;
|
||||
opinfo["return"] = null;
|
||||
opinfo["ops"] = [];
|
||||
this.stack.push(opinfo);
|
||||
break;
|
||||
case "RETURN":
|
||||
var out = log.stack.peek(0).valueOf();
|
||||
var outsize = log.stack.peek(1).valueOf();
|
||||
frame.return = log.memory.slice(out, out + outsize);
|
||||
break;
|
||||
case "STOP": case "SUICIDE":
|
||||
frame.return = log.memory.slice(0, 0);
|
||||
break;
|
||||
case "JUMPDEST":
|
||||
opinfo["pc"] = log.getPC();
|
||||
}
|
||||
if(log.op.isPush()) {
|
||||
opinfo["len"] = log.op.toNumber() - 0x5e;
|
||||
}
|
||||
frame.ops.push(opinfo);
|
||||
} else {
|
||||
this.stack = this.stack.slice(0, log.getDepth());
|
||||
}
|
||||
}
|
||||
}
|
29
eth/tracers/internal/tracers/noop_tracer.js
Normal file
29
eth/tracers/internal/tracers/noop_tracer.js
Normal file
@ -0,0 +1,29 @@
|
||||
// Copyright 2017 The go-ethereum Authors
|
||||
// This file is part of the go-ethereum library.
|
||||
//
|
||||
// The go-ethereum library is free software: you can redistribute it and/or modify
|
||||
// it under the terms of the GNU Lesser General Public License as published by
|
||||
// the Free Software Foundation, either version 3 of the License, or
|
||||
// (at your option) any later version.
|
||||
//
|
||||
// The go-ethereum library is distributed in the hope that it will be useful,
|
||||
// but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||
// GNU Lesser General Public License for more details.
|
||||
//
|
||||
// You should have received a copy of the GNU Lesser General Public License
|
||||
// along with the go-ethereum library. If not, see <http://www.gnu.org/licenses/>.
|
||||
|
||||
// noopTracer is just the barebone boilerplate code required from a JavaScript
|
||||
// object to be usable as a transaction tracer.
|
||||
{
|
||||
// step is invoked for every opcode that the VM executes.
|
||||
step: function(log, db) { },
|
||||
|
||||
// fault is invoked when the actual execution of an opcode fails.
|
||||
fault: function(log, db) { },
|
||||
|
||||
// result is invoked when all the opcodes have been iterated over and returns
|
||||
// the final result of the tracing.
|
||||
result: function(ctx, db) { }
|
||||
}
|
32
eth/tracers/internal/tracers/opcount_tracer.js
Normal file
32
eth/tracers/internal/tracers/opcount_tracer.js
Normal file
@ -0,0 +1,32 @@
|
||||
// Copyright 2017 The go-ethereum Authors
|
||||
// This file is part of the go-ethereum library.
|
||||
//
|
||||
// The go-ethereum library is free software: you can redistribute it and/or modify
|
||||
// it under the terms of the GNU Lesser General Public License as published by
|
||||
// the Free Software Foundation, either version 3 of the License, or
|
||||
// (at your option) any later version.
|
||||
//
|
||||
// The go-ethereum library is distributed in the hope that it will be useful,
|
||||
// but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||
// GNU Lesser General Public License for more details.
|
||||
//
|
||||
// You should have received a copy of the GNU Lesser General Public License
|
||||
// along with the go-ethereum library. If not, see <http://www.gnu.org/licenses/>.
|
||||
|
||||
// opcountTracer is a sample tracer that just counts the number of instructions
|
||||
// executed by the EVM before the transaction terminated.
|
||||
{
|
||||
// count tracks the number of EVM instructions executed.
|
||||
count: 0,
|
||||
|
||||
// step is invoked for every opcode that the VM executes.
|
||||
step: function(log, db) { this.count++ },
|
||||
|
||||
// fault is invoked when the actual execution of an opcode fails.
|
||||
fault: function(log, db) { },
|
||||
|
||||
// result is invoked when all the opcodes have been iterated over and returns
|
||||
// the final result of the tracing.
|
||||
result: function(ctx, db) { return this.count }
|
||||
}
|
103
eth/tracers/internal/tracers/prestate_tracer.js
Normal file
103
eth/tracers/internal/tracers/prestate_tracer.js
Normal file
@ -0,0 +1,103 @@
|
||||
// Copyright 2017 The go-ethereum Authors
|
||||
// This file is part of the go-ethereum library.
|
||||
//
|
||||
// The go-ethereum library is free software: you can redistribute it and/or modify
|
||||
// it under the terms of the GNU Lesser General Public License as published by
|
||||
// the Free Software Foundation, either version 3 of the License, or
|
||||
// (at your option) any later version.
|
||||
//
|
||||
// The go-ethereum library is distributed in the hope that it will be useful,
|
||||
// but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||
// GNU Lesser General Public License for more details.
|
||||
//
|
||||
// You should have received a copy of the GNU Lesser General Public License
|
||||
// along with the go-ethereum library. If not, see <http://www.gnu.org/licenses/>.
|
||||
|
||||
// prestateTracer outputs sufficient information to create a local execution of
|
||||
// the transaction from a custom assembled genesis block.
|
||||
{
|
||||
// prestate is the genesis that we're building.
|
||||
prestate: null,
|
||||
|
||||
// lookupAccount injects the specified account into the prestate object.
|
||||
lookupAccount: function(addr, db){
|
||||
var acc = toHex(addr);
|
||||
if (this.prestate[acc] === undefined) {
|
||||
this.prestate[acc] = {
|
||||
balance: '0x' + db.getBalance(addr).toString(16),
|
||||
nonce: db.getNonce(addr),
|
||||
code: toHex(db.getCode(addr)),
|
||||
storage: {}
|
||||
};
|
||||
}
|
||||
},
|
||||
|
||||
// lookupStorage injects the specified storage entry of the given account into
|
||||
// the prestate object.
|
||||
lookupStorage: function(addr, key, db){
|
||||
var acc = toHex(addr);
|
||||
var idx = toHex(key);
|
||||
|
||||
if (this.prestate[acc].storage[idx] === undefined) {
|
||||
var val = toHex(db.getState(addr, key));
|
||||
if (val != "0x0000000000000000000000000000000000000000000000000000000000000000") {
|
||||
this.prestate[acc].storage[idx] = toHex(db.getState(addr, key));
|
||||
}
|
||||
}
|
||||
},
|
||||
|
||||
// result is invoked when all the opcodes have been iterated over and returns
|
||||
// the final result of the tracing.
|
||||
result: function(ctx, db) {
|
||||
// At this point, we need to deduct the 'value' from the
|
||||
// outer transaction, and move it back to the origin
|
||||
this.lookupAccount(ctx.from, db);
|
||||
|
||||
var fromBal = bigInt(this.prestate[toHex(ctx.from)].balance.slice(2), 16);
|
||||
var toBal = bigInt(this.prestate[toHex(ctx.to)].balance.slice(2), 16);
|
||||
|
||||
this.prestate[toHex(ctx.to)].balance = '0x'+toBal.subtract(ctx.value).toString(16);
|
||||
this.prestate[toHex(ctx.from)].balance = '0x'+fromBal.add(ctx.value).toString(16);
|
||||
|
||||
// Decrement the caller's nonce, and remove empty create targets
|
||||
this.prestate[toHex(ctx.from)].nonce--;
|
||||
if (ctx.type == 'CREATE') {
|
||||
// We can blibdly delete the contract prestate, as any existing state would
|
||||
// have caused the transaction to be rejected as invalid in the first place.
|
||||
delete this.prestate[toHex(ctx.to)];
|
||||
}
|
||||
// Return the assembled allocations (prestate)
|
||||
return this.prestate;
|
||||
},
|
||||
|
||||
// step is invoked for every opcode that the VM executes.
|
||||
step: function(log, db) {
|
||||
// Add the current account if we just started tracing
|
||||
if (this.prestate === null){
|
||||
this.prestate = {};
|
||||
// Balance will potentially be wrong here, since this will include the value
|
||||
// sent along with the message. We fix that in 'result()'.
|
||||
this.lookupAccount(log.contract.getAddress(), db);
|
||||
}
|
||||
// Whenever new state is accessed, add it to the prestate
|
||||
switch (log.op.toString()) {
|
||||
case "EXTCODECOPY": case "EXTCODESIZE": case "BALANCE":
|
||||
this.lookupAccount(toAddress(log.stack.peek(0).toString(16)), db);
|
||||
break;
|
||||
case "CREATE":
|
||||
var from = log.contract.getAddress();
|
||||
this.lookupAccount(toContract(from, db.getNonce(from)), db);
|
||||
break;
|
||||
case "CALL": case "CALLCODE": case "DELEGATECALL": case "STATICCALL":
|
||||
this.lookupAccount(toAddress(log.stack.peek(1).toString(16)), db);
|
||||
break;
|
||||
case 'SSTORE':case 'SLOAD':
|
||||
this.lookupStorage(log.contract.getAddress(), toWord(log.stack.peek(0).toString(16)), db);
|
||||
break;
|
||||
}
|
||||
},
|
||||
|
||||
// fault is invoked when the actual execution of an opcode fails.
|
||||
fault: function(log, db) {}
|
||||
}
|
21
eth/tracers/internal/tracers/tracers.go
Normal file
21
eth/tracers/internal/tracers/tracers.go
Normal file
@ -0,0 +1,21 @@
|
||||
// Copyright 2017 The go-ethereum Authors
|
||||
// This file is part of the go-ethereum library.
|
||||
//
|
||||
// The go-ethereum library is free software: you can redistribute it and/or modify
|
||||
// it under the terms of the GNU Lesser General Public License as published by
|
||||
// the Free Software Foundation, either version 3 of the License, or
|
||||
// (at your option) any later version.
|
||||
//
|
||||
// The go-ethereum library is distributed in the hope that it will be useful,
|
||||
// but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||
// GNU Lesser General Public License for more details.
|
||||
//
|
||||
// You should have received a copy of the GNU Lesser General Public License
|
||||
// along with the go-ethereum library. If not, see <http://www.gnu.org/licenses/>.
|
||||
|
||||
//go:generate go-bindata -nometadata -o assets.go -pkg tracers -ignore ((tracers)|(assets)).go ./...
|
||||
//go:generate gofmt -s -w assets.go
|
||||
|
||||
// Package tracers contains the actual JavaScript tracer assets.
|
||||
package tracers
|
58
eth/tracers/testdata/call_tracer_create.json
vendored
Normal file
58
eth/tracers/testdata/call_tracer_create.json
vendored
Normal file
@ -0,0 +1,58 @@
|
||||
{
|
||||
"context": {
|
||||
"difficulty": "3755480783",
|
||||
"gasLimit": "5401723",
|
||||
"miner": "0xd049bfd667cb46aa3ef5df0da3e57db3be39e511",
|
||||
"number": "2294702",
|
||||
"timestamp": "1513676146"
|
||||
},
|
||||
"genesis": {
|
||||
"alloc": {
|
||||
"0x13e4acefe6a6700604929946e70e6443e4e73447": {
|
||||
"balance": "0xcf3e0938579f000",
|
||||
"code": "0x",
|
||||
"nonce": "9",
|
||||
"storage": {}
|
||||
},
|
||||
"0x7dc9c9730689ff0b0fd506c67db815f12d90a448": {
|
||||
"balance": "0x0",
|
||||
"code": "0x",
|
||||
"nonce": "0",
|
||||
"storage": {}
|
||||
}
|
||||
},
|
||||
"config": {
|
||||
"byzantiumBlock": 1700000,
|
||||
"chainId": 3,
|
||||
"daoForkSupport": true,
|
||||
"eip150Block": 0,
|
||||
"eip150Hash": "0x41941023680923e0fe4d74a34bdac8141f2540e3ae90623718e47d66d1ca4a2d",
|
||||
"eip155Block": 10,
|
||||
"eip158Block": 10,
|
||||
"ethash": {},
|
||||
"homesteadBlock": 0
|
||||
},
|
||||
"difficulty": "3757315409",
|
||||
"extraData": "0x566961425443",
|
||||
"gasLimit": "5406414",
|
||||
"hash": "0xae107f592eebdd9ff8d6ba00363676096e6afb0e1007a7d3d0af88173077378d",
|
||||
"miner": "0xd049bfd667cb46aa3ef5df0da3e57db3be39e511",
|
||||
"mixHash": "0xc927aa05a38bc3de864e95c33b3ae559d3f39c4ccd51cef6f113f9c50ba0caf1",
|
||||
"nonce": "0x93363bbd2c95f410",
|
||||
"number": "2294701",
|
||||
"stateRoot": "0x6b6737d5bde8058990483e915866bd1578014baeff57bd5e4ed228a2bfad635c",
|
||||
"timestamp": "1513676127",
|
||||
"totalDifficulty": "7160808139332585"
|
||||
},
|
||||
"input": "0xf907ef098504e3b29200830897be8080b9079c606060405260405160208061077c83398101604052808051906020019091905050600160009054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff163373ffffffffffffffffffffffffffffffffffffffff161415151561007d57600080fd5b336000806101000a81548173ffffffffffffffffffffffffffffffffffffffff021916908373ffffffffffffffffffffffffffffffffffffffff16021790555080600160006101000a81548173ffffffffffffffffffffffffffffffffffffffff021916908373ffffffffffffffffffffffffffffffffffffffff1602179055506001600460006101000a81548160ff02191690831515021790555050610653806101296000396000f300606060405260043610610083576000357c0100000000000000000000000000000000000000000000000000000000900463ffffffff16806305e4382a146100855780631c02708d146100ae5780632e1a7d4d146100c35780635114cb52146100e6578063a37dda2c146100fe578063ae200e7914610153578063b5769f70146101a8575b005b341561009057600080fd5b6100986101d1565b6040518082815260200191505060405180910390f35b34156100b957600080fd5b6100c16101d7565b005b34156100ce57600080fd5b6100e460048080359060200190919050506102eb565b005b6100fc6004808035906020019091905050610513565b005b341561010957600080fd5b6101116105d6565b604051808273ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff16815260200191505060405180910390f35b341561015e57600080fd5b6101666105fc565b604051808273ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff16815260200191505060405180910390f35b34156101b357600080fd5b6101bb610621565b6040518082815260200191505060405180910390f35b60025481565b60011515600460009054906101000a900460ff1615151415156101f957600080fd5b6000809054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff163373ffffffffffffffffffffffffffffffffffffffff1614806102a15750600160009054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff163373ffffffffffffffffffffffffffffffffffffffff16145b15156102ac57600080fd5b6000600460006101000a81548160ff0219169083151502179055506003543073ffffffffffffffffffffffffffffffffffffffff163103600281905550565b6000809054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff163373ffffffffffffffffffffffffffffffffffffffff1614806103935750600160009054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff163373ffffffffffffffffffffffffffffffffffffffff16145b151561039e57600080fd5b6000809054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff163373ffffffffffffffffffffffffffffffffffffffff16141561048357600060025411801561040757506002548111155b151561041257600080fd5b80600254036002819055506000809054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff166108fc829081150290604051600060405180830381858888f19350505050151561047e57600080fd5b610510565b600060035411801561049757506003548111155b15156104a257600080fd5b8060035403600381905550600160009054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff166108fc829081150290604051600060405180830381858888f19350505050151561050f57600080fd5b5b50565b60011515600460009054906101000a900460ff16151514151561053557600080fd5b6000809054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff163373ffffffffffffffffffffffffffffffffffffffff1614801561059657506003548160035401115b80156105bd575080600354013073ffffffffffffffffffffffffffffffffffffffff163110155b15156105c857600080fd5b806003540160038190555050565b600160009054906101000a900473ffffffffffffffffffffffffffffffffffffffff1681565b6000809054906101000a900473ffffffffffffffffffffffffffffffffffffffff1681565b600354815600a165627a7a72305820c3b849e8440987ce43eae3097b77672a69234d516351368b03fe5b7de03807910029000000000000000000000000c65e620a3a55451316168d57e268f5702ef56a1129a01060f46676a5dff6f407f0f51eb6f37f5c8c54e238c70221e18e65fc29d3ea65a0557b01c50ff4ffaac8ed6e5d31237a4ecbac843ab1bfe8bb0165a0060df7c54f",
|
||||
"result": {
|
||||
"from": "0x13e4acefe6a6700604929946e70e6443e4e73447",
|
||||
"gas": "0x5e106",
|
||||
"gasUsed": "0x5e106",
|
||||
"input": "0x606060405260405160208061077c83398101604052808051906020019091905050600160009054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff163373ffffffffffffffffffffffffffffffffffffffff161415151561007d57600080fd5b336000806101000a81548173ffffffffffffffffffffffffffffffffffffffff021916908373ffffffffffffffffffffffffffffffffffffffff16021790555080600160006101000a81548173ffffffffffffffffffffffffffffffffffffffff021916908373ffffffffffffffffffffffffffffffffffffffff1602179055506001600460006101000a81548160ff02191690831515021790555050610653806101296000396000f300606060405260043610610083576000357c0100000000000000000000000000000000000000000000000000000000900463ffffffff16806305e4382a146100855780631c02708d146100ae5780632e1a7d4d146100c35780635114cb52146100e6578063a37dda2c146100fe578063ae200e7914610153578063b5769f70146101a8575b005b341561009057600080fd5b6100986101d1565b6040518082815260200191505060405180910390f35b34156100b957600080fd5b6100c16101d7565b005b34156100ce57600080fd5b6100e460048080359060200190919050506102eb565b005b6100fc6004808035906020019091905050610513565b005b341561010957600080fd5b6101116105d6565b604051808273ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff16815260200191505060405180910390f35b341561015e57600080fd5b6101666105fc565b604051808273ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff16815260200191505060405180910390f35b34156101b357600080fd5b6101bb610621565b6040518082815260200191505060405180910390f35b60025481565b60011515600460009054906101000a900460ff1615151415156101f957600080fd5b6000809054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff163373ffffffffffffffffffffffffffffffffffffffff1614806102a15750600160009054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff163373ffffffffffffffffffffffffffffffffffffffff16145b15156102ac57600080fd5b6000600460006101000a81548160ff0219169083151502179055506003543073ffffffffffffffffffffffffffffffffffffffff163103600281905550565b6000809054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff163373ffffffffffffffffffffffffffffffffffffffff1614806103935750600160009054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff163373ffffffffffffffffffffffffffffffffffffffff16145b151561039e57600080fd5b6000809054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff163373ffffffffffffffffffffffffffffffffffffffff16141561048357600060025411801561040757506002548111155b151561041257600080fd5b80600254036002819055506000809054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff166108fc829081150290604051600060405180830381858888f19350505050151561047e57600080fd5b610510565b600060035411801561049757506003548111155b15156104a257600080fd5b8060035403600381905550600160009054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff166108fc829081150290604051600060405180830381858888f19350505050151561050f57600080fd5b5b50565b60011515600460009054906101000a900460ff16151514151561053557600080fd5b6000809054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff163373ffffffffffffffffffffffffffffffffffffffff1614801561059657506003548160035401115b80156105bd575080600354013073ffffffffffffffffffffffffffffffffffffffff163110155b15156105c857600080fd5b806003540160038190555050565b600160009054906101000a900473ffffffffffffffffffffffffffffffffffffffff1681565b6000809054906101000a900473ffffffffffffffffffffffffffffffffffffffff1681565b600354815600a165627a7a72305820c3b849e8440987ce43eae3097b77672a69234d516351368b03fe5b7de03807910029000000000000000000000000c65e620a3a55451316168d57e268f5702ef56a11",
|
||||
"output": "0x606060405260043610610083576000357c0100000000000000000000000000000000000000000000000000000000900463ffffffff16806305e4382a146100855780631c02708d146100ae5780632e1a7d4d146100c35780635114cb52146100e6578063a37dda2c146100fe578063ae200e7914610153578063b5769f70146101a8575b005b341561009057600080fd5b6100986101d1565b6040518082815260200191505060405180910390f35b34156100b957600080fd5b6100c16101d7565b005b34156100ce57600080fd5b6100e460048080359060200190919050506102eb565b005b6100fc6004808035906020019091905050610513565b005b341561010957600080fd5b6101116105d6565b604051808273ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff16815260200191505060405180910390f35b341561015e57600080fd5b6101666105fc565b604051808273ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff16815260200191505060405180910390f35b34156101b357600080fd5b6101bb610621565b6040518082815260200191505060405180910390f35b60025481565b60011515600460009054906101000a900460ff1615151415156101f957600080fd5b6000809054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff163373ffffffffffffffffffffffffffffffffffffffff1614806102a15750600160009054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff163373ffffffffffffffffffffffffffffffffffffffff16145b15156102ac57600080fd5b6000600460006101000a81548160ff0219169083151502179055506003543073ffffffffffffffffffffffffffffffffffffffff163103600281905550565b6000809054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff163373ffffffffffffffffffffffffffffffffffffffff1614806103935750600160009054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff163373ffffffffffffffffffffffffffffffffffffffff16145b151561039e57600080fd5b6000809054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff163373ffffffffffffffffffffffffffffffffffffffff16141561048357600060025411801561040757506002548111155b151561041257600080fd5b80600254036002819055506000809054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff166108fc829081150290604051600060405180830381858888f19350505050151561047e57600080fd5b610510565b600060035411801561049757506003548111155b15156104a257600080fd5b8060035403600381905550600160009054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff166108fc829081150290604051600060405180830381858888f19350505050151561050f57600080fd5b5b50565b60011515600460009054906101000a900460ff16151514151561053557600080fd5b6000809054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff163373ffffffffffffffffffffffffffffffffffffffff1614801561059657506003548160035401115b80156105bd575080600354013073ffffffffffffffffffffffffffffffffffffffff163110155b15156105c857600080fd5b806003540160038190555050565b600160009054906101000a900473ffffffffffffffffffffffffffffffffffffffff1681565b6000809054906101000a900473ffffffffffffffffffffffffffffffffffffffff1681565b600354815600a165627a7a72305820c3b849e8440987ce43eae3097b77672a69234d516351368b03fe5b7de03807910029",
|
||||
"to": "0x7dc9c9730689ff0b0fd506c67db815f12d90a448",
|
||||
"type": "CREATE",
|
||||
"value": "0x0"
|
||||
}
|
||||
}
|
415
eth/tracers/testdata/call_tracer_deep_calls.json
vendored
Normal file
415
eth/tracers/testdata/call_tracer_deep_calls.json
vendored
Normal file
File diff suppressed because one or more lines are too long
97
eth/tracers/testdata/call_tracer_delegatecall.json
vendored
Normal file
97
eth/tracers/testdata/call_tracer_delegatecall.json
vendored
Normal file
File diff suppressed because one or more lines are too long
77
eth/tracers/testdata/call_tracer_inner_create_oog_outer_throw.json
vendored
Normal file
77
eth/tracers/testdata/call_tracer_inner_create_oog_outer_throw.json
vendored
Normal file
File diff suppressed because one or more lines are too long
81
eth/tracers/testdata/call_tracer_inner_throw_outer_revert.json
vendored
Normal file
81
eth/tracers/testdata/call_tracer_inner_throw_outer_revert.json
vendored
Normal file
File diff suppressed because one or more lines are too long
60
eth/tracers/testdata/call_tracer_oog.json
vendored
Normal file
60
eth/tracers/testdata/call_tracer_oog.json
vendored
Normal file
File diff suppressed because one or more lines are too long
58
eth/tracers/testdata/call_tracer_revert.json
vendored
Normal file
58
eth/tracers/testdata/call_tracer_revert.json
vendored
Normal file
@ -0,0 +1,58 @@
|
||||
{
|
||||
"context": {
|
||||
"difficulty": "3665057456",
|
||||
"gasLimit": "5232723",
|
||||
"miner": "0xf4d8e706cfb25c0decbbdd4d2e2cc10c66376a3f",
|
||||
"number": "2294501",
|
||||
"timestamp": "1513673601"
|
||||
},
|
||||
"genesis": {
|
||||
"alloc": {
|
||||
"0x0f6cef2b7fbb504782e35aa82a2207e816a2b7a9": {
|
||||
"balance": "0x2a3fc32bcc019283",
|
||||
"code": "0x",
|
||||
"nonce": "10",
|
||||
"storage": {}
|
||||
},
|
||||
"0xabbcd5b340c80b5f1c0545c04c987b87310296ae": {
|
||||
"balance": "0x0",
|
||||
"code": "0x606060405236156100755763ffffffff7c01000000000000000000000000000000000000000000000000000000006000350416632d0335ab811461007a578063548db174146100ab5780637f649783146100fc578063b092145e1461014d578063c3f44c0a14610186578063c47cf5de14610203575b600080fd5b341561008557600080fd5b610099600160a060020a0360043516610270565b60405190815260200160405180910390f35b34156100b657600080fd5b6100fa600460248135818101908301358060208181020160405190810160405280939291908181526020018383602002808284375094965061028f95505050505050565b005b341561010757600080fd5b6100fa600460248135818101908301358060208181020160405190810160405280939291908181526020018383602002808284375094965061029e95505050505050565b005b341561015857600080fd5b610172600160a060020a03600435811690602435166102ad565b604051901515815260200160405180910390f35b341561019157600080fd5b6100fa6004803560ff1690602480359160443591606435600160a060020a0316919060a49060843590810190830135806020601f8201819004810201604051908101604052818152929190602084018383808284375094965050509235600160a060020a031692506102cd915050565b005b341561020e57600080fd5b61025460046024813581810190830135806020601f8201819004810201604051908101604052818152929190602084018383808284375094965061056a95505050505050565b604051600160a060020a03909116815260200160405180910390f35b600160a060020a0381166000908152602081905260409020545b919050565b61029a816000610594565b5b50565b61029a816001610594565b5b50565b600160209081526000928352604080842090915290825290205460ff1681565b60008080600160a060020a038416158061030d5750600160a060020a038085166000908152600160209081526040808320339094168352929052205460ff165b151561031857600080fd5b6103218561056a565b600160a060020a038116600090815260208190526040808220549295507f19000000000000000000000000000000000000000000000000000000000000009230918891908b908b90517fff000000000000000000000000000000000000000000000000000000000000008089168252871660018201526c01000000000000000000000000600160a060020a038088168202600284015286811682026016840152602a8301869052841602604a820152605e810182805190602001908083835b6020831061040057805182525b601f1990920191602091820191016103e0565b6001836020036101000a0380198251168184511617909252505050919091019850604097505050505050505051809103902091506001828a8a8a6040516000815260200160405260006040516020015260405193845260ff90921660208085019190915260408085019290925260608401929092526080909201915160208103908084039060008661646e5a03f1151561049957600080fd5b5050602060405103519050600160a060020a03838116908216146104bc57600080fd5b600160a060020a0380841660009081526020819052604090819020805460010190559087169086905180828051906020019080838360005b8381101561050d5780820151818401525b6020016104f4565b50505050905090810190601f16801561053a5780820380516001836020036101000a031916815260200191505b5091505060006040518083038160008661646e5a03f1915050151561055e57600080fd5b5b505050505050505050565b600060248251101561057e5750600061028a565b600160a060020a0360248301511690505b919050565b60005b825181101561060157600160a060020a033316600090815260016020526040812083918584815181106105c657fe5b90602001906020020151600160a060020a031681526020810191909152604001600020805460ff19169115159190911790555b600101610597565b5b5050505600a165627a7a723058200027e8b695e9d2dea9f3629519022a69f3a1d23055ce86406e686ea54f31ee9c0029",
|
||||
"nonce": "1",
|
||||
"storage": {}
|
||||
}
|
||||
},
|
||||
"config": {
|
||||
"byzantiumBlock": 1700000,
|
||||
"chainId": 3,
|
||||
"daoForkSupport": true,
|
||||
"eip150Block": 0,
|
||||
"eip150Hash": "0x41941023680923e0fe4d74a34bdac8141f2540e3ae90623718e47d66d1ca4a2d",
|
||||
"eip155Block": 10,
|
||||
"eip158Block": 10,
|
||||
"ethash": {},
|
||||
"homesteadBlock": 0
|
||||
},
|
||||
"difficulty": "3672229776",
|
||||
"extraData": "0x4554482e45544846414e532e4f52472d4641313738394444",
|
||||
"gasLimit": "5227619",
|
||||
"hash": "0xa07b3d6c6bf63f5f981016db9f2d1d93033833f2c17e8bf7209e85f1faf08076",
|
||||
"miner": "0xbbf5029fd710d227630c8b7d338051b8e76d50b3",
|
||||
"mixHash": "0x806e151ce2817be922e93e8d5921fa0f0d0fd213d6b2b9a3fa17458e74a163d0",
|
||||
"nonce": "0xbc5d43adc2c30c7d",
|
||||
"number": "2294500",
|
||||
"stateRoot": "0xca645b335888352ef9d8b1ef083e9019648180b259026572e3139717270de97d",
|
||||
"timestamp": "1513673552",
|
||||
"totalDifficulty": "7160066586979149"
|
||||
},
|
||||
"input": "0xf9018b0a8505d21dba00832dc6c094abbcd5b340c80b5f1c0545c04c987b87310296ae80b9012473b40a5c000000000000000000000000400de2e016bda6577407dfc379faba9899bc73ef0000000000000000000000002cc31912b2b0f3075a87b3640923d45a26cef3ee000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000800000000000000000000000000000000000000000000000000000000000000064d79d8e6c7265636f76657279416464726573730000000000000000000000000000000000000000000000000000000000383e3ec32dc0f66d8fe60dbdc2f6815bdf73a988383e3ec32dc0f66d8fe60dbdc2f6815bdf73a988000000000000000000000000000000000000000000000000000000000000000000000000000000001ba0fd659d76a4edbd2a823e324c93f78ad6803b30ff4a9c8bce71ba82798975c70ca06571eecc0b765688ec6c78942c5ee8b585e00988c0141b518287e9be919bc48a",
|
||||
"result": {
|
||||
"error": "execution reverted",
|
||||
"from": "0x0f6cef2b7fbb504782e35aa82a2207e816a2b7a9",
|
||||
"gas": "0x2d55e8",
|
||||
"gasUsed": "0xc3",
|
||||
"input": "0x73b40a5c000000000000000000000000400de2e016bda6577407dfc379faba9899bc73ef0000000000000000000000002cc31912b2b0f3075a87b3640923d45a26cef3ee000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000800000000000000000000000000000000000000000000000000000000000000064d79d8e6c7265636f76657279416464726573730000000000000000000000000000000000000000000000000000000000383e3ec32dc0f66d8fe60dbdc2f6815bdf73a988383e3ec32dc0f66d8fe60dbdc2f6815bdf73a98800000000000000000000000000000000000000000000000000000000000000000000000000000000",
|
||||
"to": "0xabbcd5b340c80b5f1c0545c04c987b87310296ae",
|
||||
"type": "CALL",
|
||||
"value": "0x0"
|
||||
}
|
||||
}
|
78
eth/tracers/testdata/call_tracer_simple.json
vendored
Normal file
78
eth/tracers/testdata/call_tracer_simple.json
vendored
Normal file
@ -0,0 +1,78 @@
|
||||
{
|
||||
"context": {
|
||||
"difficulty": "3502894804",
|
||||
"gasLimit": "4722976",
|
||||
"miner": "0x1585936b53834b021f68cc13eeefdec2efc8e724",
|
||||
"number": "2289806",
|
||||
"timestamp": "1513601314"
|
||||
},
|
||||
"genesis": {
|
||||
"alloc": {
|
||||
"0x0024f658a46fbb89d8ac105e98d7ac7cbbaf27c5": {
|
||||
"balance": "0x0",
|
||||
"code": "0x",
|
||||
"nonce": "22",
|
||||
"storage": {}
|
||||
},
|
||||
"0x3b873a919aa0512d5a0f09e6dcceaa4a6727fafe": {
|
||||
"balance": "0x4d87094125a369d9bd5",
|
||||
"code": "0x606060405236156100935763ffffffff60e060020a60003504166311ee8382811461009c57806313af4035146100be5780631f5e8f4c146100ee57806324daddc5146101125780634921a91a1461013b57806363e4bff414610157578063764978f91461017f578063893d20e8146101a1578063ba40aaa1146101cd578063cebc9a82146101f4578063e177246e14610216575b61009a5b5b565b005b34156100a457fe5b6100ac61023d565b60408051918252519081900360200190f35b34156100c657fe5b6100da600160a060020a0360043516610244565b604080519115158252519081900360200190f35b34156100f657fe5b6100da610307565b604080519115158252519081900360200190f35b341561011a57fe5b6100da6004351515610318565b604080519115158252519081900360200190f35b6100da6103d6565b604080519115158252519081900360200190f35b6100da600160a060020a0360043516610420565b604080519115158252519081900360200190f35b341561018757fe5b6100ac61046c565b60408051918252519081900360200190f35b34156101a957fe5b6101b1610473565b60408051600160a060020a039092168252519081900360200190f35b34156101d557fe5b6100da600435610483565b604080519115158252519081900360200190f35b34156101fc57fe5b6100ac61050d565b60408051918252519081900360200190f35b341561021e57fe5b6100da600435610514565b604080519115158252519081900360200190f35b6003545b90565b60006000610250610473565b600160a060020a031633600160a060020a03161415156102705760006000fd5b600160a060020a03831615156102865760006000fd5b50600054600160a060020a0390811690831681146102fb57604051600160a060020a0380851691908316907ffcf23a92150d56e85e3a3d33b357493246e55783095eb6a733eb8439ffc752c890600090a360008054600160a060020a031916600160a060020a03851617905560019150610300565b600091505b5b50919050565b60005460a060020a900460ff165b90565b60006000610324610473565b600160a060020a031633600160a060020a03161415156103445760006000fd5b5060005460a060020a900460ff16801515831515146102fb576000546040805160a060020a90920460ff1615158252841515602083015280517fe6cd46a119083b86efc6884b970bfa30c1708f53ba57b86716f15b2f4551a9539281900390910190a16000805460a060020a60ff02191660a060020a8515150217905560019150610300565b600091505b5b50919050565b60006103e0610307565b801561040557506103ef610473565b600160a060020a031633600160a060020a031614155b156104105760006000fd5b610419336105a0565b90505b5b90565b600061042a610307565b801561044f5750610439610473565b600160a060020a031633600160a060020a031614155b1561045a5760006000fd5b610463826105a0565b90505b5b919050565b6001545b90565b600054600160a060020a03165b90565b6000600061048f610473565b600160a060020a031633600160a060020a03161415156104af5760006000fd5b506001548281146102fb57604080518281526020810185905281517f79a3746dde45672c9e8ab3644b8bb9c399a103da2dc94b56ba09777330a83509929181900390910190a160018381559150610300565b600091505b5b50919050565b6002545b90565b60006000610520610473565b600160a060020a031633600160a060020a03161415156105405760006000fd5b506002548281146102fb57604080518281526020810185905281517ff6991a728965fedd6e927fdf16bdad42d8995970b4b31b8a2bf88767516e2494929181900390910190a1600283905560019150610300565b600091505b5b50919050565b60006000426105ad61023d565b116102fb576105c46105bd61050d565b4201610652565b6105cc61046c565b604051909150600160a060020a038416908290600081818185876187965a03f1925050501561063d57604080518281529051600160a060020a038516917f9bca65ce52fdef8a470977b51f247a2295123a4807dfa9e502edf0d30722da3b919081900360200190a260019150610300565b6102fb42610652565b5b600091505b50919050565b60038190555b505600a165627a7a72305820f3c973c8b7ed1f62000b6701bd5b708469e19d0f1d73fde378a56c07fd0b19090029",
|
||||
"nonce": "1",
|
||||
"storage": {
|
||||
"0x0000000000000000000000000000000000000000000000000000000000000000": "0x000000000000000000000001b436ba50d378d4bbc8660d312a13df6af6e89dfb",
|
||||
"0x0000000000000000000000000000000000000000000000000000000000000001": "0x00000000000000000000000000000000000000000000000006f05b59d3b20000",
|
||||
"0x0000000000000000000000000000000000000000000000000000000000000002": "0x000000000000000000000000000000000000000000000000000000000000003c",
|
||||
"0x0000000000000000000000000000000000000000000000000000000000000003": "0x000000000000000000000000000000000000000000000000000000005a37b834"
|
||||
}
|
||||
},
|
||||
"0xb436ba50d378d4bbc8660d312a13df6af6e89dfb": {
|
||||
"balance": "0x1780d77678137ac1b775",
|
||||
"code": "0x",
|
||||
"nonce": "29072",
|
||||
"storage": {}
|
||||
}
|
||||
},
|
||||
"config": {
|
||||
"byzantiumBlock": 1700000,
|
||||
"chainId": 3,
|
||||
"daoForkSupport": true,
|
||||
"eip150Block": 0,
|
||||
"eip150Hash": "0x41941023680923e0fe4d74a34bdac8141f2540e3ae90623718e47d66d1ca4a2d",
|
||||
"eip155Block": 10,
|
||||
"eip158Block": 10,
|
||||
"ethash": {},
|
||||
"homesteadBlock": 0
|
||||
},
|
||||
"difficulty": "3509749784",
|
||||
"extraData": "0x4554482e45544846414e532e4f52472d4641313738394444",
|
||||
"gasLimit": "4727564",
|
||||
"hash": "0x609948ac3bd3c00b7736b933248891d6c901ee28f066241bddb28f4e00a9f440",
|
||||
"miner": "0xbbf5029fd710d227630c8b7d338051b8e76d50b3",
|
||||
"mixHash": "0xb131e4507c93c7377de00e7c271bf409ec7492767142ff0f45c882f8068c2ada",
|
||||
"nonce": "0x4eb12e19c16d43da",
|
||||
"number": "2289805",
|
||||
"stateRoot": "0xc7f10f352bff82fac3c2999d3085093d12652e19c7fd32591de49dc5d91b4f1f",
|
||||
"timestamp": "1513601261",
|
||||
"totalDifficulty": "7143276353481064"
|
||||
},
|
||||
"input": "0xf88b8271908506fc23ac0083015f90943b873a919aa0512d5a0f09e6dcceaa4a6727fafe80a463e4bff40000000000000000000000000024f658a46fbb89d8ac105e98d7ac7cbbaf27c52aa0bdce0b59e8761854e857fe64015f06dd08a4fbb7624f6094893a79a72e6ad6bea01d9dde033cff7bb235a3163f348a6d7ab8d6b52bc0963a95b91612e40ca766a4",
|
||||
"result": {
|
||||
"calls": [
|
||||
{
|
||||
"from": "0x3b873a919aa0512d5a0f09e6dcceaa4a6727fafe",
|
||||
"input": "0x",
|
||||
"to": "0x0024f658a46fbb89d8ac105e98d7ac7cbbaf27c5",
|
||||
"type": "CALL",
|
||||
"value": "0x6f05b59d3b20000"
|
||||
}
|
||||
],
|
||||
"from": "0xb436ba50d378d4bbc8660d312a13df6af6e89dfb",
|
||||
"gas": "0x10738",
|
||||
"gasUsed": "0x3ef9",
|
||||
"input": "0x63e4bff40000000000000000000000000024f658a46fbb89d8ac105e98d7ac7cbbaf27c5",
|
||||
"output": "0x0000000000000000000000000000000000000000000000000000000000000001",
|
||||
"to": "0x3b873a919aa0512d5a0f09e6dcceaa4a6727fafe",
|
||||
"type": "CALL",
|
||||
"value": "0x0"
|
||||
}
|
||||
}
|
62
eth/tracers/testdata/call_tracer_throw.json
vendored
Normal file
62
eth/tracers/testdata/call_tracer_throw.json
vendored
Normal file
File diff suppressed because one or more lines are too long
618
eth/tracers/tracer.go
Normal file
618
eth/tracers/tracer.go
Normal file
File diff suppressed because one or more lines are too long
140
eth/tracers/tracer_test.go
Normal file
140
eth/tracers/tracer_test.go
Normal file
@ -0,0 +1,140 @@
|
||||
// Copyright 2016 The go-ethereum Authors
|
||||
// This file is part of the go-ethereum library.
|
||||
//
|
||||
// The go-ethereum library is free software: you can redistribute it and/or modify
|
||||
// it under the terms of the GNU Lesser General Public License as published by
|
||||
// the Free Software Foundation, either version 3 of the License, or
|
||||
// (at your option) any later version.
|
||||
//
|
||||
// The go-ethereum library is distributed in the hope that it will be useful,
|
||||
// but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||
// GNU Lesser General Public License for more details.
|
||||
//
|
||||
// You should have received a copy of the GNU Lesser General Public License
|
||||
// along with the go-ethereum library. If not, see <http://www.gnu.org/licenses/>.
|
||||
|
||||
package tracers
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"encoding/json"
|
||||
"errors"
|
||||
"math/big"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/ethereum/go-ethereum/common"
|
||||
"github.com/ethereum/go-ethereum/core/vm"
|
||||
"github.com/ethereum/go-ethereum/params"
|
||||
)
|
||||
|
||||
type account struct{}
|
||||
|
||||
func (account) SubBalance(amount *big.Int) {}
|
||||
func (account) AddBalance(amount *big.Int) {}
|
||||
func (account) SetAddress(common.Address) {}
|
||||
func (account) Value() *big.Int { return nil }
|
||||
func (account) SetBalance(*big.Int) {}
|
||||
func (account) SetNonce(uint64) {}
|
||||
func (account) Balance() *big.Int { return nil }
|
||||
func (account) Address() common.Address { return common.Address{} }
|
||||
func (account) ReturnGas(*big.Int) {}
|
||||
func (account) SetCode(common.Hash, []byte) {}
|
||||
func (account) ForEachStorage(cb func(key, value common.Hash) bool) {}
|
||||
|
||||
func runTrace(tracer *Tracer) (json.RawMessage, error) {
|
||||
env := vm.NewEVM(vm.Context{BlockNumber: big.NewInt(1)}, nil, params.TestChainConfig, vm.Config{Debug: true, Tracer: tracer})
|
||||
|
||||
contract := vm.NewContract(account{}, account{}, big.NewInt(0), 10000)
|
||||
contract.Code = []byte{byte(vm.PUSH1), 0x1, byte(vm.PUSH1), 0x1, 0x0}
|
||||
|
||||
_, err := env.Interpreter().Run(contract, []byte{})
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return tracer.GetResult()
|
||||
}
|
||||
|
||||
func TestTracing(t *testing.T) {
|
||||
tracer, err := New("{count: 0, step: function() { this.count += 1; }, fault: function() {}, result: function() { return this.count; }}")
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
||||
ret, err := runTrace(tracer)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
if !bytes.Equal(ret, []byte("3")) {
|
||||
t.Errorf("Expected return value to be 3, got %s", string(ret))
|
||||
}
|
||||
}
|
||||
|
||||
func TestStack(t *testing.T) {
|
||||
tracer, err := New("{depths: [], step: function(log) { this.depths.push(log.stack.length()); }, fault: function() {}, result: function() { return this.depths; }}")
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
||||
ret, err := runTrace(tracer)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
if !bytes.Equal(ret, []byte("[0,1,2]")) {
|
||||
t.Errorf("Expected return value to be [0,1,2], got %s", string(ret))
|
||||
}
|
||||
}
|
||||
|
||||
func TestOpcodes(t *testing.T) {
|
||||
tracer, err := New("{opcodes: [], step: function(log) { this.opcodes.push(log.op.toString()); }, fault: function() {}, result: function() { return this.opcodes; }}")
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
||||
ret, err := runTrace(tracer)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
if !bytes.Equal(ret, []byte("[\"PUSH1\",\"PUSH1\",\"STOP\"]")) {
|
||||
t.Errorf("Expected return value to be [\"PUSH1\",\"PUSH1\",\"STOP\"], got %s", string(ret))
|
||||
}
|
||||
}
|
||||
|
||||
func TestHalt(t *testing.T) {
|
||||
t.Skip("duktape doesn't support abortion")
|
||||
|
||||
timeout := errors.New("stahp")
|
||||
tracer, err := New("{step: function() { while(1); }, result: function() { return null; }}")
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
||||
go func() {
|
||||
time.Sleep(1 * time.Second)
|
||||
tracer.Stop(timeout)
|
||||
}()
|
||||
|
||||
if _, err = runTrace(tracer); err.Error() != "stahp in server-side tracer function 'step'" {
|
||||
t.Errorf("Expected timeout error, got %v", err)
|
||||
}
|
||||
}
|
||||
|
||||
func TestHaltBetweenSteps(t *testing.T) {
|
||||
tracer, err := New("{step: function() {}, fault: function() {}, result: function() { return null; }}")
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
||||
env := vm.NewEVM(vm.Context{BlockNumber: big.NewInt(1)}, nil, params.TestChainConfig, vm.Config{Debug: true, Tracer: tracer})
|
||||
contract := vm.NewContract(&account{}, &account{}, big.NewInt(0), 0)
|
||||
|
||||
tracer.CaptureState(env, 0, 0, 0, 0, nil, nil, contract, 0, nil)
|
||||
timeout := errors.New("stahp")
|
||||
tracer.Stop(timeout)
|
||||
tracer.CaptureState(env, 0, 0, 0, 0, nil, nil, contract, 0, nil)
|
||||
|
||||
if _, err := tracer.GetResult(); err.Error() != timeout.Error() {
|
||||
t.Errorf("Expected timeout error, got %v", err)
|
||||
}
|
||||
}
|
53
eth/tracers/tracers.go
Normal file
53
eth/tracers/tracers.go
Normal file
@ -0,0 +1,53 @@
|
||||
// Copyright 2017 The go-ethereum Authors
|
||||
// This file is part of the go-ethereum library.
|
||||
//
|
||||
// The go-ethereum library is free software: you can redistribute it and/or modify
|
||||
// it under the terms of the GNU Lesser General Public License as published by
|
||||
// the Free Software Foundation, either version 3 of the License, or
|
||||
// (at your option) any later version.
|
||||
//
|
||||
// The go-ethereum library is distributed in the hope that it will be useful,
|
||||
// but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||
// GNU Lesser General Public License for more details.
|
||||
//
|
||||
// You should have received a copy of the GNU Lesser General Public License
|
||||
// along with the go-ethereum library. If not, see <http://www.gnu.org/licenses/>.
|
||||
|
||||
// Package tracers is a collection of JavaScript transaction tracers.
|
||||
package tracers
|
||||
|
||||
import (
|
||||
"strings"
|
||||
"unicode"
|
||||
|
||||
"github.com/ethereum/go-ethereum/eth/tracers/internal/tracers"
|
||||
)
|
||||
|
||||
// all contains all the built in JavaScript tracers by name.
|
||||
var all = make(map[string]string)
|
||||
|
||||
// camel converts a snake cased input string into a camel cased output.
|
||||
func camel(str string) string {
|
||||
pieces := strings.Split(str, "_")
|
||||
for i := 1; i < len(pieces); i++ {
|
||||
pieces[i] = string(unicode.ToUpper(rune(pieces[i][0]))) + pieces[i][1:]
|
||||
}
|
||||
return strings.Join(pieces, "")
|
||||
}
|
||||
|
||||
// init retrieves the JavaScript transaction tracers included in go-ethereum.
|
||||
func init() {
|
||||
for _, file := range tracers.AssetNames() {
|
||||
name := camel(strings.TrimSuffix(file, ".js"))
|
||||
all[name] = string(tracers.MustAsset(file))
|
||||
}
|
||||
}
|
||||
|
||||
// tracer retrieves a specific JavaScript tracer by name.
|
||||
func tracer(name string) (string, bool) {
|
||||
if tracer, ok := all[name]; ok {
|
||||
return tracer, true
|
||||
}
|
||||
return "", false
|
||||
}
|
194
eth/tracers/tracers_test.go
Normal file
194
eth/tracers/tracers_test.go
Normal file
@ -0,0 +1,194 @@
|
||||
// Copyright 2017 The go-ethereum Authors
|
||||
// This file is part of the go-ethereum library.
|
||||
//
|
||||
// The go-ethereum library is free software: you can redistribute it and/or modify
|
||||
// it under the terms of the GNU Lesser General Public License as published by
|
||||
// the Free Software Foundation, either version 3 of the License, or
|
||||
// (at your option) any later version.
|
||||
//
|
||||
// The go-ethereum library is distributed in the hope that it will be useful,
|
||||
// but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||
// GNU Lesser General Public License for more details.
|
||||
//
|
||||
// You should have received a copy of the GNU Lesser General Public License
|
||||
// along with the go-ethereum library. If not, see <http://www.gnu.org/licenses/>.
|
||||
|
||||
package tracers
|
||||
|
||||
import (
|
||||
"encoding/json"
|
||||
"io/ioutil"
|
||||
"math/big"
|
||||
"path/filepath"
|
||||
"reflect"
|
||||
"strings"
|
||||
"testing"
|
||||
|
||||
"github.com/ethereum/go-ethereum/common"
|
||||
"github.com/ethereum/go-ethereum/common/hexutil"
|
||||
"github.com/ethereum/go-ethereum/common/math"
|
||||
"github.com/ethereum/go-ethereum/core"
|
||||
"github.com/ethereum/go-ethereum/core/types"
|
||||
"github.com/ethereum/go-ethereum/core/vm"
|
||||
"github.com/ethereum/go-ethereum/ethdb"
|
||||
"github.com/ethereum/go-ethereum/rlp"
|
||||
"github.com/ethereum/go-ethereum/tests"
|
||||
)
|
||||
|
||||
// To generate a new callTracer test, copy paste the makeTest method below into
|
||||
// a Geth console and call it with a transaction hash you which to export.
|
||||
|
||||
/*
|
||||
// makeTest generates a callTracer test by running a prestate reassembled and a
|
||||
// call trace run, assembling all the gathered information into a test case.
|
||||
var makeTest = function(tx, rewind) {
|
||||
// Generate the genesis block from the block, transaction and prestate data
|
||||
var block = eth.getBlock(eth.getTransaction(tx).blockHash);
|
||||
var genesis = eth.getBlock(block.parentHash);
|
||||
|
||||
delete genesis.gasUsed;
|
||||
delete genesis.logsBloom;
|
||||
delete genesis.parentHash;
|
||||
delete genesis.receiptsRoot;
|
||||
delete genesis.sha3Uncles;
|
||||
delete genesis.size;
|
||||
delete genesis.transactions;
|
||||
delete genesis.transactionsRoot;
|
||||
delete genesis.uncles;
|
||||
|
||||
genesis.gasLimit = genesis.gasLimit.toString();
|
||||
genesis.number = genesis.number.toString();
|
||||
genesis.timestamp = genesis.timestamp.toString();
|
||||
|
||||
genesis.alloc = debug.traceTransaction(tx, {tracer: "prestateTracer", rewind: rewind});
|
||||
for (var key in genesis.alloc) {
|
||||
genesis.alloc[key].nonce = genesis.alloc[key].nonce.toString();
|
||||
}
|
||||
genesis.config = admin.nodeInfo.protocols.eth.config;
|
||||
|
||||
// Generate the call trace and produce the test input
|
||||
var result = debug.traceTransaction(tx, {tracer: "callTracer", rewind: rewind});
|
||||
delete result.time;
|
||||
|
||||
console.log(JSON.stringify({
|
||||
genesis: genesis,
|
||||
context: {
|
||||
number: block.number.toString(),
|
||||
difficulty: block.difficulty,
|
||||
timestamp: block.timestamp.toString(),
|
||||
gasLimit: block.gasLimit.toString(),
|
||||
miner: block.miner,
|
||||
},
|
||||
input: eth.getRawTransaction(tx),
|
||||
result: result,
|
||||
}, null, 2));
|
||||
}
|
||||
*/
|
||||
|
||||
// callTrace is the result of a callTracer run.
|
||||
type callTrace struct {
|
||||
Type string `json:"type"`
|
||||
From common.Address `json:"from"`
|
||||
To common.Address `json:"to"`
|
||||
Input hexutil.Bytes `json:"input"`
|
||||
Output hexutil.Bytes `json:"output"`
|
||||
Gas *hexutil.Uint64 `json:"gas,omitempty"`
|
||||
GasUsed *hexutil.Uint64 `json:"gasUsed,omitempty"`
|
||||
Value *hexutil.Big `json:"value,omitempty"`
|
||||
Error string `json:"error,omitempty"`
|
||||
Calls []callTrace `json:"calls,omitempty"`
|
||||
}
|
||||
|
||||
type callContext struct {
|
||||
Number math.HexOrDecimal64 `json:"number"`
|
||||
Difficulty *math.HexOrDecimal256 `json:"difficulty"`
|
||||
Time math.HexOrDecimal64 `json:"timestamp"`
|
||||
GasLimit math.HexOrDecimal64 `json:"gasLimit"`
|
||||
Miner common.Address `json:"miner"`
|
||||
}
|
||||
|
||||
// callTracerTest defines a single test to check the call tracer against.
|
||||
type callTracerTest struct {
|
||||
Genesis *core.Genesis `json:"genesis"`
|
||||
Context *callContext `json:"context"`
|
||||
Input string `json:"input"`
|
||||
Result *callTrace `json:"result"`
|
||||
}
|
||||
|
||||
// Iterates over all the input-output datasets in the tracer test harness and
|
||||
// runs the JavaScript tracers against them.
|
||||
func TestCallTracer(t *testing.T) {
|
||||
files, err := ioutil.ReadDir("testdata")
|
||||
if err != nil {
|
||||
t.Fatalf("failed to retrieve tracer test suite: %v", err)
|
||||
}
|
||||
for _, file := range files {
|
||||
if !strings.HasPrefix(file.Name(), "call_tracer_") {
|
||||
continue
|
||||
}
|
||||
file := file // capture range variable
|
||||
t.Run(camel(strings.TrimSuffix(strings.TrimPrefix(file.Name(), "call_tracer_"), ".json")), func(t *testing.T) {
|
||||
t.Parallel()
|
||||
|
||||
// Call tracer test found, read if from disk
|
||||
blob, err := ioutil.ReadFile(filepath.Join("testdata", file.Name()))
|
||||
if err != nil {
|
||||
t.Fatalf("failed to read testcase: %v", err)
|
||||
}
|
||||
test := new(callTracerTest)
|
||||
if err := json.Unmarshal(blob, test); err != nil {
|
||||
t.Fatalf("failed to parse testcase: %v", err)
|
||||
}
|
||||
// Configure a blockchain with the given prestate
|
||||
tx := new(types.Transaction)
|
||||
if err := rlp.DecodeBytes(common.FromHex(test.Input), tx); err != nil {
|
||||
t.Fatalf("failed to parse testcase input: %v", err)
|
||||
}
|
||||
signer := types.MakeSigner(test.Genesis.Config, new(big.Int).SetUint64(uint64(test.Context.Number)))
|
||||
origin, _ := signer.Sender(tx)
|
||||
|
||||
context := vm.Context{
|
||||
CanTransfer: core.CanTransfer,
|
||||
Transfer: core.Transfer,
|
||||
Origin: origin,
|
||||
Coinbase: test.Context.Miner,
|
||||
BlockNumber: new(big.Int).SetUint64(uint64(test.Context.Number)),
|
||||
Time: new(big.Int).SetUint64(uint64(test.Context.Time)),
|
||||
Difficulty: (*big.Int)(test.Context.Difficulty),
|
||||
GasLimit: new(big.Int).SetUint64(uint64(test.Context.GasLimit)),
|
||||
GasPrice: tx.GasPrice(),
|
||||
}
|
||||
db, _ := ethdb.NewMemDatabase()
|
||||
statedb := tests.MakePreState(db, test.Genesis.Alloc)
|
||||
|
||||
// Create the tracer, the EVM environment and run it
|
||||
tracer, err := New("callTracer")
|
||||
if err != nil {
|
||||
t.Fatalf("failed to create call tracer: %v", err)
|
||||
}
|
||||
evm := vm.NewEVM(context, statedb, test.Genesis.Config, vm.Config{Debug: true, Tracer: tracer})
|
||||
|
||||
msg, err := tx.AsMessage(signer)
|
||||
if err != nil {
|
||||
t.Fatalf("failed to prepare transaction for tracing: %v", err)
|
||||
}
|
||||
st := core.NewStateTransition(evm, msg, new(core.GasPool).AddGas(tx.Gas()))
|
||||
if _, _, _, _, err = st.TransitionDb(); err != nil {
|
||||
t.Fatalf("failed to execute transaction: %v", err)
|
||||
}
|
||||
// Retrieve the trace result and compare against the etalon
|
||||
res, err := tracer.GetResult()
|
||||
if err != nil {
|
||||
t.Fatalf("failed to retrieve trace result: %v", err)
|
||||
}
|
||||
ret := new(callTrace)
|
||||
if err := json.Unmarshal(res, ret); err != nil {
|
||||
t.Fatalf("failed to unmarshal trace result: %v", err)
|
||||
}
|
||||
if !reflect.DeepEqual(ret, test.Result) {
|
||||
t.Fatalf("trace mismatch: have %+v, want %+v", ret, test.Result)
|
||||
}
|
||||
})
|
||||
}
|
||||
}
|
Reference in New Issue
Block a user