Skip to content

Commit

Permalink
core: switch EVM tx context in ApplyMessage (ethereum#30809)
Browse files Browse the repository at this point in the history
This change relocates the EVM tx context switching to the ApplyMessage function.
With this change, we can remove a lot of EVM.SetTxContext calls before
message execution.

### Tracing API changes

- This PR replaces the `GasPrice` field of the `VMContext` struct with
  `BaseFee`. Users may instead take the effective gas price from
  `tx.EffectiveGasTipValue(env.BaseFee)`.

---------

Co-authored-by: Sina Mahmoodi <[email protected]>
  • Loading branch information
rjl493456442 and s1na authored Nov 29, 2024
1 parent 03c37cd commit a793bc7
Show file tree
Hide file tree
Showing 20 changed files with 57 additions and 105 deletions.
7 changes: 2 additions & 5 deletions cmd/evm/internal/t8ntool/execution.go
Original file line number Diff line number Diff line change
Expand Up @@ -253,16 +253,13 @@ func (pre *Prestate) Apply(vmConfig vm.Config, chainConfig *params.ChainConfig,
statedb.SetTxContext(tx.Hash(), txIndex)

var (
txContext = core.NewEVMTxContext(msg)
snapshot = statedb.Snapshot()
prevGas = gaspool.Gas()
snapshot = statedb.Snapshot()
prevGas = gaspool.Gas()
)
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)
Expand Down
2 changes: 1 addition & 1 deletion core/block_validator.go
Original file line number Diff line number Diff line change
Expand Up @@ -93,7 +93,7 @@ func (v *BlockValidator) ValidateBody(block *types.Block) error {
}

// The individual checks for blob validity (version-check + not empty)
// happens in StateTransition.
// happens in state transition.
}

// Check blob gas usage.
Expand Down
16 changes: 4 additions & 12 deletions core/state_prefetcher.go
Original file line number Diff line number Diff line change
Expand Up @@ -65,7 +65,10 @@ 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, gaspool, evm); err != nil {

// We attempt to apply a transaction. The goal is not to execute
// the transaction successfully, rather to warm up touched data slots.
if _, err := ApplyMessage(evm, msg, gaspool); err != nil {
return // Ugh, something went horribly wrong, bail out
}
// If we're pre-byzantium, pre-load trie nodes for the intermediate root
Expand All @@ -78,14 +81,3 @@ func (p *statePrefetcher) Prefetch(block *types.Block, statedb *state.StateDB, c
statedb.IntermediateRoot(true)
}
}

// 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, gaspool *GasPool, evm *vm.EVM) error {
// Update the evm with the new transaction context.
evm.SetTxContext(NewEVMTxContext(msg))
// Add addresses to access list if applicable
_, err := ApplyMessage(evm, msg, gaspool)
return err
}
6 changes: 0 additions & 6 deletions core/state_processor.go
Original file line number Diff line number Diff line change
Expand Up @@ -140,17 +140,11 @@ func ApplyTransactionWithEVM(msg *Message, gp *GasPool, statedb *state.StateDB,
defer func() { hooks.OnTxEnd(receipt, err) }()
}
}

// Create a new context to be used in the EVM environment.
txContext := NewEVMTxContext(msg)
evm.SetTxContext(txContext)

// Apply the transaction to the current state (included in the env).
result, err := ApplyMessage(evm, msg, gp)
if err != nil {
return nil, err
}

