Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
2 changes: 1 addition & 1 deletion ante/evm/11_emit_event.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,7 @@ func EmitTxHashEvent(ctx sdk.Context, msg *evmtypes.MsgEthereumTx, blockTxIndex
ctx.EventManager().EmitEvent(
sdk.NewEvent(
evmtypes.EventTypeEthereumTx,
sdk.NewAttribute(evmtypes.AttributeKeyEthereumTxHash, msg.Hash().String()),
sdk.NewAttribute(evmtypes.AttributeKeyEthereumTxHash, msg.AsTransaction().Hash().String()),
sdk.NewAttribute(evmtypes.AttributeKeyTxIndex, strconv.FormatUint(blockTxIndex, 10)), // #nosec G115
),
)
Expand Down
2 changes: 1 addition & 1 deletion ante/tx_listener.go
Original file line number Diff line number Diff line change
Expand Up @@ -27,7 +27,7 @@ func (d TxListenerDecorator) AnteHandle(ctx sdk.Context, tx sdk.Tx, simulate boo
if ctx.IsCheckTx() && !simulate && d.pendingTxListener != nil {
for _, msg := range tx.GetMsgs() {
if ethTx, ok := msg.(*evmtypes.MsgEthereumTx); ok {
d.pendingTxListener(ethTx.Hash())
d.pendingTxListener(ethTx.AsTransaction().Hash())
}
}
}
Expand Down
2 changes: 1 addition & 1 deletion indexer/kv_indexer.go
Original file line number Diff line number Diff line change
Expand Up @@ -82,7 +82,7 @@ func (kv *KVIndexer) IndexBlock(block *cmttypes.Block, txResults []*abci.ExecTxR
var cumulativeGasUsed uint64
for msgIndex, msg := range tx.GetMsgs() {
ethMsg := msg.(*evmtypes.MsgEthereumTx)
txHash := ethMsg.Hash()
txHash := ethMsg.AsTransaction().Hash()

txResult := servertypes.TxResult{
Height: height,
Expand Down
4 changes: 2 additions & 2 deletions mempool/mempool.go
Original file line number Diff line number Diff line change
Expand Up @@ -224,7 +224,7 @@ func (m *ExperimentalEVMMempool) Insert(goCtx context.Context, tx sdk.Tx) error
ethMsg, err := m.getEVMMessage(tx)
if err == nil {
// Insert into EVM pool
hash := ethMsg.Hash()
hash := ethMsg.AsTransaction().Hash()
m.logger.Debug("inserting EVM transaction", "tx_hash", hash)
ethTxs := []*ethtypes.Transaction{ethMsg.AsTransaction()}
errs := m.txPool.Add(ethTxs, true)
Expand Down Expand Up @@ -320,7 +320,7 @@ func (m *ExperimentalEVMMempool) Remove(tx sdk.Tx) error {
// We should not do this with EVM transactions because removing them causes the subsequent ones to
// be dequeued as temporarily invalid, only to be requeued a block later.
// The EVM mempool handles removal based on account nonce automatically.
hash := msg.Hash()
hash := msg.AsTransaction().Hash()
if m.shouldRemoveFromEVMPool(tx) {
m.logger.Debug("manually removing EVM transaction", "tx_hash", hash)
m.legacyTxPool.RemoveTx(hash, false, true)
Expand Down
3 changes: 2 additions & 1 deletion rpc/backend/backend.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@ import (

evmmempool "github.com/cosmos/evm/mempool"
"github.com/cosmos/evm/rpc/types"
"github.com/cosmos/evm/rpc/types/interfaces"
"github.com/cosmos/evm/server/config"
servertypes "github.com/cosmos/evm/server/types"
evmtypes "github.com/cosmos/evm/x/vm/types"
Expand Down Expand Up @@ -74,7 +75,7 @@ type EVMBackend interface {
CometBlockByHash(blockHash common.Hash) (*tmrpctypes.ResultBlock, error)
BlockNumberFromComet(blockNrOrHash types.BlockNumberOrHash) (types.BlockNumber, error)
BlockNumberFromCometByHash(blockHash common.Hash) (*big.Int, error)
EthMsgsFromCometBlock(block *tmrpctypes.ResultBlock, blockRes *tmrpctypes.ResultBlockResults) []*evmtypes.MsgEthereumTx
EthMsgsFromCometBlock(block *tmrpctypes.ResultBlock, blockRes *tmrpctypes.ResultBlockResults) []interfaces.IMsgEthereumTx
BlockBloomFromCometBlock(blockRes *tmrpctypes.ResultBlockResults) (ethtypes.Bloom, error)
HeaderByNumber(blockNum types.BlockNumber) (*ethtypes.Header, error)
HeaderByHash(blockHash common.Hash) (*ethtypes.Header, error)
Expand Down
26 changes: 14 additions & 12 deletions rpc/backend/comet_to_eth.go
Original file line number Diff line number Diff line change
Expand Up @@ -14,6 +14,7 @@ import (
cmtrpctypes "github.com/cometbft/cometbft/rpc/core/types"

rpctypes "github.com/cosmos/evm/rpc/types"
"github.com/cosmos/evm/rpc/types/interfaces"
evmtypes "github.com/cosmos/evm/x/vm/types"

sdk "github.com/cosmos/cosmos-sdk/types"
Expand Down Expand Up @@ -87,8 +88,8 @@ func (b *Backend) BlockNumberFromCometByHash(blockHash common.Hash) (*big.Int, e
func (b *Backend) EthMsgsFromCometBlock(
resBlock *cmtrpctypes.ResultBlock,
blockRes *cmtrpctypes.ResultBlockResults,
) []*evmtypes.MsgEthereumTx {
var result []*evmtypes.MsgEthereumTx
) []interfaces.IMsgEthereumTx {
var result []interfaces.IMsgEthereumTx
block := resBlock.Block

txResults := blockRes.TxsResults
Expand All @@ -110,7 +111,7 @@ func (b *Backend) EthMsgsFromCometBlock(
}

for _, msg := range tx.GetMsgs() {
ethMsg, ok := msg.(*evmtypes.MsgEthereumTx)
ethMsg, ok := msg.(interfaces.IMsgEthereumTx)
if !ok {
continue
}
Expand Down Expand Up @@ -225,7 +226,7 @@ func (b *Backend) MinerFromCometBlock(
func (b *Backend) ReceiptsFromCometBlock(
resBlock *cmtrpctypes.ResultBlock,
blockRes *cmtrpctypes.ResultBlockResults,
msgs []*evmtypes.MsgEthereumTx,
msgs []interfaces.IMsgEthereumTx,
) ([]*ethtypes.Receipt, error) {
baseFee, err := b.BaseFee(blockRes)
if err != nil {
Expand All @@ -237,18 +238,19 @@ func (b *Backend) ReceiptsFromCometBlock(
receipts := make([]*ethtypes.Receipt, len(msgs))
cumulatedGasUsed := uint64(0)
for i, ethMsg := range msgs {
txResult, err := b.GetTxByEthHash(ethMsg.Hash())
tx := ethMsg.AsTransaction()
txResult, err := b.GetTxByEthHash(tx.Hash())
if err != nil {
return nil, fmt.Errorf("tx not found: hash=%s, error=%s", ethMsg.Hash(), err.Error())
return nil, fmt.Errorf("tx not found: hash=%s, error=%s", tx.Hash().Hex(), err.Error())
}

cumulatedGasUsed += txResult.GasUsed

var effectiveGasPrice *big.Int
if baseFee != nil {
effectiveGasPrice = rpctypes.EffectiveGasPrice(ethMsg.Raw.Transaction, baseFee)
effectiveGasPrice = rpctypes.EffectiveGasPrice(tx, baseFee)
} else {
effectiveGasPrice = ethMsg.Raw.GasFeeCap()
effectiveGasPrice = tx.GasFeeCap()
}

var status uint64
Expand All @@ -259,8 +261,8 @@ func (b *Backend) ReceiptsFromCometBlock(
}

contractAddress := common.Address{}
if ethMsg.Raw.To() == nil {
contractAddress = crypto.CreateAddress(ethMsg.GetSender(), ethMsg.Raw.Nonce())
if tx.To() == nil {
contractAddress = crypto.CreateAddress(common.Address(ethMsg.GetFrom().Bytes()), tx.Nonce())
}

msgIndex := int(txResult.MsgIndex) // #nosec G115 -- checked for int overflow already
Expand All @@ -277,15 +279,15 @@ func (b *Backend) ReceiptsFromCometBlock(

receipt := &ethtypes.Receipt{
// Consensus fields: These fields are defined by the Yellow Paper
Type: ethMsg.Raw.Type(),
Type: tx.Type(),
PostState: nil,
Status: status, // convert to 1=success, 0=failure
CumulativeGasUsed: cumulatedGasUsed,
Bloom: bloom,
Logs: logs,

// Implementation fields: These fields are added by geth when processing a transaction.
TxHash: ethMsg.Hash(),
TxHash: tx.Hash(),
ContractAddress: contractAddress,
GasUsed: txResult.GasUsed,
EffectiveGasPrice: effectiveGasPrice,
Expand Down
15 changes: 8 additions & 7 deletions rpc/backend/tx_info.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@ import (

"github.com/cosmos/evm/mempool/txpool"
rpctypes "github.com/cosmos/evm/rpc/types"
"github.com/cosmos/evm/rpc/types/interfaces"
servertypes "github.com/cosmos/evm/server/types"
"github.com/cosmos/evm/utils"
evmtypes "github.com/cosmos/evm/x/vm/types"
Expand Down Expand Up @@ -47,7 +48,7 @@ func (b *Backend) GetTransactionByHash(txHash common.Hash) (*rpctypes.RPCTransac
}

// the `res.MsgIndex` is inferred from tx index, should be within the bound.
msg, ok := tx.GetMsgs()[res.MsgIndex].(*evmtypes.MsgEthereumTx)
msg, ok := tx.GetMsgs()[res.MsgIndex].(interfaces.IMsgEthereumTx)
if !ok {
return nil, errors.New("invalid ethereum tx")
}
Expand All @@ -62,7 +63,7 @@ func (b *Backend) GetTransactionByHash(txHash common.Hash) (*rpctypes.RPCTransac
// Fallback to find tx index by iterating all valid eth transactions
msgs := b.EthMsgsFromCometBlock(block, blockRes)
for i := range msgs {
if msgs[i].Hash() == txHash {
if msgs[i].AsTransaction().Hash() == txHash {
if i > math.MaxInt32 {
return nil, errors.New("tx index overflow")
}
Expand Down Expand Up @@ -113,7 +114,7 @@ func (b *Backend) GetTransactionByHashPending(txHash common.Hash) (*rpctypes.RPC
continue
}

if msg.Hash() == txHash {
if msg.AsTransaction().Hash() == txHash {
// use zero block values since it's not included in a block yet
return rpctypes.NewTransactionFromMsg(
msg,
Expand Down Expand Up @@ -192,8 +193,8 @@ func (b *Backend) GetTransactionReceipt(hash common.Hash) (map[string]interface{
return nil, fmt.Errorf("block result not found at height %d: %w", res.Height, err)
}

ethMsg := tx.GetMsgs()[res.MsgIndex].(*evmtypes.MsgEthereumTx)
receipts, err := b.ReceiptsFromCometBlock(resBlock, blockRes, []*evmtypes.MsgEthereumTx{ethMsg})
ethMsg := tx.GetMsgs()[res.MsgIndex].(interfaces.IMsgEthereumTx)
receipts, err := b.ReceiptsFromCometBlock(resBlock, blockRes, []interfaces.IMsgEthereumTx{ethMsg})
if err != nil {
return nil, fmt.Errorf("failed to get receipts from comet block")
}
Expand Down Expand Up @@ -364,7 +365,7 @@ func (b *Backend) GetTransactionByBlockAndIndex(block *cmtrpctypes.ResultBlock,
return nil, nil
}

var msg *evmtypes.MsgEthereumTx
var msg interfaces.IMsgEthereumTx
// find in tx indexer
res, err := b.GetTxByTxIndex(block.Block.Height, uint(idx))
if err == nil {
Expand All @@ -376,7 +377,7 @@ func (b *Backend) GetTransactionByBlockAndIndex(block *cmtrpctypes.ResultBlock,

var ok bool
// msgIndex is inferred from tx events, should be within bound.
msg, ok = tx.GetMsgs()[res.MsgIndex].(*evmtypes.MsgEthereumTx)
msg, ok = tx.GetMsgs()[res.MsgIndex].(interfaces.IMsgEthereumTx)
if !ok {
b.Logger.Debug("invalid ethereum tx", "height", block.Block.Header, "index", idx)
return nil, nil
Expand Down
5 changes: 3 additions & 2 deletions rpc/backend/utils.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@ import (
cmtrpctypes "github.com/cometbft/cometbft/rpc/core/types"

"github.com/cosmos/evm/rpc/types"
"github.com/cosmos/evm/rpc/types/interfaces"
"github.com/cosmos/evm/utils"
feemarkettypes "github.com/cosmos/evm/x/feemarket/types"
evmtypes "github.com/cosmos/evm/x/vm/types"
Expand Down Expand Up @@ -91,7 +92,7 @@ func (b *Backend) getAccountNonce(accAddr common.Address, pending bool, height i
// only supports `MsgEthereumTx` style tx
for _, tx := range pendingTxs {
for _, msg := range (*tx).GetMsgs() {
ethMsg, ok := msg.(*evmtypes.MsgEthereumTx)
ethMsg, ok := msg.(interfaces.IMsgEthereumTx)
if !ok {
// not ethereum tx
break
Expand Down Expand Up @@ -247,7 +248,7 @@ func (b *Backend) ProcessBlock(
}
txGasUsed := uint64(cometTxResult.GasUsed) // #nosec G115
for _, msg := range tx.GetMsgs() {
ethMsg, ok := msg.(*evmtypes.MsgEthereumTx)
ethMsg, ok := msg.(interfaces.IMsgEthereumTx)
if !ok {
continue
}
Expand Down
3 changes: 2 additions & 1 deletion rpc/types/events.go
Original file line number Diff line number Diff line change
Expand Up @@ -9,6 +9,7 @@ import (
abci "github.com/cometbft/cometbft/abci/types"
cmtrpctypes "github.com/cometbft/cometbft/rpc/core/types"

"github.com/cosmos/evm/rpc/types/interfaces"
"github.com/cosmos/evm/server/types"
evmtypes "github.com/cosmos/evm/x/vm/types"

Expand Down Expand Up @@ -133,7 +134,7 @@ func ParseTxResult(result *abci.ExecTxResult, tx sdk.Tx) (*ParsedTxs, error) {
p.Txs[i].Failed = true

// replace gasUsed with gasLimit because that's what's actually deducted.
gasLimit := tx.GetMsgs()[i].(*evmtypes.MsgEthereumTx).GetGas()
gasLimit := tx.GetMsgs()[i].(interfaces.IMsgEthereumTx).GetGas()
p.Txs[i].GasUsed = gasLimit
}
}
Expand Down
22 changes: 22 additions & 0 deletions rpc/types/interfaces/interface.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,22 @@
package interfaces

import (
"math/big"

sdk "github.com/cosmos/cosmos-sdk/types"
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/core"
ethtypes "github.com/ethereum/go-ethereum/core/types"
)

type IMsgEthereumTx interface {
FromEthereumTx(tx *ethtypes.Transaction)
FromSignedEthereumTx(tx *ethtypes.Transaction, signer ethtypes.Signer) error
GetFrom() sdk.AccAddress
GetGas() uint64
GetEffectiveFee(baseFee *big.Int) *big.Int
AsTransaction() *ethtypes.Transaction
GetSenderLegacy(signer ethtypes.Signer) (common.Address, error)
AsMessage(baseFee *big.Int) *core.Message
UnmarshalBinary(b []byte, signer ethtypes.Signer) error
}
11 changes: 6 additions & 5 deletions rpc/types/utils.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@ import (
cmtrpccore "github.com/cometbft/cometbft/rpc/core/types"
cmttypes "github.com/cometbft/cometbft/types"

"github.com/cosmos/evm/rpc/types/interfaces"
feemarkettypes "github.com/cosmos/evm/x/feemarket/types"
evmtypes "github.com/cosmos/evm/x/vm/types"

Expand All @@ -38,15 +39,15 @@ const ExceedBlockGasLimitError = "out of gas in location: block gas meter; gasWa
const StateDBCommitError = "failed to commit stateDB"

// RawTxToEthTx returns a evm MsgEthereum transaction from raw tx bytes.
func RawTxToEthTx(clientCtx client.Context, txBz cmttypes.Tx) ([]*evmtypes.MsgEthereumTx, error) {
func RawTxToEthTx(clientCtx client.Context, txBz cmttypes.Tx) ([]interfaces.IMsgEthereumTx, error) {
tx, err := clientCtx.TxConfig.TxDecoder()(txBz)
if err != nil {
return nil, errorsmod.Wrap(errortypes.ErrJSONUnmarshal, err.Error())
}

ethTxs := make([]*evmtypes.MsgEthereumTx, len(tx.GetMsgs()))
ethTxs := make([]interfaces.IMsgEthereumTx, len(tx.GetMsgs()))
for i, msg := range tx.GetMsgs() {
ethTx, ok := msg.(*evmtypes.MsgEthereumTx)
ethTx, ok := msg.(interfaces.IMsgEthereumTx)
if !ok {
return nil, fmt.Errorf("invalid message type %T, expected %T", msg, &evmtypes.MsgEthereumTx{})
}
Expand Down Expand Up @@ -155,7 +156,7 @@ func MakeHeader(
// NewTransactionFromMsg returns a transaction that will serialize to the RPC
// representation, with the given location metadata set (if available).
func NewTransactionFromMsg(
msg *evmtypes.MsgEthereumTx,
msg interfaces.IMsgEthereumTx,
blockHash common.Hash,
blockNumber, blockTime, index uint64,
baseFee *big.Int,
Expand Down Expand Up @@ -429,7 +430,7 @@ func RPCMarshalHeader(head *ethtypes.Header, blockHash []byte) map[string]interf
//
// This method refers to go-ethereum v1.16.3 internal package method - RPCMarshalBlock
// (https://github.com/ethereum/go-ethereum/blob/d818a9af7bd5919808df78f31580f59382c53150/internal/ethapi/api.go#L929-L962)
func RPCMarshalBlock(block *ethtypes.Block, cmtBlock *cmtrpccore.ResultBlock, msgs []*evmtypes.MsgEthereumTx, inclTx bool, fullTx bool, config *ethparams.ChainConfig) (map[string]interface{}, error) {
func RPCMarshalBlock(block *ethtypes.Block, cmtBlock *cmtrpccore.ResultBlock, msgs []interfaces.IMsgEthereumTx, inclTx bool, fullTx bool, config *ethparams.ChainConfig) (map[string]interface{}, error) {
blockHash := cmtBlock.BlockID.Hash.Bytes()
fields := RPCMarshalHeader(block.Header(), blockHash)
fields["size"] = hexutil.Uint64(block.Size())
Expand Down
14 changes: 6 additions & 8 deletions x/vm/types/msg.go
Original file line number Diff line number Diff line change
Expand Up @@ -13,6 +13,7 @@ import (
protov2 "google.golang.org/protobuf/proto"

evmapi "github.com/cosmos/evm/api/cosmos/evm/vm/v1"
"github.com/cosmos/evm/rpc/types/interfaces"

errorsmod "cosmossdk.io/errors"
sdkmath "cosmossdk.io/math"
Expand All @@ -30,10 +31,11 @@ import (
)

var (
_ sdk.Msg = &MsgEthereumTx{}
_ sdk.Tx = &MsgEthereumTx{}
_ ante.GasTx = &MsgEthereumTx{}
_ sdk.Msg = &MsgUpdateParams{}
_ sdk.Msg = &MsgEthereumTx{}
_ sdk.Tx = &MsgEthereumTx{}
_ ante.GasTx = &MsgEthereumTx{}
_ interfaces.IMsgEthereumTx = &MsgEthereumTx{}
_ sdk.Msg = &MsgUpdateParams{}
)

// message type and route constants
Expand Down Expand Up @@ -298,10 +300,6 @@ func (msg *MsgEthereumTx) UnmarshalBinary(b []byte, signer ethtypes.Signer) erro
return msg.FromSignedEthereumTx(tx, signer)
}

func (msg *MsgEthereumTx) Hash() common.Hash {
return msg.AsTransaction().Hash()
}

// BuildTx builds the canonical cosmos tx from ethereum msg
func (msg *MsgEthereumTx) BuildTx(b client.TxBuilder, evmDenom string) (signing.Tx, error) {
return msg.BuildTxWithEvmParams(b, Params{
Expand Down