Skip to content

Commit

Permalink
eip-4844: excessDataGas param added to block execution functions (#7217)
Browse files Browse the repository at this point in the history
Small addition to block execution functions. Now they expect
excessDataGas param so the BlockContext can be created with it.
  • Loading branch information
racytech authored Mar 30, 2023
1 parent f590ba3 commit 04b5790
Show file tree
Hide file tree
Showing 10 changed files with 45 additions and 30 deletions.
4 changes: 2 additions & 2 deletions cmd/rpcdaemon/commands/trace_filtering.go
Original file line number Diff line number Diff line change
Expand Up @@ -961,7 +961,7 @@ func (api *TraceAPIImpl) callManyTransactions(
}
engine := api.engine()
consensusHeaderReader := stagedsync.NewChainReaderImpl(cfg, dbtx, nil)
err = core.InitializeBlockExecution(engine.(consensus.Engine), consensusHeaderReader, nil, block.HeaderNoCopy(), block.Transactions(), block.Uncles(), cfg, stateDb)
err = core.InitializeBlockExecution(engine.(consensus.Engine), consensusHeaderReader, nil, block.HeaderNoCopy(), block.Transactions(), block.Uncles(), cfg, stateDb, nil /*excessDataGas*/)
if err != nil {
return nil, err
}
Expand All @@ -982,7 +982,7 @@ func (api *TraceAPIImpl) callManyTransactions(
// gnosis might have a fee free account here
if msg.FeeCap().IsZero() && engine != nil {
syscall := func(contract common.Address, data []byte) ([]byte, error) {
return core.SysCallContract(contract, data, *cfg, stateDb, header, engine, true /* constCall */)
return core.SysCallContract(contract, data, *cfg, stateDb, header, engine, true /* constCall */, nil /*excessDataGas*/)
}
msg.SetIsFree(engine.IsServiceTransaction(msg.From(), syscall))
}
Expand Down
2 changes: 1 addition & 1 deletion cmd/rpcdaemon/commands/tracing.go
Original file line number Diff line number Diff line change
Expand Up @@ -120,7 +120,7 @@ func (api *PrivateDebugAPIImpl) traceBlock(ctx context.Context, blockNrOrHash rp

if msg.FeeCap().IsZero() && engine != nil {
syscall := func(contract common.Address, data []byte) ([]byte, error) {
return core.SysCallContract(contract, data, *chainConfig, ibs, block.Header(), engine, true /* constCall */)
return core.SysCallContract(contract, data, *chainConfig, ibs, block.Header(), engine, true /* constCall */, nil /*excessDataGas*/)
}
msg.SetIsFree(engine.IsServiceTransaction(msg.From(), syscall))
}
Expand Down
4 changes: 2 additions & 2 deletions cmd/state/exec3/state.go
Original file line number Diff line number Diff line change
Expand Up @@ -163,15 +163,15 @@ func (rw *Worker) RunTxTaskNoLock(txTask *exec22.TxTask) {
systemcontracts.UpgradeBuildInSystemContract(rw.chainConfig, header.Number, ibs)
}
syscall := func(contract libcommon.Address, data []byte) ([]byte, error) {
return core.SysCallContract(contract, data, *rw.chainConfig, ibs, header, rw.engine, false /* constCall */)
return core.SysCallContract(contract, data, *rw.chainConfig, ibs, header, rw.engine, false /* constCall */, nil /*excessDataGas*/)
}
rw.engine.Initialize(rw.chainConfig, rw.chain, rw.epoch, header, ibs, txTask.Txs, txTask.Uncles, syscall)
} else if txTask.Final {
if txTask.BlockNum > 0 {
//fmt.Printf("txNum=%d, blockNum=%d, finalisation of the block\n", txTask.TxNum, txTask.BlockNum)
// End of block transaction in a block
syscall := func(contract libcommon.Address, data []byte) ([]byte, error) {
return core.SysCallContract(contract, data, *rw.chainConfig, ibs, header, rw.engine, false /* constCall */)
return core.SysCallContract(contract, data, *rw.chainConfig, ibs, header, rw.engine, false /* constCall */, nil /*excessDataGas*/)
}

if _, _, err := rw.engine.Finalize(rw.chainConfig, types.CopyHeader(header), ibs, txTask.Txs, txTask.Uncles, nil /* receipts */, txTask.Withdrawals, rw.epoch, rw.chain, syscall); err != nil {
Expand Down
4 changes: 2 additions & 2 deletions cmd/state/exec3/state_recon.go
Original file line number Diff line number Diff line change
Expand Up @@ -317,7 +317,7 @@ func (rw *ReconWorker) runTxTask(txTask *exec22.TxTask) error {
//fmt.Printf("txNum=%d, blockNum=%d, finalisation of the block\n", txTask.TxNum, txTask.BlockNum)
// End of block transaction in a block
syscall := func(contract libcommon.Address, data []byte) ([]byte, error) {
return core.SysCallContract(contract, data, *rw.chainConfig, ibs, txTask.Header, rw.engine, false /* constCall */)
return core.SysCallContract(contract, data, *rw.chainConfig, ibs, txTask.Header, rw.engine, false /* constCall */, nil /*excessDataGas*/)
}
if _, _, err := rw.engine.Finalize(rw.chainConfig, types.CopyHeader(txTask.Header), ibs, txTask.Txs, txTask.Uncles, nil /* receipts */, txTask.Withdrawals, rw.epoch, rw.chain, syscall); err != nil {
if _, readError := rw.stateReader.ReadError(); !readError {
Expand All @@ -331,7 +331,7 @@ func (rw *ReconWorker) runTxTask(txTask *exec22.TxTask) error {
systemcontracts.UpgradeBuildInSystemContract(rw.chainConfig, txTask.Header.Number, ibs)
}
syscall := func(contract libcommon.Address, data []byte) ([]byte, error) {
return core.SysCallContract(contract, data, *rw.chainConfig, ibs, txTask.Header, rw.engine, false /* constCall */)
return core.SysCallContract(contract, data, *rw.chainConfig, ibs, txTask.Header, rw.engine, false /* constCall */, nil /*excessDataGas*/)
}

rw.engine.Initialize(rw.chainConfig, rw.chain, rw.epoch, txTask.Header, ibs, txTask.Txs, txTask.Uncles, syscall)
Expand Down
47 changes: 30 additions & 17 deletions core/blockchain.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@ package core

import (
"fmt"
"math/big"
"time"

"github.com/ledgerwatch/erigon-lib/chain"
Expand Down Expand Up @@ -98,7 +99,7 @@ func ExecuteBlockEphemerallyForBSC(
)

if !vmConfig.ReadOnly {
if err := InitializeBlockExecution(engine, chainReader, epochReader, block.Header(), block.Transactions(), block.Uncles(), chainConfig, ibs); err != nil {
if err := InitializeBlockExecution(engine, chainReader, epochReader, block.Header(), block.Transactions(), block.Uncles(), chainConfig, ibs, nil /*excessDataGas*/); err != nil {
return nil, err
}
}
Expand Down Expand Up @@ -161,7 +162,7 @@ func ExecuteBlockEphemerallyForBSC(
// otherwise it causes block verification error.
header.GasUsed = *usedGas
syscall := func(contract libcommon.Address, data []byte) ([]byte, error) {
return SysCallContract(contract, data, *chainConfig, ibs, header, engine, false /* constCall */)
return SysCallContract(contract, data, *chainConfig, ibs, header, engine, false /* constCall */, nil /*excessDataGas*/)
}
outTxs, outReceipts, err := engine.Finalize(chainConfig, header, ibs, block.Transactions(), block.Uncles(), receipts, block.Withdrawals(), epochReader, chainReader, syscall)
if err != nil {
Expand Down Expand Up @@ -249,7 +250,7 @@ func ExecuteBlockEphemerally(
)

if !vmConfig.ReadOnly {
if err := InitializeBlockExecution(engine, chainReader, epochReader, block.Header(), block.Transactions(), block.Uncles(), chainConfig, ibs); err != nil {
if err := InitializeBlockExecution(engine, chainReader, epochReader, block.Header(), block.Transactions(), block.Uncles(), chainConfig, ibs, nil /*excessDataGas*/); err != nil {
return nil, err
}
}
Expand Down Expand Up @@ -310,7 +311,7 @@ func ExecuteBlockEphemerally(
}
if !vmConfig.ReadOnly {
txs := block.Transactions()
if _, _, _, err := FinalizeBlockExecution(engine, stateReader, block.Header(), txs, block.Uncles(), stateWriter, chainConfig, ibs, receipts, block.Withdrawals(), epochReader, chainReader, false); err != nil {
if _, _, _, err := FinalizeBlockExecution(engine, stateReader, block.Header(), txs, block.Uncles(), stateWriter, chainConfig, ibs, receipts, block.Withdrawals(), epochReader, chainReader, false, nil /*excessDataGas*/); err != nil {
return nil, err
}
}
Expand Down Expand Up @@ -360,7 +361,7 @@ func ExecuteBlockEphemerallyBor(
)

if !vmConfig.ReadOnly {
if err := InitializeBlockExecution(engine, chainReader, epochReader, block.Header(), block.Transactions(), block.Uncles(), chainConfig, ibs); err != nil {
if err := InitializeBlockExecution(engine, chainReader, epochReader, block.Header(), block.Transactions(), block.Uncles(), chainConfig, ibs, nil /*excessDataGas*/); err != nil {
return nil, err
}
}
Expand Down Expand Up @@ -421,7 +422,7 @@ func ExecuteBlockEphemerallyBor(
}
if !vmConfig.ReadOnly {
txs := block.Transactions()
if _, _, _, err := FinalizeBlockExecution(engine, stateReader, block.Header(), txs, block.Uncles(), stateWriter, chainConfig, ibs, receipts, block.Withdrawals(), epochReader, chainReader, false); err != nil {
if _, _, _, err := FinalizeBlockExecution(engine, stateReader, block.Header(), txs, block.Uncles(), stateWriter, chainConfig, ibs, receipts, block.Withdrawals(), epochReader, chainReader, false, nil /*excessDataGas*/); err != nil {
return nil, err
}
}
Expand Down Expand Up @@ -467,7 +468,7 @@ func rlpHash(x interface{}) (h libcommon.Hash) {
return h
}

func SysCallContract(contract libcommon.Address, data []byte, chainConfig chain.Config, ibs *state.IntraBlockState, header *types.Header, engine consensus.EngineReader, constCall bool) (result []byte, err error) {
func SysCallContract(contract libcommon.Address, data []byte, chainConfig chain.Config, ibs *state.IntraBlockState, header *types.Header, engine consensus.EngineReader, constCall bool, excessDataGas *big.Int) (result []byte, err error) {
if chainConfig.DAOForkBlock != nil && chainConfig.DAOForkBlock.Cmp(header.Number) == 0 {
misc.ApplyDAOHardFork(ibs)
}
Expand All @@ -493,7 +494,7 @@ func SysCallContract(contract libcommon.Address, data []byte, chainConfig chain.
author = &state.SystemAddress
txContext = NewEVMTxContext(msg)
}
blockContext := NewEVMBlockContext(header, GetHashFn(header, nil), engine, author, nil /*excessDataGas*/)
blockContext := NewEVMBlockContext(header, GetHashFn(header, nil), engine, author, excessDataGas)
evm := vm.NewEVM(blockContext, txContext, ibs, &chainConfig, vmConfig)

ret, _, err := evm.Call(
Expand All @@ -511,7 +512,7 @@ func SysCallContract(contract libcommon.Address, data []byte, chainConfig chain.
}

// SysCreate is a special (system) contract creation methods for genesis constructors.
func SysCreate(contract libcommon.Address, data []byte, chainConfig chain.Config, ibs *state.IntraBlockState, header *types.Header) (result []byte, err error) {
func SysCreate(contract libcommon.Address, data []byte, chainConfig chain.Config, ibs *state.IntraBlockState, header *types.Header, excessDataGas *big.Int) (result []byte, err error) {
if chainConfig.DAOForkBlock != nil && chainConfig.DAOForkBlock.Cmp(header.Number) == 0 {
misc.ApplyDAOHardFork(ibs)
}
Expand All @@ -529,7 +530,7 @@ func SysCreate(contract libcommon.Address, data []byte, chainConfig chain.Config
// Create a new context to be used in the EVM environment
author := &contract
txContext := NewEVMTxContext(msg)
blockContext := NewEVMBlockContext(header, GetHashFn(header, nil), nil, author, nil /*excessDataGas*/)
blockContext := NewEVMBlockContext(header, GetHashFn(header, nil), nil, author, excessDataGas)
evm := vm.NewEVM(blockContext, txContext, ibs, &chainConfig, vmConfig)

ret, _, err := evm.SysCreate(
Expand All @@ -542,7 +543,7 @@ func SysCreate(contract libcommon.Address, data []byte, chainConfig chain.Config
return ret, err
}

func CallContract(contract libcommon.Address, data []byte, chainConfig chain.Config, ibs *state.IntraBlockState, header *types.Header, engine consensus.Engine) (result []byte, err error) {
func CallContract(contract libcommon.Address, data []byte, chainConfig chain.Config, ibs *state.IntraBlockState, header *types.Header, engine consensus.Engine, excessDataGas *big.Int) (result []byte, err error) {
gp := new(GasPool)
gp.AddGas(50_000_000)
var gasUsed uint64
Expand Down Expand Up @@ -571,12 +572,24 @@ func CallContractTx(contract libcommon.Address, data []byte, ibs *state.IntraBlo
return tx.FakeSign(from)
}

func FinalizeBlockExecution(engine consensus.Engine, stateReader state.StateReader, header *types.Header,
txs types.Transactions, uncles []*types.Header, stateWriter state.WriterWithChangeSets, cc *chain.Config, ibs *state.IntraBlockState,
receipts types.Receipts, withdrawals []*types.Withdrawal, e consensus.EpochReader, headerReader consensus.ChainHeaderReader, isMining bool,
func FinalizeBlockExecution(
engine consensus.Engine,
stateReader state.StateReader,
header *types.Header,
txs types.Transactions,
uncles []*types.Header,
stateWriter state.WriterWithChangeSets,
cc *chain.Config,
ibs *state.IntraBlockState,
receipts types.Receipts,
withdrawals []*types.Withdrawal,
e consensus.EpochReader,
headerReader consensus.ChainHeaderReader,
isMining bool,
excessDataGas *big.Int,
) (newBlock *types.Block, newTxs types.Transactions, newReceipt types.Receipts, err error) {
syscall := func(contract libcommon.Address, data []byte) ([]byte, error) {
return SysCallContract(contract, data, *cc, ibs, header, engine, false /* constCall */)
return SysCallContract(contract, data, *cc, ibs, header, engine, false /* constCall */, excessDataGas)
}
if isMining {
newBlock, newTxs, newReceipt, err = engine.FinalizeAndAssemble(cc, header, ibs, txs, uncles, receipts, withdrawals, e, headerReader, syscall, nil)
Expand All @@ -597,9 +610,9 @@ func FinalizeBlockExecution(engine consensus.Engine, stateReader state.StateRead
return newBlock, newTxs, newReceipt, nil
}

func InitializeBlockExecution(engine consensus.Engine, chain consensus.ChainHeaderReader, epochReader consensus.EpochReader, header *types.Header, txs types.Transactions, uncles []*types.Header, cc *chain.Config, ibs *state.IntraBlockState) error {
func InitializeBlockExecution(engine consensus.Engine, chain consensus.ChainHeaderReader, epochReader consensus.EpochReader, header *types.Header, txs types.Transactions, uncles []*types.Header, cc *chain.Config, ibs *state.IntraBlockState, excessDataGas *big.Int) error {
engine.Initialize(cc, chain, epochReader, header, ibs, txs, uncles, func(contract libcommon.Address, data []byte) ([]byte, error) {
return SysCallContract(contract, data, *cc, ibs, header, engine, false /* constCall */)
return SysCallContract(contract, data, *cc, ibs, header, engine, false /* constCall */, excessDataGas)
})
noop := state.NewNoopWriter()
ibs.FinalizeTx(cc.Rules(header.Number.Uint64(), header.Time), noop)
Expand Down
2 changes: 2 additions & 0 deletions core/evm.go
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,8 @@ import (
)

// NewEVMBlockContext creates a new context for use in the EVM.
// excessDataGas must be set to the excessDataGas value from the *parent* block header, and can be
// nil if the parent block is not of EIP-4844 type. It is read only.
func NewEVMBlockContext(header *types.Header, blockHashFunc func(n uint64) libcommon.Hash, engine consensus.EngineReader, author *libcommon.Address, excessDataGas *big.Int) evmtypes.BlockContext {
// If we don't have an explicit author (i.e. not mining), extract from the header
var beneficiary libcommon.Address
Expand Down
2 changes: 1 addition & 1 deletion core/genesis_write.go
Original file line number Diff line number Diff line change
Expand Up @@ -583,7 +583,7 @@ func GenesisToBlock(g *types.Genesis, tmpDir string) (*types.Block, *state.Intra
}

if len(account.Constructor) > 0 {
if _, err = SysCreate(addr, account.Constructor, *g.Config, statedb, head); err != nil {
if _, err = SysCreate(addr, account.Constructor, *g.Config, statedb, head, nil /*excessDataGas*/); err != nil {
return
}
}
Expand Down
2 changes: 1 addition & 1 deletion core/state_processor.go
Original file line number Diff line number Diff line change
Expand Up @@ -43,7 +43,7 @@ func applyTransaction(config *chain.Config, engine consensus.EngineReader, gp *G
if msg.FeeCap().IsZero() && engine != nil {
// Only zero-gas transactions may be service ones
syscall := func(contract libcommon.Address, data []byte) ([]byte, error) {
return SysCallContract(contract, data, *config, ibs, header, engine, true /* constCall */)
return SysCallContract(contract, data, *config, ibs, header, engine, true /* constCall */, nil /*excessDataGas*/)
}
msg.SetIsFree(engine.IsServiceTransaction(msg.From(), syscall))
}
Expand Down
2 changes: 1 addition & 1 deletion eth/stagedsync/stage_mining_exec.go
Original file line number Diff line number Diff line change
Expand Up @@ -173,7 +173,7 @@ func SpawnMiningExecStage(s *StageState, tx kv.RwTx, cfg MiningExecCfg, quit <-c

var err error
_, current.Txs, current.Receipts, err = core.FinalizeBlockExecution(cfg.engine, stateReader, current.Header, current.Txs, current.Uncles, stateWriter,
&cfg.chainConfig, ibs, current.Receipts, current.Withdrawals, EpochReaderImpl{tx: tx}, ChainReaderImpl{config: &cfg.chainConfig, tx: tx, blockReader: cfg.blockReader}, true)
&cfg.chainConfig, ibs, current.Receipts, current.Withdrawals, EpochReaderImpl{tx: tx}, ChainReaderImpl{config: &cfg.chainConfig, tx: tx, blockReader: cfg.blockReader}, true, nil /*excessDataGas*/)
if err != nil {
return err
}
Expand Down
6 changes: 3 additions & 3 deletions turbo/transactions/tracing.go
Original file line number Diff line number Diff line change
Expand Up @@ -64,7 +64,7 @@ func ComputeTxEnv(ctx context.Context, engine consensus.EngineReader, block *typ
msg, _ := txn.AsMessage(*signer, block.BaseFee(), rules)
if msg.FeeCap().IsZero() && engine != nil {
syscall := func(contract libcommon.Address, data []byte) ([]byte, error) {
return core.SysCallContract(contract, data, *cfg, statedb, header, engine, true /* constCall */)
return core.SysCallContract(contract, data, *cfg, statedb, header, engine, true /* constCall */, nil /*excessDataGas*/)
}
msg.SetIsFree(engine.IsServiceTransaction(msg.From(), syscall))
}
Expand All @@ -77,7 +77,7 @@ func ComputeTxEnv(ctx context.Context, engine consensus.EngineReader, block *typ

consensusHeaderReader := stagedsync.NewChainReaderImpl(cfg, dbtx, nil)

core.InitializeBlockExecution(engine.(consensus.Engine), consensusHeaderReader, nil, header, block.Transactions(), block.Uncles(), cfg, statedb)
core.InitializeBlockExecution(engine.(consensus.Engine), consensusHeaderReader, nil, header, block.Transactions(), block.Uncles(), cfg, statedb, nil /*excessDataGas*/)

for idx, txn := range block.Transactions() {
select {
Expand All @@ -91,7 +91,7 @@ func ComputeTxEnv(ctx context.Context, engine consensus.EngineReader, block *typ
msg, _ := txn.AsMessage(*signer, block.BaseFee(), rules)
if msg.FeeCap().IsZero() && engine != nil {
syscall := func(contract libcommon.Address, data []byte) ([]byte, error) {
return core.SysCallContract(contract, data, *cfg, statedb, header, engine, true /* constCall */)
return core.SysCallContract(contract, data, *cfg, statedb, header, engine, true /* constCall */, nil /*excessDataGas*/)
}
msg.SetIsFree(engine.IsServiceTransaction(msg.From(), syscall))
}
Expand Down

0 comments on commit 04b5790

Please sign in to comment.