// Update the state with pending changes.
var root []byte
if evm.ChainConfig().IsByzantium(blockNumber) {
Expand Down
29 changes: 15 additions & 14 deletions core/state_transition.go
Original file line number Diff line number Diff line change
Expand Up @@ -187,10 +187,11 @@ func TransactionToMessage(tx *types.Transaction, s types.Signer, baseFee *big.In
// indicates a core error meaning that the message would always fail for that particular
// state and would never be accepted within a block.
func ApplyMessage(evm *vm.EVM, msg *Message, gp *GasPool) (*ExecutionResult, error) {
return NewStateTransition(evm, msg, gp).TransitionDb()
evm.SetTxContext(NewEVMTxContext(msg))
return newStateTransition(evm, msg, gp).execute()
}

// StateTransition represents a state transition.
// stateTransition represents a state transition.
//
// == The State Transitioning Model
//
Expand All @@ -212,7 +213,7 @@ func ApplyMessage(evm *vm.EVM, msg *Message, gp *GasPool) (*ExecutionResult, err
//
// 5. Run Script section
// 6. Derive new state root
type StateTransition struct {
type stateTransition struct {
gp *GasPool
msg *Message
gasRemaining uint64
Expand All @@ -221,9 +222,9 @@ type StateTransition struct {
evm *vm.EVM
}

// NewStateTransition initialises and returns a new state transition object.
func NewStateTransition(evm *vm.EVM, msg *Message, gp *GasPool) *StateTransition {
return &StateTransition{
// newStateTransition initialises and returns a new state transition object.
func newStateTransition(evm *vm.EVM, msg *Message, gp *GasPool) *stateTransition {
return &stateTransition{
gp: gp,
evm: evm,
msg: msg,
Expand All @@ -232,14 +233,14 @@ func NewStateTransition(evm *vm.EVM, msg *Message, gp *GasPool) *StateTransition
}

// to returns the recipient of the message.
func (st *StateTransition) to() common.Address {
func (st *stateTransition) to() common.Address {
if st.msg == nil || st.msg.To == nil /* contract creation */ {
return common.Address{}
}
return *st.msg.To
}

func (st *StateTransition) buyGas() error {
func (st *stateTransition) buyGas() error {
mgval := new(big.Int).SetUint64(st.msg.GasLimit)
mgval.Mul(mgval, st.msg.GasPrice)
balanceCheck := new(big.Int).Set(mgval)
Expand Down Expand Up @@ -283,7 +284,7 @@ func (st *StateTransition) buyGas() error {
return nil
}

func (st *StateTransition) preCheck() error {
func (st *stateTransition) preCheck() error {
// Only check transactions that are not fake
msg := st.msg
if !msg.SkipNonceChecks {
Expand Down Expand Up @@ -368,7 +369,7 @@ func (st *StateTransition) preCheck() error {
return st.buyGas()
}

// TransitionDb will transition the state by applying the current message and
// execute will transition the state by applying the current message and
// returning the evm execution result with following fields.
//
// - used gas: total gas used (including gas being refunded)
Expand All @@ -378,7 +379,7 @@ func (st *StateTransition) preCheck() error {
//
// However if any consensus issue encountered, return the error directly with
// nil evm execution result.
func (st *StateTransition) TransitionDb() (*ExecutionResult, error) {
func (st *stateTransition) execute() (*ExecutionResult, error) {
// First check this message satisfies all consensus rules before
// applying the message. The rules include these clauses
//
Expand Down Expand Up @@ -493,7 +494,7 @@ func (st *StateTransition) TransitionDb() (*ExecutionResult, error) {
}, nil
}

func (st *StateTransition) refundGas(refundQuotient uint64) uint64 {
func (st *stateTransition) refundGas(refundQuotient uint64) uint64 {
// Apply refund counter, capped to a refund quotient
refund := st.gasUsed() / refundQuotient
if refund > st.state.GetRefund() {
Expand Down Expand Up @@ -523,11 +524,11 @@ func (st *StateTransition) refundGas(refundQuotient uint64) uint64 {
}

// gasUsed returns the amount of gas used up by the state transition.
func (st *StateTransition) gasUsed() uint64 {
func (st *stateTransition) gasUsed() uint64 {
return st.initialGas - st.gasRemaining
}

// blobGasUsed returns the amount of blob gas used by the message.
func (st *StateTransition) blobGasUsed() uint64 {
func (st *stateTransition) blobGasUsed() uint64 {
return uint64(len(st.msg.BlobHashes) * params.BlobTxBlobGasPerBlob)
}
5 changes: 2 additions & 3 deletions core/tracing/hooks.go
Original file line number Diff line number Diff line change
Expand Up @@ -55,9 +55,8 @@ type VMContext struct {
BlockNumber *big.Int
Time uint64
Random *common.Hash
// Effective tx gas price
GasPrice *big.Int
StateDB StateDB
BaseFee *big.Int
StateDB StateDB
}

// BlockEvent is emitted upon tracing an incoming block.
Expand Down
2 changes: 1 addition & 1 deletion core/vm/evm.go
Original file line number Diff line number Diff line change
Expand Up @@ -605,7 +605,7 @@ func (evm *EVM) GetVMContext() *tracing.VMContext {
BlockNumber: evm.Context.BlockNumber,
Time: evm.Context.Time,
Random: evm.Context.Random,
GasPrice: evm.TxContext.GasPrice,
BaseFee: evm.Context.BaseFee,
StateDB: evm.StateDB,
}
}
8 changes: 3 additions & 5 deletions eth/gasestimator/gasestimator.go
Original file line number Diff line number Diff line change
Expand Up @@ -217,21 +217,19 @@ func execute(ctx context.Context, call *core.Message, opts *Options, gasLimit ui
func run(ctx context.Context, call *core.Message, opts *Options) (*core.ExecutionResult, error) {
// Assemble the call and the call context
var (
msgContext = core.NewEVMTxContext(call)
evmContext = core.NewEVMBlockContext(opts.Header, opts.Chain, nil)

dirtyState = opts.State.Copy()
)
// Lower the basefee to 0 to avoid breaking EVM
// invariants (basefee < feecap).
if msgContext.GasPrice.Sign() == 0 {
if call.GasPrice.Sign() == 0 {
evmContext.BaseFee = new(big.Int)
}
if msgContext.BlobFeeCap != nil && msgContext.BlobFeeCap.BitLen() == 0 {
if call.BlobGasFeeCap != nil && call.BlobGasFeeCap.BitLen() == 0 {
evmContext.BlobBaseFee = new(big.Int)
}
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.
Expand Down
2 changes: 0 additions & 2 deletions eth/state_accessor.go
Original file line number Diff line number Diff line change
Expand Up @@ -255,8 +255,6 @@ func (eth *Ethereum) stateAtTransaction(ctx context.Context, block *types.Block,
}
// 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
statedb.SetTxContext(tx.Hash(), idx)
Expand Down
20 changes: 6 additions & 14 deletions eth/tracers/api.go
Original file line number Diff line number Diff line change
Expand Up @@ -546,11 +546,7 @@ func (api *API) IntermediateRoots(ctx context.Context, hash common.Hash, config
if err := ctx.Err(); err != nil {
return nil, err
}
var (
msg, _ = core.TransactionToMessage(tx, signer, block.BaseFee())
txContext = core.NewEVMTxContext(msg)
)
evm.SetTxContext(txContext)
msg, _ := core.TransactionToMessage(tx, signer, block.BaseFee())
statedb.SetTxContext(tx.Hash(), i)
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)
Expand Down Expand Up @@ -708,7 +704,6 @@ txloop:
// Generate the next state snapshot fast without tracing
msg, _ := core.TransactionToMessage(tx, signer, block.BaseFee())
statedb.SetTxContext(tx.Hash(), i)
evm.SetTxContext(core.NewEVMTxContext(msg))
if _, err := core.ApplyMessage(evm, msg, new(core.GasPool).AddGas(msg.GasLimit)); err != nil {
failed = err
break txloop
Expand Down Expand Up @@ -792,12 +787,11 @@ func (api *API) standardTraceBlockToFile(ctx context.Context, block *types.Block
for i, tx := range block.Transactions() {
// Prepare the transaction for un-traced execution
var (
msg, _ = core.TransactionToMessage(tx, signer, block.BaseFee())
txContext = core.NewEVMTxContext(msg)
vmConf vm.Config
dump *os.File
writer *bufio.Writer
err error
msg, _ = core.TransactionToMessage(tx, signer, block.BaseFee())
vmConf vm.Config
dump *os.File
writer *bufio.Writer
err error
)
// If the transaction needs tracing, swap out the configs
if tx.Hash() == txHash || txHash == (common.Hash{}) {
Expand All @@ -820,7 +814,6 @@ func (api *API) standardTraceBlockToFile(ctx context.Context, block *types.Block
}
}
// Execute the transaction and flush any traces to disk
evm.SetTxContext(txContext)
statedb.SetTxContext(tx.Hash(), i)
if vmConf.Tracer.OnTxStart != nil {
vmConf.Tracer.OnTxStart(evm.GetVMContext(), tx, msg.From)
Expand Down Expand Up @@ -1016,7 +1009,6 @@ func (api *API) traceTx(ctx context.Context, tx *types.Transaction, message *cor
}
// The actual TxContext will be created as part of ApplyTransactionWithEVM.
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 {
Expand Down
2 changes: 0 additions & 2 deletions eth/tracers/api_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -177,8 +177,6 @@ func (b *testBackend) StateAtTransaction(ctx context.Context, block *types.Block
return tx, context, statedb, release, 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)
}
Expand Down
33 changes: 14 additions & 19 deletions eth/tracers/internal/tracetest/calltrace_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -133,7 +133,6 @@ func testCallTracer(tracerName string, dirPath string, t *testing.T) {
t.Fatalf("failed to prepare transaction for tracing: %v", err)
}
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 {
Expand Down Expand Up @@ -206,11 +205,6 @@ func benchTracer(tracerName string, test *callTracerTest, b *testing.B) {
b.Fatalf("failed to parse testcase input: %v", err)
}
signer := types.MakeSigner(test.Genesis.Config, new(big.Int).SetUint64(uint64(test.Context.Number)), uint64(test.Context.Time))
origin, _ := signer.Sender(tx)
txContext := vm.TxContext{
Origin: origin,
GasPrice: tx.GasPrice(),
}
context := test.Context.toBlockContext(test.Genesis)
msg, err := core.TransactionToMessage(tx, signer, context.BaseFee)
if err != nil {
Expand All @@ -222,19 +216,25 @@ 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)
evm := vm.NewEVM(context, state.StateDB, test.Genesis.Config, vm.Config{})

for i := 0; i < b.N; i++ {
snap := state.StateDB.Snapshot()
st := core.NewStateTransition(evm, msg, new(core.GasPool).AddGas(tx.Gas()))
if _, err = st.TransitionDb(); err != nil {
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.Config.Tracer = tracer.Hooks
if tracer.OnTxStart != nil {
tracer.OnTxStart(evm.GetVMContext(), tx, msg.From)
}
_, err = core.ApplyMessage(evm, msg, new(core.GasPool).AddGas(tx.Gas()))
if err != nil {
b.Fatalf("failed to execute transaction: %v", err)
}
if tracer.OnTxEnd != nil {
tracer.OnTxEnd(&types.Receipt{GasUsed: tx.Gas()}, nil)
}
if _, err = tracer.GetResult(); err != nil {
b.Fatal(err)
}
Expand Down Expand Up @@ -372,12 +372,7 @@ func TestInternals(t *testing.T) {
if err != nil {
t.Fatalf("test %v: failed to sign transaction: %v", tc.name, err)
}
txContext := vm.TxContext{
Origin: origin,
GasPrice: tx.GasPrice(),
}
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)
Expand Down
1 change: 0 additions & 1 deletion eth/tracers/internal/tracetest/flat_calltrace_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -99,7 +99,6 @@ func flatCallTracerTestRunner(tracerName string, filename string, dirPath string
return fmt.Errorf("failed to prepare transaction for tracing: %v", err)
}
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 {
Expand Down
1 change: 0 additions & 1 deletion eth/tracers/internal/tracetest/prestate_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -107,7 +107,6 @@ func testPrestateDiffTracer(tracerName string, dirPath string, t *testing.T) {
t.Fatalf("failed to prepare transaction for tracing: %v", err)
}
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 {
Expand Down
2 changes: 1 addition & 1 deletion eth/tracers/js/goja.go
Original file line number Diff line number Diff line change
Expand Up @@ -260,7 +260,7 @@ func (t *jsTracer) OnTxStart(env *tracing.VMContext, tx *types.Transaction, from
t.activePrecompiles = vm.ActivePrecompiles(rules)
t.ctx["block"] = t.vm.ToValue(t.env.BlockNumber.Uint64())
t.ctx["gas"] = t.vm.ToValue(tx.Gas())
gasPriceBig, err := t.toBig(t.vm, env.GasPrice.String())
gasPriceBig, err := t.toBig(t.vm, tx.EffectiveGasTipValue(env.BaseFee).String())
if err != nil {
t.err = err
return
Expand Down
Loading

0 comments on commit a793bc7

Please sign in to comment.