Skip to content

Commit

Permalink
Internal txs and vmErr logic
Browse files Browse the repository at this point in the history
  • Loading branch information
quentinlesceller committed Aug 17, 2022
1 parent d3ad55e commit fbdcb59
Show file tree
Hide file tree
Showing 9 changed files with 202 additions and 48 deletions.
2 changes: 1 addition & 1 deletion core/blockchain.go
Original file line number Diff line number Diff line change
Expand Up @@ -1664,7 +1664,7 @@ func (bc *BlockChain) insertChain(chain types.Blocks, verifySeals, setHead bool)

// Process block using the parent state as reference point
substart := time.Now()
receipts, logs, usedGas, err := bc.processor.Process(block, statedb, bc.vmConfig)
receipts, logs, _, _, usedGas, err := bc.processor.Process(block, statedb, bc.vmConfig)
if err != nil {
bc.reportBlock(block, receipts, err)
atomic.StoreUint32(&followupInterrupt, 1)
Expand Down
46 changes: 23 additions & 23 deletions core/blockchain_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -159,7 +159,7 @@ func testBlockChainImport(chain types.Blocks, blockchain *BlockChain) error {
if err != nil {
return err
}
receipts, _, usedGas, err := blockchain.processor.Process(block, statedb, vm.Config{})
receipts, _, _, _, usedGas, err := blockchain.processor.Process(block, statedb, vm.Config{})
if err != nil {
blockchain.reportBlock(block, receipts, err)
return err
Expand Down Expand Up @@ -1862,8 +1862,8 @@ func TestInsertReceiptChainRollback(t *testing.T) {
// overtake the 'canon' chain until after it's passed canon by about 200 blocks.
//
// Details at:
// - https://github.com/blockcypher/go-ethereum/issues/18977
// - https://github.com/blockcypher/go-ethereum/pull/18988
// - https://github.com/blockcypher/go-ethereum/issues/18977
// - https://github.com/blockcypher/go-ethereum/pull/18988
func TestLowDiffLongChain(t *testing.T) {
// Generate a canonical chain to act as the main dataset
engine := ethash.NewFaker()
Expand Down Expand Up @@ -2026,7 +2026,8 @@ func testSideImport(t *testing.T, numCanonBlocksInSidechain, blocksBetweenCommon
// That is: the sidechain for import contains some blocks already present in canon chain.
// So the blocks are
// [ Cn, Cn+1, Cc, Sn+3 ... Sm]
// ^ ^ ^ pruned
//
// ^ ^ ^ pruned
func TestPrunedImportSide(t *testing.T) {
//glogger := log.NewGlogHandler(log.StreamHandler(os.Stdout, log.TerminalFormat(false)))
//glogger.Verbosity(3)
Expand Down Expand Up @@ -2808,9 +2809,9 @@ func BenchmarkBlockChain_1x1000Executions(b *testing.B) {
// This internally leads to a sidechain import, since the blocks trigger an
// ErrPrunedAncestor error.
// This may e.g. happen if
// 1. Downloader rollbacks a batch of inserted blocks and exits
// 2. Downloader starts to sync again
// 3. The blocks fetched are all known and canonical blocks
// 1. Downloader rollbacks a batch of inserted blocks and exits
// 2. Downloader starts to sync again
// 3. The blocks fetched are all known and canonical blocks
func TestSideImportPrunedBlocks(t *testing.T) {
// Generate a canonical chain to act as the main dataset
engine := ethash.NewFaker()
Expand Down Expand Up @@ -3323,20 +3324,19 @@ func TestDeleteRecreateSlotsAcrossManyBlocks(t *testing.T) {

// TestInitThenFailCreateContract tests a pretty notorious case that happened
// on mainnet over blocks 7338108, 7338110 and 7338115.
// - Block 7338108: address e771789f5cccac282f23bb7add5690e1f6ca467c is initiated
// with 0.001 ether (thus created but no code)
// - Block 7338110: a CREATE2 is attempted. The CREATE2 would deploy code on
// the same address e771789f5cccac282f23bb7add5690e1f6ca467c. However, the
// deployment fails due to OOG during initcode execution
// - Block 7338115: another tx checks the balance of
// e771789f5cccac282f23bb7add5690e1f6ca467c, and the snapshotter returned it as
// zero.
// - Block 7338108: address e771789f5cccac282f23bb7add5690e1f6ca467c is initiated
// with 0.001 ether (thus created but no code)
// - Block 7338110: a CREATE2 is attempted. The CREATE2 would deploy code on
// the same address e771789f5cccac282f23bb7add5690e1f6ca467c. However, the
// deployment fails due to OOG during initcode execution
// - Block 7338115: another tx checks the balance of
// e771789f5cccac282f23bb7add5690e1f6ca467c, and the snapshotter returned it as
// zero.
//
// The problem being that the snapshotter maintains a destructset, and adds items
// to the destructset in case something is created "onto" an existing item.
// We need to either roll back the snapDestructs, or not place it into snapDestructs
// in the first place.
//
func TestInitThenFailCreateContract(t *testing.T) {
var (
// Generate a canonical chain to act as the main dataset
Expand Down Expand Up @@ -3524,13 +3524,13 @@ func TestEIP2718Transition(t *testing.T) {

// TestEIP1559Transition tests the following:
//
// 1. A transaction whose gasFeeCap is greater than the baseFee is valid.
// 2. Gas accounting for access lists on EIP-1559 transactions is correct.
// 3. Only the transaction's tip will be received by the coinbase.
// 4. The transaction sender pays for both the tip and baseFee.
// 5. The coinbase receives only the partially realized tip when
// gasFeeCap - gasTipCap < baseFee.
// 6. Legacy transaction behave as expected (e.g. gasPrice = gasFeeCap = gasTipCap).
// 1. A transaction whose gasFeeCap is greater than the baseFee is valid.
// 2. Gas accounting for access lists on EIP-1559 transactions is correct.
// 3. Only the transaction's tip will be received by the coinbase.
// 4. The transaction sender pays for both the tip and baseFee.
// 5. The coinbase receives only the partially realized tip when
// gasFeeCap - gasTipCap < baseFee.
// 6. Legacy transaction behave as expected (e.g. gasPrice = gasFeeCap = gasTipCap).
func TestEIP1559Transition(t *testing.T) {
var (
aa = common.HexToAddress("0x000000000000000000000000000000000000aaaa")
Expand Down
2 changes: 1 addition & 1 deletion core/chain_makers.go
Original file line number Diff line number Diff line change
Expand Up @@ -103,7 +103,7 @@ func (b *BlockGen) AddTxWithChain(bc *BlockChain, tx *types.Transaction) {
b.SetCoinbase(common.Address{})
}
b.statedb.Prepare(tx.Hash(), len(b.txs))
receipt, err := ApplyTransaction(b.config, bc, &b.header.Coinbase, b.gasPool, b.statedb, b.header, tx, &b.header.GasUsed, vm.Config{})
receipt, _, _, err := ApplyTransaction(b.config, bc, &b.header.Coinbase, b.gasPool, b.statedb, b.header, tx, &b.header.GasUsed, vm.Config{})
if err != nil {
panic(err)
}
Expand Down
84 changes: 84 additions & 0 deletions core/internals_processor.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,84 @@
package core

import (
"math/big"

"github.com/blockcypher/go-ethereum/common"
"github.com/blockcypher/go-ethereum/core/types"
"github.com/blockcypher/go-ethereum/core/vm"
)

// Implementation of evm.InternalTxListener

type InternalTxWatcher struct {
internals types.InternalTransactions
}

func NewInternalTxWatcher() *InternalTxWatcher {
return &InternalTxWatcher{
internals: make(types.InternalTransactions, 0),
}
}

// Public API: for users
func (self *InternalTxWatcher) SetParentHash(ph common.Hash) {
for i := range self.internals {
self.internals[i].ParentHash = ph
}
}

func (self *InternalTxWatcher) InternalTransactions() types.InternalTransactions {
return self.internals
}

// Public API: For interfacing with EVM
func (self *InternalTxWatcher) RegisterCall(nonce uint64, gasPrice *big.Int, gas uint64, srcAddr, dstAddr common.Address, value *big.Int, data []byte, depth uint64) {
self.internals = append(self.internals,
types.NewInternalTransaction(nonce, gasPrice, gas,
srcAddr, dstAddr, value, data, depth, self.index(), "call"))
}

func (self *InternalTxWatcher) RegisterStaticCall(nonce uint64, gasPrice *big.Int, gas uint64, srcAddr, dstAddr common.Address, data []byte, depth uint64) {

self.internals = append(self.internals,
types.NewInternalTransaction(nonce, gasPrice, gas,
srcAddr, dstAddr, big.NewInt(0), data, depth, self.index(),
"staticcall"))
}

func (self *InternalTxWatcher) RegisterCallCode(nonce uint64, gasPrice *big.Int, gas uint64, contractAddr common.Address, value *big.Int, data []byte, depth uint64) {
self.internals = append(self.internals,
types.NewInternalTransaction(nonce, gasPrice, gas,
contractAddr, contractAddr, value, data, depth, self.index(),
"call"))
}

func (self *InternalTxWatcher) RegisterCreate(nonce uint64, gasPrice *big.Int, gas uint64, srcAddr, newContractAddr common.Address, value *big.Int, data []byte, depth uint64) {
self.internals = append(self.internals,
types.NewInternalTransaction(nonce, gasPrice, gas,
srcAddr, newContractAddr, value, data, depth, self.index(),
"create"))
}

func (self *InternalTxWatcher) RegisterDelegateCall(nonce uint64, gasPrice *big.Int, gas uint64, callerAddr common.Address, value *big.Int, data []byte, depth uint64) {
self.internals = append(self.internals,
types.NewInternalTransaction(nonce, gasPrice, gas,
callerAddr, callerAddr, value, data, depth, self.index(), "call"))
}

func (self *InternalTxWatcher) RegisterSuicide(nonce uint64, gasPrice *big.Int, gas uint64, contractAddr, creatorAddr common.Address, remainingValue *big.Int, depth uint64) {
self.internals = append(self.internals,
types.NewInternalTransaction(nonce, gasPrice, gas,
contractAddr, creatorAddr, remainingValue,
append([]byte{byte(vm.SELFDESTRUCT)}, creatorAddr[:]...),
depth, self.index(), "suicide"))
}

// Utilities
func (self *InternalTxWatcher) index() uint64 {
return uint64(len(self.internals))
}

func toBigInt(g uint64) *big.Int {
return big.NewInt(0).SetUint64(g)
}
33 changes: 23 additions & 10 deletions core/state_processor.go
Original file line number Diff line number Diff line change
Expand Up @@ -63,7 +63,7 @@ func NewStateProcessor(config *params.ChainConfig, bc blockchain, engine consens
// Process returns the receipts and logs accumulated during the process and
// returns the amount of gas that was used in the process. If any of the
// transactions failed to execute due to insufficient gas it will return an error.
func (p *StateProcessor) Process(block *types.Block, statedb *state.StateDB, cfg vm.Config) (types.Receipts, []*types.Log, uint64, error) {
func (p *StateProcessor) Process(block *types.Block, statedb *state.StateDB, cfg vm.Config) (types.Receipts, []*types.Log, []types.InternalTransactions, []string, uint64, error) {
var (
receipts types.Receipts
usedGas = new(uint64)
Expand All @@ -72,6 +72,8 @@ func (p *StateProcessor) Process(block *types.Block, statedb *state.StateDB, cfg
blockNumber = block.Number()
allLogs []*types.Log
gp = new(GasPool).AddGas(block.GasLimit())
intTxs []types.InternalTransactions
vmerrs []string
)
// Mutate the block and state according to any hard-fork specs
if p.config.DAOForkSupport && p.config.DAOForkBlock != nil && p.config.DAOForkBlock.Cmp(block.Number()) == 0 {
Expand All @@ -83,31 +85,36 @@ func (p *StateProcessor) Process(block *types.Block, statedb *state.StateDB, cfg
for i, tx := range block.Transactions() {
msg, err := tx.AsMessage(types.MakeSigner(p.config, header.Number), header.BaseFee)
if err != nil {
return nil, nil, 0, fmt.Errorf("could not apply tx %d [%v]: %w", i, tx.Hash().Hex(), err)
return nil, nil, nil, nil, 0, fmt.Errorf("could not apply tx %d [%v]: %w", i, tx.Hash().Hex(), err)
}
statedb.Prepare(tx.Hash(), i)
receipt, err := applyTransaction(msg, p.config, p.bc, nil, gp, statedb, blockNumber, blockHash, tx, usedGas, vmenv)
receipt, internals, vmerr, err := applyTransaction(msg, p.config, p.bc, nil, gp, statedb, blockNumber, blockHash, tx, usedGas, vmenv)
if err != nil {
return nil, nil, 0, fmt.Errorf("could not apply tx %d [%v]: %w", i, tx.Hash().Hex(), err)
return nil, nil, nil, nil, 0, fmt.Errorf("could not apply tx %d [%v]: %w", i, tx.Hash().Hex(), err)
}
receipts = append(receipts, receipt)
allLogs = append(allLogs, receipt.Logs...)
intTxs = append(intTxs, internals)
vmerrs = append(vmerrs, vmerr)
}
// Finalize the block, applying any consensus engine specific extras (e.g. block rewards)
p.engine.Finalize(p.bc, header, statedb, block.Transactions(), block.Uncles())

return receipts, allLogs, *usedGas, nil
return receipts, allLogs, intTxs, vmerrs, *usedGas, nil
}

func applyTransaction(msg types.Message, config *params.ChainConfig, bc ChainContext, author *common.Address, gp *GasPool, statedb *state.StateDB, blockNumber *big.Int, blockHash common.Hash, tx *types.Transaction, usedGas *uint64, evm *vm.EVM) (*types.Receipt, error) {
func applyTransaction(msg types.Message, config *params.ChainConfig, bc ChainContext, author *common.Address, gp *GasPool, statedb *state.StateDB, blockNumber *big.Int, blockHash common.Hash, tx *types.Transaction, usedGas *uint64, evm *vm.EVM) (*types.Receipt, types.InternalTransactions, string, error) {
// Create a new context to be used in the EVM environment.
txContext := NewEVMTxContext(msg)
evm.Reset(txContext, statedb)

itx := NewInternalTxWatcher()
evm.AddListener(itx)

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

// Update the state with pending changes.
Expand Down Expand Up @@ -141,17 +148,23 @@ func applyTransaction(msg types.Message, config *params.ChainConfig, bc ChainCon
receipt.BlockHash = blockHash
receipt.BlockNumber = blockNumber
receipt.TransactionIndex = uint(statedb.TxIndex())
return receipt, err

itx.SetParentHash(tx.Hash())
if result.Err != nil {
return receipt, itx.InternalTransactions(), result.Err.Error(), err
}

return receipt, itx.InternalTransactions(), "", err
}

// ApplyTransaction attempts to apply a transaction to the given state database
// 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, bc ChainContext, author *common.Address, gp *GasPool, statedb *state.StateDB, header *types.Header, tx *types.Transaction, usedGas *uint64, cfg vm.Config) (*types.Receipt, types.InternalTransactions, string, error) {
msg, err := tx.AsMessage(types.MakeSigner(config, header.Number), header.BaseFee)
if err != nil {
return nil, err
return nil, nil, "", err
}
// Create a new context to be used in the EVM environment
blockContext := NewEVMBlockContext(header, bc, author)
Expand Down
2 changes: 1 addition & 1 deletion core/types.go
Original file line number Diff line number Diff line change
Expand Up @@ -47,5 +47,5 @@ type Processor interface {
// Process processes the state changes according to the Ethereum rules by running
// the transaction messages using the statedb and applying any rewards to both
// the processor (coinbase) and any included uncles.
Process(block *types.Block, statedb *state.StateDB, cfg vm.Config) (types.Receipts, []*types.Log, uint64, error)
Process(block *types.Block, statedb *state.StateDB, cfg vm.Config) (types.Receipts, []*types.Log, []types.InternalTransactions, []string, uint64, error)
}
57 changes: 57 additions & 0 deletions core/types/internal_tx.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,57 @@
package types

import (
"math/big"

"github.com/blockcypher/go-ethereum/common"
)

type InternalTransaction struct {
Transaction Transaction

Sender common.Address
ParentHash common.Hash
Depth uint64
Index uint64
Note string
Rejected bool
}

type InternalTransactions []*InternalTransaction

func NewInternalTransaction(accountNonce uint64, price *big.Int,
gasLimit uint64, sender common.Address,
recipient common.Address, amount *big.Int, payload []byte,
depth, index uint64, note string) *InternalTransaction {

tx := NewTransaction(accountNonce, recipient, amount, gasLimit, price, payload)
var h common.Hash
return &InternalTransaction{*tx, sender, h, depth, index, note, false}
}

func (tx *InternalTransaction) Reject() {
tx.Rejected = true
}

func (tx *InternalTransaction) Hash() common.Hash {
rej := byte(0)
if tx.Rejected {
rej = byte(1)
}

data := []interface{}{
tx.Transaction.Nonce(),
tx.ParentHash,
tx.Sender,
*tx.Transaction.To(),
tx.Transaction.Value(),
tx.Transaction.GasPrice(),
tx.Transaction.Gas(),
tx.Transaction.Data(),
tx.Note,
tx.Depth,
tx.Index,
rej,
}
return rlpHash(data)
}
20 changes: 10 additions & 10 deletions eth/state_accessor.go
Original file line number Diff line number Diff line change
Expand Up @@ -36,15 +36,15 @@ import (
// base layer statedb can be passed then it's regarded as the statedb of the
// parent block.
// Parameters:
// - block: The block for which we want the state (== state at the stateRoot of the parent)
// - reexec: The maximum number of blocks to reprocess trying to obtain the desired state
// - base: If the caller is tracing multiple blocks, the caller can provide the parent state
// continuously from the callsite.
// - checklive: if true, then the live 'blockchain' state database is used. If the caller want to
// perform Commit or other 'save-to-disk' changes, this should be set to false to avoid
// storing trash persistently
// - preferDisk: this arg can be used by the caller to signal that even though the 'base' is provided,
// it would be preferrable to start from a fresh state, if we have it on disk.
// - block: The block for which we want the state (== state at the stateRoot of the parent)
// - reexec: The maximum number of blocks to reprocess trying to obtain the desired state
// - base: If the caller is tracing multiple blocks, the caller can provide the parent state
// continuously from the callsite.
// - checklive: if true, then the live 'blockchain' state database is used. If the caller want to
// perform Commit or other 'save-to-disk' changes, this should be set to false to avoid
// storing trash persistently
// - preferDisk: this arg can be used by the caller to signal that even though the 'base' is provided,
// it would be preferrable to start from a fresh state, if we have it on disk.
func (eth *Ethereum) StateAtBlock(block *types.Block, reexec uint64, base *state.StateDB, checkLive bool, preferDisk bool) (statedb *state.StateDB, err error) {
var (
current *types.Block
Expand Down Expand Up @@ -131,7 +131,7 @@ func (eth *Ethereum) StateAtBlock(block *types.Block, reexec uint64, base *state
if current = eth.blockchain.GetBlockByNumber(next); current == nil {
return nil, fmt.Errorf("block #%d not found", next)
}
_, _, _, err := eth.blockchain.Processor().Process(current, statedb, vm.Config{})
_, _, _, _, _, err := eth.blockchain.Processor().Process(current, statedb, vm.Config{})
if err != nil {
return nil, fmt.Errorf("processing block %d failed: %v", current.NumberU64(), err)
}
Expand Down

0 comments on commit fbdcb59

Please sign in to comment.