evm: refactor dup state transition code (#674)

* Problem: state transition code is duplicated

Closes: #672

Solution:
- move gas refund out from ApplyMessage
- move check into ApplyMessage
- move evm construction into ApplyMessage
- ensure context stack is clean after ApplyMessage return

fix unit tests

undo rename

add underflow check

* improve performance

- don't duplicate params loading
- passing EVMConfig around as pointer
This commit is contained in:
yihuang 2021-10-23 01:21:03 +08:00 committed by GitHub
parent b1aedf9a2a
commit 1fe07edbf9
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
10 changed files with 196 additions and 189 deletions

View File

@ -59,6 +59,8 @@ Ref: https://keepachangelog.com/en/1.0.0/
* (rpc) [tharsis#679](https://github.com/tharsis/ethermint/pull/679) Fix file close handle.
* (rpc) [tharsis#671](https://github.com/tharsis/ethermint/pull/671) Don't pass base fee externally for `EthCall`/`EthEstimateGas` apis.
* (evm) [tharsis#674](https://github.com/tharsis/ethermint/pull/674) Refactor `ApplyMessage`, remove
`ApplyNativeMessage`.
## [v0.7.1] - 2021-10-08

View File

@ -18,7 +18,6 @@ import (
"github.com/ethereum/go-ethereum/core"
ethtypes "github.com/ethereum/go-ethereum/core/types"
"github.com/ethereum/go-ethereum/core/vm"
"github.com/ethereum/go-ethereum/params"
)
// EVMKeeper defines the expected keeper interface used on the Eth AnteHandler
@ -29,7 +28,7 @@ type EVMKeeper interface {
GetParams(ctx sdk.Context) evmtypes.Params
WithContext(ctx sdk.Context)
ResetRefundTransient(ctx sdk.Context)
NewEVM(msg core.Message, config *params.ChainConfig, params evmtypes.Params, coinbase common.Address, baseFee *big.Int, tracer vm.Tracer) *vm.EVM
NewEVM(msg core.Message, cfg *evmtypes.EVMConfig, tracer vm.Tracer) *vm.EVM
GetCodeHash(addr common.Address) common.Hash
DeductTxCostsFromUserBalance(
ctx sdk.Context, msgEthTx evmtypes.MsgEthereumTx, txData evmtypes.TxData, denom string, homestead, istanbul, london bool,
@ -372,7 +371,13 @@ func (ctd CanTransferDecorator) AnteHandle(ctx sdk.Context, tx sdk.Tx, simulate
}
// NOTE: pass in an empty coinbase address and nil tracer as we don't need them for the check below
evm := ctd.evmKeeper.NewEVM(coreMsg, ethCfg, params, common.Address{}, baseFee, evmtypes.NewNoOpTracer())
cfg := &evmtypes.EVMConfig{
ChainConfig: ethCfg,
Params: params,
CoinBase: common.Address{},
BaseFee: baseFee,
}
evm := ctd.evmKeeper.NewEVM(coreMsg, cfg, evmtypes.NewNoOpTracer())
// check that caller has enough balance to cover asset transfer for **topmost** call
// NOTE: here the gas consumed is from the context with the infinite gas meter

View File

@ -177,7 +177,7 @@ func (f *Filter) blockLogs(header *ethtypes.Header) ([]*ethtypes.Log, error) {
return []*ethtypes.Log{}, errors.Wrapf(err, "failed to fetch logs block number %d", header.Number.Int64())
}
var unfiltered []*ethtypes.Log
unfiltered := make([]*ethtypes.Log, 0)
for _, logs := range logsList {
unfiltered = append(unfiltered, logs...)
}

View File

@ -234,18 +234,7 @@ func (k Keeper) EthCall(c context.Context, req *types.EthCallRequest) (*types.Ms
return nil, status.Error(codes.InvalidArgument, err.Error())
}
coinbase, err := k.GetCoinbaseAddress(ctx)
if err != nil {
return nil, status.Error(codes.Internal, err.Error())
}
tracer := types.NewTracer(k.tracer, msg, ethCfg, ctx.BlockHeight(), k.debug)
evm := k.NewEVM(msg, ethCfg, params, coinbase, baseFee, tracer)
// pass true means execute in query mode, which don't do actual gas refund.
res, err := k.ApplyMessage(evm, msg, ethCfg, true)
k.ctxStack.RevertAll()
res, err := k.ApplyMessage(msg, nil, false)
if err != nil {
return nil, status.Error(codes.Internal, err.Error())
}
@ -300,16 +289,12 @@ func (k Keeper) EstimateGas(c context.Context, req *types.EthCallRequest) (*type
}
cap = hi
params := k.GetParams(ctx)
ethCfg := params.ChainConfig.EthereumConfig(k.eip155ChainID)
coinbase, err := k.GetCoinbaseAddress(ctx)
cfg, err := k.EVMConfig(ctx)
if err != nil {
return nil, status.Error(codes.Internal, err.Error())
return nil, status.Error(codes.Internal, "failed to load evm config")
}
var baseFee *big.Int
if types.IsLondon(ethCfg, ctx.BlockHeight()) {
if types.IsLondon(cfg.ChainConfig, ctx.BlockHeight()) {
baseFee = k.feeMarketKeeper.GetBaseFee(ctx)
}
@ -325,14 +310,7 @@ func (k Keeper) EstimateGas(c context.Context, req *types.EthCallRequest) (*type
return false, nil, err
}
tracer := types.NewTracer(k.tracer, msg, ethCfg, k.Ctx().BlockHeight(), k.debug)
evm := k.NewEVM(msg, ethCfg, params, coinbase, baseFee, tracer)
// pass true means execute in query mode, which don't do actual gas refund.
rsp, err = k.ApplyMessage(evm, msg, ethCfg, true)
k.ctxStack.RevertAll()
rsp, err = k.ApplyMessageWithConfig(msg, nil, false, cfg)
if err != nil {
if errors.Is(stacktrace.RootCause(err), core.ErrIntrinsicGas) {
@ -384,18 +362,13 @@ func (k Keeper) TraceTx(c context.Context, req *types.QueryTraceTxRequest) (*typ
ctx := sdk.UnwrapSDKContext(c)
k.WithContext(ctx)
coinbase, err := k.GetCoinbaseAddress(ctx)
if err != nil {
return nil, status.Error(codes.Internal, err.Error())
}
params := k.GetParams(ctx)
ethCfg := params.ChainConfig.EthereumConfig(k.eip155ChainID)
signer := ethtypes.MakeSigner(ethCfg, big.NewInt(ctx.BlockHeight()))
tx := req.Msg.AsTransaction()
baseFee := k.feeMarketKeeper.GetBaseFee(ctx)
result, err := k.traceTx(ctx, coinbase, signer, req.TxIndex, params, ethCfg, tx, baseFee, req.TraceConfig)
result, err := k.traceTx(ctx, signer, req.TxIndex, ethCfg, tx, baseFee, req.TraceConfig)
if err != nil {
return nil, err
}
@ -412,10 +385,8 @@ func (k Keeper) TraceTx(c context.Context, req *types.QueryTraceTxRequest) (*typ
func (k *Keeper) traceTx(
ctx sdk.Context,
coinbase common.Address,
signer ethtypes.Signer,
txIndex uint64,
params types.Params,
ethCfg *ethparams.ChainConfig,
tx *ethtypes.Transaction,
baseFee *big.Int,
@ -488,12 +459,10 @@ func (k *Keeper) traceTx(
tracer = types.NewTracer(types.TracerStruct, msg, ethCfg, ctx.BlockHeight(), true)
}
evm := k.NewEVM(msg, ethCfg, params, coinbase, baseFee, tracer)
k.SetTxHashTransient(txHash)
k.SetTxIndexTransient(txIndex)
res, err := k.ApplyMessage(evm, msg, ethCfg, true)
res, err := k.ApplyMessage(msg, tracer, false)
if err != nil {
return nil, status.Error(codes.Internal, err.Error())
}

View File

@ -9,10 +9,13 @@ import (
sdk "github.com/cosmos/cosmos-sdk/types"
paramtypes "github.com/cosmos/cosmos-sdk/x/params/types"
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/core"
ethtypes "github.com/ethereum/go-ethereum/core/types"
"github.com/ethereum/go-ethereum/core/vm"
"github.com/palantir/stacktrace"
"github.com/tendermint/tendermint/libs/log"
"github.com/ethereum/go-ethereum/params"
ethermint "github.com/tharsis/ethermint/types"
"github.com/tharsis/ethermint/x/evm/types"
)
@ -377,3 +380,8 @@ func (k *Keeper) PostTxProcessing(txHash common.Hash, logs []*ethtypes.Log) erro
}
return k.hooks.PostTxProcessing(k.Ctx(), txHash, logs)
}
// Tracer return a default vm.Tracer based on current keeper state
func (k Keeper) Tracer(msg core.Message, ethCfg *params.ChainConfig) vm.Tracer {
return types.NewTracer(k.tracer, msg, ethCfg, k.Ctx().BlockHeight(), k.debug)
}

View File

@ -21,34 +21,57 @@ import (
"github.com/ethereum/go-ethereum/params"
)
// EVMConfig creates the EVMConfig based on current state
func (k *Keeper) EVMConfig(ctx sdk.Context) (*types.EVMConfig, error) {
params := k.GetParams(ctx)
ethCfg := params.ChainConfig.EthereumConfig(k.eip155ChainID)
// get the coinbase address from the block proposer
coinbase, err := k.GetCoinbaseAddress(ctx)
if err != nil {
return nil, stacktrace.Propagate(err, "failed to obtain coinbase address")
}
var baseFee *big.Int
if types.IsLondon(ethCfg, ctx.BlockHeight()) {
baseFee = k.feeMarketKeeper.GetBaseFee(ctx)
}
return &types.EVMConfig{
Params: params,
ChainConfig: ethCfg,
CoinBase: coinbase,
BaseFee: baseFee,
}, nil
}
// NewEVM generates a go-ethereum VM from the provided Message fields and the chain parameters
// (ChainConfig and module Params). It additionally sets the validator operator address as the
// coinbase address to make it available for the COINBASE opcode, even though there is no
// beneficiary of the coinbase transaction (since we're not mining).
func (k *Keeper) NewEVM(
msg core.Message,
config *params.ChainConfig,
params types.Params,
coinbase common.Address,
baseFee *big.Int,
cfg *types.EVMConfig,
tracer vm.Tracer,
) *vm.EVM {
blockCtx := vm.BlockContext{
CanTransfer: core.CanTransfer,
Transfer: core.Transfer,
GetHash: k.GetHashFn(),
Coinbase: coinbase,
Coinbase: cfg.CoinBase,
GasLimit: ethermint.BlockGasLimit(k.Ctx()),
BlockNumber: big.NewInt(k.Ctx().BlockHeight()),
Time: big.NewInt(k.Ctx().BlockHeader().Time.Unix()),
Difficulty: big.NewInt(0), // unused. Only required in PoW context
BaseFee: baseFee,
BaseFee: cfg.BaseFee,
}
txCtx := core.NewEVMTxContext(msg)
vmConfig := k.VMConfig(msg, params, tracer)
return vm.NewEVM(blockCtx, txCtx, k, config, vmConfig)
if tracer == nil {
tracer = k.Tracer(msg, cfg.ChainConfig)
}
vmConfig := k.VMConfig(msg, cfg.Params, tracer)
return vm.NewEVM(blockCtx, txCtx, k, cfg.ChainConfig, vmConfig)
}
// VMConfig creates an EVM configuration from the debug setting and the extra EIPs enabled on the
@ -142,25 +165,20 @@ func (k Keeper) GetHashFn() vm.GetHashFunc {
// For relevant discussion see: https://github.com/cosmos/cosmos-sdk/discussions/9072
func (k *Keeper) ApplyTransaction(tx *ethtypes.Transaction) (*types.MsgEthereumTxResponse, error) {
ctx := k.Ctx()
params := k.GetParams(ctx)
// ensure keeper state error is cleared
defer k.ClearStateError()
// return error if contract creation or call are disabled through governance
if !params.EnableCreate && tx.To() == nil {
return nil, stacktrace.Propagate(types.ErrCreateDisabled, "failed to create new contract")
} else if !params.EnableCall && tx.To() != nil {
return nil, stacktrace.Propagate(types.ErrCallDisabled, "failed to call contract")
cfg, err := k.EVMConfig(ctx)
if err != nil {
return nil, stacktrace.Propagate(err, "failed to load evm config")
}
ethCfg := params.ChainConfig.EthereumConfig(k.eip155ChainID)
// get the latest signer according to the chain rules from the config
signer := ethtypes.MakeSigner(ethCfg, big.NewInt(ctx.BlockHeight()))
signer := ethtypes.MakeSigner(cfg.ChainConfig, big.NewInt(ctx.BlockHeight()))
var baseFee *big.Int
if types.IsLondon(ethCfg, ctx.BlockHeight()) {
if types.IsLondon(cfg.ChainConfig, ctx.BlockHeight()) {
baseFee = k.feeMarketKeeper.GetBaseFee(ctx)
}
@ -169,67 +187,52 @@ func (k *Keeper) ApplyTransaction(tx *ethtypes.Transaction) (*types.MsgEthereumT
return nil, stacktrace.Propagate(err, "failed to return ethereum transaction as core message")
}
// get the coinbase address from the block proposer
coinbase, err := k.GetCoinbaseAddress(ctx)
if err != nil {
return nil, stacktrace.Propagate(err, "failed to obtain coinbase address")
}
// create an ethereum EVM instance and run the message
tracer := types.NewTracer(k.tracer, msg, ethCfg, ctx.BlockHeight(), k.debug)
evm := k.NewEVM(msg, ethCfg, params, coinbase, baseFee, tracer)
txHash := tx.Hash()
// set the transaction hash and index to the impermanent (transient) block state so that it's also
// available on the StateDB functions (eg: AddLog)
k.SetTxHashTransient(txHash)
if !k.ctxStack.IsEmpty() {
panic("context stack shouldn't be dirty before apply message")
}
// revision is -1 for empty stack
revision := -1
// snapshot to contain the tx processing and post processing in same scope
var commit func()
if k.hooks != nil {
// snapshot to contain the tx processing and post processing in same scope
revision = k.Snapshot()
// Create a cache context to revert state when tx hooks fails,
// the cache context is only committed when both tx and hooks executed successfully.
// Didn't use `Snapshot` because the context stack has exponential complexity on certain operations,
// thus restricted to be used only inside `ApplyMessage`.
var cacheCtx sdk.Context
cacheCtx, commit = ctx.CacheContext()
k.WithContext(cacheCtx)
defer (func() {
k.WithContext(ctx)
})()
}
// pass false to execute in real mode, which do actual gas refunding
res, err := k.ApplyMessage(evm, msg, ethCfg, false)
res, err := k.ApplyMessageWithConfig(msg, nil, true, cfg)
if err != nil {
return nil, stacktrace.Propagate(err, "failed to apply ethereum core message")
}
res.Hash = txHash.Hex()
// The state is reverted (i.e `RevertToSnapshot`) for the VM error cases, so it's safe to call the commit here.
// NOTE: revision is >= 0 only when the EVM hooks are not empty
if revision >= 0 {
// Flatten the cache contexts to improve the efficiency of following DB operations.
// Only commit the cache layers created by the EVM contract execution
// FIXME: some operations under deep context stack are extremely slow,
// see `benchmark_test.go:BenchmarkDeepContextStack13`.
if err = k.ctxStack.CommitToRevision(revision); err != nil {
return nil, stacktrace.Propagate(err, "failed to commit ethereum core message")
}
} else {
// All cache layers are created by the EVM contract execution. So it is safe to commit them all
k.CommitCachedContexts()
// refund gas prior to handling the vm error in order to match the Ethereum gas consumption instead of the default SDK one.
err = k.RefundGas(msg, msg.Gas()-res.GasUsed, cfg.Params.EvmDenom)
if err != nil {
return nil, stacktrace.Propagate(err, "failed to refund gas leftover gas to sender %s", msg.From())
}
res.Hash = txHash.Hex()
logs := k.GetTxLogsTransient(txHash)
if !res.Failed() {
// Only call hooks if tx executed successfully.
if err = k.PostTxProcessing(txHash, logs); err != nil {
// If hooks return error, revert the whole tx.
k.RevertToSnapshot(revision)
res.VmError = types.ErrPostTxProcessing.Error()
k.Logger(ctx).Error("tx post processing failed", "error", err)
} else {
// PostTxProcessing is successful, commit the leftover contexts
k.CommitCachedContexts()
k.Logger(k.Ctx()).Error("tx post processing failed", "error", err)
} else if commit != nil {
// PostTxProcessing is successful, commit the cache context
commit()
ctx.EventManager().EmitEvents(k.Ctx().EventManager().Events())
}
}
@ -248,14 +251,21 @@ func (k *Keeper) ApplyTransaction(tx *ethtypes.Transaction) (*types.MsgEthereumT
return res, nil
}
// ApplyMessage computes the new state by applying the given message against the existing state.
// ApplyMessageWithConfig computes the new state by applying the given message against the existing state.
// If the message fails, the VM execution error with the reason will be returned to the client
// and the transaction won't be committed to the store.
//
// Reverted state
//
// The transaction is never "reverted" since there is no snapshot + rollback performed on the StateDB.
// Only successful transactions are written to the store during DeliverTx mode.
// The snapshot and rollback are supported by the `ContextStack`, which should be only used inside `ApplyMessage`,
// because some operations has exponential computational complexity with deep stack.
//
// Different Callers
//
// It's called in three scenarios:
// 1. `ApplyTransaction`, in the transaction processing flow.
// 2. `EthCall/EthEstimateGas` grpc query handler.
// 3. Called by other native modules directly.
//
// Prechecks and Preprocessing
//
@ -273,24 +283,40 @@ func (k *Keeper) ApplyTransaction(tx *ethtypes.Transaction) (*types.MsgEthereumT
//
// 1. set up the initial access list (iff fork > Berlin)
//
// Query mode
// Tracer parameter
//
// The gRPC query endpoint from 'eth_call' calls this method in query mode, and since the query handler don't call AnteHandler,
// so we don't do real gas refund in that case.
func (k *Keeper) ApplyMessage(evm *vm.EVM, msg core.Message, cfg *params.ChainConfig, query bool) (*types.MsgEthereumTxResponse, error) {
// It should be a `vm.Tracer` object or nil, if pass `nil`, it'll create a default one based on keeper options.
//
// Commit parameter
//
// If commit is true, the cache context stack will be committed, otherwise discarded.
func (k *Keeper) ApplyMessageWithConfig(msg core.Message, tracer vm.Tracer, commit bool, cfg *types.EVMConfig) (*types.MsgEthereumTxResponse, error) {
var (
ret []byte // return bytes from evm execution
vmErr error // vm errors do not effect consensus and are therefore not assigned to err
)
if !k.ctxStack.IsEmpty() {
panic("context stack shouldn't be dirty before apply message")
}
evm := k.NewEVM(msg, cfg, tracer)
// ensure keeper state error is cleared
defer k.ClearStateError()
// return error if contract creation or call are disabled through governance
if !cfg.Params.EnableCreate && msg.To() == nil {
return nil, stacktrace.Propagate(types.ErrCreateDisabled, "failed to create new contract")
} else if !cfg.Params.EnableCall && msg.To() != nil {
return nil, stacktrace.Propagate(types.ErrCallDisabled, "failed to call contract")
}
sender := vm.AccountRef(msg.From())
contractCreation := msg.To() == nil
isLondon := cfg.IsLondon(evm.Context.BlockNumber)
isLondon := cfg.ChainConfig.IsLondon(evm.Context.BlockNumber)
intrinsicGas, err := k.GetEthIntrinsicGas(msg, cfg, contractCreation)
intrinsicGas, err := k.GetEthIntrinsicGas(msg, cfg.ChainConfig, contractCreation)
if err != nil {
// should have already been checked on Ante Handler
return nil, stacktrace.Propagate(err, "intrinsic gas failed")
@ -304,7 +330,7 @@ func (k *Keeper) ApplyMessage(evm *vm.EVM, msg core.Message, cfg *params.ChainCo
// access list preparaion is moved from ante handler to here, because it's needed when `ApplyMessage` is called
// under contexts where ante handlers are not run, for example `eth_call` and `eth_estimateGas`.
if rules := cfg.Rules(big.NewInt(k.Ctx().BlockHeight())); rules.IsBerlin {
if rules := cfg.ChainConfig.Rules(big.NewInt(k.Ctx().BlockHeight())); rules.IsBerlin {
k.PrepareAccessList(msg.From(), msg.To(), vm.ActivePrecompiles(rules), msg.AccessList())
}
@ -321,19 +347,16 @@ func (k *Keeper) ApplyMessage(evm *vm.EVM, msg core.Message, cfg *params.ChainCo
refundQuotient = params.RefundQuotientEIP3529
}
if query {
// gRPC query handlers don't go through the AnteHandler to deduct the gas fee from the sender or have access historical state.
// We don't refund gas to the sender.
// For more info, see: https://github.com/tharsis/ethermint/issues/229 and https://github.com/cosmos/cosmos-sdk/issues/9636
gasConsumed := msg.Gas() - leftoverGas
leftoverGas += k.GasToRefund(gasConsumed, refundQuotient)
} else {
// refund gas prior to handling the vm error in order to match the Ethereum gas consumption instead of the default SDK one.
leftoverGas, err = k.RefundGas(msg, leftoverGas, refundQuotient)
if err != nil {
return nil, stacktrace.Propagate(err, "failed to refund gas leftover gas to sender %s", msg.From())
}
// calculate gas refund
if msg.Gas() < leftoverGas {
return nil, stacktrace.Propagate(types.ErrGasOverflow, "apply message")
}
gasUsed := msg.Gas() - leftoverGas
refund := k.GasToRefund(gasUsed, refundQuotient)
if refund > gasUsed {
return nil, stacktrace.Propagate(types.ErrGasOverflow, "apply message")
}
gasUsed -= refund
// EVM execution error needs to be available for the JSON-RPC client
var vmError string
@ -341,7 +364,14 @@ func (k *Keeper) ApplyMessage(evm *vm.EVM, msg core.Message, cfg *params.ChainCo
vmError = vmErr.Error()
}
gasUsed := msg.Gas() - leftoverGas
// The context stack is designed specifically for `StateDB` interface, it should only be used in `ApplyMessage`,
// after return, the stack should be clean, the cached states are either committed or discarded.
if commit {
k.CommitCachedContexts()
} else {
k.ctxStack.RevertAll()
}
return &types.MsgEthereumTxResponse{
GasUsed: gasUsed,
VmError: vmError,
@ -349,42 +379,13 @@ func (k *Keeper) ApplyMessage(evm *vm.EVM, msg core.Message, cfg *params.ChainCo
}, nil
}
// ApplyNativeMessage executes an ethereum message on the EVM. It is meant to be called from an internal
// native Cosmos SDK module.
func (k *Keeper) ApplyNativeMessage(msg core.Message) (*types.MsgEthereumTxResponse, error) {
// TODO: clean up and remove duplicate code.
ctx := k.Ctx()
params := k.GetParams(ctx)
// return error if contract creation or call are disabled through governance
if !params.EnableCreate && msg.To() == nil {
return nil, stacktrace.Propagate(types.ErrCreateDisabled, "failed to create new contract")
} else if !params.EnableCall && msg.To() != nil {
return nil, stacktrace.Propagate(types.ErrCallDisabled, "failed to call contract")
}
ethCfg := params.ChainConfig.EthereumConfig(k.eip155ChainID)
// get the coinbase address from the block proposer
coinbase, err := k.GetCoinbaseAddress(ctx)
// ApplyMessage calls ApplyMessageWithConfig with default EVMConfig
func (k *Keeper) ApplyMessage(msg core.Message, tracer vm.Tracer, commit bool) (*types.MsgEthereumTxResponse, error) {
cfg, err := k.EVMConfig(k.Ctx())
if err != nil {
return nil, stacktrace.Propagate(err, "failed to obtain coinbase address")
return nil, stacktrace.Propagate(err, "failed to load evm config")
}
var baseFee *big.Int
if types.IsLondon(ethCfg, ctx.BlockHeight()) {
baseFee = k.feeMarketKeeper.GetBaseFee(ctx)
}
tracer := types.NewTracer(k.tracer, msg, ethCfg, ctx.BlockHeight(), k.debug)
evm := k.NewEVM(msg, ethCfg, params, coinbase, baseFee, tracer)
ret, err := k.ApplyMessage(evm, msg, ethCfg, true)
if err != nil {
return nil, err
}
k.CommitCachedContexts()
return ret, nil
return k.ApplyMessageWithConfig(msg, tracer, commit, cfg)
}
// GetEthIntrinsicGas returns the intrinsic gas cost for the transaction
@ -413,46 +414,30 @@ func (k *Keeper) GasToRefund(gasConsumed, refundQuotient uint64) uint64 {
// consumed in the transaction. Additionally, the function sets the total gas consumed to the value
// returned by the EVM execution, thus ignoring the previous intrinsic gas consumed during in the
// AnteHandler.
// NOTE: DO NOT pass 0 to refundQuotient
func (k *Keeper) RefundGas(msg core.Message, leftoverGas, refundQuotient uint64) (uint64, error) {
// safety check: leftover gas after execution should never exceed the gas limit defined on the message
if leftoverGas > msg.Gas() {
return leftoverGas, stacktrace.Propagate(
sdkerrors.Wrapf(types.ErrInconsistentGas, "leftover gas cannot be greater than gas limit (%d > %d)", leftoverGas, msg.Gas()),
"failed to update gas consumed after refund of leftover gas",
)
}
gasConsumed := msg.Gas() - leftoverGas
// calculate available gas to refund and add it to the leftover gas amount
refund := k.GasToRefund(gasConsumed, refundQuotient)
leftoverGas += refund
func (k *Keeper) RefundGas(msg core.Message, leftoverGas uint64, denom string) error {
// Return EVM tokens for remaining gas, exchanged at the original rate.
remaining := new(big.Int).Mul(new(big.Int).SetUint64(leftoverGas), msg.GasPrice())
switch remaining.Sign() {
case -1:
// negative refund errors
return leftoverGas, sdkerrors.Wrapf(types.ErrInvalidRefund, "refunded amount value cannot be negative %d", remaining.Int64())
return sdkerrors.Wrapf(types.ErrInvalidRefund, "refunded amount value cannot be negative %d", remaining.Int64())
case 1:
// positive amount refund
params := k.GetParams(k.Ctx())
refundedCoins := sdk.Coins{sdk.NewCoin(params.EvmDenom, sdk.NewIntFromBigInt(remaining))}
refundedCoins := sdk.Coins{sdk.NewCoin(denom, sdk.NewIntFromBigInt(remaining))}
// refund to sender from the fee collector module account, which is the escrow account in charge of collecting tx fees
err := k.bankKeeper.SendCoinsFromModuleToAccount(k.Ctx(), authtypes.FeeCollectorName, msg.From().Bytes(), refundedCoins)
if err != nil {
err = sdkerrors.Wrapf(sdkerrors.ErrInsufficientFunds, "fee collector account failed to refund fees: %s", err.Error())
return leftoverGas, stacktrace.Propagate(err, "failed to refund %d leftover gas (%s)", leftoverGas, refundedCoins.String())
return stacktrace.Propagate(err, "failed to refund %d leftover gas (%s)", leftoverGas, refundedCoins.String())
}
default:
// no refund, consume gas and update the tx gas meter
}
return leftoverGas, nil
return nil
}
// ResetGasMeterAndConsumeGas reset first the gas meter consumed value to zero and set it back to the new value

View File

@ -222,7 +222,7 @@ func BenchmarkApplyTransactionWithDynamicFeeTx(b *testing.B) {
}
}
func BenchmarkApplyNativeMessage(b *testing.B) {
func BenchmarkApplyMessage(b *testing.B) {
suite := KeeperTestSuite{}
suite.DoSetupTest(b)
@ -249,7 +249,7 @@ func BenchmarkApplyNativeMessage(b *testing.B) {
require.NoError(b, err)
b.StartTimer()
resp, err := suite.app.EvmKeeper.ApplyNativeMessage(m)
resp, err := suite.app.EvmKeeper.ApplyMessage(m, nil, true)
b.StopTimer()
require.NoError(b, err)
@ -257,7 +257,7 @@ func BenchmarkApplyNativeMessage(b *testing.B) {
}
}
func BenchmarkApplyNativeMessageWithLegacyTx(b *testing.B) {
func BenchmarkApplyMessageWithLegacyTx(b *testing.B) {
suite := KeeperTestSuite{}
suite.DoSetupTest(b)
@ -284,7 +284,7 @@ func BenchmarkApplyNativeMessageWithLegacyTx(b *testing.B) {
require.NoError(b, err)
b.StartTimer()
resp, err := suite.app.EvmKeeper.ApplyNativeMessage(m)
resp, err := suite.app.EvmKeeper.ApplyMessage(m, nil, true)
b.StopTimer()
require.NoError(b, err)
@ -292,7 +292,7 @@ func BenchmarkApplyNativeMessageWithLegacyTx(b *testing.B) {
}
}
func BenchmarkApplyNativeMessageWithDynamicFeeTx(b *testing.B) {
func BenchmarkApplyMessageWithDynamicFeeTx(b *testing.B) {
suite := KeeperTestSuite{dynamicTxFee: true}
suite.DoSetupTest(b)
@ -319,7 +319,7 @@ func BenchmarkApplyNativeMessageWithDynamicFeeTx(b *testing.B) {
require.NoError(b, err)
b.StartTimer()
resp, err := suite.app.EvmKeeper.ApplyNativeMessage(m)
resp, err := suite.app.EvmKeeper.ApplyMessage(m, nil, true)
b.StopTimer()
require.NoError(b, err)

View File

@ -15,6 +15,7 @@ import (
tmproto "github.com/tendermint/tendermint/proto/tendermint/types"
tmtypes "github.com/tendermint/tendermint/types"
"github.com/tharsis/ethermint/tests"
"github.com/tharsis/ethermint/x/evm/types"
)
func (suite *KeeperTestSuite) TestGetHashFn() {
@ -379,15 +380,15 @@ func (suite *KeeperTestSuite) TestRefundGas() {
"leftoverGas equal to tx gas limit, insufficient fee collector account",
params.TxGas,
params.RefundQuotient,
false,
params.TxGas,
true,
0,
},
{
"leftoverGas less than to tx gas limit",
params.TxGas - 1,
params.RefundQuotient,
true,
params.TxGas - 1,
0,
},
{
"no leftoverGas, refund half used gas ",
@ -422,14 +423,20 @@ func (suite *KeeperTestSuite) TestRefundGas() {
suite.app.EvmKeeper.AddRefund(params.TxGas)
gr, err := suite.app.EvmKeeper.RefundGas(m, tc.leftoverGas, tc.refundQuotient)
if tc.leftoverGas > m.Gas() {
return
}
gasUsed := m.Gas() - tc.leftoverGas
refund := suite.app.EvmKeeper.GasToRefund(gasUsed, tc.refundQuotient)
suite.Require().Equal(tc.expGasRefund, refund)
err = suite.app.EvmKeeper.RefundGas(m, refund, "aphoton")
if tc.noError {
suite.Require().NoError(err)
} else {
suite.Require().Error(err)
}
suite.Require().Equal(tc.expGasRefund, gr)
})
}
suite.mintFeeCollector = false
@ -495,3 +502,13 @@ func (suite *KeeperTestSuite) TestResetGasMeterAndConsumeGas() {
})
}
}
func (suite *KeeperTestSuite) TestEVMConfig() {
suite.SetupTest()
cfg, err := suite.app.EvmKeeper.EVMConfig(suite.ctx)
suite.Require().NoError(err)
suite.Require().Equal(types.DefaultParams(), cfg.Params)
suite.Require().Equal((*big.Int)(nil), cfg.BaseFee)
suite.Require().Equal(suite.address, cfg.CoinBase)
suite.Require().Equal(types.DefaultParams().ChainConfig.EthereumConfig(big.NewInt(9000)), cfg.ChainConfig)
}

17
x/evm/types/config.go Normal file
View File

@ -0,0 +1,17 @@
package types
import (
"math/big"
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/params"
)
// EVMConfig encapulates common parameters needed to create an EVM to execute a message
// It's mainly to reduce the number of method parameters
type EVMConfig struct {
Params Params
ChainConfig *params.ChainConfig
CoinBase common.Address
BaseFee *big.Int
}

View File

@ -30,6 +30,7 @@ const (
codeErrInconsistentGas
codeErrInvalidGasCap
codeErrInvalidBaseFee
codeErrGasOverflow
)
var ErrPostTxProcessing = errors.New("failed to execute post processing")
@ -85,6 +86,9 @@ var (
// ErrInvalidBaseFee returns an error if a the base fee cap value is invalid
ErrInvalidBaseFee = sdkerrors.Register(ModuleName, codeErrInvalidBaseFee, "invalid base fee")
// ErrGasOverflow returns an error if gas computation overlow/underflow
ErrGasOverflow = sdkerrors.Register(ModuleName, codeErrGasOverflow, "gas computation overflow/underflow")
)
// NewExecErrorWithReason unpacks the revert return bytes and returns a wrapped error