E3: fix unwind changes visibility (#6147)

This commit is contained in:
Alex Sharov 2022-11-30 08:31:13 +07:00 committed by GitHub
parent ae6d00747e
commit 16cd87748f
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
45 changed files with 599 additions and 573 deletions

View File

@ -36,8 +36,8 @@ jobs:
- name: test-integration
run: make test-integration
# - name: history-v3-test-integration
# run: make test3-integration
- name: history-v3-test-integration
run: make test3-integration
tests-windows:
strategy:

View File

@ -518,13 +518,14 @@ func loopExec(db kv.RwDB, ctx context.Context, unwind uint64) error {
syncCfg.ExecWorkerCount = int(workers)
syncCfg.ReconWorkerCount = int(reconWorkers)
initialCycle := false
cfg := stagedsync.StageExecuteBlocksCfg(db, pm, batchSize, nil, chainConfig, engine, vmConfig, nil,
/*stateStream=*/ false,
/*badBlockHalt=*/ false, historyV3, dirs, getBlockReader(db), nil, genesis, syncCfg, agg)
// set block limit of execute stage
sync.MockExecFunc(stages.Execution, func(firstCycle bool, badBlockUnwind bool, stageState *stagedsync.StageState, unwinder stagedsync.Unwinder, tx kv.RwTx, quiet bool) error {
if err = stagedsync.SpawnExecuteBlocksStage(stageState, sync, tx, to, ctx, cfg, false /* initialCycle */, false /* quiet */); err != nil {
if err = stagedsync.SpawnExecuteBlocksStage(stageState, sync, tx, to, ctx, cfg, initialCycle, false /* quiet */); err != nil {
return fmt.Errorf("spawnExecuteBlocksStage: %w", err)
}
return nil
@ -539,7 +540,7 @@ func loopExec(db kv.RwDB, ctx context.Context, unwind uint64) error {
_ = sync.SetCurrentStage(stages.Execution)
t := time.Now()
if err = sync.Run(db, tx, false /* firstCycle */, false /* quiet */); err != nil {
if err = sync.Run(db, tx, initialCycle, false /* quiet */); err != nil {
return err
}
fmt.Printf("loop time: %s\n", time.Since(t))

View File

@ -99,15 +99,15 @@ func TestTraceBlockByHash(t *testing.T) {
for _, tt := range debugTraceTransactionTests {
var buf bytes.Buffer
stream := jsoniter.NewStream(jsoniter.ConfigDefault, &buf, 4096)
tx, err := ethApi.GetTransactionByHash(context.Background(), common.HexToHash(tt.txHash))
tx, err := ethApi.GetTransactionByHash(m.Ctx, common.HexToHash(tt.txHash))
if err != nil {
t.Errorf("traceBlock %s: %v", tt.txHash, err)
}
txcount, err := ethApi.GetBlockTransactionCountByHash(context.Background(), *tx.BlockHash)
txcount, err := ethApi.GetBlockTransactionCountByHash(m.Ctx, *tx.BlockHash)
if err != nil {
t.Errorf("traceBlock %s: %v", tt.txHash, err)
}
err = api.TraceBlockByHash(context.Background(), *tx.BlockHash, &tracers.TraceConfig{}, stream)
err = api.TraceBlockByHash(m.Ctx, *tx.BlockHash, &tracers.TraceConfig{}, stream)
if err != nil {
t.Errorf("traceBlock %s: %v", tt.txHash, err)
}

View File

@ -15,12 +15,12 @@ import (
"github.com/ledgerwatch/erigon/core/state"
"github.com/ledgerwatch/erigon/core/types"
"github.com/ledgerwatch/erigon/core/vm"
"github.com/ledgerwatch/erigon/crypto"
"github.com/ledgerwatch/erigon/rpc"
"github.com/ledgerwatch/erigon/turbo/adapter/ethapi"
"github.com/ledgerwatch/erigon/turbo/rpchelper"
"github.com/ledgerwatch/erigon/turbo/transactions"
"github.com/ledgerwatch/log/v3"
"golang.org/x/crypto/sha3"
)
func (api *APIImpl) CallBundle(ctx context.Context, txHashes []common.Hash, stateBlockNumberOrHash rpc.BlockNumberOrHash, timeoutMilliSecondsPtr *int64) (map[string]interface{}, error) {
@ -149,7 +149,9 @@ func (api *APIImpl) CallBundle(ctx context.Context, txHashes []common.Hash, stat
results := []map[string]interface{}{}
bundleHash := sha3.NewLegacyKeccak256()
bundleHash := crypto.NewLegacyKeccak256()
defer crypto.ReturnToPoolKeccak256(bundleHash)
for _, txn := range txs {
msg, err := txn.AsMessage(*signer, nil, rules)
if err != nil {

View File

@ -16,6 +16,7 @@ import (
"github.com/ledgerwatch/erigon/core/state"
"github.com/ledgerwatch/erigon/core/types"
"github.com/ledgerwatch/erigon/core/vm"
"github.com/ledgerwatch/erigon/core/vm/evmtypes"
"github.com/ledgerwatch/erigon/rpc"
"github.com/ledgerwatch/erigon/turbo/adapter/ethapi"
"github.com/ledgerwatch/erigon/turbo/rpchelper"
@ -42,7 +43,7 @@ type StateContext struct {
TransactionIndex *int
}
func blockHeaderOverride(blockCtx *vm.BlockContext, blockOverride BlockOverrides, overrideBlockHash map[uint64]common.Hash) {
func blockHeaderOverride(blockCtx *evmtypes.BlockContext, blockOverride BlockOverrides, overrideBlockHash map[uint64]common.Hash) {
if blockOverride.BlockNumber != nil {
blockCtx.BlockNumber = uint64(*blockOverride.BlockNumber)
}
@ -73,8 +74,8 @@ func (api *APIImpl) CallMany(ctx context.Context, bundles []Bundle, simulateCont
hash common.Hash
replayTransactions types.Transactions
evm *vm.EVM
blockCtx vm.BlockContext
txCtx vm.TxContext
blockCtx evmtypes.BlockContext
txCtx evmtypes.TxContext
overrideBlockHash map[uint64]common.Hash
baseFee uint256.Int
)
@ -162,7 +163,7 @@ func (api *APIImpl) CallMany(ctx context.Context, bundles []Bundle, simulateCont
baseFee.SetFromBig(parent.BaseFee)
}
blockCtx = vm.BlockContext{
blockCtx = evmtypes.BlockContext{
CanTransfer: core.CanTransfer,
Transfer: core.Transfer,
GetHash: getHash,

View File

@ -22,6 +22,7 @@ import (
"github.com/ledgerwatch/erigon/core/state"
"github.com/ledgerwatch/erigon/core/types"
"github.com/ledgerwatch/erigon/core/vm"
"github.com/ledgerwatch/erigon/core/vm/evmtypes"
"github.com/ledgerwatch/erigon/eth/filters"
"github.com/ledgerwatch/erigon/ethdb/cbor"
"github.com/ledgerwatch/erigon/params"
@ -346,6 +347,9 @@ func (api *APIImpl) getLogsV3(ctx context.Context, tx kv.Tx, begin, end uint64,
addrMap[v] = struct{}{}
}
evm := vm.NewEVM(evmtypes.BlockContext{}, evmtypes.TxContext{}, nil, chainConfig, vm.Config{})
vmConfig := vm.Config{SkipAnalysis: skipAnalysis}
var minTxNumInBlock, maxTxNumInBlock uint64 // end is an inclusive bound
var blockNum uint64
var ok bool
@ -374,7 +378,7 @@ func (api *APIImpl) getLogsV3(ctx context.Context, tx kv.Tx, begin, end uint64,
blockHash = header.Hash()
signer = types.MakeSigner(chainConfig, blockNum)
rules = chainConfig.Rules(blockNum)
skipAnalysis = core.SkipAnalysis(chainConfig, blockNum)
vmConfig.SkipAnalysis = core.SkipAnalysis(chainConfig, blockNum)
minTxNumInBlock, err = rawdb.TxNums.Min(tx, blockNum)
if err != nil {
@ -401,13 +405,13 @@ func (api *APIImpl) getLogsV3(ctx context.Context, tx kv.Tx, begin, end uint64,
if err != nil {
return nil, err
}
blockCtx, txCtx := transactions.GetEvmContext(msg, header, true /* requireCanonical */, tx, api._blockReader)
vmConfig := vm.Config{SkipAnalysis: skipAnalysis}
ibs := state.New(stateReader)
evm := vm.NewEVM(blockCtx, txCtx, ibs, chainConfig, vmConfig)
ibs.Prepare(txHash, blockHash, txIndex)
blockCtx, txCtx := transactions.GetEvmContext(msg, header, true /* requireCanonical */, tx, api._blockReader)
evm.ResetBetweenBlocks(blockCtx, txCtx, ibs, vmConfig, rules)
gp := new(core.GasPool).AddGas(msg.Gas())
ibs.Prepare(txHash, blockHash, txIndex)
_, err = core.ApplyMessage(evm, msg, gp, true /* refunds */, false /* gasBailout */)
if err != nil {
return nil, fmt.Errorf("%w: blockNum=%d, txNum=%d", err, blockNum, txNum)

View File

@ -8,6 +8,7 @@ import (
"github.com/holiman/uint256"
jsoniter "github.com/json-iterator/go"
"github.com/ledgerwatch/erigon/core/vm/evmtypes"
"github.com/ledgerwatch/log/v3"
"github.com/ledgerwatch/erigon/common"
@ -93,7 +94,7 @@ func (api *PrivateDebugAPIImpl) traceBlock(ctx context.Context, blockNrOrHash rp
}
ibs.Prepare(txn.Hash(), block.Hash(), idx)
msg, _ := txn.AsMessage(*signer, block.BaseFee(), rules)
txCtx := vm.TxContext{
txCtx := evmtypes.TxContext{
TxHash: txn.Hash(),
Origin: msg.From(),
GasPrice: msg.GasPrice().ToBig(),
@ -250,8 +251,8 @@ func (api *PrivateDebugAPIImpl) TraceCallMany(ctx context.Context, bundles []Bun
hash common.Hash
replayTransactions types.Transactions
evm *vm.EVM
blockCtx vm.BlockContext
txCtx vm.TxContext
blockCtx evmtypes.BlockContext
txCtx evmtypes.TxContext
overrideBlockHash map[uint64]common.Hash
baseFee uint256.Int
)
@ -346,7 +347,7 @@ func (api *PrivateDebugAPIImpl) TraceCallMany(ctx context.Context, bundles []Bun
baseFee.SetFromBig(parent.BaseFee)
}
blockCtx = vm.BlockContext{
blockCtx = evmtypes.BlockContext{
CanTransfer: core.CanTransfer,
Transfer: core.Transfer,
GetHash: getHash,

View File

@ -9,6 +9,7 @@ import (
"github.com/ledgerwatch/erigon-lib/common/datadir"
"github.com/ledgerwatch/erigon-lib/kv/memdb"
libstate "github.com/ledgerwatch/erigon-lib/state"
"github.com/ledgerwatch/erigon/cmd/state/exec22"
"github.com/ledgerwatch/erigon/common"
"github.com/ledgerwatch/erigon/core"
"github.com/ledgerwatch/erigon/core/state"
@ -114,7 +115,7 @@ func ReplayTx(genesis *core.Genesis) error {
}
defer agg.Close()
ac := agg.MakeContext()
workCh := make(chan *state.TxTask)
workCh := make(chan *exec22.TxTask)
rs := state.NewReconState(workCh)
if err = replayTxNum(ctx, allSnapshots, blockReader, txNum, txNums, rs, ac); err != nil {
return err

View File

@ -0,0 +1,89 @@
package exec22
import (
"bytes"
"github.com/holiman/uint256"
"github.com/ledgerwatch/erigon/common"
"github.com/ledgerwatch/erigon/core/types"
"github.com/ledgerwatch/erigon/core/types/accounts"
"github.com/ledgerwatch/erigon/core/vm/evmtypes"
"github.com/ledgerwatch/erigon/params"
)
// ReadWriteSet contains ReadSet, WriteSet and BalanceIncrease of a transaction,
// which is processed by a single thread that writes into the ReconState1 and
// flushes to the database
type TxTask struct {
TxNum uint64
BlockNum uint64
Rules *params.Rules
Header *types.Header
Txs types.Transactions
Uncles []*types.Header
Coinbase common.Address
BlockHash common.Hash
Sender *common.Address
SkipAnalysis bool
TxIndex int // -1 for block initialisation
Final bool
Tx types.Transaction
GetHashFn func(n uint64) common.Hash
TxAsMessage types.Message
EvmBlockContext evmtypes.BlockContext
BalanceIncreaseSet map[common.Address]uint256.Int
ReadLists map[string]*KvList
WriteLists map[string]*KvList
AccountPrevs map[string][]byte
AccountDels map[string]*accounts.Account
StoragePrevs map[string][]byte
CodePrevs map[string]uint64
ResultsSize int64
Error error
Logs []*types.Log
TraceFroms map[common.Address]struct{}
TraceTos map[common.Address]struct{}
}
type TxTaskQueue []*TxTask
func (h TxTaskQueue) Len() int {
return len(h)
}
func (h TxTaskQueue) Less(i, j int) bool {
return h[i].TxNum < h[j].TxNum
}
func (h TxTaskQueue) Swap(i, j int) {
h[i], h[j] = h[j], h[i]
}
func (h *TxTaskQueue) Push(a interface{}) {
*h = append(*h, a.(*TxTask))
}
func (h *TxTaskQueue) Pop() interface{} {
c := *h
*h = c[:len(c)-1]
return c[len(c)-1]
}
// KvList sort.Interface to sort write list by keys
type KvList struct {
Keys, Vals [][]byte
}
func (l KvList) Len() int {
return len(l.Keys)
}
func (l KvList) Less(i, j int) bool {
return bytes.Compare(l.Keys[i], l.Keys[j]) < 0
}
func (l *KvList) Swap(i, j int) {
l.Keys[i], l.Keys[j] = l.Keys[j], l.Keys[i]
l.Vals[i], l.Vals[j] = l.Vals[j], l.Vals[i]
}

View File

@ -3,10 +3,10 @@ package exec3
import (
"context"
"math/big"
"runtime/debug"
"sync"
"github.com/ledgerwatch/erigon-lib/kv"
"github.com/ledgerwatch/erigon/cmd/state/exec22"
"github.com/ledgerwatch/erigon/common"
"github.com/ledgerwatch/erigon/consensus"
"github.com/ledgerwatch/erigon/consensus/misc"
@ -16,41 +16,46 @@ import (
"github.com/ledgerwatch/erigon/core/systemcontracts"
"github.com/ledgerwatch/erigon/core/types"
"github.com/ledgerwatch/erigon/core/vm"
"github.com/ledgerwatch/erigon/core/vm/evmtypes"
"github.com/ledgerwatch/erigon/params"
"github.com/ledgerwatch/erigon/turbo/services"
"github.com/ledgerwatch/log/v3"
)
type Worker22 struct {
type Worker struct {
lock sync.Locker
wg *sync.WaitGroup
chainDb kv.RoDB
chainTx kv.Tx
background bool
blockReader services.FullBlockReader
rs *state.State22
stateWriter *state.StateWriter22
stateReader *state.StateReader22
chainConfig *params.ChainConfig
getHeader func(hash common.Hash, number uint64) *types.Header
ctx context.Context
engine consensus.Engine
logger log.Logger
genesis *core.Genesis
resultCh chan *state.TxTask
resultCh chan *exec22.TxTask
epoch EpochReader
chain ChainReader
isPoSA bool
posa consensus.PoSA
starkNetEvm *vm.CVMAdapter
evm *vm.EVM
}
func NewWorker22(lock sync.Locker, chainDb kv.RoDB, wg *sync.WaitGroup, rs *state.State22, blockReader services.FullBlockReader, chainConfig *params.ChainConfig, logger log.Logger, genesis *core.Genesis, resultCh chan *state.TxTask, engine consensus.Engine) *Worker22 {
func NewWorker(lock sync.Locker, background bool, chainDb kv.RoDB, wg *sync.WaitGroup, rs *state.State22, blockReader services.FullBlockReader, chainConfig *params.ChainConfig, logger log.Logger, genesis *core.Genesis, resultCh chan *exec22.TxTask, engine consensus.Engine) *Worker {
ctx := context.Background()
w := &Worker22{
w := &Worker{
lock: lock,
chainDb: chainDb,
wg: wg,
rs: rs,
background: background,
blockReader: blockReader,
stateWriter: state.NewStateWriter22(rs),
stateReader: state.NewStateReader22(rs),
@ -61,22 +66,19 @@ func NewWorker22(lock sync.Locker, chainDb kv.RoDB, wg *sync.WaitGroup, rs *stat
genesis: genesis,
resultCh: resultCh,
engine: engine,
}
w.getHeader = func(hash common.Hash, number uint64) *types.Header {
h, err := blockReader.Header(ctx, w.chainTx, hash, number)
if err != nil {
panic(err)
}
return h
starkNetEvm: &vm.CVMAdapter{Cvm: vm.NewCVM(nil)},
evm: vm.NewEVM(evmtypes.BlockContext{}, evmtypes.TxContext{}, nil, chainConfig, vm.Config{}),
}
w.posa, w.isPoSA = engine.(consensus.PoSA)
return w
}
func (rw *Worker22) Tx() kv.Tx { return rw.chainTx }
func (rw *Worker22) ResetTx(chainTx kv.Tx) {
if rw.chainTx != nil {
func (rw *Worker) Tx() kv.Tx { return rw.chainTx }
func (rw *Worker) ResetTx(chainTx kv.Tx) {
if rw.background && rw.chainTx != nil {
rw.chainTx.Rollback()
rw.chainTx = nil
}
@ -88,7 +90,7 @@ func (rw *Worker22) ResetTx(chainTx kv.Tx) {
}
}
func (rw *Worker22) Run() {
func (rw *Worker) Run() {
defer rw.wg.Done()
for txTask, ok := rw.rs.Schedule(); ok; txTask, ok = rw.rs.Schedule() {
rw.RunTxTask(txTask)
@ -96,10 +98,10 @@ func (rw *Worker22) Run() {
}
}
func (rw *Worker22) RunTxTask(txTask *state.TxTask) {
func (rw *Worker) RunTxTask(txTask *exec22.TxTask) {
rw.lock.Lock()
defer rw.lock.Unlock()
if rw.chainTx == nil {
if rw.background && rw.chainTx == nil {
var err error
if rw.chainTx, err = rw.chainDb.BeginRo(rw.ctx); err != nil {
panic(err)
@ -173,17 +175,16 @@ func (rw *Worker22) RunTxTask(txTask *state.TxTask) {
gp := new(core.GasPool).AddGas(txTask.Tx.GetGas())
ct := NewCallTracer()
vmConfig := vm.Config{Debug: true, Tracer: ct, SkipAnalysis: txTask.SkipAnalysis}
getHashFn := core.GetHashFn(header, rw.getHeader)
ibs.Prepare(txHash, txTask.BlockHash, txTask.TxIndex)
msg := txTask.TxAsMessage
var vmenv vm.VMInterface
if txTask.Tx.IsStarkNet() {
vmenv = &vm.CVMAdapter{Cvm: vm.NewCVM(ibs)}
rw.starkNetEvm.Reset(evmtypes.TxContext{}, ibs)
vmenv = rw.starkNetEvm
} else {
blockContext := core.NewEVMBlockContext(header, getHashFn, rw.engine, nil /* author */)
txContext := core.NewEVMTxContext(msg)
vmenv = vm.NewEVM(blockContext, txContext, ibs, rw.chainConfig, vmConfig)
rw.evm.ResetBetweenBlocks(txTask.EvmBlockContext, core.NewEVMTxContext(msg), ibs, vmConfig, txTask.Rules)
vmenv = rw.evm
}
if _, err = core.ApplyMessage(vmenv, msg, gp, true /* refunds */, false /* gasBailout */); err != nil {
txTask.Error = err
@ -298,22 +299,19 @@ func (cr EpochReader) FindBeforeOrEqualNumber(number uint64) (blockNum uint64, b
return rawdb.FindEpochBeforeOrEqualNumber(cr.tx, number)
}
func NewWorkersPool(lock sync.Locker, chainDb kv.RoDB, wg *sync.WaitGroup, rs *state.State22, blockReader services.FullBlockReader, chainConfig *params.ChainConfig, logger log.Logger, genesis *core.Genesis, engine consensus.Engine, workerCount int) (reconWorkers []*Worker22, resultCh chan *state.TxTask, clear func()) {
queueSize := workerCount * 64
reconWorkers = make([]*Worker22, workerCount)
resultCh = make(chan *state.TxTask, queueSize)
func NewWorkersPool(lock sync.Locker, background bool, chainDb kv.RoDB, wg *sync.WaitGroup, rs *state.State22, blockReader services.FullBlockReader, chainConfig *params.ChainConfig, logger log.Logger, genesis *core.Genesis, engine consensus.Engine, workerCount int) (reconWorkers []*Worker, resultCh chan *exec22.TxTask, clear func()) {
queueSize := workerCount * 4
reconWorkers = make([]*Worker, workerCount)
resultCh = make(chan *exec22.TxTask, queueSize)
for i := 0; i < workerCount; i++ {
reconWorkers[i] = NewWorker22(lock, chainDb, wg, rs, blockReader, chainConfig, logger, genesis, resultCh, engine)
reconWorkers[i] = NewWorker(lock, background, chainDb, wg, rs, blockReader, chainConfig, logger, genesis, resultCh, engine)
}
clear = func() {
if rec := recover(); rec != nil {
log.Error("Some panic happen", "panic", rec, "stack", debug.Stack())
}
for _, w := range reconWorkers {
w.ResetTx(nil)
}
}
if workerCount > 1 {
if background {
wg.Add(workerCount)
for i := 0; i < workerCount; i++ {
go reconWorkers[i].Run()

View File

@ -11,13 +11,13 @@ import (
"github.com/ledgerwatch/erigon-lib/etl"
"github.com/ledgerwatch/erigon-lib/kv"
"github.com/ledgerwatch/erigon-lib/state"
"github.com/ledgerwatch/erigon/cmd/state/exec22"
"github.com/ledgerwatch/erigon/common"
"github.com/ledgerwatch/erigon/consensus"
"github.com/ledgerwatch/erigon/consensus/misc"
"github.com/ledgerwatch/erigon/core"
state2 "github.com/ledgerwatch/erigon/core/state"
"github.com/ledgerwatch/erigon/core/systemcontracts"
"github.com/ledgerwatch/erigon/core/types"
"github.com/ledgerwatch/erigon/core/types/accounts"
"github.com/ledgerwatch/erigon/core/vm"
"github.com/ledgerwatch/erigon/params"
@ -229,7 +229,6 @@ type ReconWorker struct {
blockReader services.FullBlockReader
stateWriter *state2.StateReconWriter
stateReader *state2.HistoryReaderNoState
getHeader func(hash common.Hash, number uint64) *types.Header
ctx context.Context
engine consensus.Engine
chainConfig *params.ChainConfig
@ -273,13 +272,6 @@ func (rw *ReconWorker) SetTx(tx kv.Tx) {
func (rw *ReconWorker) Run() {
defer rw.wg.Done()
rw.getHeader = func(hash common.Hash, number uint64) *types.Header {
h, err := rw.blockReader.Header(rw.ctx, nil, hash, number)
if err != nil {
panic(err)
}
return h
}
for txTask, ok := rw.rs.Schedule(); ok; txTask, ok = rw.rs.Schedule() {
rw.runTxTask(txTask)
}
@ -287,7 +279,7 @@ func (rw *ReconWorker) Run() {
var noop = state2.NewNoopWriter()
func (rw *ReconWorker) runTxTask(txTask *state2.TxTask) {
func (rw *ReconWorker) runTxTask(txTask *exec22.TxTask) {
rw.lock.Lock()
defer rw.lock.Unlock()
rw.stateReader.SetTxNum(txTask.TxNum)
@ -341,7 +333,6 @@ func (rw *ReconWorker) runTxTask(txTask *state2.TxTask) {
}
gp := new(core.GasPool).AddGas(txTask.Tx.GetGas())
vmConfig := vm.Config{NoReceipts: true, SkipAnalysis: txTask.SkipAnalysis}
getHashFn := core.GetHashFn(txTask.Header, rw.getHeader)
ibs.Prepare(txTask.Tx.Hash(), txTask.BlockHash, txTask.TxIndex)
msg := txTask.TxAsMessage
@ -349,9 +340,7 @@ func (rw *ReconWorker) runTxTask(txTask *state2.TxTask) {
if txTask.Tx.IsStarkNet() {
vmenv = &vm.CVMAdapter{Cvm: vm.NewCVM(ibs)}
} else {
blockContext := core.NewEVMBlockContext(txTask.Header, getHashFn, rw.engine, nil /* author */)
txContext := core.NewEVMTxContext(msg)
vmenv = vm.NewEVM(blockContext, txContext, ibs, rw.chainConfig, vmConfig)
vmenv = vm.NewEVM(txTask.EvmBlockContext, core.NewEVMTxContext(msg), ibs, rw.chainConfig, vmConfig)
}
//fmt.Printf("txNum=%d, blockNum=%d, txIndex=%d, evm=%p\n", txTask.TxNum, txTask.BlockNum, txTask.TxIndex, vmenv)
_, err = core.ApplyMessage(vmenv, msg, gp, true /* refunds */, false /* gasBailout */)

View File

@ -32,7 +32,6 @@ import (
"github.com/ledgerwatch/erigon/rlp"
"github.com/ledgerwatch/erigon/rpc"
"github.com/ledgerwatch/log/v3"
"golang.org/x/crypto/sha3"
)
const (
@ -145,7 +144,9 @@ func ecrecover(header *types.Header, sigcache *lru.ARCCache, c *params.BorConfig
// SealHash returns the hash of a block prior to it being sealed.
func SealHash(header *types.Header, c *params.BorConfig) (hash common.Hash) {
hasher := sha3.NewLegacyKeccak256()
hasher := crypto.NewLegacyKeccak256()
defer crypto.ReturnToPoolKeccak256(hasher)
encodeSigHeader(hasher, header, c)
hasher.Sum(hash[:0])
return hash

View File

@ -29,10 +29,8 @@ import (
"github.com/goccy/go-json"
lru "github.com/hashicorp/golang-lru"
"github.com/ledgerwatch/erigon-lib/kv"
"golang.org/x/crypto/sha3"
libcommon "github.com/ledgerwatch/erigon-lib/common"
"github.com/ledgerwatch/erigon-lib/kv"
"github.com/ledgerwatch/erigon/common"
"github.com/ledgerwatch/erigon/common/dbutils"
"github.com/ledgerwatch/erigon/common/debug"
@ -526,7 +524,9 @@ func (c *Clique) APIs(chain consensus.ChainHeaderReader) []rpc.API {
// SealHash returns the hash of a block prior to it being sealed.
func SealHash(header *types.Header) (hash common.Hash) {
hasher := sha3.NewLegacyKeccak256()
hasher := crypto.NewLegacyKeccak256()
defer crypto.ReturnToPoolKeccak256(hasher)
encodeSigHeader(hasher, header)
hasher.Sum(hash[:0])
return hash

View File

@ -20,7 +20,6 @@ import (
"github.com/holiman/uint256"
"github.com/ledgerwatch/erigon-lib/kv"
"github.com/ledgerwatch/log/v3"
"golang.org/x/crypto/sha3"
"golang.org/x/exp/slices"
"github.com/ledgerwatch/erigon/accounts/abi"
@ -174,7 +173,9 @@ func ecrecover(header *types.Header, sigCache *lru.ARCCache, chainId *big.Int) (
// SealHash returns the hash of a block prior to it being sealed.
func SealHash(header *types.Header, chainId *big.Int) (hash common.Hash) {
hasher := sha3.NewLegacyKeccak256()
hasher := crypto.NewLegacyKeccak256()
defer crypto.ReturnToPoolKeccak256(hasher)
encodeSigHeader(hasher, header, chainId)
hasher.Sum(hash[:0])
return hash

View File

@ -22,6 +22,7 @@ import (
"time"
"github.com/ledgerwatch/erigon/core/systemcontracts"
"github.com/ledgerwatch/erigon/core/vm/evmtypes"
"github.com/ledgerwatch/erigon/rlp"
"golang.org/x/crypto/sha3"
"golang.org/x/exp/slices"
@ -481,11 +482,11 @@ func SysCallContract(contract common.Address, data []byte, chainConfig params.Ch
vmConfig := vm.Config{NoReceipts: true, RestoreState: constCall}
// Create a new context to be used in the EVM environment
isBor := chainConfig.Bor != nil
var txContext vm.TxContext
var txContext evmtypes.TxContext
var author *common.Address
if isBor {
author = &header.Coinbase
txContext = vm.TxContext{}
txContext = evmtypes.TxContext{}
} else {
author = &state.SystemAddress
txContext = NewEVMTxContext(msg)

View File

@ -4,7 +4,7 @@ import (
"github.com/holiman/uint256"
"github.com/ledgerwatch/erigon/common"
"github.com/ledgerwatch/erigon/core/types"
"github.com/ledgerwatch/erigon/core/vm"
"github.com/ledgerwatch/erigon/core/vm/evmtypes"
)
var transferLogSig = common.HexToHash("0xe6497e3ee548a3372136af2fcb0696db31fc6cf20260707645068bd3fe97f3c4")
@ -14,7 +14,7 @@ var zero = uint256.NewInt(0)
// AddTransferLog adds transfer log into state
func AddTransferLog(
state vm.IntraBlockState,
state evmtypes.IntraBlockState,
sender,
recipient common.Address,
@ -43,7 +43,7 @@ func AddTransferLog(
// AddFeeTransferLog adds transfer log into state
// Deprecating transfer log and will be removed in future fork. PLEASE DO NOT USE this transfer log going forward. Parameters won't get updated as expected going forward with EIP1559
func AddFeeTransferLog(
state vm.IntraBlockState,
state evmtypes.IntraBlockState,
sender,
recipient common.Address,
@ -71,7 +71,7 @@ func AddFeeTransferLog(
// addTransferLog adds transfer log into state
func addTransferLog(
state vm.IntraBlockState,
state evmtypes.IntraBlockState,
eventSig common.Hash,
sender,

View File

@ -25,12 +25,12 @@ import (
"github.com/ledgerwatch/erigon/consensus"
"github.com/ledgerwatch/erigon/consensus/serenity"
"github.com/ledgerwatch/erigon/core/types"
"github.com/ledgerwatch/erigon/core/vm"
"github.com/ledgerwatch/erigon/core/vm/evmtypes"
"github.com/ledgerwatch/erigon/params"
)
// NewEVMBlockContext creates a new context for use in the EVM.
func NewEVMBlockContext(header *types.Header, blockHashFunc func(n uint64) common.Hash, engine consensus.Engine, author *common.Address) vm.BlockContext {
func NewEVMBlockContext(header *types.Header, blockHashFunc func(n uint64) common.Hash, engine consensus.Engine, author *common.Address) evmtypes.BlockContext {
// If we don't have an explicit author (i.e. not mining), extract from the header
var beneficiary common.Address
if author == nil {
@ -52,14 +52,14 @@ func NewEVMBlockContext(header *types.Header, blockHashFunc func(n uint64) commo
prevRandDao = &header.MixDigest
}
var transferFunc vm.TransferFunc
var transferFunc evmtypes.TransferFunc
if engine != nil && engine.Type() == params.BorConsensus {
transferFunc = BorTransfer
} else {
transferFunc = Transfer
}
return vm.BlockContext{
return evmtypes.BlockContext{
CanTransfer: CanTransfer,
Transfer: transferFunc,
GetHash: blockHashFunc,
@ -74,8 +74,8 @@ func NewEVMBlockContext(header *types.Header, blockHashFunc func(n uint64) commo
}
// NewEVMTxContext creates a new transaction context for a single transaction.
func NewEVMTxContext(msg Message) vm.TxContext {
return vm.TxContext{
func NewEVMTxContext(msg Message) evmtypes.TxContext {
return evmtypes.TxContext{
Origin: msg.From(),
GasPrice: msg.GasPrice().ToBig(),
}
@ -117,12 +117,12 @@ func GetHashFn(ref *types.Header, getHeader func(hash common.Hash, number uint64
// CanTransfer checks whether there are enough funds in the address' account to make a transfer.
// This does not take the necessary gas in to account to make the transfer valid.
func CanTransfer(db vm.IntraBlockState, addr common.Address, amount *uint256.Int) bool {
func CanTransfer(db evmtypes.IntraBlockState, addr common.Address, amount *uint256.Int) bool {
return !db.GetBalance(addr).Lt(amount)
}
// Transfer subtracts amount from sender and adds amount to recipient using the given Db
func Transfer(db vm.IntraBlockState, sender, recipient common.Address, amount *uint256.Int, bailout bool) {
func Transfer(db evmtypes.IntraBlockState, sender, recipient common.Address, amount *uint256.Int, bailout bool) {
if !bailout {
db.SubBalance(sender, amount)
}
@ -130,7 +130,7 @@ func Transfer(db vm.IntraBlockState, sender, recipient common.Address, amount *u
}
// BorTransfer transfer in Bor
func BorTransfer(db vm.IntraBlockState, sender, recipient common.Address, amount *uint256.Int, bailout bool) {
func BorTransfer(db evmtypes.IntraBlockState, sender, recipient common.Address, amount *uint256.Int, bailout bool) {
// get inputs before
input1 := db.GetBalance(sender).Clone()
input2 := db.GetBalance(recipient).Clone()

View File

@ -0,0 +1 @@
package exec

View File

@ -15,81 +15,23 @@ import (
"github.com/ledgerwatch/erigon-lib/etl"
"github.com/ledgerwatch/erigon-lib/kv"
libstate "github.com/ledgerwatch/erigon-lib/state"
"github.com/ledgerwatch/erigon/cmd/state/exec22"
"github.com/ledgerwatch/erigon/common"
"github.com/ledgerwatch/erigon/common/dbutils"
"github.com/ledgerwatch/erigon/core/types"
"github.com/ledgerwatch/erigon/core/types/accounts"
"github.com/ledgerwatch/erigon/params"
"github.com/ledgerwatch/erigon/turbo/shards"
atomic2 "go.uber.org/atomic"
)
// ReadWriteSet contains ReadSet, WriteSet and BalanceIncrease of a transaction,
// which is processed by a single thread that writes into the ReconState1 and
// flushes to the database
type TxTask struct {
TxNum uint64
BlockNum uint64
Rules *params.Rules
Header *types.Header
Txs types.Transactions
Uncles []*types.Header
Coinbase common.Address
BlockHash common.Hash
Sender *common.Address
SkipAnalysis bool
TxIndex int // -1 for block initialisation
Final bool
Tx types.Transaction
GetHashFn func(n uint64) common.Hash
TxAsMessage types.Message
BalanceIncreaseSet map[common.Address]uint256.Int
ReadLists map[string]*KvList
WriteLists map[string]*KvList
AccountPrevs map[string][]byte
AccountDels map[string]*accounts.Account
StoragePrevs map[string][]byte
CodePrevs map[string]uint64
ResultsSize int64
Error error
Logs []*types.Log
TraceFroms map[common.Address]struct{}
TraceTos map[common.Address]struct{}
}
type TxTaskQueue []*TxTask
func (h TxTaskQueue) Len() int {
return len(h)
}
func (h TxTaskQueue) Less(i, j int) bool {
return h[i].TxNum < h[j].TxNum
}
func (h TxTaskQueue) Swap(i, j int) {
h[i], h[j] = h[j], h[i]
}
func (h *TxTaskQueue) Push(a interface{}) {
*h = append(*h, a.(*TxTask))
}
func (h *TxTaskQueue) Pop() interface{} {
c := *h
*h = c[:len(c)-1]
return c[len(c)-1]
}
const CodeSizeTable = "CodeSize"
type State22 struct {
lock sync.RWMutex
receiveWork *sync.Cond
triggers map[uint64]*TxTask
triggers map[uint64]*exec22.TxTask
senderTxNums map[common.Address]uint64
triggerLock sync.RWMutex
queue TxTaskQueue
queue exec22.TxTaskQueue
queueLock sync.Mutex
changes map[string]*btree.BTreeG[statePair]
sizeEstimate uint64
@ -107,7 +49,7 @@ func stateItemLess(i, j statePair) bool {
func NewState22() *State22 {
rs := &State22{
triggers: map[uint64]*TxTask{},
triggers: map[uint64]*exec22.TxTask{},
senderTxNums: map[common.Address]uint64{},
changes: map[string]*btree.BTreeG[statePair]{},
txsDone: atomic2.NewUint64(0),
@ -180,19 +122,19 @@ func (rs *State22) Flush(rwTx kv.RwTx) error {
return nil
}
func (rs *State22) Schedule() (*TxTask, bool) {
func (rs *State22) Schedule() (*exec22.TxTask, bool) {
rs.queueLock.Lock()
defer rs.queueLock.Unlock()
for !rs.finished && rs.queue.Len() == 0 {
rs.receiveWork.Wait()
}
if rs.queue.Len() > 0 {
return heap.Pop(&rs.queue).(*TxTask), true
return heap.Pop(&rs.queue).(*exec22.TxTask), true
}
return nil, false
}
func (rs *State22) RegisterSender(txTask *TxTask) bool {
func (rs *State22) RegisterSender(txTask *exec22.TxTask) bool {
rs.triggerLock.Lock()
defer rs.triggerLock.Unlock()
lastTxNum, deferral := rs.senderTxNums[*txTask.Sender]
@ -227,16 +169,22 @@ func (rs *State22) CommitTxNum(sender *common.Address, txNum uint64) uint64 {
return count
}
func (rs *State22) queuePush(t *TxTask) {
func (rs *State22) queuePush(t *exec22.TxTask) {
rs.queueLock.Lock()
heap.Push(&rs.queue, t)
rs.queueLock.Unlock()
}
func (rs *State22) AddWork(txTask *TxTask) {
func (rs *State22) AddWork(txTask *exec22.TxTask) {
txTask.BalanceIncreaseSet = nil
txTask.ReadLists = nil
txTask.WriteLists = nil
if txTask.ReadLists != nil {
returnReadList(txTask.ReadLists)
txTask.ReadLists = nil
}
if txTask.WriteLists != nil {
returnWriteList(txTask.WriteLists)
txTask.WriteLists = nil
}
txTask.ResultsSize = 0
txTask.Logs = nil
txTask.TraceFroms = nil
@ -261,7 +209,7 @@ func (rs *State22) Finish() {
rs.receiveWork.Broadcast()
}
func (rs *State22) appplyState1(roTx kv.Tx, txTask *TxTask, agg *libstate.Aggregator22) error {
func (rs *State22) appplyState1(roTx kv.Tx, txTask *exec22.TxTask, agg *libstate.Aggregator22) error {
rs.lock.RLock()
defer rs.lock.RUnlock()
@ -371,7 +319,7 @@ func (rs *State22) appplyState1(roTx kv.Tx, txTask *TxTask, agg *libstate.Aggreg
return nil
}
func (rs *State22) appplyState(roTx kv.Tx, txTask *TxTask, agg *libstate.Aggregator22) error {
func (rs *State22) appplyState(roTx kv.Tx, txTask *exec22.TxTask, agg *libstate.Aggregator22) error {
emptyRemoval := txTask.Rules.IsSpuriousDragon
rs.lock.Lock()
defer rs.lock.Unlock()
@ -419,7 +367,7 @@ func (rs *State22) appplyState(roTx kv.Tx, txTask *TxTask, agg *libstate.Aggrega
return nil
}
func (rs *State22) ApplyState(roTx kv.Tx, txTask *TxTask, agg *libstate.Aggregator22) error {
func (rs *State22) ApplyState(roTx kv.Tx, txTask *exec22.TxTask, agg *libstate.Aggregator22) error {
agg.SetTxNum(txTask.TxNum)
if err := rs.appplyState1(roTx, txTask, agg); err != nil {
return err
@ -427,10 +375,15 @@ func (rs *State22) ApplyState(roTx kv.Tx, txTask *TxTask, agg *libstate.Aggregat
if err := rs.appplyState(roTx, txTask, agg); err != nil {
return err
}
returnReadList(txTask.ReadLists)
returnWriteList(txTask.WriteLists)
txTask.ReadLists, txTask.WriteLists = nil, nil
return nil
}
func (rs *State22) ApplyHistory(roTx kv.Tx, txTask *TxTask, agg *libstate.Aggregator22) error {
func (rs *State22) ApplyHistory(roTx kv.Tx, txTask *exec22.TxTask, agg *libstate.Aggregator22) error {
for addrS, enc0 := range txTask.AccountPrevs {
if err := agg.AddAccountPrev([]byte(addrS), enc0); err != nil {
return err
@ -573,7 +526,7 @@ func (rs *State22) SizeEstimate() uint64 {
return r
}
func (rs *State22) ReadsValid(readLists map[string]*KvList) bool {
func (rs *State22) ReadsValid(readLists map[string]*exec22.KvList) bool {
search := statePair{}
var t *btree.BTreeG[statePair]
@ -608,28 +561,10 @@ func (rs *State22) ReadsValid(readLists map[string]*KvList) bool {
return true
}
// KvList sort.Interface to sort write list by keys
type KvList struct {
Keys, Vals [][]byte
}
func (l KvList) Len() int {
return len(l.Keys)
}
func (l KvList) Less(i, j int) bool {
return bytes.Compare(l.Keys[i], l.Keys[j]) < 0
}
func (l *KvList) Swap(i, j int) {
l.Keys[i], l.Keys[j] = l.Keys[j], l.Keys[i]
l.Vals[i], l.Vals[j] = l.Vals[j], l.Vals[i]
}
type StateWriter22 struct {
rs *State22
txNum uint64
writeLists map[string]*KvList
writeLists map[string]*exec22.KvList
accountPrevs map[string][]byte
accountDels map[string]*accounts.Account
storagePrevs map[string][]byte
@ -638,13 +573,8 @@ type StateWriter22 struct {
func NewStateWriter22(rs *State22) *StateWriter22 {
return &StateWriter22{
rs: rs,
writeLists: map[string]*KvList{
kv.PlainState: {},
kv.Code: {},
kv.PlainContractCode: {},
kv.IncarnationMap: {},
},
rs: rs,
writeLists: newWriteList(),
accountPrevs: map[string][]byte{},
accountDels: map[string]*accounts.Account{},
storagePrevs: map[string][]byte{},
@ -657,19 +587,14 @@ func (w *StateWriter22) SetTxNum(txNum uint64) {
}
func (w *StateWriter22) ResetWriteSet() {
w.writeLists = map[string]*KvList{
kv.PlainState: {},
kv.Code: {},
kv.PlainContractCode: {},
kv.IncarnationMap: {},
}
w.writeLists = newWriteList()
w.accountPrevs = map[string][]byte{}
w.accountDels = map[string]*accounts.Account{}
w.storagePrevs = map[string][]byte{}
w.codePrevs = map[string]uint64{}
}
func (w *StateWriter22) WriteSet() map[string]*KvList {
func (w *StateWriter22) WriteSet() map[string]*exec22.KvList {
return w.writeLists
}
@ -740,18 +665,13 @@ type StateReader22 struct {
trace bool
rs *State22
composite []byte
readLists map[string]*KvList
readLists map[string]*exec22.KvList
}
func NewStateReader22(rs *State22) *StateReader22 {
return &StateReader22{
rs: rs,
readLists: map[string]*KvList{
kv.PlainState: {},
kv.Code: {},
CodeSizeTable: {},
kv.IncarnationMap: {},
},
rs: rs,
readLists: newReadList(),
}
}
@ -764,15 +684,10 @@ func (r *StateReader22) SetTx(tx kv.Tx) {
}
func (r *StateReader22) ResetReadSet() {
r.readLists = map[string]*KvList{
kv.PlainState: {},
kv.Code: {},
CodeSizeTable: {},
kv.IncarnationMap: {},
}
r.readLists = newReadList()
}
func (r *StateReader22) ReadSet() map[string]*KvList {
func (r *StateReader22) ReadSet() map[string]*exec22.KvList {
return r.readLists
}
@ -893,3 +808,43 @@ func (r *StateReader22) ReadAccountIncarnation(address common.Address) (uint64,
}
return binary.BigEndian.Uint64(enc), nil
}
var writeListPool = sync.Pool{
New: func() any {
return map[string]*exec22.KvList{
kv.PlainState: {},
kv.Code: {},
kv.PlainContractCode: {},
kv.IncarnationMap: {},
}
},
}
func newWriteList() map[string]*exec22.KvList {
w := writeListPool.Get().(map[string]*exec22.KvList)
for _, tbl := range w {
tbl.Keys, tbl.Vals = tbl.Keys[:0], tbl.Vals[:0]
}
return w
}
func returnWriteList(w map[string]*exec22.KvList) { writeListPool.Put(w) }
var readListPool = sync.Pool{
New: func() any {
return map[string]*exec22.KvList{
kv.PlainState: {},
kv.Code: {},
CodeSizeTable: {},
kv.IncarnationMap: {},
}
},
}
func newReadList() map[string]*exec22.KvList {
w := readListPool.Get().(map[string]*exec22.KvList)
for _, tbl := range w {
tbl.Keys, tbl.Vals = tbl.Keys[:0], tbl.Vals[:0]
}
return w
}
func returnReadList(w map[string]*exec22.KvList) { readListPool.Put(w) }

View File

@ -13,6 +13,7 @@ import (
"github.com/holiman/uint256"
"github.com/ledgerwatch/erigon-lib/kv"
libstate "github.com/ledgerwatch/erigon-lib/state"
"github.com/ledgerwatch/erigon/cmd/state/exec22"
"github.com/ledgerwatch/erigon/common"
"github.com/ledgerwatch/erigon/common/dbutils"
"github.com/ledgerwatch/erigon/core/types/accounts"
@ -41,9 +42,9 @@ func ReconnLess(i, thanItem reconPair) bool {
type ReconnWork struct {
lock sync.RWMutex
doneBitmap roaring64.Bitmap
triggers map[uint64][]*TxTask
workCh chan *TxTask
queue TxTaskQueue
triggers map[uint64][]*exec22.TxTask
workCh chan *exec22.TxTask
queue exec22.TxTaskQueue
rollbackCount uint64
}
@ -56,11 +57,11 @@ type ReconState struct {
sizeEstimate uint64
}
func NewReconState(workCh chan *TxTask) *ReconState {
func NewReconState(workCh chan *exec22.TxTask) *ReconState {
rs := &ReconState{
ReconnWork: &ReconnWork{
workCh: workCh,
triggers: map[uint64][]*TxTask{},
triggers: map[uint64][]*exec22.TxTask{},
},
changes: map[string]*btree.BTreeG[reconPair]{},
}
@ -130,7 +131,7 @@ func (rs *ReconState) Flush(rwTx kv.RwTx) error {
return nil
}
func (rs *ReconnWork) Schedule() (*TxTask, bool) {
func (rs *ReconnWork) Schedule() (*exec22.TxTask, bool) {
rs.lock.Lock()
defer rs.lock.Unlock()
for rs.queue.Len() < 16 {
@ -142,7 +143,7 @@ func (rs *ReconnWork) Schedule() (*TxTask, bool) {
heap.Push(&rs.queue, txTask)
}
if rs.queue.Len() > 0 {
return heap.Pop(&rs.queue).(*TxTask), true
return heap.Pop(&rs.queue).(*exec22.TxTask), true
}
return nil, false
}
@ -159,7 +160,7 @@ func (rs *ReconnWork) CommitTxNum(txNum uint64) {
rs.doneBitmap.Add(txNum)
}
func (rs *ReconnWork) RollbackTx(txTask *TxTask, dependency uint64) {
func (rs *ReconnWork) RollbackTx(txTask *exec22.TxTask, dependency uint64) {
rs.lock.Lock()
defer rs.lock.Unlock()
if rs.doneBitmap.Contains(dependency) {

View File

@ -22,6 +22,7 @@ import (
"github.com/ledgerwatch/erigon/core/state"
"github.com/ledgerwatch/erigon/core/types"
"github.com/ledgerwatch/erigon/core/vm"
"github.com/ledgerwatch/erigon/core/vm/evmtypes"
"github.com/ledgerwatch/erigon/crypto"
"github.com/ledgerwatch/erigon/params"
)
@ -108,7 +109,7 @@ func ApplyTransaction(config *params.ChainConfig, blockHashFunc func(n uint64) c
vmenv = &vm.CVMAdapter{Cvm: vm.NewCVM(ibs)}
} else {
blockContext := NewEVMBlockContext(header, blockHashFunc, engine, author)
vmenv = vm.NewEVM(blockContext, vm.TxContext{}, ibs, config, cfg)
vmenv = vm.NewEVM(blockContext, evmtypes.TxContext{}, ibs, config, cfg)
}
return applyTransaction(config, engine, gp, ibs, stateWriter, header, tx, usedGas, vmenv, cfg)

View File

@ -20,6 +20,7 @@ import (
"fmt"
"github.com/holiman/uint256"
"github.com/ledgerwatch/erigon/core/vm/evmtypes"
"github.com/ledgerwatch/erigon/common"
"github.com/ledgerwatch/erigon/common/math"
@ -63,7 +64,7 @@ type StateTransition struct {
initialGas uint64
value *uint256.Int
data []byte
state vm.IntraBlockState
state evmtypes.IntraBlockState
evm vm.VMInterface
//some pre-allocated intermediate variables

View File

@ -136,11 +136,10 @@ func Bloom9(data []byte) []byte {
// bloomValues returns the bytes (index-value pairs) to set for the given data
func bloomValues(data []byte, hashbuf []byte) (uint, byte, uint, byte, uint, byte) {
sha := hasherPool.Get().(crypto.KeccakState)
sha.Reset()
sha := crypto.NewKeccakState()
sha.Write(data) //nolint:errcheck
sha.Read(hashbuf) //nolint:errcheck
hasherPool.Put(sha)
crypto.ReturnToPoolKeccak256(sha)
// The actual bits to flip
v1 := byte(1 << (hashbuf[1] & 0x7))
v2 := byte(1 << (hashbuf[3] & 0x7))

View File

@ -20,14 +20,12 @@ import (
"bytes"
"fmt"
"io"
"sync"
"github.com/ledgerwatch/erigon/common"
"github.com/ledgerwatch/erigon/crypto"
"github.com/ledgerwatch/erigon/rlp"
"github.com/ledgerwatch/erigon/turbo/rlphacks"
"github.com/ledgerwatch/erigon/turbo/trie"
"golang.org/x/crypto/sha3"
)
type DerivableList interface {
@ -160,37 +158,26 @@ func intsize(i uint) (size int) {
}
}
// hasherPool holds LegacyKeccak hashers.
var hasherPool = sync.Pool{
New: func() interface{} {
return sha3.NewLegacyKeccak256()
},
}
func RawRlpHash(rawRlpData rlp.RawValue) (h common.Hash) {
sha := hasherPool.Get().(crypto.KeccakState)
defer hasherPool.Put(sha)
sha.Reset()
sha := crypto.NewKeccakState()
sha.Write(rawRlpData) //nolint:errcheck
sha.Read(h[:]) //nolint:errcheck
crypto.ReturnToPoolKeccak256(sha)
return h
}
func rlpHash(x interface{}) (h common.Hash) {
sha := hasherPool.Get().(crypto.KeccakState)
defer hasherPool.Put(sha)
sha.Reset()
sha := crypto.NewKeccakState()
rlp.Encode(sha, x) //nolint:errcheck
sha.Read(h[:]) //nolint:errcheck
crypto.ReturnToPoolKeccak256(sha)
return h
}
// prefixedRlpHash writes the prefix into the hasher before rlp-encoding the
// given interface. It's used for typed transactions.
func prefixedRlpHash(prefix byte, x interface{}) (h common.Hash) {
sha := hasherPool.Get().(crypto.KeccakState)
defer hasherPool.Put(sha)
sha.Reset()
sha := crypto.NewKeccakState()
//nolint:errcheck
sha.Write([]byte{prefix})
if err := rlp.Encode(sha, x); err != nil {
@ -198,5 +185,6 @@ func prefixedRlpHash(prefix byte, x interface{}) (h common.Hash) {
}
//nolint:errcheck
sha.Read(h[:])
crypto.ReturnToPoolKeccak256(sha)
return h
}

View File

@ -4,10 +4,11 @@ import (
"fmt"
"github.com/ledgerwatch/erigon/common"
"github.com/ledgerwatch/erigon/core/vm/evmtypes"
"github.com/ledgerwatch/erigon/crypto"
)
func NewCVM(state IntraBlockState) *CVM {
func NewCVM(state evmtypes.IntraBlockState) *CVM {
cvm := &CVM{
intraBlockState: state,
}
@ -17,7 +18,7 @@ func NewCVM(state IntraBlockState) *CVM {
type CVM struct {
config Config
intraBlockState IntraBlockState
intraBlockState evmtypes.IntraBlockState
}
func (cvm *CVM) Create(caller ContractRef, code []byte) ([]byte, common.Address, error) {
@ -34,7 +35,7 @@ func (cvm *CVM) Config() Config {
return cvm.config
}
func (cvm *CVM) IntraBlockState() IntraBlockState {
func (cvm *CVM) IntraBlockState() evmtypes.IntraBlockState {
return cvm.intraBlockState
}

View File

@ -5,6 +5,7 @@ import (
"github.com/holiman/uint256"
"github.com/ledgerwatch/erigon/common"
"github.com/ledgerwatch/erigon/core/vm/evmtypes"
"github.com/ledgerwatch/erigon/params"
)
@ -14,7 +15,7 @@ type CVMAdapter struct {
Cvm *CVM
}
func (c *CVMAdapter) Reset(txCtx TxContext, ibs IntraBlockState) {
func (c *CVMAdapter) Reset(txCtx evmtypes.TxContext, ibs evmtypes.IntraBlockState) {
c.Cvm.intraBlockState = ibs
}
@ -42,14 +43,14 @@ func (cvm *CVMAdapter) ChainRules() *params.Rules {
return &params.Rules{}
}
func (cvm *CVMAdapter) Context() BlockContext {
return BlockContext{}
func (cvm *CVMAdapter) Context() evmtypes.BlockContext {
return evmtypes.BlockContext{}
}
func (cvm *CVMAdapter) IntraBlockState() IntraBlockState {
func (cvm *CVMAdapter) IntraBlockState() evmtypes.IntraBlockState {
return cvm.Cvm.IntraBlockState()
}
func (cvm *CVMAdapter) TxContext() TxContext {
return TxContext{}
func (cvm *CVMAdapter) TxContext() evmtypes.TxContext {
return evmtypes.TxContext{}
}

View File

@ -22,6 +22,7 @@ import (
"time"
"github.com/holiman/uint256"
"github.com/ledgerwatch/erigon/core/vm/evmtypes"
"github.com/ledgerwatch/erigon/common"
"github.com/ledgerwatch/erigon/common/u256"
@ -33,16 +34,6 @@ import (
// deployed contract addresses (relevant after the account abstraction).
var emptyCodeHash = crypto.Keccak256Hash(nil)
type (
// CanTransferFunc is the signature of a transfer guard function
CanTransferFunc func(IntraBlockState, common.Address, *uint256.Int) bool
// TransferFunc is the signature of a transfer function
TransferFunc func(IntraBlockState, common.Address, common.Address, *uint256.Int, bool)
// GetHashFunc returns the nth block hash in the blockchain
// and is used by the BLOCKHASH EVM op code.
GetHashFunc func(uint64) common.Hash
)
func (evm *EVM) precompile(addr common.Address) (PrecompiledContract, bool) {
var precompiles map[common.Address]PrecompiledContract
switch {
@ -72,37 +63,6 @@ func run(evm *EVM, contract *Contract, input []byte, readOnly bool) ([]byte, err
return evm.interpreter.Run(contract, input, readOnly)
}
// BlockContext provides the EVM with auxiliary information. Once provided
// it shouldn't be modified.
type BlockContext struct {
// CanTransfer returns whether the account contains
// sufficient ether to transfer the value
CanTransfer CanTransferFunc
// Transfer transfers ether from one account to the other
Transfer TransferFunc
// GetHash returns the hash corresponding to n
GetHash GetHashFunc
// Block information
Coinbase common.Address // Provides information for COINBASE
GasLimit uint64 // Provides information for GASLIMIT
MaxGasLimit bool // Use GasLimit override for 2^256-1 (to be compatible with OpenEthereum's trace_call)
BlockNumber uint64 // Provides information for NUMBER
Time uint64 // Provides information for TIME
Difficulty *big.Int // Provides information for DIFFICULTY
BaseFee *uint256.Int // Provides information for BASEFEE
PrevRanDao *common.Hash // Provides information for PREVRANDAO
}
// TxContext provides the EVM with information about a transaction.
// All fields can change between transactions.
type TxContext struct {
// Message information
TxHash common.Hash
Origin common.Address // Provides information for ORIGIN
GasPrice *big.Int // Provides information for GASPRICE
}
// EVM is the Ethereum Virtual Machine base object and provides
// the necessary tools to run a contract on the given state with
// the provided context. It should be noted that any error
@ -114,10 +74,10 @@ type TxContext struct {
// The EVM should never be reused and is not thread safe.
type EVM struct {
// Context provides auxiliary blockchain related information
context BlockContext
txContext TxContext
context evmtypes.BlockContext
txContext evmtypes.TxContext
// IntraBlockState gives access to the underlying state
intraBlockState IntraBlockState
intraBlockState evmtypes.IntraBlockState
// Depth is the current call stack
depth int
@ -142,7 +102,7 @@ type EVM struct {
// NewEVM returns a new EVM. The returned EVM is not thread safe and should
// only ever be used *once*.
func NewEVM(blockCtx BlockContext, txCtx TxContext, state IntraBlockState, chainConfig *params.ChainConfig, vmConfig Config) *EVM {
func NewEVM(blockCtx evmtypes.BlockContext, txCtx evmtypes.TxContext, state evmtypes.IntraBlockState, chainConfig *params.ChainConfig, vmConfig Config) *EVM {
evm := &EVM{
context: blockCtx,
txContext: txCtx,
@ -159,7 +119,7 @@ func NewEVM(blockCtx BlockContext, txCtx TxContext, state IntraBlockState, chain
// Reset resets the EVM with a new transaction context.Reset
// This is not threadsafe and should only be done very cautiously.
func (evm *EVM) Reset(txCtx TxContext, ibs IntraBlockState) {
func (evm *EVM) Reset(txCtx evmtypes.TxContext, ibs evmtypes.IntraBlockState) {
evm.txContext = txCtx
evm.intraBlockState = ibs
@ -167,6 +127,19 @@ func (evm *EVM) Reset(txCtx TxContext, ibs IntraBlockState) {
atomic.StoreInt32(&evm.abort, 0)
}
func (evm *EVM) ResetBetweenBlocks(blockCtx evmtypes.BlockContext, txCtx evmtypes.TxContext, ibs evmtypes.IntraBlockState, vmConfig Config, chainRules *params.Rules) {
evm.context = blockCtx
evm.txContext = txCtx
evm.intraBlockState = ibs
evm.config = vmConfig
evm.chainRules = chainRules
evm.interpreter = NewEVMInterpreter(evm, vmConfig)
// ensure the evm is reset to be used again
atomic.StoreInt32(&evm.abort, 0)
}
// Cancel cancels any running EVM operation. This may be called concurrently and
// it's safe to be called multiple times.
func (evm *EVM) Cancel() {
@ -467,14 +440,14 @@ func (evm *EVM) ChainRules() *params.Rules {
return evm.chainRules
}
func (evm *EVM) Context() BlockContext {
func (evm *EVM) Context() evmtypes.BlockContext {
return evm.context
}
func (evm *EVM) TxContext() TxContext {
func (evm *EVM) TxContext() evmtypes.TxContext {
return evm.txContext
}
func (evm *EVM) IntraBlockState() IntraBlockState {
func (evm *EVM) IntraBlockState() evmtypes.IntraBlockState {
return evm.intraBlockState
}

View File

@ -4,6 +4,7 @@ import (
"fmt"
"testing"
"github.com/ledgerwatch/erigon/core/vm/evmtypes"
"github.com/ledgerwatch/erigon/params"
"github.com/holiman/uint256"
@ -12,7 +13,7 @@ import (
func TestInterpreterReadonly(t *testing.T) {
rapid.Check(t, func(t *rapid.T) {
env := NewEVM(BlockContext{}, TxContext{}, &dummyStatedb{}, params.TestChainConfig, Config{})
env := NewEVM(evmtypes.BlockContext{}, evmtypes.TxContext{}, &dummyStatedb{}, params.TestChainConfig, Config{})
isEVMSliceTest := rapid.SliceOfN(rapid.Bool(), 1, -1).Draw(t, "tevm")
readOnlySliceTest := rapid.SliceOfN(rapid.Bool(), len(isEVMSliceTest), len(isEVMSliceTest)).Draw(t, "readonly")
@ -267,7 +268,7 @@ func TestReadonlyBasicCases(t *testing.T) {
t.Run(testcase.testName+evmsTestcase.suffix, func(t *testing.T) {
readonlySliceTest := testcase.readonlySliceTest
env := NewEVM(BlockContext{}, TxContext{}, &dummyStatedb{}, params.TestChainConfig, Config{})
env := NewEVM(evmtypes.BlockContext{}, evmtypes.TxContext{}, &dummyStatedb{}, params.TestChainConfig, Config{})
readonliesGot := make([]*readOnlyState, len(testcase.readonlySliceTest))
isEVMGot := make([]bool, len(evmsTestcase.emvs))

View File

@ -0,0 +1,100 @@
package evmtypes
import (
"math/big"
"github.com/holiman/uint256"
"github.com/ledgerwatch/erigon/common"
"github.com/ledgerwatch/erigon/core/types"
)
// BlockContext provides the EVM with auxiliary information. Once provided
// it shouldn't be modified.
type BlockContext struct {
// CanTransfer returns whether the account contains
// sufficient ether to transfer the value
CanTransfer CanTransferFunc
// Transfer transfers ether from one account to the other
Transfer TransferFunc
// GetHash returns the hash corresponding to n
GetHash GetHashFunc
// Block information
Coinbase common.Address // Provides information for COINBASE
GasLimit uint64 // Provides information for GASLIMIT
MaxGasLimit bool // Use GasLimit override for 2^256-1 (to be compatible with OpenEthereum's trace_call)
BlockNumber uint64 // Provides information for NUMBER
Time uint64 // Provides information for TIME
Difficulty *big.Int // Provides information for DIFFICULTY
BaseFee *uint256.Int // Provides information for BASEFEE
PrevRanDao *common.Hash // Provides information for PREVRANDAO
}
// TxContext provides the EVM with information about a transaction.
// All fields can change between transactions.
type TxContext struct {
// Message information
TxHash common.Hash
Origin common.Address // Provides information for ORIGIN
GasPrice *big.Int // Provides information for GASPRICE
}
type (
// CanTransferFunc is the signature of a transfer guard function
CanTransferFunc func(IntraBlockState, common.Address, *uint256.Int) bool
// TransferFunc is the signature of a transfer function
TransferFunc func(IntraBlockState, common.Address, common.Address, *uint256.Int, bool)
// GetHashFunc returns the nth block hash in the blockchain
// and is used by the BLOCKHASH EVM op code.
GetHashFunc func(uint64) common.Hash
)
// IntraBlockState is an EVM database for full state querying.
type IntraBlockState interface {
CreateAccount(common.Address, bool)
SubBalance(common.Address, *uint256.Int)
AddBalance(common.Address, *uint256.Int)
GetBalance(common.Address) *uint256.Int
GetNonce(common.Address) uint64
SetNonce(common.Address, uint64)
GetCodeHash(common.Address) common.Hash
GetCode(common.Address) []byte
SetCode(common.Address, []byte)
GetCodeSize(common.Address) int
AddRefund(uint64)
SubRefund(uint64)
GetRefund() uint64
GetCommittedState(common.Address, *common.Hash, *uint256.Int)
GetState(address common.Address, slot *common.Hash, outValue *uint256.Int)
SetState(common.Address, *common.Hash, uint256.Int)
Suicide(common.Address) bool
HasSuicided(common.Address) bool
// Exist reports whether the given account exists in state.
// Notably this should also return true for suicided accounts.
Exist(common.Address) bool
// Empty returns whether the given account is empty. Empty
// is defined according to EIP161 (balance = nonce = code = 0).
Empty(common.Address) bool
PrepareAccessList(sender common.Address, dest *common.Address, precompiles []common.Address, txAccesses types.AccessList)
AddressInAccessList(addr common.Address) bool
SlotInAccessList(addr common.Address, slot common.Hash) (addressOk bool, slotOk bool)
// AddAddressToAccessList adds the given address to the access list. This operation is safe to perform
// even if the feature/fork is not active yet
AddAddressToAccessList(addr common.Address)
// AddSlotToAccessList adds the given (address,slot) to the access list. This operation is safe to perform
// even if the feature/fork is not active yet
AddSlotToAccessList(addr common.Address, slot common.Hash)
RevertToSnapshot(int)
Snapshot() int
AddLog(*types.Log)
}

View File

@ -27,6 +27,7 @@ import (
"github.com/ledgerwatch/erigon/common"
"github.com/ledgerwatch/erigon/common/hexutil"
"github.com/ledgerwatch/erigon/core/state"
"github.com/ledgerwatch/erigon/core/vm/evmtypes"
"github.com/ledgerwatch/erigon/params"
)
@ -95,11 +96,11 @@ func TestEIP2200(t *testing.T) {
s.SetState(address, &common.Hash{}, *uint256.NewInt(uint64(tt.original)))
_ = s.CommitBlock(params.AllProtocolChanges.Rules(0), state.NewPlainStateWriter(tx, tx, 0))
vmctx := BlockContext{
CanTransfer: func(IntraBlockState, common.Address, *uint256.Int) bool { return true },
Transfer: func(IntraBlockState, common.Address, common.Address, *uint256.Int, bool) {},
vmctx := evmtypes.BlockContext{
CanTransfer: func(evmtypes.IntraBlockState, common.Address, *uint256.Int) bool { return true },
Transfer: func(evmtypes.IntraBlockState, common.Address, common.Address, *uint256.Int, bool) {},
}
vmenv := NewEVM(vmctx, TxContext{}, s, params.AllProtocolChanges, Config{ExtraEips: []int{2200}})
vmenv := NewEVM(vmctx, evmtypes.TxContext{}, s, params.AllProtocolChanges, Config{ExtraEips: []int{2200}})
_, gas, err := vmenv.Call(AccountRef(common.Address{}), address, nil, tt.gaspool, new(uint256.Int), false /* bailout */)
if !errors.Is(err, tt.failure) {
@ -140,16 +141,16 @@ func TestCreateGas(t *testing.T) {
s.SetCode(address, hexutil.MustDecode(tt.code))
_ = s.CommitBlock(params.TestChainConfig.Rules(0), state.NewPlainStateWriter(tx, tx, 0))
vmctx := BlockContext{
CanTransfer: func(IntraBlockState, common.Address, *uint256.Int) bool { return true },
Transfer: func(IntraBlockState, common.Address, common.Address, *uint256.Int, bool) {},
vmctx := evmtypes.BlockContext{
CanTransfer: func(evmtypes.IntraBlockState, common.Address, *uint256.Int) bool { return true },
Transfer: func(evmtypes.IntraBlockState, common.Address, common.Address, *uint256.Int, bool) {},
}
config := Config{}
if tt.eip3860 {
config.ExtraEips = []int{3860}
}
vmenv := NewEVM(vmctx, TxContext{}, s, params.TestChainConfig, config)
vmenv := NewEVM(vmctx, evmtypes.TxContext{}, s, params.TestChainConfig, config)
var startGas uint64 = math.MaxUint64
_, gas, err := vmenv.Call(AccountRef(common.Address{}), address, nil, startGas, new(uint256.Int), false /* bailout */)

View File

@ -25,6 +25,7 @@ import (
"testing"
"github.com/holiman/uint256"
"github.com/ledgerwatch/erigon/core/vm/evmtypes"
"github.com/ledgerwatch/erigon/common"
"github.com/ledgerwatch/erigon/core/vm/stack"
@ -95,7 +96,7 @@ func init() {
func testTwoOperandOp(t *testing.T, tests []TwoOperandTestcase, opFn executionFunc, name string) {
var (
env = NewEVM(BlockContext{}, TxContext{}, nil, params.TestChainConfig, Config{})
env = NewEVM(evmtypes.BlockContext{}, evmtypes.TxContext{}, nil, params.TestChainConfig, Config{})
stack = stack.New()
pc = uint64(0)
evmInterpreter = env.interpreter.(*EVMInterpreter)
@ -194,7 +195,7 @@ func TestSAR(t *testing.T) {
func TestAddMod(t *testing.T) {
var (
env = NewEVM(BlockContext{}, TxContext{}, nil, params.TestChainConfig, Config{})
env = NewEVM(evmtypes.BlockContext{}, evmtypes.TxContext{}, nil, params.TestChainConfig, Config{})
stack = stack.New()
evmInterpreter = NewEVMInterpreter(env, env.Config())
pc = uint64(0)
@ -281,7 +282,7 @@ func TestJsonTestcases(t *testing.T) {
func opBenchmark(bench *testing.B, op executionFunc, args ...string) {
var (
env = NewEVM(BlockContext{}, TxContext{}, nil, params.TestChainConfig, Config{})
env = NewEVM(evmtypes.BlockContext{}, evmtypes.TxContext{}, nil, params.TestChainConfig, Config{})
stack = stack.New()
evmInterpreter = NewEVMInterpreter(env, env.Config())
)
@ -515,7 +516,7 @@ func BenchmarkOpIsZero(b *testing.B) {
func TestOpMstore(t *testing.T) {
var (
env = NewEVM(BlockContext{}, TxContext{}, nil, params.TestChainConfig, Config{})
env = NewEVM(evmtypes.BlockContext{}, evmtypes.TxContext{}, nil, params.TestChainConfig, Config{})
stack = stack.New()
mem = NewMemory()
evmInterpreter = NewEVMInterpreter(env, env.Config())
@ -539,7 +540,7 @@ func TestOpMstore(t *testing.T) {
func BenchmarkOpMstore(bench *testing.B) {
var (
env = NewEVM(BlockContext{}, TxContext{}, nil, params.TestChainConfig, Config{})
env = NewEVM(evmtypes.BlockContext{}, evmtypes.TxContext{}, nil, params.TestChainConfig, Config{})
stack = stack.New()
mem = NewMemory()
evmInterpreter = NewEVMInterpreter(env, env.Config())
@ -560,7 +561,7 @@ func BenchmarkOpMstore(bench *testing.B) {
func BenchmarkOpKeccak256(bench *testing.B) {
var (
env = NewEVM(BlockContext{}, TxContext{}, nil, params.TestChainConfig, Config{})
env = NewEVM(evmtypes.BlockContext{}, evmtypes.TxContext{}, nil, params.TestChainConfig, Config{})
stack = stack.New()
mem = NewMemory()
evmInterpreter = NewEVMInterpreter(env, env.Config())

View File

@ -19,64 +19,14 @@ package vm
import (
"math/big"
"github.com/ledgerwatch/erigon/core/vm/evmtypes"
"github.com/ledgerwatch/erigon/params"
"github.com/holiman/uint256"
"github.com/ledgerwatch/erigon/common"
"github.com/ledgerwatch/erigon/core/types"
)
// IntraBlockState is an EVM database for full state querying.
type IntraBlockState interface {
CreateAccount(common.Address, bool)
SubBalance(common.Address, *uint256.Int)
AddBalance(common.Address, *uint256.Int)
GetBalance(common.Address) *uint256.Int
GetNonce(common.Address) uint64
SetNonce(common.Address, uint64)
GetCodeHash(common.Address) common.Hash
GetCode(common.Address) []byte
SetCode(common.Address, []byte)
GetCodeSize(common.Address) int
AddRefund(uint64)
SubRefund(uint64)
GetRefund() uint64
GetCommittedState(common.Address, *common.Hash, *uint256.Int)
GetState(address common.Address, slot *common.Hash, outValue *uint256.Int)
SetState(common.Address, *common.Hash, uint256.Int)
Suicide(common.Address) bool
HasSuicided(common.Address) bool
// Exist reports whether the given account exists in state.
// Notably this should also return true for suicided accounts.
Exist(common.Address) bool
// Empty returns whether the given account is empty. Empty
// is defined according to EIP161 (balance = nonce = code = 0).
Empty(common.Address) bool
PrepareAccessList(sender common.Address, dest *common.Address, precompiles []common.Address, txAccesses types.AccessList)
AddressInAccessList(addr common.Address) bool
SlotInAccessList(addr common.Address, slot common.Hash) (addressOk bool, slotOk bool)
// AddAddressToAccessList adds the given address to the access list. This operation is safe to perform
// even if the feature/fork is not active yet
AddAddressToAccessList(addr common.Address)
// AddSlotToAccessList adds the given (address,slot) to the access list. This operation is safe to perform
// even if the feature/fork is not active yet
AddSlotToAccessList(addr common.Address, slot common.Hash)
RevertToSnapshot(int)
Snapshot() int
AddLog(*types.Log)
}
// CallContext provides a basic interface for the EVM calling conventions. The EVM
// depends on this context being implemented for doing subcalls and initialising new EVM contracts.
type CallContext interface {
@ -91,13 +41,13 @@ type CallContext interface {
}
type VMInterface interface {
Reset(txCtx TxContext, ibs IntraBlockState)
Reset(txCtx evmtypes.TxContext, ibs evmtypes.IntraBlockState)
Create(caller ContractRef, code []byte, gas uint64, value *uint256.Int) (ret []byte, contractAddr common.Address, leftOverGas uint64, err error)
Call(caller ContractRef, addr common.Address, input []byte, gas uint64, value *uint256.Int, bailout bool) (ret []byte, leftOverGas uint64, err error)
Config() Config
ChainConfig() *params.ChainConfig
ChainRules() *params.Rules
Context() BlockContext
IntraBlockState() IntraBlockState
TxContext() TxContext
Context() evmtypes.BlockContext
IntraBlockState() evmtypes.IntraBlockState
TxContext() evmtypes.TxContext
}

View File

@ -21,6 +21,7 @@ import (
"testing"
"github.com/holiman/uint256"
"github.com/ledgerwatch/erigon/core/vm/evmtypes"
"github.com/ledgerwatch/erigon/common"
"github.com/ledgerwatch/erigon/core/vm/stack"
@ -29,7 +30,7 @@ import (
func TestStoreCapture(t *testing.T) {
var (
env = NewEVM(BlockContext{}, TxContext{}, &dummyStatedb{}, params.TestChainConfig, Config{})
env = NewEVM(evmtypes.BlockContext{}, evmtypes.TxContext{}, &dummyStatedb{}, params.TestChainConfig, Config{})
logger = NewStructLogger(nil)
mem = NewMemory()
stack = stack.New()

View File

@ -19,15 +19,16 @@ package runtime
import (
"github.com/ledgerwatch/erigon/core"
"github.com/ledgerwatch/erigon/core/vm"
"github.com/ledgerwatch/erigon/core/vm/evmtypes"
)
func NewEnv(cfg *Config) *vm.EVM {
txContext := vm.TxContext{
txContext := evmtypes.TxContext{
Origin: cfg.Origin,
GasPrice: cfg.GasPrice,
}
blockContext := vm.BlockContext{
blockContext := evmtypes.BlockContext{
CanTransfer: core.CanTransfer,
Transfer: core.Transfer,
GetHash: cfg.GetHashFn,

View File

@ -66,7 +66,7 @@ type KeccakState interface {
// NewKeccakState creates a new KeccakState
func NewKeccakState() KeccakState {
return sha3.NewLegacyKeccak256().(KeccakState)
return NewLegacyKeccak256().(KeccakState)
}
// HashData hashes the provided data using the KeccakState and returns a 32 byte hash
@ -87,6 +87,7 @@ func Keccak256(data ...[]byte) []byte {
d.Write(b)
}
d.Read(b) //nolint:errcheck
ReturnToPoolKeccak256(d)
return b
}
@ -98,6 +99,7 @@ func Keccak256Hash(data ...[]byte) (h common.Hash) {
d.Write(b)
}
d.Read(h[:]) //nolint:errcheck
ReturnToPoolKeccak256(d)
return h
}

22
crypto/pool.go Normal file
View File

@ -0,0 +1,22 @@
package crypto
import (
"hash"
"sync"
"golang.org/x/crypto/sha3"
)
// hasherPool holds LegacyKeccak hashers.
var hasherPool = sync.Pool{
New: func() interface{} {
return sha3.NewLegacyKeccak256()
},
}
func NewLegacyKeccak256() hash.Hash {
h := hasherPool.Get().(hash.Hash)
h.Reset()
return h
}
func ReturnToPoolKeccak256(h hash.Hash) { hasherPool.Put(h) }

View File

@ -23,6 +23,7 @@ import (
"github.com/ledgerwatch/erigon-lib/kv"
kv2 "github.com/ledgerwatch/erigon-lib/kv/mdbx"
state2 "github.com/ledgerwatch/erigon-lib/state"
"github.com/ledgerwatch/erigon/cmd/state/exec22"
"github.com/ledgerwatch/erigon/cmd/state/exec3"
common2 "github.com/ledgerwatch/erigon/common"
"github.com/ledgerwatch/erigon/common/math"
@ -57,7 +58,7 @@ type Progress struct {
logPrefix string
}
func (p *Progress) Log(rs *state.State22, rwsLen int, queueSize, count, inputBlockNum, outputBlockNum, outTxNum, repeatCount uint64, resultsSize uint64, resultCh chan *state.TxTask, idxStepsAmountInDB float64) {
func (p *Progress) Log(rs *state.State22, rwsLen int, queueSize, count, inputBlockNum, outputBlockNum, outTxNum, repeatCount uint64, resultsSize uint64, resultCh chan *exec22.TxTask, idxStepsAmountInDB float64) {
ExecStepsInDB.Set(uint64(idxStepsAmountInDB * 100))
var m runtime.MemStats
common.ReadMemStats(&m)
@ -100,12 +101,11 @@ func (p *Progress) Log(rs *state.State22, rwsLen int, queueSize, count, inputBlo
func Exec3(ctx context.Context,
execStage *StageState, workerCount int, batchSize datasize.ByteSize, chainDb kv.RwDB, applyTx kv.RwTx,
rs *state.State22, blockReader services.FullBlockReader,
parallel bool, rs *state.State22, blockReader services.FullBlockReader,
logger log.Logger, agg *state2.Aggregator22, engine consensus.Engine,
maxBlockNum uint64, chainConfig *params.ChainConfig,
genesis *core.Genesis,
) (err error) {
parallel := workerCount > 1
useExternalTx := applyTx != nil
if !useExternalTx && !parallel {
applyTx, err = chainDb.BeginRw(ctx)
@ -113,9 +113,10 @@ func Exec3(ctx context.Context,
return err
}
defer applyTx.Rollback()
}
if !useExternalTx && blockReader.(WithSnapshots).Snapshots().Cfg().Enabled {
defer blockReader.(WithSnapshots).Snapshots().EnableMadvNormal().DisableReadAhead()
} else {
if blockReader.(WithSnapshots).Snapshots().Cfg().Enabled {
defer blockReader.(WithSnapshots).Snapshots().EnableMadvNormal().DisableReadAhead()
}
}
var block, stageProgress uint64
@ -125,7 +126,7 @@ func Exec3(ctx context.Context,
stageProgress = execStage.BlockNumber
block = execStage.BlockNumber + 1
}
if !parallel || useExternalTx {
if applyTx != nil {
agg.SetTx(applyTx)
_maxTxNum, err := rawdb.TxNums.Max(applyTx, maxBlockNum)
if err != nil {
@ -168,7 +169,7 @@ func Exec3(ctx context.Context,
var repeatCount, triggerCount = atomic2.NewUint64(0), atomic2.NewUint64(0)
var resultsSize = atomic2.NewInt64(0)
var lock sync.RWMutex
var rws state.TxTaskQueue
var rws exec22.TxTaskQueue
var rwsLock sync.RWMutex
// erigon3 execution doesn't support power-off shutdown yet. it need to do quite a lot of work on exit
@ -178,7 +179,7 @@ func Exec3(ctx context.Context,
ctx = context.Background()
queueSize := workerCount * 4
var wg sync.WaitGroup
execWorkers, resultCh, clear := exec3.NewWorkersPool(lock.RLocker(), chainDb, &wg, rs, blockReader, chainConfig, logger, genesis, engine, workerCount)
execWorkers, resultCh, clear := exec3.NewWorkersPool(lock.RLocker(), parallel, chainDb, &wg, rs, blockReader, chainConfig, logger, genesis, engine, workerCount)
defer clear()
commitThreshold := batchSize.Bytes()
@ -192,66 +193,29 @@ func Exec3(ctx context.Context,
heap.Init(&rws)
agg.SetTxNum(inputTxNum)
applyWg := sync.WaitGroup{} // to wait for finishing of applyLoop after applyCtx cancel
taskCh := make(chan *state.TxTask, 1024)
//defer close(taskCh)
var applyLoop func(ctx context.Context)
if parallel {
applyLoop = func(ctx context.Context) {
defer applyWg.Done()
tx, err := chainDb.BeginRo(ctx)
if err != nil {
panic(err)
}
defer tx.Rollback()
notifyReceived := func() { rwsReceiveCond.Signal() }
for outputTxNum.Load() < maxTxNum.Load() {
select {
case <-ctx.Done():
return
case txTask := <-resultCh:
func() {
rwsLock.Lock()
defer rwsLock.Unlock()
resultsSize.Add(txTask.ResultsSize)
heap.Push(&rws, txTask)
processResultQueue(&rws, outputTxNum, rs, agg, tx, triggerCount, outputBlockNum, repeatCount, resultsSize, notifyReceived)
syncMetrics[stages.Execution].Set(outputBlockNum.Load())
}()
}
}
applyLoop := func(ctx context.Context) {
defer applyWg.Done()
tx, err := chainDb.BeginRo(ctx)
if err != nil {
panic(err)
}
} else {
applyLoop = func(ctx context.Context) {
defer applyWg.Done()
tx, err := chainDb.BeginRo(ctx)
if err != nil {
panic(err)
}
defer tx.Rollback()
execWorkers[0].ResetTx(nil)
defer tx.Rollback()
for {
select {
case <-ctx.Done():
return
case txTask, ok := <-taskCh:
if !ok { // no more tasks
return
}
execWorkers[0].RunTxTask(txTask)
if err := rs.ApplyState(tx, txTask, agg); err != nil {
panic(fmt.Errorf("State22.Apply: %w", err))
}
rs.CommitTxNum(txTask.Sender, txTask.TxNum)
outputTxNum.Inc()
outputBlockNum.Store(txTask.BlockNum)
rwsReceiveCond.Signal()
if err := rs.ApplyHistory(tx, txTask, agg); err != nil {
panic(fmt.Errorf("State22.Apply: %w", err))
}
}
notifyReceived := func() { rwsReceiveCond.Signal() }
for outputTxNum.Load() < maxTxNum.Load() {
select {
case <-ctx.Done():
return
case txTask := <-resultCh:
func() {
rwsLock.Lock()
defer rwsLock.Unlock()
resultsSize.Add(txTask.ResultsSize)
heap.Push(&rws, txTask)
processResultQueue(&rws, outputTxNum, rs, agg, tx, triggerCount, outputBlockNum, repeatCount, resultsSize, notifyReceived)
syncMetrics[stages.Execution].Set(outputBlockNum.Load())
}()
}
}
}
@ -353,7 +317,7 @@ func Exec3(ctx context.Context,
// Drain results queue as well
for rws.Len() > 0 {
txTask := heap.Pop(&rws).(*state.TxTask)
txTask := heap.Pop(&rws).(*exec22.TxTask)
resultsSize.Add(-txTask.ResultsSize)
rs.AddWork(txTask)
}
@ -432,6 +396,7 @@ func Exec3(ctx context.Context,
if !parallel {
defer agg.StartWrites().FinishWrites()
execWorkers[0].ResetTx(applyTx)
}
if block < blockReader.(WithSnapshots).Snapshots().BlocksAvailable() {
@ -439,23 +404,25 @@ func Exec3(ctx context.Context,
defer agg.KeepInDB(ethconfig.HistoryV3AggregationStep)
}
if !parallel {
applyWg.Add(1)
go applyLoop(ctx)
}
getHeaderFunc := func(hash common2.Hash, number uint64) (h *types.Header) {
if err = chainDb.View(ctx, func(tx kv.Tx) error {
h, err = blockReader.Header(ctx, tx, hash, number)
if err != nil {
return err
if parallel {
if err = chainDb.View(ctx, func(tx kv.Tx) error {
h, err = blockReader.Header(ctx, tx, hash, number)
if err != nil {
return err
}
return nil
}); err != nil {
panic(err)
}
return nil
}); err != nil {
panic(err)
return h
} else {
h, err = blockReader.Header(ctx, applyTx, hash, number)
if err != nil {
panic(err)
}
return h
}
return h
}
var b *types.Block
var blockNum uint64
@ -485,6 +452,7 @@ loop:
defer getHashFnMute.Unlock()
return f(n)
}
blockContext := core.NewEVMBlockContext(header, getHashFn, engine, nil /* author */)
if parallel {
func() {
@ -504,19 +472,20 @@ loop:
for txIndex := -1; txIndex <= len(txs); txIndex++ {
// Do not oversend, wait for the result heap to go under certain size
txTask := &state.TxTask{
BlockNum: blockNum,
Header: header,
Coinbase: b.Coinbase(),
Uncles: b.Uncles(),
Rules: rules,
Txs: txs,
TxNum: inputTxNum,
TxIndex: txIndex,
BlockHash: b.Hash(),
SkipAnalysis: skipAnalysis,
Final: txIndex == len(txs),
GetHashFn: getHashFn,
txTask := &exec22.TxTask{
BlockNum: blockNum,
Header: header,
Coinbase: b.Coinbase(),
Uncles: b.Uncles(),
Rules: rules,
Txs: txs,
TxNum: inputTxNum,
TxIndex: txIndex,
BlockHash: b.Hash(),
SkipAnalysis: skipAnalysis,
Final: txIndex == len(txs),
GetHashFn: getHashFn,
EvmBlockContext: blockContext,
}
if txIndex >= 0 && txIndex < len(txs) {
txTask.Tx = txs[txIndex]
@ -538,7 +507,16 @@ loop:
}
if !parallel {
count++
taskCh <- txTask
execWorkers[0].RunTxTask(txTask)
if err := rs.ApplyState(applyTx, txTask, agg); err != nil {
panic(fmt.Errorf("State22.Apply: %w", err))
}
rs.CommitTxNum(txTask.Sender, txTask.TxNum)
outputTxNum.Inc()
outputBlockNum.Store(txTask.BlockNum)
if err := rs.ApplyHistory(applyTx, txTask, agg); err != nil {
panic(fmt.Errorf("State22.Apply: %w", err))
}
}
stageProgress = blockNum
inputTxNum++
@ -554,45 +532,12 @@ loop:
stepsInDB := idxStepsInDB(applyTx)
progress.Log(rs, rws.Len(), uint64(queueSize), count, inputBlockNum.Load(), outputBlockNum.Load(), outputTxNum.Load(), repeatCount.Load(), uint64(resultsSize.Load()), resultCh, stepsInDB)
if rs.SizeEstimate() < commitThreshold {
// too much steps in db will slow-down everything: flush and prune
// it means better spend time for pruning, before flushing more data to db
// also better do it now - instead of before Commit() - because Commit does block execution
if stepsInDB > 5 && rs.SizeEstimate() < uint64(float64(commitThreshold)*0.2) {
if err = agg.Prune(ctx, ethconfig.HistoryV3AggregationStep*2); err != nil { // prune part of retired data, before commit
panic(err)
}
} else if stepsInDB > 2 {
t := time.Now()
if err = agg.Prune(ctx, ethconfig.HistoryV3AggregationStep/10); err != nil { // prune part of retired data, before commit
panic(err)
}
if time.Since(t) > 10*time.Second && // allready spent much time on this cycle, let's print for user regular logs
rs.SizeEstimate() < uint64(float64(commitThreshold)*0.8) { // batch is 80%-full - means commit soon, time to flush indices
break
}
}
// rotate indices-WAL, execution will work on new WAL while rwTx-thread can flush indices-WAL to db or prune db.
if err := agg.Flush(applyTx); err != nil {
panic(err)
}
break
}
close(taskCh)
applyWg.Wait()
taskCh = make(chan *state.TxTask, 1024)
var t1, t2, t3, t4 time.Duration
commitStart := time.Now()
if err := func() error {
rwsLock.Lock()
defer rwsLock.Unlock()
rwsReceiveCond.Signal()
lock.Lock() // This is to prevent workers from starting work on any new txTask
defer lock.Unlock()
t1 = time.Since(commitStart)
tt := time.Now()
if err := rs.Flush(applyTx); err != nil {
@ -611,22 +556,6 @@ loop:
}
applyTx.CollectMetrics()
//TODO: can't commit - because we are in the middle of the block. Need make sure that we are always processed whole block.
tt = time.Now()
if !useExternalTx {
if err = applyTx.Commit(); err != nil {
return err
}
t4 = time.Since(tt)
//execWorkers[0].ResetTx(nil)
if applyTx, err = chainDb.BeginRw(ctx); err != nil {
return err
}
agg.SetTx(applyTx)
applyWg.Add(1)
go applyLoop(ctx)
}
return nil
}(); err != nil {
@ -652,8 +581,6 @@ loop:
if parallel {
wg.Wait()
} else {
close(taskCh)
applyWg.Wait()
if err = rs.Flush(applyTx); err != nil {
return err
}
@ -691,10 +618,10 @@ func blockWithSenders(db kv.RoDB, tx kv.Tx, blockReader services.BlockReader, bl
return b, nil
}
func processResultQueue(rws *state.TxTaskQueue, outputTxNum *atomic2.Uint64, rs *state.State22, agg *state2.Aggregator22, applyTx kv.Tx,
func processResultQueue(rws *exec22.TxTaskQueue, outputTxNum *atomic2.Uint64, rs *state.State22, agg *state2.Aggregator22, applyTx kv.Tx,
triggerCount, outputBlockNum, repeatCount *atomic2.Uint64, resultsSize *atomic2.Int64, onSuccess func()) {
for rws.Len() > 0 && (*rws)[0].TxNum == outputTxNum.Load() {
txTask := heap.Pop(rws).(*state.TxTask)
txTask := heap.Pop(rws).(*exec22.TxTask)
resultsSize.Add(-txTask.ResultsSize)
if txTask.Error == nil && rs.ReadsValid(txTask.ReadLists) {
if err := rs.ApplyState(applyTx, txTask, agg); err != nil {
@ -755,7 +682,7 @@ func ReconstituteState(ctx context.Context, s *StageState, dirs datadir.Dirs, wo
log.Info(fmt.Sprintf("[%s] Blocks execution, reconstitution", s.LogPrefix()), "fromBlock", s.BlockNumber, "toBlock", blockNum, "toTxNum", txNum)
var wg sync.WaitGroup
workCh := make(chan *state.TxTask, workerCount*4)
workCh := make(chan *exec22.TxTask, workerCount*4)
rs := state.NewReconState(workCh)
var fromKey, toKey []byte
bigCount := big.NewInt(int64(workerCount))
@ -1070,23 +997,25 @@ func ReconstituteState(ctx context.Context, s *StageState, dirs datadir.Dirs, wo
defer getHashFnMute.Unlock()
return f(n)
}
blockContext := core.NewEVMBlockContext(header, getHashFn, engine, nil /* author */)
for txIndex := -1; txIndex <= len(txs); txIndex++ {
if bitmap.Contains(inputTxNum) {
binary.BigEndian.PutUint64(txKey[:], inputTxNum)
txTask := &state.TxTask{
BlockNum: bn,
Header: header,
Coinbase: b.Coinbase(),
Uncles: b.Uncles(),
Rules: rules,
TxNum: inputTxNum,
Txs: txs,
TxIndex: txIndex,
BlockHash: b.Hash(),
SkipAnalysis: skipAnalysis,
Final: txIndex == len(txs),
GetHashFn: getHashFn,
txTask := &exec22.TxTask{
BlockNum: bn,
Header: header,
Coinbase: b.Coinbase(),
Uncles: b.Uncles(),
Rules: rules,
TxNum: inputTxNum,
Txs: txs,
TxIndex: txIndex,
BlockHash: b.Hash(),
SkipAnalysis: skipAnalysis,
Final: txIndex == len(txs),
GetHashFn: getHashFn,
EvmBlockContext: blockContext,
}
if txIndex >= 0 && txIndex < len(txs) {
txTask.Tx = txs[txIndex]

View File

@ -273,7 +273,8 @@ func ExecBlock22(s *StageState, u Unwinder, tx kv.RwTx, toBlock uint64, ctx cont
log.Info(fmt.Sprintf("[%s] Blocks execution", logPrefix), "from", s.BlockNumber, "to", to)
}
rs := state.NewState22()
if err := Exec3(ctx, s, workersCount, cfg.batchSize, cfg.db, tx, rs,
parallel := initialCycle && tx == nil
if err := Exec3(ctx, s, workersCount, cfg.batchSize, cfg.db, tx, parallel, rs,
cfg.blockReader, log.New(), cfg.agg, cfg.engine,
to,
cfg.chainConfig, cfg.genesis); err != nil {
@ -730,7 +731,7 @@ func PruneExecutionStage(s *PruneState, tx kv.RwTx, cfg ExecuteBlockCfg, ctx con
if cfg.historyV3 {
cfg.agg.SetTx(tx)
if err = cfg.agg.Prune(ctx, ethconfig.HistoryV3AggregationStep/10); err != nil { // prune part of retired data, before commit
if err = cfg.agg.Prune(ctx, 100); err != nil { // prune part of retired data, before commit
return err
}
} else {

View File

@ -9,6 +9,7 @@ import (
"github.com/ledgerwatch/erigon-lib/kv"
"github.com/ledgerwatch/erigon-lib/kv/memdb"
libstate "github.com/ledgerwatch/erigon-lib/state"
"github.com/ledgerwatch/erigon/cmd/state/exec22"
"github.com/ledgerwatch/erigon/common/changeset"
"github.com/ledgerwatch/erigon/core/rawdb"
"github.com/ledgerwatch/erigon/core/state"
@ -133,7 +134,7 @@ func apply(tx kv.RwTx, agg *libstate.Aggregator22) (beforeBlock, afterBlock test
stateWriter.SetTxNum(n)
stateWriter.ResetWriteSet()
}, func(n, from, numberOfBlocks uint64) {
txTask := &state.TxTask{
txTask := &exec22.TxTask{
BlockNum: n,
Rules: params.TestRules,
TxNum: n,

View File

@ -26,6 +26,7 @@ import (
"unsafe"
"github.com/ledgerwatch/erigon/core"
"github.com/ledgerwatch/erigon/core/vm/evmtypes"
"github.com/holiman/uint256"
@ -191,7 +192,7 @@ func (sw *stackWrapper) pushObject(vm *JSVM) {
// dbWrapper provides a JavaScript wrapper around vm.Database.
type dbWrapper struct {
db vm.IntraBlockState
db evmtypes.IntraBlockState
}
// pushObject assembles a JSVM object wrapping a swappable database and pushes it

View File

@ -27,6 +27,7 @@ import (
"github.com/ledgerwatch/erigon/common"
"github.com/ledgerwatch/erigon/core/state"
"github.com/ledgerwatch/erigon/core/vm"
"github.com/ledgerwatch/erigon/core/vm/evmtypes"
"github.com/ledgerwatch/erigon/params"
)
@ -52,14 +53,14 @@ func (*dummyStatedb) GetRefund() uint64 { return 1337
func (*dummyStatedb) GetBalance(addr common.Address) *uint256.Int { return uint256.NewInt(0) }
type vmContext struct {
blockCtx vm.BlockContext
txCtx vm.TxContext
blockCtx evmtypes.BlockContext
txCtx evmtypes.TxContext
}
func testCtx() *vmContext {
return &vmContext{blockCtx: vm.BlockContext{
return &vmContext{blockCtx: evmtypes.BlockContext{
BlockNumber: 1,
}, txCtx: vm.TxContext{GasPrice: big.NewInt(100000)}}
}, txCtx: evmtypes.TxContext{GasPrice: big.NewInt(100000)}}
}
func runTrace(tracer *Tracer, vmctx *vmContext) (json.RawMessage, error) {
@ -83,9 +84,9 @@ func runTrace(tracer *Tracer, vmctx *vmContext) (json.RawMessage, error) {
func TestTracer(t *testing.T) {
execTracer := func(code string) ([]byte, string) {
t.Helper()
ctx := &vmContext{blockCtx: vm.BlockContext{
ctx := &vmContext{blockCtx: evmtypes.BlockContext{
BlockNumber: 1,
}, txCtx: vm.TxContext{GasPrice: big.NewInt(100000)}}
}, txCtx: evmtypes.TxContext{GasPrice: big.NewInt(100000)}}
tracer, err := New(code, new(Context))
if err != nil {
t.Fatal(err)
@ -155,9 +156,9 @@ func TestHaltBetweenSteps(t *testing.T) {
if err != nil {
t.Fatal(err)
}
env := vm.NewEVM(vm.BlockContext{
env := vm.NewEVM(evmtypes.BlockContext{
BlockNumber: 1,
}, vm.TxContext{}, &dummyStatedb{}, params.TestChainConfig, vm.Config{Debug: true, Tracer: tracer})
}, evmtypes.TxContext{}, &dummyStatedb{}, params.TestChainConfig, vm.Config{Debug: true, Tracer: tracer})
contract := vm.NewContract(&account{}, &account{}, uint256.NewInt(0), 0, false)
tracer.CaptureState(env, 0, 0, 0, 0, &vm.ScopeContext{Contract: contract}, nil, 0, nil) //nolint:errcheck
@ -183,7 +184,7 @@ func TestNoStepExec(t *testing.T) {
}
execTracer := func(code string) []byte {
t.Helper()
ctx := &vmContext{blockCtx: vm.BlockContext{BlockNumber: 1}, txCtx: vm.TxContext{GasPrice: big.NewInt(100000)}}
ctx := &vmContext{blockCtx: evmtypes.BlockContext{BlockNumber: 1}, txCtx: evmtypes.TxContext{GasPrice: big.NewInt(100000)}}
tracer, err := New(code, new(Context))
if err != nil {
t.Fatal(err)

View File

@ -35,6 +35,7 @@ import (
"github.com/ledgerwatch/erigon/core"
"github.com/ledgerwatch/erigon/core/types"
"github.com/ledgerwatch/erigon/core/vm"
"github.com/ledgerwatch/erigon/core/vm/evmtypes"
"github.com/ledgerwatch/erigon/crypto"
"github.com/ledgerwatch/erigon/params"
"github.com/ledgerwatch/erigon/rlp"
@ -147,11 +148,11 @@ func TestPrestateTracerCreate2(t *testing.T) {
result: 0x60f3f640a8508fC6a86d45DF051962668E1e8AC7
*/
origin, _ := signer.Sender(txn)
txContext := vm.TxContext{
txContext := evmtypes.TxContext{
Origin: origin,
GasPrice: big.NewInt(1),
}
context := vm.BlockContext{
context := evmtypes.BlockContext{
CanTransfer: core.CanTransfer,
Transfer: core.Transfer,
Coinbase: common.Address{},
@ -239,11 +240,11 @@ func TestCallTracer(t *testing.T) {
}
signer := types.MakeSigner(test.Genesis.Config, uint64(test.Context.Number))
origin, _ := signer.Sender(txn)
txContext := vm.TxContext{
txContext := evmtypes.TxContext{
Origin: origin,
GasPrice: big.NewInt(int64(txn.GetPrice().Uint64())),
}
context := vm.BlockContext{
context := evmtypes.BlockContext{
CanTransfer: core.CanTransfer,
Transfer: core.Transfer,
Coinbase: test.Context.Miner,

View File

@ -1838,6 +1838,7 @@ func HeadersIdx(ctx context.Context, segmentFilePath string, firstBlockNumInSegm
p.Total.Store(uint64(d.Count()))
hasher := crypto.NewKeccakState()
defer crypto.ReturnToPoolKeccak256(hasher)
var h common.Hash
if err := Idx(ctx, d, firstBlockNumInSegment, tmpDir, log.LvlDebug, func(idx *recsplit.RecSplit, i, offset uint64, word []byte) error {
p.Processed.Inc()

View File

@ -7,6 +7,7 @@ import (
"github.com/holiman/uint256"
"github.com/ledgerwatch/erigon-lib/kv"
"github.com/ledgerwatch/erigon/core/vm/evmtypes"
"github.com/ledgerwatch/log/v3"
"github.com/ledgerwatch/erigon/common"
@ -101,7 +102,7 @@ func DoCall(
return result, nil
}
func GetEvmContext(msg core.Message, header *types.Header, requireCanonical bool, tx kv.Tx, headerReader services.HeaderReader) (vm.BlockContext, vm.TxContext) {
func GetEvmContext(msg core.Message, header *types.Header, requireCanonical bool, tx kv.Tx, headerReader services.HeaderReader) (evmtypes.BlockContext, evmtypes.TxContext) {
var baseFee uint256.Int
if header.BaseFee != nil {
overflow := baseFee.SetFromBig(header.BaseFee)
@ -110,7 +111,7 @@ func GetEvmContext(msg core.Message, header *types.Header, requireCanonical bool
}
}
return core.NewEVMBlockContext(header, getHashGetter(requireCanonical, tx, headerReader), ethash.NewFaker() /* TODO Discover correcrt engine type */, nil /* author */),
vm.TxContext{
evmtypes.TxContext{
Origin: msg.From(),
GasPrice: msg.GasPrice().ToBig(),
}

View File

@ -19,6 +19,7 @@ import (
"github.com/ledgerwatch/erigon/core/state"
"github.com/ledgerwatch/erigon/core/types"
"github.com/ledgerwatch/erigon/core/vm"
"github.com/ledgerwatch/erigon/core/vm/evmtypes"
"github.com/ledgerwatch/erigon/eth/tracers"
"github.com/ledgerwatch/erigon/params"
"github.com/ledgerwatch/erigon/turbo/rpchelper"
@ -34,20 +35,20 @@ type BlockGetter interface {
}
// ComputeTxEnv returns the execution environment of a certain transaction.
func ComputeTxEnv(ctx context.Context, block *types.Block, cfg *params.ChainConfig, headerReader services.HeaderReader, dbtx kv.Tx, txIndex uint64, agg *state2.Aggregator22, historyV3 bool) (core.Message, vm.BlockContext, vm.TxContext, *state.IntraBlockState, state.StateReader, error) {
func ComputeTxEnv(ctx context.Context, block *types.Block, cfg *params.ChainConfig, headerReader services.HeaderReader, dbtx kv.Tx, txIndex uint64, agg *state2.Aggregator22, historyV3 bool) (core.Message, evmtypes.BlockContext, evmtypes.TxContext, *state.IntraBlockState, state.StateReader, error) {
header := block.HeaderNoCopy()
reader, err := rpchelper.CreateHistoryStateReader(dbtx, block.NumberU64(), txIndex, agg, historyV3)
if err != nil {
return nil, vm.BlockContext{}, vm.TxContext{}, nil, nil, err
return nil, evmtypes.BlockContext{}, evmtypes.TxContext{}, nil, nil, err
}
if historyV3 {
//engine := ethash.NewFaker()
ibs := state.New(reader)
if txIndex == 0 && len(block.Transactions()) == 0 {
return nil, vm.BlockContext{}, vm.TxContext{}, ibs, reader, nil
return nil, evmtypes.BlockContext{}, evmtypes.TxContext{}, ibs, reader, nil
}
if int(txIndex) > block.Transactions().Len() {
return nil, vm.BlockContext{}, vm.TxContext{}, ibs, reader, nil
return nil, evmtypes.BlockContext{}, evmtypes.TxContext{}, ibs, reader, nil
}
txn := block.Transactions()[txIndex]
signer := types.MakeSigner(cfg, block.NumberU64())
@ -67,19 +68,19 @@ func ComputeTxEnv(ctx context.Context, block *types.Block, cfg *params.ChainConf
statedb := state.New(reader)
if txIndex == 0 && len(block.Transactions()) == 0 {
return nil, vm.BlockContext{}, vm.TxContext{}, statedb, reader, nil
return nil, evmtypes.BlockContext{}, evmtypes.TxContext{}, statedb, reader, nil
}
// Recompute transactions up to the target index.
signer := types.MakeSigner(cfg, block.NumberU64())
BlockContext := core.NewEVMBlockContext(header, core.GetHashFn(header, getHeader), engine, nil)
vmenv := vm.NewEVM(BlockContext, vm.TxContext{}, statedb, cfg, vm.Config{})
vmenv := vm.NewEVM(BlockContext, evmtypes.TxContext{}, statedb, cfg, vm.Config{})
rules := vmenv.ChainRules()
for idx, tx := range block.Transactions() {
select {
default:
case <-ctx.Done():
return nil, vm.BlockContext{}, vm.TxContext{}, nil, nil, ctx.Err()
return nil, evmtypes.BlockContext{}, evmtypes.TxContext{}, nil, nil, ctx.Err()
}
statedb.Prepare(tx.Hash(), block.Hash(), idx)
@ -92,7 +93,7 @@ func ComputeTxEnv(ctx context.Context, block *types.Block, cfg *params.ChainConf
vmenv.Reset(TxContext, statedb)
// Not yet the searched for transaction, execute on top of the current state
if _, err := core.ApplyMessage(vmenv, msg, new(core.GasPool).AddGas(tx.GetGas()), true /* refunds */, false /* gasBailout */); err != nil {
return nil, vm.BlockContext{}, vm.TxContext{}, nil, nil, fmt.Errorf("transaction %x failed: %w", tx.Hash(), err)
return nil, evmtypes.BlockContext{}, evmtypes.TxContext{}, nil, nil, fmt.Errorf("transaction %x failed: %w", tx.Hash(), err)
}
// Ensure any modifications are committed to the state
// Only delete empty objects if EIP161 (part of Spurious Dragon) is in effect
@ -100,10 +101,10 @@ func ComputeTxEnv(ctx context.Context, block *types.Block, cfg *params.ChainConf
if idx+1 == len(block.Transactions()) {
// Return the state from evaluating all txs in the block, note no msg or TxContext in this case
return nil, BlockContext, vm.TxContext{}, statedb, reader, nil
return nil, BlockContext, evmtypes.TxContext{}, statedb, reader, nil
}
}
return nil, vm.BlockContext{}, vm.TxContext{}, nil, nil, fmt.Errorf("transaction index %d out of range for block %x", txIndex, block.Hash())
return nil, evmtypes.BlockContext{}, evmtypes.TxContext{}, nil, nil, fmt.Errorf("transaction index %d out of range for block %x", txIndex, block.Hash())
}
// TraceTx configures a new tracer according to the provided configuration, and
@ -112,9 +113,9 @@ func ComputeTxEnv(ctx context.Context, block *types.Block, cfg *params.ChainConf
func TraceTx(
ctx context.Context,
message core.Message,
blockCtx vm.BlockContext,
txCtx vm.TxContext,
ibs vm.IntraBlockState,
blockCtx evmtypes.BlockContext,
txCtx evmtypes.TxContext,
ibs evmtypes.IntraBlockState,
config *tracers.TraceConfig,
chainConfig *params.ChainConfig,
stream *jsoniter.Stream,