From e3d61e6db028c412f74bc4d4c7e117a9e29d0de0 Mon Sep 17 00:00:00 2001 From: rjl493456442 Date: Wed, 20 Nov 2024 19:35:52 +0800 Subject: [PATCH] core, eth, internal, cmd: rework EVM constructor (#30745) This pull request refactors the EVM constructor by removing the TxContext parameter. The EVM object is frequently overused. Ideally, only a single EVM instance should be created and reused throughout the entire state transition of a block, with the transaction context switched as needed by calling evm.SetTxContext. Unfortunately, in some parts of the code, the EVM object is repeatedly created, resulting in unnecessary complexity. This pull request is the first step towards gradually improving and simplifying this setup. --------- Co-authored-by: Martin Holst Swende --- cmd/evm/internal/t8ntool/execution.go | 18 +-- core/chain_makers.go | 20 +-- core/state_prefetcher.go | 8 +- core/state_processor.go | 35 ++--- core/verkle_witness_test.go | 2 +- core/vm/evm.go | 21 +-- core/vm/gas_table_test.go | 10 +- core/vm/instructions_test.go | 128 ++++++++---------- core/vm/interpreter_test.go | 2 +- core/vm/runtime/env.go | 4 +- eth/api_backend.go | 5 +- eth/gasestimator/gasestimator.go | 3 +- eth/state_accessor.go | 24 ++-- eth/tracers/api.go | 57 ++++---- eth/tracers/api_test.go | 13 +- .../internal/tracetest/calltrace_test.go | 19 ++- .../internal/tracetest/flat_calltrace_test.go | 3 +- .../internal/tracetest/prestate_test.go | 3 +- eth/tracers/js/tracer_test.go | 17 ++- eth/tracers/logger/logger_test.go | 6 +- eth/tracers/tracers_test.go | 3 +- internal/ethapi/api.go | 6 +- internal/ethapi/api_test.go | 5 +- internal/ethapi/backend.go | 2 +- internal/ethapi/simulate.go | 4 +- internal/ethapi/transaction_args_test.go | 2 +- miner/worker.go | 19 +-- tests/state_test.go | 3 +- tests/state_test_util.go | 3 +- 29 files changed, 220 insertions(+), 225 deletions(-) diff --git a/cmd/evm/internal/t8ntool/execution.go b/cmd/evm/internal/t8ntool/execution.go index f80dd02c67..a868fbf11c 100644 --- a/cmd/evm/internal/t8ntool/execution.go +++ b/cmd/evm/internal/t8ntool/execution.go @@ -201,15 +201,14 @@ func (pre *Prestate) Apply(vmConfig vm.Config, chainConfig *params.ChainConfig, chainConfig.DAOForkBlock.Cmp(new(big.Int).SetUint64(pre.Env.Number)) == 0 { misc.ApplyDAOHardFork(statedb) } + evm := vm.NewEVM(vmContext, statedb, chainConfig, vmConfig) if beaconRoot := pre.Env.ParentBeaconBlockRoot; beaconRoot != nil { - evm := vm.NewEVM(vmContext, vm.TxContext{}, statedb, chainConfig, vmConfig) core.ProcessBeaconBlockRoot(*beaconRoot, evm, statedb) } if pre.Env.BlockHashes != nil && chainConfig.IsPrague(new(big.Int).SetUint64(pre.Env.Number), pre.Env.Timestamp) { var ( prevNumber = pre.Env.Number - 1 prevHash = pre.Env.BlockHashes[math.HexOrDecimal64(prevNumber)] - evm = vm.NewEVM(vmContext, vm.TxContext{}, statedb, chainConfig, vmConfig) ) core.ProcessParentBlockHash(prevHash, evm, statedb) } @@ -246,8 +245,10 @@ func (pre *Prestate) Apply(vmConfig vm.Config, chainConfig *params.ChainConfig, if err != nil { return nil, nil, nil, err } + // TODO (rjl493456442) it's a bit weird to reset the tracer in the + // middle of block execution, please improve it somehow. if tracer != nil { - vmConfig.Tracer = tracer.Hooks + evm.SetTracer(tracer.Hooks) } statedb.SetTxContext(tx.Hash(), txIndex) @@ -256,12 +257,12 @@ func (pre *Prestate) Apply(vmConfig vm.Config, chainConfig *params.ChainConfig, snapshot = statedb.Snapshot() prevGas = gaspool.Gas() ) - evm := vm.NewEVM(vmContext, txContext, statedb, chainConfig, vmConfig) - if tracer != nil && tracer.OnTxStart != nil { tracer.OnTxStart(evm.GetVMContext(), tx, msg.From) } // (ret []byte, usedGas uint64, failed bool, err error) + + evm.SetTxContext(txContext) msgResult, err := core.ApplyMessage(evm, msg, gaspool) if err != nil { statedb.RevertToSnapshot(snapshot) @@ -375,12 +376,11 @@ func (pre *Prestate) Apply(vmConfig vm.Config, chainConfig *params.ChainConfig, return nil, nil, nil, NewError(ErrorEVM, fmt.Errorf("could not parse requests logs: %v", err)) } requests = append(requests, depositRequests) - // create EVM for system calls - vmenv := vm.NewEVM(vmContext, vm.TxContext{}, statedb, chainConfig, vm.Config{}) + // EIP-7002 withdrawals - requests = append(requests, core.ProcessWithdrawalQueue(vmenv, statedb)) + requests = append(requests, core.ProcessWithdrawalQueue(evm, statedb)) // EIP-7251 consolidations - requests = append(requests, core.ProcessConsolidationQueue(vmenv, statedb)) + requests = append(requests, core.ProcessConsolidationQueue(evm, statedb)) } // Commit block diff --git a/core/chain_makers.go b/core/chain_makers.go index 586979e772..07448abc3d 100644 --- a/core/chain_makers.go +++ b/core/chain_makers.go @@ -100,9 +100,9 @@ func (b *BlockGen) SetParentBeaconRoot(root common.Hash) { b.header.ParentBeaconRoot = &root var ( blockContext = NewEVMBlockContext(b.header, b.cm, &b.header.Coinbase) - vmenv = vm.NewEVM(blockContext, vm.TxContext{}, b.statedb, b.cm.config, vm.Config{}) + evm = vm.NewEVM(blockContext, b.statedb, b.cm.config, vm.Config{}) ) - ProcessBeaconBlockRoot(root, vmenv, b.statedb) + ProcessBeaconBlockRoot(root, evm, b.statedb) } // addTx adds a transaction to the generated block. If no coinbase has @@ -116,8 +116,12 @@ func (b *BlockGen) addTx(bc *BlockChain, vmConfig vm.Config, tx *types.Transacti if b.gasPool == nil { b.SetCoinbase(common.Address{}) } + var ( + blockContext = NewEVMBlockContext(b.header, bc, &b.header.Coinbase) + evm = vm.NewEVM(blockContext, b.statedb, b.cm.config, vmConfig) + ) b.statedb.SetTxContext(tx.Hash(), len(b.txs)) - receipt, err := ApplyTransaction(b.cm.config, bc, &b.header.Coinbase, b.gasPool, b.statedb, b.header, tx, &b.header.GasUsed, vmConfig) + receipt, err := ApplyTransaction(b.cm.config, evm, b.gasPool, b.statedb, b.header, tx, &b.header.GasUsed) if err != nil { panic(err) } @@ -360,12 +364,12 @@ func GenerateChain(config *params.ChainConfig, parent *types.Block, engine conse requests = append(requests, depositRequests) // create EVM for system calls blockContext := NewEVMBlockContext(b.header, cm, &b.header.Coinbase) - vmenv := vm.NewEVM(blockContext, vm.TxContext{}, statedb, cm.config, vm.Config{}) + evm := vm.NewEVM(blockContext, statedb, cm.config, vm.Config{}) // EIP-7002 withdrawals - withdrawalRequests := ProcessWithdrawalQueue(vmenv, statedb) + withdrawalRequests := ProcessWithdrawalQueue(evm, statedb) requests = append(requests, withdrawalRequests) // EIP-7251 consolidations - consolidationRequests := ProcessConsolidationQueue(vmenv, statedb) + consolidationRequests := ProcessConsolidationQueue(evm, statedb) requests = append(requests, consolidationRequests) } if requests != nil { @@ -466,8 +470,8 @@ func GenerateVerkleChain(config *params.ChainConfig, parent *types.Block, engine if config.IsPrague(b.header.Number, b.header.Time) { // EIP-2935 blockContext := NewEVMBlockContext(b.header, cm, &b.header.Coinbase) - vmenv := vm.NewEVM(blockContext, vm.TxContext{}, statedb, cm.config, vm.Config{}) - ProcessParentBlockHash(b.header.ParentHash, vmenv, statedb) + evm := vm.NewEVM(blockContext, statedb, cm.config, vm.Config{}) + ProcessParentBlockHash(b.header.ParentHash, evm, statedb) } // Execute any user modifications to the block. diff --git a/core/state_prefetcher.go b/core/state_prefetcher.go index 31405fa078..3b987bb289 100644 --- a/core/state_prefetcher.go +++ b/core/state_prefetcher.go @@ -49,7 +49,7 @@ func (p *statePrefetcher) Prefetch(block *types.Block, statedb *state.StateDB, c header = block.Header() gaspool = new(GasPool).AddGas(block.GasLimit()) blockContext = NewEVMBlockContext(header, p.chain, nil) - evm = vm.NewEVM(blockContext, vm.TxContext{}, statedb, p.config, cfg) + evm = vm.NewEVM(blockContext, statedb, p.config, cfg) signer = types.MakeSigner(p.config, header.Number, header.Time) ) // Iterate over and process the individual transactions @@ -65,7 +65,7 @@ func (p *statePrefetcher) Prefetch(block *types.Block, statedb *state.StateDB, c return // Also invalid block, bail out } statedb.SetTxContext(tx.Hash(), i) - if err := precacheTransaction(msg, p.config, gaspool, statedb, header, evm); err != nil { + if err := precacheTransaction(msg, gaspool, evm); err != nil { return // Ugh, something went horribly wrong, bail out } // If we're pre-byzantium, pre-load trie nodes for the intermediate root @@ -82,9 +82,9 @@ func (p *statePrefetcher) Prefetch(block *types.Block, statedb *state.StateDB, c // precacheTransaction attempts to apply a transaction to the given state database // and uses the input parameters for its environment. The goal is not to execute // the transaction successfully, rather to warm up touched data slots. -func precacheTransaction(msg *Message, config *params.ChainConfig, gaspool *GasPool, statedb *state.StateDB, header *types.Header, evm *vm.EVM) error { +func precacheTransaction(msg *Message, gaspool *GasPool, evm *vm.EVM) error { // Update the evm with the new transaction context. - evm.Reset(NewEVMTxContext(msg), statedb) + evm.SetTxContext(NewEVMTxContext(msg)) // Add addresses to access list if applicable _, err := ApplyMessage(evm, msg, gaspool) return err diff --git a/core/state_processor.go b/core/state_processor.go index c04049e986..74985b69a1 100644 --- a/core/state_processor.go +++ b/core/state_processor.go @@ -74,18 +74,18 @@ func (p *StateProcessor) Process(block *types.Block, statedb *state.StateDB, cfg ) // Apply pre-execution system calls. - context = NewEVMBlockContext(header, p.chain, nil) - - vmenv := vm.NewEVM(context, vm.TxContext{}, statedb, p.config, cfg) var tracingStateDB = vm.StateDB(statedb) if hooks := cfg.Tracer; hooks != nil { tracingStateDB = state.NewHookedState(statedb, hooks) } + context = NewEVMBlockContext(header, p.chain, nil) + evm := vm.NewEVM(context, tracingStateDB, p.config, cfg) + if beaconRoot := block.BeaconRoot(); beaconRoot != nil { - ProcessBeaconBlockRoot(*beaconRoot, vmenv, tracingStateDB) + ProcessBeaconBlockRoot(*beaconRoot, evm, tracingStateDB) } if p.config.IsPrague(block.Number(), block.Time()) { - ProcessParentBlockHash(block.ParentHash(), vmenv, tracingStateDB) + ProcessParentBlockHash(block.ParentHash(), evm, tracingStateDB) } // Iterate over and process the individual transactions @@ -96,7 +96,7 @@ func (p *StateProcessor) Process(block *types.Block, statedb *state.StateDB, cfg } statedb.SetTxContext(tx.Hash(), i) - receipt, err := ApplyTransactionWithEVM(msg, p.config, gp, statedb, blockNumber, blockHash, tx, usedGas, vmenv) + receipt, err := ApplyTransactionWithEVM(msg, p.config, gp, statedb, blockNumber, blockHash, tx, usedGas, evm) if err != nil { return nil, fmt.Errorf("could not apply tx %d [%v]: %w", i, tx.Hash().Hex(), err) } @@ -113,10 +113,10 @@ func (p *StateProcessor) Process(block *types.Block, statedb *state.StateDB, cfg } requests = append(requests, depositRequests) // EIP-7002 withdrawals - withdrawalRequests := ProcessWithdrawalQueue(vmenv, tracingStateDB) + withdrawalRequests := ProcessWithdrawalQueue(evm, tracingStateDB) requests = append(requests, withdrawalRequests) // EIP-7251 consolidations - consolidationRequests := ProcessConsolidationQueue(vmenv, tracingStateDB) + consolidationRequests := ProcessConsolidationQueue(evm, tracingStateDB) requests = append(requests, consolidationRequests) } @@ -135,9 +135,7 @@ func (p *StateProcessor) Process(block *types.Block, statedb *state.StateDB, cfg // and uses the input parameters for its environment similar to ApplyTransaction. However, // this method takes an already created EVM instance as input. func ApplyTransactionWithEVM(msg *Message, config *params.ChainConfig, gp *GasPool, statedb *state.StateDB, blockNumber *big.Int, blockHash common.Hash, tx *types.Transaction, usedGas *uint64, evm *vm.EVM) (receipt *types.Receipt, err error) { - var tracingStateDB = vm.StateDB(statedb) if hooks := evm.Config.Tracer; hooks != nil { - tracingStateDB = state.NewHookedState(statedb, hooks) if hooks.OnTxStart != nil { hooks.OnTxStart(evm.GetVMContext(), tx, msg.From) } @@ -148,7 +146,7 @@ func ApplyTransactionWithEVM(msg *Message, config *params.ChainConfig, gp *GasPo // Create a new context to be used in the EVM environment. txContext := NewEVMTxContext(msg) - evm.Reset(txContext, tracingStateDB) + evm.SetTxContext(txContext) // Apply the transaction to the current state (included in the env). result, err := ApplyMessage(evm, msg, gp) @@ -159,7 +157,7 @@ func ApplyTransactionWithEVM(msg *Message, config *params.ChainConfig, gp *GasPo // Update the state with pending changes. var root []byte if config.IsByzantium(blockNumber) { - tracingStateDB.Finalise(true) + evm.StateDB.Finalise(true) } else { root = statedb.IntermediateRoot(config.IsEIP158(blockNumber)).Bytes() } @@ -210,16 +208,13 @@ func MakeReceipt(evm *vm.EVM, result *ExecutionResult, statedb *state.StateDB, b // and uses the input parameters for its environment. It returns the receipt // for the transaction, gas used and an error if the transaction failed, // indicating the block was invalid. -func ApplyTransaction(config *params.ChainConfig, bc ChainContext, author *common.Address, gp *GasPool, statedb *state.StateDB, header *types.Header, tx *types.Transaction, usedGas *uint64, cfg vm.Config) (*types.Receipt, error) { +func ApplyTransaction(config *params.ChainConfig, evm *vm.EVM, gp *GasPool, statedb *state.StateDB, header *types.Header, tx *types.Transaction, usedGas *uint64) (*types.Receipt, error) { msg, err := TransactionToMessage(tx, types.MakeSigner(config, header.Number, header.Time), header.BaseFee) if err != nil { return nil, err } // Create a new context to be used in the EVM environment - blockContext := NewEVMBlockContext(header, bc, author) - txContext := NewEVMTxContext(msg) - vmenv := vm.NewEVM(blockContext, txContext, statedb, config, cfg) - return ApplyTransactionWithEVM(msg, config, gp, statedb, header.Number, header.Hash(), tx, usedGas, vmenv) + return ApplyTransactionWithEVM(msg, config, gp, statedb, header.Number, header.Hash(), tx, usedGas, evm) } // ProcessBeaconBlockRoot applies the EIP-4788 system call to the beacon block root @@ -242,7 +237,7 @@ func ProcessBeaconBlockRoot(beaconRoot common.Hash, vmenv *vm.EVM, statedb vm.St To: ¶ms.BeaconRootsAddress, Data: beaconRoot[:], } - vmenv.Reset(NewEVMTxContext(msg), statedb) + vmenv.SetTxContext(NewEVMTxContext(msg)) statedb.AddAddressToAccessList(params.BeaconRootsAddress) _, _, _ = vmenv.Call(vm.AccountRef(msg.From), *msg.To, msg.Data, 30_000_000, common.U2560) statedb.Finalise(true) @@ -268,7 +263,7 @@ func ProcessParentBlockHash(prevHash common.Hash, vmenv *vm.EVM, statedb vm.Stat To: ¶ms.HistoryStorageAddress, Data: prevHash.Bytes(), } - vmenv.Reset(NewEVMTxContext(msg), statedb) + vmenv.SetTxContext(NewEVMTxContext(msg)) statedb.AddAddressToAccessList(params.HistoryStorageAddress) _, _, _ = vmenv.Call(vm.AccountRef(msg.From), *msg.To, msg.Data, 30_000_000, common.U2560) statedb.Finalise(true) @@ -304,7 +299,7 @@ func processRequestsSystemCall(vmenv *vm.EVM, statedb vm.StateDB, requestType by GasTipCap: common.Big0, To: &addr, } - vmenv.Reset(NewEVMTxContext(msg), statedb) + vmenv.SetTxContext(NewEVMTxContext(msg)) statedb.AddAddressToAccessList(addr) ret, _, _ := vmenv.Call(vm.AccountRef(msg.From), *msg.To, msg.Data, 30_000_000, common.U2560) statedb.Finalise(true) diff --git a/core/verkle_witness_test.go b/core/verkle_witness_test.go index b3088f0f90..80996eee33 100644 --- a/core/verkle_witness_test.go +++ b/core/verkle_witness_test.go @@ -225,7 +225,7 @@ func TestProcessParentBlockHash(t *testing.T) { for i := 1; i <= num; i++ { header := &types.Header{ParentHash: common.Hash{byte(i)}, Number: big.NewInt(int64(i)), Difficulty: new(big.Int)} vmContext := NewEVMBlockContext(header, nil, new(common.Address)) - evm := vm.NewEVM(vmContext, vm.TxContext{}, statedb, params.MergedTestChainConfig, vm.Config{}) + evm := vm.NewEVM(vmContext, statedb, params.MergedTestChainConfig, vm.Config{}) ProcessParentBlockHash(header.ParentHash, evm, statedb) } // Read block hashes for block 0 .. num-1 diff --git a/core/vm/evm.go b/core/vm/evm.go index 0593a32a3e..34e5fa766b 100644 --- a/core/vm/evm.go +++ b/core/vm/evm.go @@ -116,12 +116,13 @@ type EVM struct { precompiles map[common.Address]PrecompiledContract } -// NewEVM returns a new EVM. The returned EVM is not thread safe and should -// only ever be used *once*. -func NewEVM(blockCtx BlockContext, txCtx TxContext, statedb StateDB, chainConfig *params.ChainConfig, config Config) *EVM { +// NewEVM constructs an EVM instance with the supplied block context, state +// database and several configs. It meant to be used throughout the entire +// state transition of a block, with the transaction context switched as +// needed by calling evm.SetTxContext. +func NewEVM(blockCtx BlockContext, statedb StateDB, chainConfig *params.ChainConfig, config Config) *EVM { evm := &EVM{ Context: blockCtx, - TxContext: txCtx, StateDB: statedb, Config: config, chainConfig: chainConfig, @@ -132,6 +133,11 @@ func NewEVM(blockCtx BlockContext, txCtx TxContext, statedb StateDB, chainConfig return evm } +// SetTracer sets the tracer for following state transition. +func (evm *EVM) SetTracer(tracer *tracing.Hooks) { + evm.Config.Tracer = tracer +} + // SetPrecompiles sets the precompiled contracts for the EVM. // This method is only used through RPC calls. // It is not thread-safe. @@ -139,14 +145,13 @@ func (evm *EVM) SetPrecompiles(precompiles PrecompiledContracts) { evm.precompiles = precompiles } -// Reset resets the EVM with a new transaction context.Reset +// SetTxContext resets the EVM with a new transaction context. // This is not threadsafe and should only be done very cautiously. -func (evm *EVM) Reset(txCtx TxContext, statedb StateDB) { +func (evm *EVM) SetTxContext(txCtx TxContext) { if evm.chainRules.IsEIP4762 { - txCtx.AccessEvents = state.NewAccessEvents(statedb.PointCache()) + txCtx.AccessEvents = state.NewAccessEvents(evm.StateDB.PointCache()) } evm.TxContext = txCtx - evm.StateDB = statedb } // Cancel cancels any running EVM operation. This may be called concurrently and diff --git a/core/vm/gas_table_test.go b/core/vm/gas_table_test.go index babe9a5b6a..be86885261 100644 --- a/core/vm/gas_table_test.go +++ b/core/vm/gas_table_test.go @@ -95,16 +95,16 @@ func TestEIP2200(t *testing.T) { CanTransfer: func(StateDB, common.Address, *uint256.Int) bool { return true }, Transfer: func(StateDB, common.Address, common.Address, *uint256.Int) {}, } - vmenv := NewEVM(vmctx, TxContext{}, statedb, params.AllEthashProtocolChanges, Config{ExtraEips: []int{2200}}) + evm := NewEVM(vmctx, statedb, params.AllEthashProtocolChanges, Config{ExtraEips: []int{2200}}) - _, gas, err := vmenv.Call(AccountRef(common.Address{}), address, nil, tt.gaspool, new(uint256.Int)) + _, gas, err := evm.Call(AccountRef(common.Address{}), address, nil, tt.gaspool, new(uint256.Int)) if !errors.Is(err, tt.failure) { t.Errorf("test %d: failure mismatch: have %v, want %v", i, err, tt.failure) } if used := tt.gaspool - gas; used != tt.used { t.Errorf("test %d: gas used mismatch: have %v, want %v", i, used, tt.used) } - if refund := vmenv.StateDB.GetRefund(); refund != tt.refund { + if refund := evm.StateDB.GetRefund(); refund != tt.refund { t.Errorf("test %d: gas refund mismatch: have %v, want %v", i, refund, tt.refund) } } @@ -151,9 +151,9 @@ func TestCreateGas(t *testing.T) { config.ExtraEips = []int{3860} } - vmenv := NewEVM(vmctx, TxContext{}, statedb, params.AllEthashProtocolChanges, config) + evm := NewEVM(vmctx, statedb, params.AllEthashProtocolChanges, config) var startGas = uint64(testGas) - ret, gas, err := vmenv.Call(AccountRef(common.Address{}), address, nil, startGas, new(uint256.Int)) + ret, gas, err := evm.Call(AccountRef(common.Address{}), address, nil, startGas, new(uint256.Int)) if err != nil { return false } diff --git a/core/vm/instructions_test.go b/core/vm/instructions_test.go index b8e62e1de5..08f2b2bfea 100644 --- a/core/vm/instructions_test.go +++ b/core/vm/instructions_test.go @@ -104,10 +104,9 @@ func init() { func testTwoOperandOp(t *testing.T, tests []TwoOperandTestcase, opFn executionFunc, name string) { var ( - env = NewEVM(BlockContext{}, TxContext{}, nil, params.TestChainConfig, Config{}) - stack = newstack() - pc = uint64(0) - evmInterpreter = env.interpreter + evm = NewEVM(BlockContext{}, nil, params.TestChainConfig, Config{}) + stack = newstack() + pc = uint64(0) ) for i, test := range tests { @@ -116,7 +115,7 @@ func testTwoOperandOp(t *testing.T, tests []TwoOperandTestcase, opFn executionFu expected := new(uint256.Int).SetBytes(common.Hex2Bytes(test.Expected)) stack.push(x) stack.push(y) - opFn(&pc, evmInterpreter, &ScopeContext{nil, stack, nil}) + opFn(&pc, evm.interpreter, &ScopeContext{nil, stack, nil}) if len(stack.data) != 1 { t.Errorf("Expected one item on stack after %v, got %d: ", name, len(stack.data)) } @@ -203,10 +202,9 @@ func TestSAR(t *testing.T) { func TestAddMod(t *testing.T) { var ( - env = NewEVM(BlockContext{}, TxContext{}, nil, params.TestChainConfig, Config{}) - stack = newstack() - evmInterpreter = NewEVMInterpreter(env) - pc = uint64(0) + evm = NewEVM(BlockContext{}, nil, params.TestChainConfig, Config{}) + stack = newstack() + pc = uint64(0) ) tests := []struct { x string @@ -231,7 +229,7 @@ func TestAddMod(t *testing.T) { stack.push(z) stack.push(y) stack.push(x) - opAddmod(&pc, evmInterpreter, &ScopeContext{nil, stack, nil}) + opAddmod(&pc, evm.interpreter, &ScopeContext{nil, stack, nil}) actual := stack.pop() if actual.Cmp(expected) != 0 { t.Errorf("Testcase %d, expected %x, got %x", i, expected, actual) @@ -247,10 +245,9 @@ func TestWriteExpectedValues(t *testing.T) { // getResult is a convenience function to generate the expected values getResult := func(args []*twoOperandParams, opFn executionFunc) []TwoOperandTestcase { var ( - env = NewEVM(BlockContext{}, TxContext{}, nil, params.TestChainConfig, Config{}) - stack = newstack() - pc = uint64(0) - interpreter = env.interpreter + evm = NewEVM(BlockContext{}, nil, params.TestChainConfig, Config{}) + stack = newstack() + pc = uint64(0) ) result := make([]TwoOperandTestcase, len(args)) for i, param := range args { @@ -258,7 +255,7 @@ func TestWriteExpectedValues(t *testing.T) { y := new(uint256.Int).SetBytes(common.Hex2Bytes(param.y)) stack.push(x) stack.push(y) - opFn(&pc, interpreter, &ScopeContext{nil, stack, nil}) + opFn(&pc, evm.interpreter, &ScopeContext{nil, stack, nil}) actual := stack.pop() result[i] = TwoOperandTestcase{param.x, param.y, fmt.Sprintf("%064x", actual)} } @@ -292,13 +289,10 @@ func TestJsonTestcases(t *testing.T) { func opBenchmark(bench *testing.B, op executionFunc, args ...string) { var ( - env = NewEVM(BlockContext{}, TxContext{}, nil, params.TestChainConfig, Config{}) - stack = newstack() - scope = &ScopeContext{nil, stack, nil} - evmInterpreter = NewEVMInterpreter(env) + evm = NewEVM(BlockContext{}, nil, params.TestChainConfig, Config{}) + stack = newstack() + scope = &ScopeContext{nil, stack, nil} ) - - env.interpreter = evmInterpreter // convert args intArgs := make([]*uint256.Int, len(args)) for i, arg := range args { @@ -310,7 +304,7 @@ func opBenchmark(bench *testing.B, op executionFunc, args ...string) { for _, arg := range intArgs { stack.push(arg) } - op(&pc, evmInterpreter, scope) + op(&pc, evm.interpreter, scope) stack.pop() } bench.StopTimer() @@ -533,25 +527,22 @@ func BenchmarkOpIsZero(b *testing.B) { func TestOpMstore(t *testing.T) { var ( - env = NewEVM(BlockContext{}, TxContext{}, nil, params.TestChainConfig, Config{}) - stack = newstack() - mem = NewMemory() - evmInterpreter = NewEVMInterpreter(env) + evm = NewEVM(BlockContext{}, nil, params.TestChainConfig, Config{}) + stack = newstack() + mem = NewMemory() ) - - env.interpreter = evmInterpreter mem.Resize(64) pc := uint64(0) v := "abcdef00000000000000abba000000000deaf000000c0de00100000000133700" stack.push(new(uint256.Int).SetBytes(common.Hex2Bytes(v))) stack.push(new(uint256.Int)) - opMstore(&pc, evmInterpreter, &ScopeContext{mem, stack, nil}) + opMstore(&pc, evm.interpreter, &ScopeContext{mem, stack, nil}) if got := common.Bytes2Hex(mem.GetCopy(0, 32)); got != v { t.Fatalf("Mstore fail, got %v, expected %v", got, v) } stack.push(new(uint256.Int).SetUint64(0x1)) stack.push(new(uint256.Int)) - opMstore(&pc, evmInterpreter, &ScopeContext{mem, stack, nil}) + opMstore(&pc, evm.interpreter, &ScopeContext{mem, stack, nil}) if common.Bytes2Hex(mem.GetCopy(0, 32)) != "0000000000000000000000000000000000000000000000000000000000000001" { t.Fatalf("Mstore failed to overwrite previous value") } @@ -559,13 +550,10 @@ func TestOpMstore(t *testing.T) { func BenchmarkOpMstore(bench *testing.B) { var ( - env = NewEVM(BlockContext{}, TxContext{}, nil, params.TestChainConfig, Config{}) - stack = newstack() - mem = NewMemory() - evmInterpreter = NewEVMInterpreter(env) + evm = NewEVM(BlockContext{}, nil, params.TestChainConfig, Config{}) + stack = newstack() + mem = NewMemory() ) - - env.interpreter = evmInterpreter mem.Resize(64) pc := uint64(0) memStart := new(uint256.Int) @@ -575,43 +563,41 @@ func BenchmarkOpMstore(bench *testing.B) { for i := 0; i < bench.N; i++ { stack.push(value) stack.push(memStart) - opMstore(&pc, evmInterpreter, &ScopeContext{mem, stack, nil}) + opMstore(&pc, evm.interpreter, &ScopeContext{mem, stack, nil}) } } func TestOpTstore(t *testing.T) { var ( - statedb, _ = state.New(types.EmptyRootHash, state.NewDatabaseForTesting()) - env = NewEVM(BlockContext{}, TxContext{}, statedb, params.TestChainConfig, Config{}) - stack = newstack() - mem = NewMemory() - evmInterpreter = NewEVMInterpreter(env) - caller = common.Address{} - to = common.Address{1} - contractRef = contractRef{caller} - contract = NewContract(contractRef, AccountRef(to), new(uint256.Int), 0) - scopeContext = ScopeContext{mem, stack, contract} - value = common.Hex2Bytes("abcdef00000000000000abba000000000deaf000000c0de00100000000133700") + statedb, _ = state.New(types.EmptyRootHash, state.NewDatabaseForTesting()) + evm = NewEVM(BlockContext{}, statedb, params.TestChainConfig, Config{}) + stack = newstack() + mem = NewMemory() + caller = common.Address{} + to = common.Address{1} + contractRef = contractRef{caller} + contract = NewContract(contractRef, AccountRef(to), new(uint256.Int), 0) + scopeContext = ScopeContext{mem, stack, contract} + value = common.Hex2Bytes("abcdef00000000000000abba000000000deaf000000c0de00100000000133700") ) // Add a stateObject for the caller and the contract being called statedb.CreateAccount(caller) statedb.CreateAccount(to) - env.interpreter = evmInterpreter pc := uint64(0) // push the value to the stack stack.push(new(uint256.Int).SetBytes(value)) // push the location to the stack stack.push(new(uint256.Int)) - opTstore(&pc, evmInterpreter, &scopeContext) + opTstore(&pc, evm.interpreter, &scopeContext) // there should be no elements on the stack after TSTORE if stack.len() != 0 { t.Fatal("stack wrong size") } // push the location to the stack stack.push(new(uint256.Int)) - opTload(&pc, evmInterpreter, &scopeContext) + opTload(&pc, evm.interpreter, &scopeContext) // there should be one element on the stack after TLOAD if stack.len() != 1 { t.Fatal("stack wrong size") @@ -624,12 +610,10 @@ func TestOpTstore(t *testing.T) { func BenchmarkOpKeccak256(bench *testing.B) { var ( - env = NewEVM(BlockContext{}, TxContext{}, nil, params.TestChainConfig, Config{}) - stack = newstack() - mem = NewMemory() - evmInterpreter = NewEVMInterpreter(env) + evm = NewEVM(BlockContext{}, nil, params.TestChainConfig, Config{}) + stack = newstack() + mem = NewMemory() ) - env.interpreter = evmInterpreter mem.Resize(32) pc := uint64(0) start := new(uint256.Int) @@ -638,7 +622,7 @@ func BenchmarkOpKeccak256(bench *testing.B) { for i := 0; i < bench.N; i++ { stack.push(uint256.NewInt(32)) stack.push(start) - opKeccak256(&pc, evmInterpreter, &ScopeContext{mem, stack, nil}) + opKeccak256(&pc, evm.interpreter, &ScopeContext{mem, stack, nil}) } } @@ -728,12 +712,11 @@ func TestRandom(t *testing.T) { {name: "hash(0x010203)", random: crypto.Keccak256Hash([]byte{0x01, 0x02, 0x03})}, } { var ( - env = NewEVM(BlockContext{Random: &tt.random}, TxContext{}, nil, params.TestChainConfig, Config{}) - stack = newstack() - pc = uint64(0) - evmInterpreter = env.interpreter + evm = NewEVM(BlockContext{Random: &tt.random}, nil, params.TestChainConfig, Config{}) + stack = newstack() + pc = uint64(0) ) - opRandom(&pc, evmInterpreter, &ScopeContext{nil, stack, nil}) + opRandom(&pc, evm.interpreter, &ScopeContext{nil, stack, nil}) if len(stack.data) != 1 { t.Errorf("Expected one item on stack after %v, got %d: ", tt.name, len(stack.data)) } @@ -769,13 +752,13 @@ func TestBlobHash(t *testing.T) { {name: "out-of-bounds (nil)", idx: 25, expect: zero, hashes: nil}, } { var ( - env = NewEVM(BlockContext{}, TxContext{BlobHashes: tt.hashes}, nil, params.TestChainConfig, Config{}) - stack = newstack() - pc = uint64(0) - evmInterpreter = env.interpreter + evm = NewEVM(BlockContext{}, nil, params.TestChainConfig, Config{}) + stack = newstack() + pc = uint64(0) ) + evm.SetTxContext(TxContext{BlobHashes: tt.hashes}) stack.push(uint256.NewInt(tt.idx)) - opBlobHash(&pc, evmInterpreter, &ScopeContext{nil, stack, nil}) + opBlobHash(&pc, evm.interpreter, &ScopeContext{nil, stack, nil}) if len(stack.data) != 1 { t.Errorf("Expected one item on stack after %v, got %d: ", tt.name, len(stack.data)) } @@ -872,10 +855,9 @@ func TestOpMCopy(t *testing.T) { }, } { var ( - env = NewEVM(BlockContext{}, TxContext{}, nil, params.TestChainConfig, Config{}) - stack = newstack() - pc = uint64(0) - evmInterpreter = env.interpreter + evm = NewEVM(BlockContext{}, nil, params.TestChainConfig, Config{}) + stack = newstack() + pc = uint64(0) ) data := common.FromHex(strings.ReplaceAll(tc.pre, " ", "")) // Set pre @@ -906,7 +888,7 @@ func TestOpMCopy(t *testing.T) { } // and the dynamic cost var haveGas uint64 - if dynamicCost, err := gasMcopy(env, nil, stack, mem, memorySize); err != nil { + if dynamicCost, err := gasMcopy(evm, nil, stack, mem, memorySize); err != nil { t.Error(err) } else { haveGas = GasFastestStep + dynamicCost @@ -916,7 +898,7 @@ func TestOpMCopy(t *testing.T) { mem.Resize(memorySize) } // Do the copy - opMcopy(&pc, evmInterpreter, &ScopeContext{mem, stack, nil}) + opMcopy(&pc, evm.interpreter, &ScopeContext{mem, stack, nil}) want := common.FromHex(strings.ReplaceAll(tc.want, " ", "")) if have := mem.store; !bytes.Equal(want, have) { t.Errorf("case %d: \nwant: %#x\nhave: %#x\n", i, want, have) diff --git a/core/vm/interpreter_test.go b/core/vm/interpreter_test.go index 7d4b2ddf36..cacad8f813 100644 --- a/core/vm/interpreter_test.go +++ b/core/vm/interpreter_test.go @@ -47,7 +47,7 @@ func TestLoopInterrupt(t *testing.T) { statedb.SetCode(address, common.Hex2Bytes(tt)) statedb.Finalise(true) - evm := NewEVM(vmctx, TxContext{}, statedb, params.AllEthashProtocolChanges, Config{}) + evm := NewEVM(vmctx, statedb, params.AllEthashProtocolChanges, Config{}) errChannel := make(chan error) timeout := make(chan bool) diff --git a/core/vm/runtime/env.go b/core/vm/runtime/env.go index 34335b8e9e..e54041f7e2 100644 --- a/core/vm/runtime/env.go +++ b/core/vm/runtime/env.go @@ -42,5 +42,7 @@ func NewEnv(cfg *Config) *vm.EVM { Random: cfg.Random, } - return vm.NewEVM(blockContext, txContext, cfg.State, cfg.ChainConfig, cfg.EVMConfig) + evm := vm.NewEVM(blockContext, cfg.State, cfg.ChainConfig, cfg.EVMConfig) + evm.SetTxContext(txContext) + return evm } diff --git a/eth/api_backend.go b/eth/api_backend.go index 4e81d68e07..be2101c6ec 100644 --- a/eth/api_backend.go +++ b/eth/api_backend.go @@ -249,18 +249,17 @@ func (b *EthAPIBackend) GetTd(ctx context.Context, hash common.Hash) *big.Int { return nil } -func (b *EthAPIBackend) GetEVM(ctx context.Context, msg *core.Message, state *state.StateDB, header *types.Header, vmConfig *vm.Config, blockCtx *vm.BlockContext) *vm.EVM { +func (b *EthAPIBackend) GetEVM(ctx context.Context, state *state.StateDB, header *types.Header, vmConfig *vm.Config, blockCtx *vm.BlockContext) *vm.EVM { if vmConfig == nil { vmConfig = b.eth.blockchain.GetVMConfig() } - txContext := core.NewEVMTxContext(msg) var context vm.BlockContext if blockCtx != nil { context = *blockCtx } else { context = core.NewEVMBlockContext(header, b.eth.BlockChain(), nil) } - return vm.NewEVM(context, txContext, state, b.ChainConfig(), *vmConfig) + return vm.NewEVM(context, state, b.ChainConfig(), *vmConfig) } func (b *EthAPIBackend) SubscribeRemovedLogsEvent(ch chan<- core.RemovedLogsEvent) event.Subscription { diff --git a/eth/gasestimator/gasestimator.go b/eth/gasestimator/gasestimator.go index d43057dda2..5729f84278 100644 --- a/eth/gasestimator/gasestimator.go +++ b/eth/gasestimator/gasestimator.go @@ -230,7 +230,8 @@ func run(ctx context.Context, call *core.Message, opts *Options) (*core.Executio if msgContext.BlobFeeCap != nil && msgContext.BlobFeeCap.BitLen() == 0 { evmContext.BlobBaseFee = new(big.Int) } - evm := vm.NewEVM(evmContext, msgContext, dirtyState, opts.Config, vm.Config{NoBaseFee: true}) + evm := vm.NewEVM(evmContext, dirtyState, opts.Config, vm.Config{NoBaseFee: true}) + evm.SetTxContext(msgContext) // Monitor the outer context and interrupt the EVM upon cancellation. To avoid // a dangling goroutine until the outer estimation finishes, create an internal // context for the lifetime of this method call. diff --git a/eth/state_accessor.go b/eth/state_accessor.go index cb5a233a83..a919030249 100644 --- a/eth/state_accessor.go +++ b/eth/state_accessor.go @@ -235,16 +235,14 @@ func (eth *Ethereum) stateAtTransaction(ctx context.Context, block *types.Block, return nil, vm.BlockContext{}, nil, nil, err } // Insert parent beacon block root in the state as per EIP-4788. + context := core.NewEVMBlockContext(block.Header(), eth.blockchain, nil) + evm := vm.NewEVM(context, statedb, eth.blockchain.Config(), vm.Config{}) if beaconRoot := block.BeaconRoot(); beaconRoot != nil { - context := core.NewEVMBlockContext(block.Header(), eth.blockchain, nil) - vmenv := vm.NewEVM(context, vm.TxContext{}, statedb, eth.blockchain.Config(), vm.Config{}) - core.ProcessBeaconBlockRoot(*beaconRoot, vmenv, statedb) + core.ProcessBeaconBlockRoot(*beaconRoot, evm, statedb) } // If prague hardfork, insert parent block hash in the state as per EIP-2935. if eth.blockchain.Config().IsPrague(block.Number(), block.Time()) { - context := core.NewEVMBlockContext(block.Header(), eth.blockchain, nil) - vmenv := vm.NewEVM(context, vm.TxContext{}, statedb, eth.blockchain.Config(), vm.Config{}) - core.ProcessParentBlockHash(block.ParentHash(), vmenv, statedb) + core.ProcessParentBlockHash(block.ParentHash(), evm, statedb) } if txIndex == 0 && len(block.Transactions()) == 0 { return nil, vm.BlockContext{}, statedb, release, nil @@ -252,22 +250,22 @@ func (eth *Ethereum) stateAtTransaction(ctx context.Context, block *types.Block, // Recompute transactions up to the target index. signer := types.MakeSigner(eth.blockchain.Config(), block.Number(), block.Time()) for idx, tx := range block.Transactions() { - // Assemble the transaction call message and return if the requested offset - msg, _ := core.TransactionToMessage(tx, signer, block.BaseFee()) - txContext := core.NewEVMTxContext(msg) - context := core.NewEVMBlockContext(block.Header(), eth.blockchain, nil) if idx == txIndex { return tx, context, statedb, release, nil } + // Assemble the transaction call message and return if the requested offset + msg, _ := core.TransactionToMessage(tx, signer, block.BaseFee()) + txContext := core.NewEVMTxContext(msg) + evm.SetTxContext(txContext) + // Not yet the searched for transaction, execute on top of the current state - vmenv := vm.NewEVM(context, txContext, statedb, eth.blockchain.Config(), vm.Config{}) statedb.SetTxContext(tx.Hash(), idx) - if _, err := core.ApplyMessage(vmenv, msg, new(core.GasPool).AddGas(tx.Gas())); err != nil { + if _, err := core.ApplyMessage(evm, msg, new(core.GasPool).AddGas(tx.Gas())); err != nil { return nil, vm.BlockContext{}, nil, nil, fmt.Errorf("transaction %#x failed: %v", tx.Hash(), err) } // Ensure any modifications are committed to the state // Only delete empty objects if EIP158/161 (a.k.a Spurious Dragon) is in effect - statedb.Finalise(vmenv.ChainConfig().IsEIP158(block.Number())) + statedb.Finalise(evm.ChainConfig().IsEIP158(block.Number())) } return nil, vm.BlockContext{}, nil, nil, fmt.Errorf("transaction index %d out of range for block %#x", txIndex, block.Hash()) } diff --git a/eth/tracers/api.go b/eth/tracers/api.go index 7d8191c25b..671a86e78a 100644 --- a/eth/tracers/api.go +++ b/eth/tracers/api.go @@ -378,16 +378,14 @@ func (api *API) traceChain(start, end *types.Block, config *TraceConfig, closed } // Insert block's parent beacon block root in the state // as per EIP-4788. + context := core.NewEVMBlockContext(next.Header(), api.chainContext(ctx), nil) + evm := vm.NewEVM(context, statedb, api.backend.ChainConfig(), vm.Config{}) if beaconRoot := next.BeaconRoot(); beaconRoot != nil { - context := core.NewEVMBlockContext(next.Header(), api.chainContext(ctx), nil) - vmenv := vm.NewEVM(context, vm.TxContext{}, statedb, api.backend.ChainConfig(), vm.Config{}) - core.ProcessBeaconBlockRoot(*beaconRoot, vmenv, statedb) + core.ProcessBeaconBlockRoot(*beaconRoot, evm, statedb) } // Insert parent hash in history contract. if api.backend.ChainConfig().IsPrague(next.Number(), next.Time()) { - context := core.NewEVMBlockContext(next.Header(), api.chainContext(ctx), nil) - vmenv := vm.NewEVM(context, vm.TxContext{}, statedb, api.backend.ChainConfig(), vm.Config{}) - core.ProcessParentBlockHash(next.ParentHash(), vmenv, statedb) + core.ProcessParentBlockHash(next.ParentHash(), evm, statedb) } // Clean out any pending release functions of trace state. Note this // step must be done after constructing tracing state, because the @@ -537,12 +535,12 @@ func (api *API) IntermediateRoots(ctx context.Context, hash common.Hash, config vmctx = core.NewEVMBlockContext(block.Header(), api.chainContext(ctx), nil) deleteEmptyObjects = chainConfig.IsEIP158(block.Number()) ) + evm := vm.NewEVM(vmctx, statedb, chainConfig, vm.Config{}) if beaconRoot := block.BeaconRoot(); beaconRoot != nil { - vmenv := vm.NewEVM(vmctx, vm.TxContext{}, statedb, chainConfig, vm.Config{}) - core.ProcessBeaconBlockRoot(*beaconRoot, vmenv, statedb) + core.ProcessBeaconBlockRoot(*beaconRoot, evm, statedb) } if chainConfig.IsPrague(block.Number(), block.Time()) { - core.ProcessParentBlockHash(block.ParentHash(), vm.NewEVM(vmctx, vm.TxContext{}, statedb, chainConfig, vm.Config{}), statedb) + core.ProcessParentBlockHash(block.ParentHash(), evm, statedb) } for i, tx := range block.Transactions() { if err := ctx.Err(); err != nil { @@ -551,10 +549,10 @@ func (api *API) IntermediateRoots(ctx context.Context, hash common.Hash, config var ( msg, _ = core.TransactionToMessage(tx, signer, block.BaseFee()) txContext = core.NewEVMTxContext(msg) - vmenv = vm.NewEVM(vmctx, txContext, statedb, chainConfig, vm.Config{}) ) + evm.SetTxContext(txContext) statedb.SetTxContext(tx.Hash(), i) - if _, err := core.ApplyMessage(vmenv, msg, new(core.GasPool).AddGas(msg.GasLimit)); err != nil { + if _, err := core.ApplyMessage(evm, msg, new(core.GasPool).AddGas(msg.GasLimit)); err != nil { log.Warn("Tracing intermediate roots did not complete", "txindex", i, "txhash", tx.Hash(), "err", err) // We intentionally don't return the error here: if we do, then the RPC server will not // return the roots. Most likely, the caller already knows that a certain transaction fails to @@ -605,13 +603,12 @@ func (api *API) traceBlock(ctx context.Context, block *types.Block, config *Trac defer release() blockCtx := core.NewEVMBlockContext(block.Header(), api.chainContext(ctx), nil) + evm := vm.NewEVM(blockCtx, statedb, api.backend.ChainConfig(), vm.Config{}) if beaconRoot := block.BeaconRoot(); beaconRoot != nil { - vmenv := vm.NewEVM(blockCtx, vm.TxContext{}, statedb, api.backend.ChainConfig(), vm.Config{}) - core.ProcessBeaconBlockRoot(*beaconRoot, vmenv, statedb) + core.ProcessBeaconBlockRoot(*beaconRoot, evm, statedb) } if api.backend.ChainConfig().IsPrague(block.Number(), block.Time()) { - vmenv := vm.NewEVM(blockCtx, vm.TxContext{}, statedb, api.backend.ChainConfig(), vm.Config{}) - core.ProcessParentBlockHash(block.ParentHash(), vmenv, statedb) + core.ProcessParentBlockHash(block.ParentHash(), evm, statedb) } // JS tracers have high overhead. In this case run a parallel @@ -695,6 +692,8 @@ func (api *API) traceBlockParallel(ctx context.Context, block *types.Block, stat // Feed the transactions into the tracers and return var failed error blockCtx := core.NewEVMBlockContext(block.Header(), api.chainContext(ctx), nil) + evm := vm.NewEVM(blockCtx, statedb, api.backend.ChainConfig(), vm.Config{}) + txloop: for i, tx := range txs { // Send the trace task over for execution @@ -709,14 +708,14 @@ txloop: // Generate the next state snapshot fast without tracing msg, _ := core.TransactionToMessage(tx, signer, block.BaseFee()) statedb.SetTxContext(tx.Hash(), i) - vmenv := vm.NewEVM(blockCtx, core.NewEVMTxContext(msg), statedb, api.backend.ChainConfig(), vm.Config{}) - if _, err := core.ApplyMessage(vmenv, msg, new(core.GasPool).AddGas(msg.GasLimit)); err != nil { + evm.SetTxContext(core.NewEVMTxContext(msg)) + if _, err := core.ApplyMessage(evm, msg, new(core.GasPool).AddGas(msg.GasLimit)); err != nil { failed = err break txloop } // Finalize the state so any modifications are written to the trie // Only delete empty objects if EIP158/161 (a.k.a Spurious Dragon) is in effect - statedb.Finalise(vmenv.ChainConfig().IsEIP158(block.Number())) + statedb.Finalise(evm.ChainConfig().IsEIP158(block.Number())) } close(jobs) @@ -783,13 +782,12 @@ func (api *API) standardTraceBlockToFile(ctx context.Context, block *types.Block // Note: This copies the config, to not screw up the main config chainConfig, canon = overrideConfig(chainConfig, config.Overrides) } + evm := vm.NewEVM(vmctx, statedb, chainConfig, vm.Config{}) if beaconRoot := block.BeaconRoot(); beaconRoot != nil { - vmenv := vm.NewEVM(vmctx, vm.TxContext{}, statedb, chainConfig, vm.Config{}) - core.ProcessBeaconBlockRoot(*beaconRoot, vmenv, statedb) + core.ProcessBeaconBlockRoot(*beaconRoot, evm, statedb) } if chainConfig.IsPrague(block.Number(), block.Time()) { - vmenv := vm.NewEVM(vmctx, vm.TxContext{}, statedb, chainConfig, vm.Config{}) - core.ProcessParentBlockHash(block.ParentHash(), vmenv, statedb) + core.ProcessParentBlockHash(block.ParentHash(), evm, statedb) } for i, tx := range block.Transactions() { // Prepare the transaction for un-traced execution @@ -822,12 +820,12 @@ func (api *API) standardTraceBlockToFile(ctx context.Context, block *types.Block } } // Execute the transaction and flush any traces to disk - vmenv := vm.NewEVM(vmctx, txContext, statedb, chainConfig, vmConf) + evm.SetTxContext(txContext) statedb.SetTxContext(tx.Hash(), i) if vmConf.Tracer.OnTxStart != nil { - vmConf.Tracer.OnTxStart(vmenv.GetVMContext(), tx, msg.From) + vmConf.Tracer.OnTxStart(evm.GetVMContext(), tx, msg.From) } - vmRet, err := core.ApplyMessage(vmenv, msg, new(core.GasPool).AddGas(msg.GasLimit)) + vmRet, err := core.ApplyMessage(evm, msg, new(core.GasPool).AddGas(msg.GasLimit)) if vmConf.Tracer.OnTxEnd != nil { vmConf.Tracer.OnTxEnd(&types.Receipt{GasUsed: vmRet.UsedGas}, err) } @@ -843,7 +841,7 @@ func (api *API) standardTraceBlockToFile(ctx context.Context, block *types.Block } // Finalize the state so any modifications are written to the trie // Only delete empty objects if EIP158/161 (a.k.a Spurious Dragon) is in effect - statedb.Finalise(vmenv.ChainConfig().IsEIP158(block.Number())) + statedb.Finalise(evm.ChainConfig().IsEIP158(block.Number())) // If we've traced the transaction we were looking for, abort if tx.Hash() == txHash { @@ -1017,7 +1015,8 @@ func (api *API) traceTx(ctx context.Context, tx *types.Transaction, message *cor } } // The actual TxContext will be created as part of ApplyTransactionWithEVM. - vmenv := vm.NewEVM(vmctx, vm.TxContext{GasPrice: message.GasPrice, BlobFeeCap: message.BlobGasFeeCap}, statedb, api.backend.ChainConfig(), vm.Config{Tracer: tracer.Hooks, NoBaseFee: true}) + evm := vm.NewEVM(vmctx, statedb, api.backend.ChainConfig(), vm.Config{Tracer: tracer.Hooks, NoBaseFee: true}) + evm.SetTxContext(vm.TxContext{GasPrice: message.GasPrice, BlobFeeCap: message.BlobGasFeeCap}) // Define a meaningful timeout of a single transaction trace if config.Timeout != nil { @@ -1031,14 +1030,14 @@ func (api *API) traceTx(ctx context.Context, tx *types.Transaction, message *cor if errors.Is(deadlineCtx.Err(), context.DeadlineExceeded) { tracer.Stop(errors.New("execution timeout")) // Stop evm execution. Note cancellation is not necessarily immediate. - vmenv.Cancel() + evm.Cancel() } }() defer cancel() // Call Prepare to clear out the statedb access list statedb.SetTxContext(txctx.TxHash, txctx.TxIndex) - _, err = core.ApplyTransactionWithEVM(message, api.backend.ChainConfig(), new(core.GasPool).AddGas(message.GasLimit), statedb, vmctx.BlockNumber, txctx.BlockHash, tx, &usedGas, vmenv) + _, err = core.ApplyTransactionWithEVM(message, api.backend.ChainConfig(), new(core.GasPool).AddGas(message.GasLimit), statedb, vmctx.BlockNumber, txctx.BlockHash, tx, &usedGas, evm) if err != nil { return nil, fmt.Errorf("tracing failed: %w", err) } diff --git a/eth/tracers/api_test.go b/eth/tracers/api_test.go index 47e3693495..2b5cfa35c8 100644 --- a/eth/tracers/api_test.go +++ b/eth/tracers/api_test.go @@ -170,18 +170,19 @@ func (b *testBackend) StateAtTransaction(ctx context.Context, block *types.Block } // Recompute transactions up to the target index. signer := types.MakeSigner(b.chainConfig, block.Number(), block.Time()) + context := core.NewEVMBlockContext(block.Header(), b.chain, nil) + evm := vm.NewEVM(context, statedb, b.chainConfig, vm.Config{}) for idx, tx := range block.Transactions() { - msg, _ := core.TransactionToMessage(tx, signer, block.BaseFee()) - txContext := core.NewEVMTxContext(msg) - context := core.NewEVMBlockContext(block.Header(), b.chain, nil) if idx == txIndex { return tx, context, statedb, release, nil } - vmenv := vm.NewEVM(context, txContext, statedb, b.chainConfig, vm.Config{}) - if _, err := core.ApplyMessage(vmenv, msg, new(core.GasPool).AddGas(tx.Gas())); err != nil { + msg, _ := core.TransactionToMessage(tx, signer, block.BaseFee()) + txContext := core.NewEVMTxContext(msg) + evm.SetTxContext(txContext) + if _, err := core.ApplyMessage(evm, msg, new(core.GasPool).AddGas(tx.Gas())); err != nil { return nil, vm.BlockContext{}, nil, nil, fmt.Errorf("transaction %#x failed: %v", tx.Hash(), err) } - statedb.Finalise(vmenv.ChainConfig().IsEIP158(block.Number())) + statedb.Finalise(evm.ChainConfig().IsEIP158(block.Number())) } return nil, vm.BlockContext{}, nil, nil, fmt.Errorf("transaction index %d out of range for block %#x", txIndex, block.Hash()) } diff --git a/eth/tracers/internal/tracetest/calltrace_test.go b/eth/tracers/internal/tracetest/calltrace_test.go index cb635e7127..869558c324 100644 --- a/eth/tracers/internal/tracetest/calltrace_test.go +++ b/eth/tracers/internal/tracetest/calltrace_test.go @@ -132,7 +132,8 @@ func testCallTracer(tracerName string, dirPath string, t *testing.T) { if err != nil { t.Fatalf("failed to prepare transaction for tracing: %v", err) } - evm := vm.NewEVM(context, core.NewEVMTxContext(msg), logState, test.Genesis.Config, vm.Config{Tracer: tracer.Hooks}) + evm := vm.NewEVM(context, logState, test.Genesis.Config, vm.Config{Tracer: tracer.Hooks}) + evm.SetTxContext(core.NewEVMTxContext(msg)) tracer.OnTxStart(evm.GetVMContext(), tx, msg.From) vmRet, err := core.ApplyMessage(evm, msg, new(core.GasPool).AddGas(tx.Gas())) if err != nil { @@ -220,12 +221,15 @@ func benchTracer(tracerName string, test *callTracerTest, b *testing.B) { b.ReportAllocs() b.ResetTimer() + + tracer, err := tracers.DefaultDirectory.New(tracerName, new(tracers.Context), nil, test.Genesis.Config) + if err != nil { + b.Fatalf("failed to create call tracer: %v", err) + } + evm := vm.NewEVM(context, state.StateDB, test.Genesis.Config, vm.Config{Tracer: tracer.Hooks}) + evm.SetTxContext(txContext) + for i := 0; i < b.N; i++ { - tracer, err := tracers.DefaultDirectory.New(tracerName, new(tracers.Context), nil, test.Genesis.Config) - if err != nil { - b.Fatalf("failed to create call tracer: %v", err) - } - evm := vm.NewEVM(context, txContext, state.StateDB, test.Genesis.Config, vm.Config{Tracer: tracer.Hooks}) snap := state.StateDB.Snapshot() st := core.NewStateTransition(evm, msg, new(core.GasPool).AddGas(tx.Gas())) if _, err = st.TransitionDb(); err != nil { @@ -372,7 +376,8 @@ func TestInternals(t *testing.T) { Origin: origin, GasPrice: tx.GasPrice(), } - evm := vm.NewEVM(context, txContext, logState, config, vm.Config{Tracer: tc.tracer.Hooks}) + evm := vm.NewEVM(context, logState, config, vm.Config{Tracer: tc.tracer.Hooks}) + evm.SetTxContext(txContext) msg, err := core.TransactionToMessage(tx, signer, big.NewInt(0)) if err != nil { t.Fatalf("test %v: failed to create message: %v", tc.name, err) diff --git a/eth/tracers/internal/tracetest/flat_calltrace_test.go b/eth/tracers/internal/tracetest/flat_calltrace_test.go index 0ec3c367bc..1dbdc7caac 100644 --- a/eth/tracers/internal/tracetest/flat_calltrace_test.go +++ b/eth/tracers/internal/tracetest/flat_calltrace_test.go @@ -98,7 +98,8 @@ func flatCallTracerTestRunner(tracerName string, filename string, dirPath string if err != nil { return fmt.Errorf("failed to prepare transaction for tracing: %v", err) } - evm := vm.NewEVM(context, core.NewEVMTxContext(msg), state.StateDB, test.Genesis.Config, vm.Config{Tracer: tracer.Hooks}) + evm := vm.NewEVM(context, state.StateDB, test.Genesis.Config, vm.Config{Tracer: tracer.Hooks}) + evm.SetTxContext(core.NewEVMTxContext(msg)) tracer.OnTxStart(evm.GetVMContext(), tx, msg.From) vmRet, err := core.ApplyMessage(evm, msg, new(core.GasPool).AddGas(tx.Gas())) if err != nil { diff --git a/eth/tracers/internal/tracetest/prestate_test.go b/eth/tracers/internal/tracetest/prestate_test.go index c6cf10a483..b8dec51db1 100644 --- a/eth/tracers/internal/tracetest/prestate_test.go +++ b/eth/tracers/internal/tracetest/prestate_test.go @@ -106,7 +106,8 @@ func testPrestateDiffTracer(tracerName string, dirPath string, t *testing.T) { if err != nil { t.Fatalf("failed to prepare transaction for tracing: %v", err) } - evm := vm.NewEVM(context, core.NewEVMTxContext(msg), state.StateDB, test.Genesis.Config, vm.Config{Tracer: tracer.Hooks}) + evm := vm.NewEVM(context, state.StateDB, test.Genesis.Config, vm.Config{Tracer: tracer.Hooks}) + evm.SetTxContext(core.NewEVMTxContext(msg)) tracer.OnTxStart(evm.GetVMContext(), tx, msg.From) vmRet, err := core.ApplyMessage(evm, msg, new(core.GasPool).AddGas(tx.Gas())) if err != nil { diff --git a/eth/tracers/js/tracer_test.go b/eth/tracers/js/tracer_test.go index ed2789d70d..54f628b5f3 100644 --- a/eth/tracers/js/tracer_test.go +++ b/eth/tracers/js/tracer_test.go @@ -64,20 +64,21 @@ func testCtx() *vmContext { func runTrace(tracer *tracers.Tracer, vmctx *vmContext, chaincfg *params.ChainConfig, contractCode []byte) (json.RawMessage, error) { var ( - env = vm.NewEVM(vmctx.blockCtx, vmctx.txCtx, &dummyStatedb{}, chaincfg, vm.Config{Tracer: tracer.Hooks}) + evm = vm.NewEVM(vmctx.blockCtx, &dummyStatedb{}, chaincfg, vm.Config{Tracer: tracer.Hooks}) gasLimit uint64 = 31000 startGas uint64 = 10000 value = uint256.NewInt(0) contract = vm.NewContract(account{}, account{}, value, startGas) ) + evm.SetTxContext(vmctx.txCtx) contract.Code = []byte{byte(vm.PUSH1), 0x1, byte(vm.PUSH1), 0x1, 0x0} if contractCode != nil { contract.Code = contractCode } - tracer.OnTxStart(env.GetVMContext(), types.NewTx(&types.LegacyTx{Gas: gasLimit}), contract.Caller()) + tracer.OnTxStart(evm.GetVMContext(), types.NewTx(&types.LegacyTx{Gas: gasLimit}), contract.Caller()) tracer.OnEnter(0, byte(vm.CALL), contract.Caller(), contract.Address(), []byte{}, startGas, value.ToBig()) - ret, err := env.Interpreter().Run(contract, []byte{}, false) + ret, err := evm.Interpreter().Run(contract, []byte{}, false) tracer.OnExit(0, ret, startGas-contract.Gas, err, true) // Rest gas assumes no refund tracer.OnTxEnd(&types.Receipt{GasUsed: gasLimit - contract.Gas}, nil) @@ -191,8 +192,9 @@ func TestHaltBetweenSteps(t *testing.T) { scope := &vm.ScopeContext{ Contract: vm.NewContract(&account{}, &account{}, uint256.NewInt(0), 0), } - env := vm.NewEVM(vm.BlockContext{BlockNumber: big.NewInt(1)}, vm.TxContext{GasPrice: big.NewInt(1)}, &dummyStatedb{}, chainConfig, vm.Config{Tracer: tracer.Hooks}) - tracer.OnTxStart(env.GetVMContext(), types.NewTx(&types.LegacyTx{}), common.Address{}) + evm := vm.NewEVM(vm.BlockContext{BlockNumber: big.NewInt(1)}, &dummyStatedb{}, chainConfig, vm.Config{Tracer: tracer.Hooks}) + evm.SetTxContext(vm.TxContext{GasPrice: big.NewInt(1)}) + tracer.OnTxStart(evm.GetVMContext(), types.NewTx(&types.LegacyTx{}), common.Address{}) tracer.OnEnter(0, byte(vm.CALL), common.Address{}, common.Address{}, []byte{}, 0, big.NewInt(0)) tracer.OnOpcode(0, 0, 0, 0, scope, nil, 0, nil) timeout := errors.New("stahp") @@ -214,8 +216,9 @@ func TestNoStepExec(t *testing.T) { if err != nil { t.Fatal(err) } - env := vm.NewEVM(vm.BlockContext{BlockNumber: big.NewInt(1)}, vm.TxContext{GasPrice: big.NewInt(100)}, &dummyStatedb{}, chainConfig, vm.Config{Tracer: tracer.Hooks}) - tracer.OnTxStart(env.GetVMContext(), types.NewTx(&types.LegacyTx{}), common.Address{}) + evm := vm.NewEVM(vm.BlockContext{BlockNumber: big.NewInt(1)}, &dummyStatedb{}, chainConfig, vm.Config{Tracer: tracer.Hooks}) + evm.SetTxContext(vm.TxContext{GasPrice: big.NewInt(100)}) + tracer.OnTxStart(evm.GetVMContext(), types.NewTx(&types.LegacyTx{}), common.Address{}) tracer.OnEnter(0, byte(vm.CALL), common.Address{}, common.Address{}, []byte{}, 1000, big.NewInt(0)) tracer.OnExit(0, nil, 0, nil, false) ret, err := tracer.GetResult() diff --git a/eth/tracers/logger/logger_test.go b/eth/tracers/logger/logger_test.go index fb1a0154e3..e7799dde35 100644 --- a/eth/tracers/logger/logger_test.go +++ b/eth/tracers/logger/logger_test.go @@ -58,13 +58,13 @@ func (*dummyStatedb) SetState(_ common.Address, _ common.Hash, _ common.Hash) co func TestStoreCapture(t *testing.T) { var ( logger = NewStructLogger(nil) - env = vm.NewEVM(vm.BlockContext{}, vm.TxContext{}, &dummyStatedb{}, params.TestChainConfig, vm.Config{Tracer: logger.Hooks()}) + evm = vm.NewEVM(vm.BlockContext{}, &dummyStatedb{}, params.TestChainConfig, vm.Config{Tracer: logger.Hooks()}) contract = vm.NewContract(&dummyContractRef{}, &dummyContractRef{}, new(uint256.Int), 100000) ) contract.Code = []byte{byte(vm.PUSH1), 0x1, byte(vm.PUSH1), 0x0, byte(vm.SSTORE)} var index common.Hash - logger.OnTxStart(env.GetVMContext(), nil, common.Address{}) - _, err := env.Interpreter().Run(contract, []byte{}, false) + logger.OnTxStart(evm.GetVMContext(), nil, common.Address{}) + _, err := evm.Interpreter().Run(contract, []byte{}, false) if err != nil { t.Fatal(err) } diff --git a/eth/tracers/tracers_test.go b/eth/tracers/tracers_test.go index 31e14b9112..72cc1bde25 100644 --- a/eth/tracers/tracers_test.go +++ b/eth/tracers/tracers_test.go @@ -89,7 +89,8 @@ func BenchmarkTransactionTrace(b *testing.B) { //EnableMemory: false, //EnableReturnData: false, }) - evm := vm.NewEVM(context, txContext, state.StateDB, params.AllEthashProtocolChanges, vm.Config{Tracer: tracer.Hooks()}) + evm := vm.NewEVM(context, state.StateDB, params.AllEthashProtocolChanges, vm.Config{Tracer: tracer.Hooks()}) + evm.SetTxContext(txContext) msg, err := core.TransactionToMessage(tx, signer, context.BaseFee) if err != nil { b.Fatalf("failed to prepare transaction for tracing: %v", err) diff --git a/internal/ethapi/api.go b/internal/ethapi/api.go index a508b0ca5b..9c08006f1e 100644 --- a/internal/ethapi/api.go +++ b/internal/ethapi/api.go @@ -864,10 +864,11 @@ func applyMessage(ctx context.Context, b Backend, args TransactionArgs, state *s if msg.BlobGasFeeCap != nil && msg.BlobGasFeeCap.BitLen() == 0 { blockContext.BlobBaseFee = new(big.Int) } - evm := b.GetEVM(ctx, msg, state, header, vmConfig, blockContext) + evm := b.GetEVM(ctx, state, header, vmConfig, blockContext) if precompiles != nil { evm.SetPrecompiles(precompiles) } + evm.SetTxContext(core.NewEVMTxContext(msg)) res, err := applyMessageWithEVM(ctx, evm, msg, timeout, gp) // If an internal state error occurred, let that have precedence. Otherwise, // a "trie root missing" type of error will masquerade as e.g. "insufficient gas" @@ -1331,7 +1332,7 @@ func AccessList(ctx context.Context, b Backend, blockNrOrHash rpc.BlockNumberOrH // Apply the transaction with the access list tracer tracer := logger.NewAccessListTracer(accessList, args.from(), to, precompiles) config := vm.Config{Tracer: tracer.Hooks(), NoBaseFee: true} - vmenv := b.GetEVM(ctx, msg, statedb, header, &config, nil) + vmenv := b.GetEVM(ctx, statedb, header, &config, nil) // Lower the basefee to 0 to avoid breaking EVM // invariants (basefee < feecap). if msg.GasPrice.Sign() == 0 { @@ -1340,6 +1341,7 @@ func AccessList(ctx context.Context, b Backend, blockNrOrHash rpc.BlockNumberOrH if msg.BlobGasFeeCap != nil && msg.BlobGasFeeCap.BitLen() == 0 { vmenv.Context.BlobBaseFee = new(big.Int) } + vmenv.SetTxContext(core.NewEVMTxContext(msg)) res, err := core.ApplyMessage(vmenv, msg, new(core.GasPool).AddGas(msg.GasLimit)) if err != nil { return nil, 0, nil, fmt.Errorf("failed to apply transaction: %v err: %v", args.ToTransaction(types.LegacyTxType).Hash(), err) diff --git a/internal/ethapi/api_test.go b/internal/ethapi/api_test.go index f570c5dc4c..1f5f2dd1d5 100644 --- a/internal/ethapi/api_test.go +++ b/internal/ethapi/api_test.go @@ -570,16 +570,15 @@ func (b testBackend) GetTd(ctx context.Context, hash common.Hash) *big.Int { } return big.NewInt(1) } -func (b testBackend) GetEVM(ctx context.Context, msg *core.Message, state *state.StateDB, header *types.Header, vmConfig *vm.Config, blockContext *vm.BlockContext) *vm.EVM { +func (b testBackend) GetEVM(ctx context.Context, state *state.StateDB, header *types.Header, vmConfig *vm.Config, blockContext *vm.BlockContext) *vm.EVM { if vmConfig == nil { vmConfig = b.chain.GetVMConfig() } - txContext := core.NewEVMTxContext(msg) context := core.NewEVMBlockContext(header, b.chain, nil) if blockContext != nil { context = *blockContext } - return vm.NewEVM(context, txContext, state, b.chain.Config(), *vmConfig) + return vm.NewEVM(context, state, b.chain.Config(), *vmConfig) } func (b testBackend) SubscribeChainEvent(ch chan<- core.ChainEvent) event.Subscription { panic("implement me") diff --git a/internal/ethapi/backend.go b/internal/ethapi/backend.go index 82465ca7d7..3d8f7aa700 100644 --- a/internal/ethapi/backend.go +++ b/internal/ethapi/backend.go @@ -68,7 +68,7 @@ type Backend interface { StateAndHeaderByNumberOrHash(ctx context.Context, blockNrOrHash rpc.BlockNumberOrHash) (*state.StateDB, *types.Header, error) Pending() (*types.Block, types.Receipts, *state.StateDB) GetReceipts(ctx context.Context, hash common.Hash) (types.Receipts, error) - GetEVM(ctx context.Context, msg *core.Message, state *state.StateDB, header *types.Header, vmConfig *vm.Config, blockCtx *vm.BlockContext) *vm.EVM + GetEVM(ctx context.Context, state *state.StateDB, header *types.Header, vmConfig *vm.Config, blockCtx *vm.BlockContext) *vm.EVM SubscribeChainEvent(ch chan<- core.ChainEvent) event.Subscription SubscribeChainHeadEvent(ch chan<- core.ChainHeadEvent) event.Subscription diff --git a/internal/ethapi/simulate.go b/internal/ethapi/simulate.go index 81b4633d42..979de352fd 100644 --- a/internal/ethapi/simulate.go +++ b/internal/ethapi/simulate.go @@ -185,12 +185,12 @@ func (sim *simulator) processBlock(ctx context.Context, block *simBlock, header, NoBaseFee: !sim.validate, Tracer: tracer.Hooks(), } - evm = vm.NewEVM(blockContext, vm.TxContext{GasPrice: new(big.Int)}, sim.state, sim.chainConfig, *vmConfig) ) var tracingStateDB = vm.StateDB(sim.state) if hooks := tracer.Hooks(); hooks != nil { tracingStateDB = state.NewHookedState(sim.state, hooks) } + evm := vm.NewEVM(blockContext, tracingStateDB, sim.chainConfig, *vmConfig) // It is possible to override precompiles with EVM bytecode, or // move them to another address. if precompiles != nil { @@ -208,7 +208,7 @@ func (sim *simulator) processBlock(ctx context.Context, block *simBlock, header, tracer.reset(tx.Hash(), uint(i)) // EoA check is always skipped, even in validation mode. msg := call.ToMessage(header.BaseFee, !sim.validate, true) - evm.Reset(core.NewEVMTxContext(msg), tracingStateDB) + evm.SetTxContext(core.NewEVMTxContext(msg)) result, err := applyMessageWithEVM(ctx, evm, msg, timeout, sim.gp) if err != nil { txErr := txValidationError(err) diff --git a/internal/ethapi/transaction_args_test.go b/internal/ethapi/transaction_args_test.go index 5f59b491e1..7172fc883f 100644 --- a/internal/ethapi/transaction_args_test.go +++ b/internal/ethapi/transaction_args_test.go @@ -370,7 +370,7 @@ func (b *backendMock) GetLogs(ctx context.Context, blockHash common.Hash, number return nil, nil } func (b *backendMock) GetTd(ctx context.Context, hash common.Hash) *big.Int { return nil } -func (b *backendMock) GetEVM(ctx context.Context, msg *core.Message, state *state.StateDB, header *types.Header, vmConfig *vm.Config, blockCtx *vm.BlockContext) *vm.EVM { +func (b *backendMock) GetEVM(ctx context.Context, state *state.StateDB, header *types.Header, vmConfig *vm.Config, blockCtx *vm.BlockContext) *vm.EVM { return nil } func (b *backendMock) SubscribeChainEvent(ch chan<- core.ChainEvent) event.Subscription { return nil } diff --git a/miner/worker.go b/miner/worker.go index db2fac3871..51ac5e2e36 100644 --- a/miner/worker.go +++ b/miner/worker.go @@ -51,6 +51,7 @@ type environment struct { tcount int // tx count in cycle gasPool *core.GasPool // available gas used to pack transactions coinbase common.Address + evm *vm.EVM header *types.Header txs []*types.Transaction @@ -126,14 +127,11 @@ func (miner *Miner) generateWork(params *generateParams, witness bool) *newPaylo return &newPayloadResult{err: err} } requests = append(requests, depositRequests) - // create EVM for system calls - blockContext := core.NewEVMBlockContext(work.header, miner.chain, &work.header.Coinbase) - vmenv := vm.NewEVM(blockContext, vm.TxContext{}, work.state, miner.chainConfig, vm.Config{}) // EIP-7002 withdrawals - withdrawalRequests := core.ProcessWithdrawalQueue(vmenv, work.state) + withdrawalRequests := core.ProcessWithdrawalQueue(work.evm, work.state) requests = append(requests, withdrawalRequests) // EIP-7251 consolidations - consolidationRequests := core.ProcessConsolidationQueue(vmenv, work.state) + consolidationRequests := core.ProcessConsolidationQueue(work.evm, work.state) requests = append(requests, consolidationRequests) } if requests != nil { @@ -233,14 +231,10 @@ func (miner *Miner) prepareWork(genParams *generateParams, witness bool) (*envir return nil, err } if header.ParentBeaconRoot != nil { - context := core.NewEVMBlockContext(header, miner.chain, nil) - vmenv := vm.NewEVM(context, vm.TxContext{}, env.state, miner.chainConfig, vm.Config{}) - core.ProcessBeaconBlockRoot(*header.ParentBeaconRoot, vmenv, env.state) + core.ProcessBeaconBlockRoot(*header.ParentBeaconRoot, env.evm, env.state) } if miner.chainConfig.IsPrague(header.Number, header.Time) { - context := core.NewEVMBlockContext(header, miner.chain, nil) - vmenv := vm.NewEVM(context, vm.TxContext{}, env.state, miner.chainConfig, vm.Config{}) - core.ProcessParentBlockHash(header.ParentHash, vmenv, env.state) + core.ProcessParentBlockHash(header.ParentHash, env.evm, env.state) } return env, nil } @@ -266,6 +260,7 @@ func (miner *Miner) makeEnv(parent *types.Header, header *types.Header, coinbase coinbase: coinbase, header: header, witness: state.Witness(), + evm: vm.NewEVM(core.NewEVMBlockContext(header, miner.chain, &coinbase), state, miner.chainConfig, vm.Config{}), }, nil } @@ -314,7 +309,7 @@ func (miner *Miner) applyTransaction(env *environment, tx *types.Transaction) (* snap = env.state.Snapshot() gp = env.gasPool.Gas() ) - receipt, err := core.ApplyTransaction(miner.chainConfig, miner.chain, &env.coinbase, env.gasPool, env.state, env.header, tx, &env.header.GasUsed, vm.Config{}) + receipt, err := core.ApplyTransaction(miner.chainConfig, env.evm, env.gasPool, env.state, env.header, tx, &env.header.GasUsed) if err != nil { env.state.RevertToSnapshot(snap) env.gasPool.SetGas(gp) diff --git a/tests/state_test.go b/tests/state_test.go index 76d5a601c7..7b82b05e58 100644 --- a/tests/state_test.go +++ b/tests/state_test.go @@ -302,7 +302,8 @@ func runBenchmark(b *testing.B, t *StateTest) { context := core.NewEVMBlockContext(block.Header(), nil, &t.json.Env.Coinbase) context.GetHash = vmTestBlockHash context.BaseFee = baseFee - evm := vm.NewEVM(context, txContext, state.StateDB, config, vmconfig) + evm := vm.NewEVM(context, state.StateDB, config, vmconfig) + evm.SetTxContext(txContext) // Create "contract" for sender to cache code analysis. sender := vm.NewContract(vm.AccountRef(msg.From), vm.AccountRef(msg.From), diff --git a/tests/state_test_util.go b/tests/state_test_util.go index af2cb63d94..446ffb40d5 100644 --- a/tests/state_test_util.go +++ b/tests/state_test_util.go @@ -293,7 +293,8 @@ func (t *StateTest) RunNoVerify(subtest StateSubtest, vmconfig vm.Config, snapsh if config.IsCancun(new(big.Int), block.Time()) && t.json.Env.ExcessBlobGas != nil { context.BlobBaseFee = eip4844.CalcBlobFee(*t.json.Env.ExcessBlobGas) } - evm := vm.NewEVM(context, txContext, st.StateDB, config, vmconfig) + evm := vm.NewEVM(context, st.StateDB, config, vmconfig) + evm.SetTxContext(txContext) if tracer := vmconfig.Tracer; tracer != nil && tracer.OnTxStart != nil { tracer.OnTxStart(evm.GetVMContext(), nil, msg.From)