diff --git a/accounts/abi/bind/backends/simulated.go b/accounts/abi/bind/backends/simulated.go
index 3c401323422..7829045edcd 100644
--- a/accounts/abi/bind/backends/simulated.go
+++ b/accounts/abi/bind/backends/simulated.go
@@ -605,7 +605,7 @@ func (b *SimulatedBackend) EstimateGas(ctx context.Context, call ethereum.CallMs
}
}
gasCap = hi
- b.pendingState.Prepare(libcommon.Hash{}, libcommon.Hash{}, len(b.pendingBlock.Transactions()))
+ b.pendingState.SetTxContext(libcommon.Hash{}, libcommon.Hash{}, len(b.pendingBlock.Transactions()))
// Create a helper to check if a gas allowance results in an executable transaction
executable := func(gas uint64) (bool, *core.ExecutionResult, error) {
@@ -715,7 +715,7 @@ func (b *SimulatedBackend) SendTransaction(ctx context.Context, tx types.Transac
return fmt.Errorf("invalid transaction nonce: got %d, want %d", tx.GetNonce(), nonce)
}
- b.pendingState.Prepare(tx.Hash(), libcommon.Hash{}, len(b.pendingBlock.Transactions()))
+ b.pendingState.SetTxContext(tx.Hash(), libcommon.Hash{}, len(b.pendingBlock.Transactions()))
//fmt.Printf("==== Start producing block %d, header: %d\n", b.pendingBlock.NumberU64(), b.pendingHeader.Number.Uint64())
if _, _, err := core.ApplyTransaction(
b.m.ChainConfig, core.GetHashFn(b.pendingHeader, b.getHeader), b.m.Engine,
diff --git a/cmd/integration/commands/state_domains.go b/cmd/integration/commands/state_domains.go
index 9d4047081b7..7eee8e4e431 100644
--- a/cmd/integration/commands/state_domains.go
+++ b/cmd/integration/commands/state_domains.go
@@ -520,7 +520,7 @@ func (b *blockProcessor) applyBlock(
for i, tx := range block.Transactions() {
if b.txNum >= b.startTxNum {
ibs := state.New(b.reader)
- ibs.Prepare(tx.Hash(), block.Hash(), i)
+ ibs.SetTxContext(tx.Hash(), block.Hash(), i)
ct := exec3.NewCallTracer()
b.vmConfig.Tracer = ct
receipt, _, err := core.ApplyTransaction(b.chainConfig, getHashFn, b.engine, nil, gp, ibs, b.writer, header, tx, usedGas, b.vmConfig, parentHeader.ExcessDataGas)
diff --git a/cmd/rpcdaemon/commands/eth_callMany.go b/cmd/rpcdaemon/commands/eth_callMany.go
index c330803a39b..5d18b2c440d 100644
--- a/cmd/rpcdaemon/commands/eth_callMany.go
+++ b/cmd/rpcdaemon/commands/eth_callMany.go
@@ -207,7 +207,7 @@ func (api *APIImpl) CallMany(ctx context.Context, bundles []Bundle, simulateCont
// and apply the message.
gp := new(core.GasPool).AddGas(math.MaxUint64)
for idx, txn := range replayTransactions {
- st.Prepare(txn.Hash(), block.Hash(), idx)
+ st.SetTxContext(txn.Hash(), block.Hash(), idx)
msg, err := txn.AsMessage(*signer, block.BaseFee(), rules)
if err != nil {
return nil, err
diff --git a/cmd/rpcdaemon/commands/eth_receipts.go b/cmd/rpcdaemon/commands/eth_receipts.go
index 3632b7b11d4..85d2bd2d49d 100644
--- a/cmd/rpcdaemon/commands/eth_receipts.go
+++ b/cmd/rpcdaemon/commands/eth_receipts.go
@@ -65,7 +65,7 @@ func (api *BaseAPI) getReceipts(ctx context.Context, tx kv.Tx, chainConfig *chai
header := block.Header()
excessDataGas := header.ParentExcessDataGas(getHeader)
for i, txn := range block.Transactions() {
- ibs.Prepare(txn.Hash(), block.Hash(), i)
+ ibs.SetTxContext(txn.Hash(), block.Hash(), i)
receipt, _, err := core.ApplyTransaction(chainConfig, core.GetHashFn(header, getHeader), engine, nil, gp, ibs, noopWriter, header, txn, usedGas, vm.Config{}, excessDataGas)
if err != nil {
return nil, err
@@ -515,7 +515,7 @@ func (e *intraBlockExec) execTx(txNum uint64, txIndex int, txn types.Transaction
e.stateReader.SetTxNum(txNum)
txHash := txn.Hash()
e.ibs.Reset()
- e.ibs.Prepare(txHash, e.blockHash, txIndex)
+ e.ibs.SetTxContext(txHash, e.blockHash, txIndex)
gp := new(core.GasPool).AddGas(txn.GetGas())
msg, err := txn.AsMessage(*e.signer, e.header.BaseFee, e.rules)
if err != nil {
diff --git a/cmd/rpcdaemon/commands/otterscan_generic_tracer.go b/cmd/rpcdaemon/commands/otterscan_generic_tracer.go
index 2f0bb9addff..c12acd10d78 100644
--- a/cmd/rpcdaemon/commands/otterscan_generic_tracer.go
+++ b/cmd/rpcdaemon/commands/otterscan_generic_tracer.go
@@ -85,7 +85,7 @@ func (api *OtterscanAPIImpl) genericTracer(dbtx kv.Tx, ctx context.Context, bloc
excessDataGas := header.ParentExcessDataGas(getHeader)
rules := chainConfig.Rules(block.NumberU64(), header.Time)
for idx, tx := range block.Transactions() {
- ibs.Prepare(tx.Hash(), block.Hash(), idx)
+ ibs.SetTxContext(tx.Hash(), block.Hash(), idx)
msg, _ := tx.AsMessage(*signer, header.BaseFee, rules)
diff --git a/cmd/rpcdaemon/commands/otterscan_search_trace.go b/cmd/rpcdaemon/commands/otterscan_search_trace.go
index 2698cafeacf..921d6934714 100644
--- a/cmd/rpcdaemon/commands/otterscan_search_trace.go
+++ b/cmd/rpcdaemon/commands/otterscan_search_trace.go
@@ -81,7 +81,7 @@ func (api *OtterscanAPIImpl) traceBlock(dbtx kv.Tx, ctx context.Context, blockNu
rules := chainConfig.Rules(block.NumberU64(), header.Time)
found := false
for idx, tx := range block.Transactions() {
- ibs.Prepare(tx.Hash(), block.Hash(), idx)
+ ibs.SetTxContext(tx.Hash(), block.Hash(), idx)
msg, _ := tx.AsMessage(*signer, header.BaseFee, rules)
diff --git a/cmd/rpcdaemon/commands/trace_adhoc.go b/cmd/rpcdaemon/commands/trace_adhoc.go
index 79988b862fa..cd63c916dff 100644
--- a/cmd/rpcdaemon/commands/trace_adhoc.go
+++ b/cmd/rpcdaemon/commands/trace_adhoc.go
@@ -973,7 +973,7 @@ func (api *TraceAPIImpl) Call(ctx context.Context, args TraceCallParam, traceTyp
gp := new(core.GasPool).AddGas(msg.Gas())
var execResult *core.ExecutionResult
- ibs.Prepare(libcommon.Hash{}, libcommon.Hash{}, 0)
+ ibs.SetTxContext(libcommon.Hash{}, libcommon.Hash{}, 0)
execResult, err = core.ApplyMessage(evm, msg, gp, true /* refunds */, true /* gasBailout */)
if err != nil {
return nil, err
@@ -1196,9 +1196,9 @@ func (api *TraceAPIImpl) doCallMany(ctx context.Context, dbtx kv.Tx, msgs []type
cloneReader = state.NewCachedReader(stateReader, cloneCache)
}
if args.txHash != nil {
- ibs.Prepare(*args.txHash, header.Hash(), txIndex)
+ ibs.SetTxContext(*args.txHash, header.Hash(), txIndex)
} else {
- ibs.Prepare(libcommon.Hash{}, header.Hash(), txIndex)
+ ibs.SetTxContext(libcommon.Hash{}, header.Hash(), txIndex)
}
execResult, err = core.ApplyMessage(evm, msg, gp, true /* refunds */, gasBailout /* gasBailout */)
if err != nil {
diff --git a/cmd/rpcdaemon/commands/trace_filtering.go b/cmd/rpcdaemon/commands/trace_filtering.go
index c96d3c0eeaa..ad8c33e94a6 100644
--- a/cmd/rpcdaemon/commands/trace_filtering.go
+++ b/cmd/rpcdaemon/commands/trace_filtering.go
@@ -836,7 +836,7 @@ func (api *TraceAPIImpl) filterV3(ctx context.Context, dbtx kv.TemporalTx, fromB
evm := vm.NewEVM(blockCtx, txCtx, ibs, chainConfig, vmConfig)
gp := new(core.GasPool).AddGas(msg.Gas())
- ibs.Prepare(txHash, lastBlockHash, txIndex)
+ ibs.SetTxContext(txHash, lastBlockHash, txIndex)
var execResult *core.ExecutionResult
execResult, err = core.ApplyMessage(evm, msg, gp, true /* refunds */, false /* gasBailout */)
if err != nil {
diff --git a/cmd/rpcdaemon/commands/tracing.go b/cmd/rpcdaemon/commands/tracing.go
index 3d84f6aadb2..23d954390aa 100644
--- a/cmd/rpcdaemon/commands/tracing.go
+++ b/cmd/rpcdaemon/commands/tracing.go
@@ -128,7 +128,7 @@ func (api *PrivateDebugAPIImpl) traceBlock(ctx context.Context, blockNrOrHash rp
stream.WriteNil()
return ctx.Err()
}
- ibs.Prepare(txn.Hash(), block.Hash(), idx)
+ ibs.SetTxContext(txn.Hash(), block.Hash(), idx)
msg, _ := txn.AsMessage(*signer, block.BaseFee(), rules)
if msg.FeeCap().IsZero() && engine != nil {
@@ -451,7 +451,7 @@ func (api *PrivateDebugAPIImpl) TraceCallMany(ctx context.Context, bundles []Bun
// and apply the message.
gp := new(core.GasPool).AddGas(math.MaxUint64)
for idx, txn := range replayTransactions {
- st.Prepare(txn.Hash(), block.Hash(), idx)
+ st.SetTxContext(txn.Hash(), block.Hash(), idx)
msg, err := txn.AsMessage(*signer, block.BaseFee(), rules)
if err != nil {
stream.WriteNil()
@@ -494,7 +494,7 @@ func (api *PrivateDebugAPIImpl) TraceCallMany(ctx context.Context, bundles []Bun
}
txCtx = core.NewEVMTxContext(msg)
ibs := evm.IntraBlockState().(*state.IntraBlockState)
- ibs.Prepare(common.Hash{}, parent.Hash(), txn_index)
+ ibs.SetTxContext(common.Hash{}, parent.Hash(), txn_index)
err = transactions.TraceTx(ctx, msg, blockCtx, txCtx, evm.IntraBlockState(), config, chainConfig, stream, api.evmCallTimeout)
if err != nil {
diff --git a/cmd/state/commands/erigon4.go b/cmd/state/commands/erigon4.go
index 7439dcb7c4f..bdcc9d35b5b 100644
--- a/cmd/state/commands/erigon4.go
+++ b/cmd/state/commands/erigon4.go
@@ -415,7 +415,7 @@ func processBlock23(startTxNum uint64, trace bool, txNumStart uint64, rw *StateR
for i, tx := range block.Transactions() {
if txNum >= startTxNum {
ibs := state.New(rw)
- ibs.Prepare(tx.Hash(), block.Hash(), i)
+ ibs.SetTxContext(tx.Hash(), block.Hash(), i)
ct := exec3.NewCallTracer()
vmConfig.Tracer = ct
receipt, _, err := core.ApplyTransaction(chainConfig, getHashFn, engine, nil, gp, ibs, ww, header, tx, usedGas, vmConfig, excessDataGas)
diff --git a/cmd/state/commands/history22.go b/cmd/state/commands/history22.go
index 1fa43d06345..94d260f1dce 100644
--- a/cmd/state/commands/history22.go
+++ b/cmd/state/commands/history22.go
@@ -256,7 +256,7 @@ func runHistory22(trace bool, blockNum, txNumStart uint64, hw *state.HistoryRead
for i, tx := range block.Transactions() {
hw.SetTxNum(txNum)
ibs := state.New(hw)
- ibs.Prepare(tx.Hash(), block.Hash(), i)
+ ibs.SetTxContext(tx.Hash(), block.Hash(), i)
receipt, _, err := core.ApplyTransaction(chainConfig, core.GetHashFn(header, getHeader), engine, nil, gp, ibs, ww, header, tx, usedGas, vmConfig, excessDataGas)
if err != nil {
return 0, nil, fmt.Errorf("could not apply tx %d [%x] failed: %w", i, tx.Hash(), err)
diff --git a/cmd/state/commands/opcode_tracer.go b/cmd/state/commands/opcode_tracer.go
index a93ecedbb98..e0c1a045acd 100644
--- a/cmd/state/commands/opcode_tracer.go
+++ b/cmd/state/commands/opcode_tracer.go
@@ -714,7 +714,7 @@ func runBlock(engine consensus.Engine, ibs *state.IntraBlockState, txnWriter sta
systemcontracts.UpgradeBuildInSystemContract(chainConfig, header.Number, ibs)
rules := chainConfig.Rules(block.NumberU64(), block.Time())
for i, tx := range block.Transactions() {
- ibs.Prepare(tx.Hash(), block.Hash(), i)
+ ibs.SetTxContext(tx.Hash(), block.Hash(), i)
receipt, _, err := core.ApplyTransaction(chainConfig, core.GetHashFn(header, getHeader), engine, nil, gp, ibs, txnWriter, header, tx, usedGas, vmConfig, excessDataGas)
if err != nil {
return nil, fmt.Errorf("could not apply tx %d [%x] failed: %w", i, tx.Hash(), err)
diff --git a/cmd/state/exec3/state.go b/cmd/state/exec3/state.go
index 13ef8cec26f..3ed5098f9d5 100644
--- a/cmd/state/exec3/state.go
+++ b/cmd/state/exec3/state.go
@@ -182,7 +182,7 @@ func (rw *Worker) RunTxTaskNoLock(txTask *exec22.TxTask) {
rw.taskGasPool.Reset(txTask.Tx.GetGas())
rw.callTracer.Reset()
vmConfig := vm.Config{Debug: true, Tracer: rw.callTracer, SkipAnalysis: txTask.SkipAnalysis}
- ibs.Prepare(txHash, txTask.BlockHash, txTask.TxIndex)
+ ibs.SetTxContext(txHash, txTask.BlockHash, txTask.TxIndex)
msg := txTask.TxAsMessage
blockContext := txTask.EvmBlockContext
diff --git a/cmd/state/exec3/state_recon.go b/cmd/state/exec3/state_recon.go
index 750363e2833..de1d991dd03 100644
--- a/cmd/state/exec3/state_recon.go
+++ b/cmd/state/exec3/state_recon.go
@@ -330,7 +330,7 @@ func (rw *ReconWorker) runTxTask(txTask *exec22.TxTask) error {
} else {
gp := new(core.GasPool).AddGas(txTask.Tx.GetGas())
vmConfig := vm.Config{NoReceipts: true, SkipAnalysis: txTask.SkipAnalysis}
- ibs.Prepare(txTask.Tx.Hash(), txTask.BlockHash, txTask.TxIndex)
+ ibs.SetTxContext(txTask.Tx.Hash(), txTask.BlockHash, txTask.TxIndex)
msg := txTask.TxAsMessage
rw.evm.ResetBetweenBlocks(txTask.EvmBlockContext, core.NewEVMTxContext(msg), ibs, vmConfig, txTask.Rules)
diff --git a/core/blockchain.go b/core/blockchain.go
index 39695039ea7..7a7687b5b0f 100644
--- a/core/blockchain.go
+++ b/core/blockchain.go
@@ -115,7 +115,7 @@ func ExecuteBlockEphemerally(
noop := state.NewNoopWriter()
//fmt.Printf("====txs processing start: %d====\n", block.NumberU64())
for i, tx := range block.Transactions() {
- ibs.Prepare(tx.Hash(), block.Hash(), i)
+ ibs.SetTxContext(tx.Hash(), block.Hash(), i)
writeTrace := false
if vmConfig.Debug && vmConfig.Tracer == nil {
tracer, err := getTracer(i, tx.Hash())
@@ -226,7 +226,7 @@ func ExecuteBlockEphemerallyBor(
noop := state.NewNoopWriter()
//fmt.Printf("====txs processing start: %d====\n", block.NumberU64())
for i, tx := range block.Transactions() {
- ibs.Prepare(tx.Hash(), block.Hash(), i)
+ ibs.SetTxContext(tx.Hash(), block.Hash(), i)
writeTrace := false
if vmConfig.Debug && vmConfig.Tracer == nil {
tracer, err := getTracer(i, tx.Hash())
diff --git a/core/chain_makers.go b/core/chain_makers.go
index 959f78073d1..1f052c78dbc 100644
--- a/core/chain_makers.go
+++ b/core/chain_makers.go
@@ -116,7 +116,7 @@ func (b *BlockGen) AddTxWithChain(getHeader func(hash libcommon.Hash, number uin
if b.gasPool == nil {
b.SetCoinbase(libcommon.Address{})
}
- b.ibs.Prepare(tx.Hash(), libcommon.Hash{}, len(b.txs))
+ b.ibs.SetTxContext(tx.Hash(), libcommon.Hash{}, len(b.txs))
receipt, _, err := ApplyTransaction(b.config, GetHashFn(b.header, getHeader), engine, &b.header.Coinbase, b.gasPool, b.ibs, state.NewNoopWriter(), b.header, tx, &b.header.GasUsed, vm.Config{}, b.parent.ExcessDataGas())
if err != nil {
panic(err)
@@ -129,7 +129,7 @@ func (b *BlockGen) AddFailedTxWithChain(getHeader func(hash libcommon.Hash, numb
if b.gasPool == nil {
b.SetCoinbase(libcommon.Address{})
}
- b.ibs.Prepare(tx.Hash(), libcommon.Hash{}, len(b.txs))
+ b.ibs.SetTxContext(tx.Hash(), libcommon.Hash{}, len(b.txs))
receipt, _, err := ApplyTransaction(b.config, GetHashFn(b.header, getHeader), engine, &b.header.Coinbase, b.gasPool, b.ibs, state.NewNoopWriter(), b.header, tx, &b.header.GasUsed, vm.Config{}, b.parent.ExcessDataGas())
_ = err // accept failed transactions
b.txs = append(b.txs, tx)
diff --git a/core/state/intra_block_state.go b/core/state/intra_block_state.go
index e916c4f1791..bd57ece5c95 100644
--- a/core/state/intra_block_state.go
+++ b/core/state/intra_block_state.go
@@ -75,13 +75,18 @@ type IntraBlockState struct {
logs map[libcommon.Hash][]*types.Log
logSize uint
+ // Per-transaction access list
+ accessList *accessList
+
+ // Transient storage
+ transientStorage transientStorage
+
// Journal of state modifications. This is the backbone of
// Snapshot and RevertToSnapshot.
journal *journal
validRevisions []revision
nextRevisionID int
trace bool
- accessList *accessList
balanceInc map[libcommon.Address]*BalanceIncrease // Map of balance increases (without first reading the account)
}
@@ -95,6 +100,7 @@ func New(stateReader StateReader) *IntraBlockState {
logs: map[libcommon.Hash][]*types.Log{},
journal: newJournal(),
accessList: newAccessList(),
+ transientStorage: newTransientStorage(),
balanceInc: map[libcommon.Address]*BalanceIncrease{},
}
}
@@ -414,6 +420,35 @@ func (sdb *IntraBlockState) Selfdestruct(addr libcommon.Address) bool {
return true
}
+// SetTransientState sets transient storage for a given account. It
+// adds the change to the journal so that it can be rolled back
+// to its previous value if there is a revert.
+func (sdb *IntraBlockState) SetTransientState(addr libcommon.Address, key libcommon.Hash, value uint256.Int) {
+ prev := sdb.GetTransientState(addr, key)
+ if prev == value {
+ return
+ }
+
+ sdb.journal.append(transientStorageChange{
+ account: &addr,
+ key: key,
+ prevalue: prev,
+ })
+
+ sdb.setTransientState(addr, key, value)
+}
+
+// setTransientState is a lower level setter for transient storage. It
+// is called during a revert to prevent modifications to the journal.
+func (sdb *IntraBlockState) setTransientState(addr libcommon.Address, key libcommon.Hash, value uint256.Int) {
+ sdb.transientStorage.Set(addr, key, value)
+}
+
+// GetTransientState gets transient storage for a given account.
+func (sdb *IntraBlockState) GetTransientState(addr libcommon.Address, key libcommon.Hash) uint256.Int {
+ return sdb.transientStorage.Get(addr, key)
+}
+
func (sdb *IntraBlockState) getStateObject(addr libcommon.Address) (stateObject *stateObject) {
// Prefer 'live' objects.
if obj := sdb.stateObjects[addr]; obj != nil {
@@ -701,13 +736,13 @@ func (sdb *IntraBlockState) Print(chainRules chain.Rules) {
}
}
-// Prepare sets the current transaction hash and index and block hash which is
-// used when the EVM emits new state logs.
-func (sdb *IntraBlockState) Prepare(thash, bhash libcommon.Hash, ti int) {
+// SetTxContext sets the current transaction hash and index and block hash which are
+// used when the EVM emits new state logs. It should be invoked before
+// transaction execution.
+func (sdb *IntraBlockState) SetTxContext(thash, bhash libcommon.Hash, ti int) {
sdb.thash = thash
sdb.bhash = bhash
sdb.txIndex = ti
- sdb.accessList = newAccessList()
}
// no not lock
@@ -717,30 +752,48 @@ func (sdb *IntraBlockState) clearJournalAndRefund() {
sdb.refund = 0
}
-// PrepareAccessList handles the preparatory steps for executing a state transition with
-// regards to both EIP-2929 and EIP-2930:
+// Prepare handles the preparatory steps for executing a state transition.
+// This method must be invoked before state transition.
+//
+// Berlin fork:
+// - Add sender to access list (EIP-2929)
+// - Add destination to access list (EIP-2929)
+// - Add precompiles to access list (EIP-2929)
+// - Add the contents of the optional tx access list (EIP-2930)
//
-// - Add sender to access list (2929)
-// - Add destination to access list (2929)
-// - Add precompiles to access list (2929)
-// - Add the contents of the optional tx access list (2930)
+// Shanghai fork:
+// - Add coinbase to access list (EIP-3651)
//
-// This method should only be called if Yolov3/Berlin/2929+2930 is applicable at the current number.
-func (sdb *IntraBlockState) PrepareAccessList(sender libcommon.Address, dst *libcommon.Address, precompiles []libcommon.Address, list types2.AccessList) {
- sdb.AddAddressToAccessList(sender)
- if dst != nil {
- sdb.AddAddressToAccessList(*dst)
- // If it's a create-tx, the destination will be added inside evm.create
- }
- for _, addr := range precompiles {
- sdb.AddAddressToAccessList(addr)
- }
- for _, el := range list {
- sdb.AddAddressToAccessList(el.Address)
- for _, key := range el.StorageKeys {
- sdb.AddSlotToAccessList(el.Address, key)
+// Cancun fork:
+// - Reset transient storage (EIP-1153)
+func (sdb *IntraBlockState) Prepare(rules *chain.Rules, sender, coinbase libcommon.Address, dst *libcommon.Address,
+ precompiles []libcommon.Address, list types2.AccessList,
+) {
+ if rules.IsBerlin {
+ // Clear out any leftover from previous executions
+ al := newAccessList()
+ sdb.accessList = al
+
+ al.AddAddress(sender)
+ if dst != nil {
+ al.AddAddress(*dst)
+ // If it's a create-tx, the destination will be added inside evm.create
+ }
+ for _, addr := range precompiles {
+ al.AddAddress(addr)
+ }
+ for _, el := range list {
+ al.AddAddress(el.Address)
+ for _, key := range el.StorageKeys {
+ al.AddSlot(el.Address, key)
+ }
+ }
+ if rules.IsShanghai { // EIP-3651: warm coinbase
+ al.AddAddress(coinbase)
}
}
+ // Reset transient storage at the beginning of transaction execution
+ sdb.transientStorage = newTransientStorage()
}
// AddAddressToAccessList adds the given address to the access list
diff --git a/core/state/intra_block_state_test.go b/core/state/intra_block_state_test.go
index 3c202c50fad..600fa75c60b 100644
--- a/core/state/intra_block_state_test.go
+++ b/core/state/intra_block_state_test.go
@@ -158,6 +158,16 @@ func newTestAction(addr libcommon.Address, r *rand.Rand) testAction {
},
args: make([]int64, 1),
},
+ {
+ name: "SetTransientState",
+ fn: func(a testAction, s *IntraBlockState) {
+ var key libcommon.Hash
+ binary.BigEndian.PutUint16(key[:], uint16(a.args[0]))
+ val := uint256.NewInt(uint64(a.args[1]))
+ s.SetTransientState(addr, key, *val)
+ },
+ args: make([]int64, 2),
+ },
}
action := actions[r.Intn(len(actions))]
var nameargs []string //nolint:prealloc
@@ -312,3 +322,27 @@ func (test *snapshotTest) checkEqual(state, checkstate *IntraBlockState) error {
}
return nil
}
+
+func TestTransientStorage(t *testing.T) {
+ state := New(nil)
+
+ key := libcommon.Hash{0x01}
+ value := uint256.NewInt(2)
+ addr := libcommon.Address{}
+
+ state.SetTransientState(addr, key, *value)
+ if exp, got := 1, state.journal.length(); exp != got {
+ t.Fatalf("journal length mismatch: have %d, want %d", got, exp)
+ }
+ // the retrieved value should equal what was set
+ if got := state.GetTransientState(addr, key); got != *value {
+ t.Fatalf("transient storage mismatch: have %x, want %x", got, value)
+ }
+
+ // revert the transient state being set and then check that the
+ // value is now the empty hash
+ state.journal.revert(state, 0)
+ if got, exp := state.GetTransientState(addr, key), (*uint256.NewInt(0)); exp != got {
+ t.Fatalf("transient storage mismatch: have %x, want %x", got, exp)
+ }
+}
diff --git a/core/state/journal.go b/core/state/journal.go
index c5faaad7bb5..4b018dcbfca 100644
--- a/core/state/journal.go
+++ b/core/state/journal.go
@@ -140,6 +140,7 @@ type (
touchChange struct {
account *libcommon.Address
}
+
// Changes to the access list
accessListAddAccountChange struct {
address *libcommon.Address
@@ -148,6 +149,12 @@ type (
address *libcommon.Address
slot *libcommon.Hash
}
+
+ transientStorageChange struct {
+ account *libcommon.Address
+ key libcommon.Hash
+ prevalue uint256.Int
+ }
)
func (ch createObjectChange) revert(s *IntraBlockState) {
@@ -249,6 +256,14 @@ func (ch fakeStorageChange) dirtied() *libcommon.Address {
return ch.account
}
+func (ch transientStorageChange) revert(s *IntraBlockState) {
+ s.setTransientState(*ch.account, ch.key, ch.prevalue)
+}
+
+func (ch transientStorageChange) dirtied() *libcommon.Address {
+ return nil
+}
+
func (ch refundChange) revert(s *IntraBlockState) {
s.refund = ch.prev
}
diff --git a/core/state/transient_storage.go b/core/state/transient_storage.go
new file mode 100644
index 00000000000..39b2535fdf5
--- /dev/null
+++ b/core/state/transient_storage.go
@@ -0,0 +1,48 @@
+// Copyright 2022 The go-ethereum Authors
+// This file is part of the go-ethereum library.
+//
+// The go-ethereum library is free software: you can redistribute it and/or modify
+// it under the terms of the GNU Lesser General Public License as published by
+// the Free Software Foundation, either version 3 of the License, or
+// (at your option) any later version.
+//
+// The go-ethereum library is distributed in the hope that it will be useful,
+// but WITHOUT ANY WARRANTY; without even the implied warranty of
+// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+// GNU Lesser General Public License for more details.
+//
+// You should have received a copy of the GNU Lesser General Public License
+// along with the go-ethereum library. If not, see .
+
+package state
+
+import (
+ "github.com/holiman/uint256"
+
+ libcommon "github.com/ledgerwatch/erigon-lib/common"
+)
+
+// transientStorage is a representation of EIP-1153 "Transient Storage".
+type transientStorage map[libcommon.Address]Storage
+
+// newTransientStorage creates a new instance of a transientStorage.
+func newTransientStorage() transientStorage {
+ return make(transientStorage)
+}
+
+// Set sets the transient-storage `value` for `key` at the given `addr`.
+func (t transientStorage) Set(addr libcommon.Address, key libcommon.Hash, value uint256.Int) {
+ if _, ok := t[addr]; !ok {
+ t[addr] = make(Storage)
+ }
+ t[addr][key] = value
+}
+
+// Get gets the transient storage for `key` at the given `addr`.
+func (t transientStorage) Get(addr libcommon.Address, key libcommon.Hash) uint256.Int {
+ val, ok := t[addr]
+ if !ok {
+ return *uint256.NewInt(0)
+ }
+ return val[key]
+}
diff --git a/core/state_transition.go b/core/state_transition.go
index 419769d0b7d..b1d2302b6ef 100644
--- a/core/state_transition.go
+++ b/core/state_transition.go
@@ -299,11 +299,13 @@ func (st *StateTransition) preCheck(gasBailout bool) error {
// However if any consensus issue encountered, return the error directly with
// nil evm execution result.
func (st *StateTransition) TransitionDb(refunds bool, gasBailout bool) (*ExecutionResult, error) {
+ coinbase := st.evm.Context().Coinbase
+
var input1 *uint256.Int
var input2 *uint256.Int
if st.isBor {
input1 = st.state.GetBalance(st.msg.From()).Clone()
- input2 = st.state.GetBalance(st.evm.Context().Coinbase).Clone()
+ input2 = st.state.GetBalance(coinbase).Clone()
}
// First check this message satisfies all consensus rules before
@@ -357,14 +359,10 @@ func (st *StateTransition) TransitionDb(refunds bool, gasBailout bool) (*Executi
return nil, fmt.Errorf("%w: code size %v limit %v", ErrMaxInitCodeSizeExceeded, len(st.data), params.MaxInitCodeSize)
}
- // Set up the initial access list.
- if rules.IsBerlin {
- st.state.PrepareAccessList(msg.From(), msg.To(), vm.ActivePrecompiles(rules), msg.AccessList())
- // EIP-3651 warm COINBASE
- if rules.IsShanghai {
- st.state.AddAddressToAccessList(st.evm.Context().Coinbase)
- }
- }
+ // Execute the preparatory steps for state transition which includes:
+ // - prepare accessList(post-berlin)
+ // - reset transient storage(eip 1153)
+ st.state.Prepare(rules, msg.From(), coinbase, msg.To(), vm.ActivePrecompiles(rules), msg.AccessList())
var (
ret []byte
@@ -400,7 +398,7 @@ func (st *StateTransition) TransitionDb(refunds bool, gasBailout bool) (*Executi
}
amount := new(uint256.Int).SetUint64(st.gasUsed())
amount.Mul(amount, effectiveTip) // gasUsed * effectiveTip = how much goes to the block producer (miner, validator)
- st.state.AddBalance(st.evm.Context().Coinbase, amount)
+ st.state.AddBalance(coinbase, amount)
if !msg.IsFree() && rules.IsLondon && rules.IsEip1559FeeCollector {
burntContractAddress := *st.evm.ChainConfig().Eip1559FeeCollector
burnAmount := new(uint256.Int).Mul(new(uint256.Int).SetUint64(st.gasUsed()), st.evm.Context().BaseFee)
@@ -415,7 +413,7 @@ func (st *StateTransition) TransitionDb(refunds bool, gasBailout bool) (*Executi
st.state,
msg.From(),
- st.evm.Context().Coinbase,
+ coinbase,
amount,
input1,
diff --git a/core/vm/eips.go b/core/vm/eips.go
index 1109aa5148a..83e37d68673 100644
--- a/core/vm/eips.go
+++ b/core/vm/eips.go
@@ -22,18 +22,21 @@ import (
"github.com/holiman/uint256"
+ libcommon "github.com/ledgerwatch/erigon-lib/common"
+
"github.com/ledgerwatch/erigon/params"
)
var activators = map[int]func(*JumpTable){
- 3855: enable3855,
3860: enable3860,
+ 3855: enable3855,
3529: enable3529,
3198: enable3198,
2929: enable2929,
2200: enable2200,
1884: enable1884,
1344: enable1344,
+ 1153: enable1153,
}
// EnableEIP enables the given EIP on the config.
@@ -168,6 +171,45 @@ func enable3198(jt *JumpTable) {
}
}
+// enable1153 applies EIP-1153 "Transient Storage"
+// - Adds TLOAD that reads from transient storage
+// - Adds TSTORE that writes to transient storage
+func enable1153(jt *JumpTable) {
+ jt[TLOAD] = &operation{
+ execute: opTload,
+ constantGas: params.WarmStorageReadCostEIP2929,
+ numPop: 1,
+ numPush: 1,
+ }
+
+ jt[TSTORE] = &operation{
+ execute: opTstore,
+ constantGas: params.WarmStorageReadCostEIP2929,
+ numPop: 2,
+ numPush: 0,
+ }
+}
+
+// opTload implements TLOAD opcode
+func opTload(pc *uint64, interpreter *EVMInterpreter, scope *ScopeContext) ([]byte, error) {
+ loc := scope.Stack.Peek()
+ hash := libcommon.Hash(loc.Bytes32())
+ val := interpreter.evm.IntraBlockState().GetTransientState(scope.Contract.Address(), hash)
+ loc.SetBytes(val.Bytes())
+ return nil, nil
+}
+
+// opTstore implements TSTORE opcode
+func opTstore(pc *uint64, interpreter *EVMInterpreter, scope *ScopeContext) ([]byte, error) {
+ if interpreter.readOnly {
+ return nil, ErrWriteProtection
+ }
+ loc := scope.Stack.Pop()
+ val := scope.Stack.Pop()
+ interpreter.evm.IntraBlockState().SetTransientState(scope.Contract.Address(), loc.Bytes32(), val)
+ return nil, nil
+}
+
// opBaseFee implements BASEFEE opcode
func opBaseFee(pc *uint64, interpreter *EVMInterpreter, callContext *ScopeContext) ([]byte, error) {
baseFee := interpreter.evm.Context().BaseFee
diff --git a/core/vm/evmtypes/evmtypes.go b/core/vm/evmtypes/evmtypes.go
index efc59612021..a8cfed5cba1 100644
--- a/core/vm/evmtypes/evmtypes.go
+++ b/core/vm/evmtypes/evmtypes.go
@@ -4,6 +4,8 @@ import (
"math/big"
"github.com/holiman/uint256"
+
+ "github.com/ledgerwatch/erigon-lib/chain"
libcommon "github.com/ledgerwatch/erigon-lib/common"
types2 "github.com/ledgerwatch/erigon-lib/types"
@@ -77,6 +79,9 @@ type IntraBlockState interface {
GetState(address libcommon.Address, slot *libcommon.Hash, outValue *uint256.Int)
SetState(libcommon.Address, *libcommon.Hash, uint256.Int)
+ GetTransientState(addr libcommon.Address, key libcommon.Hash) uint256.Int
+ SetTransientState(addr libcommon.Address, key libcommon.Hash, value uint256.Int)
+
Selfdestruct(libcommon.Address) bool
HasSelfdestructed(libcommon.Address) bool
@@ -87,7 +92,9 @@ type IntraBlockState interface {
// is defined according to EIP161 (balance = nonce = code = 0).
Empty(libcommon.Address) bool
- PrepareAccessList(sender libcommon.Address, dest *libcommon.Address, precompiles []libcommon.Address, txAccesses types2.AccessList)
+ Prepare(rules *chain.Rules, sender, coinbase libcommon.Address, dest *libcommon.Address,
+ precompiles []libcommon.Address, txAccesses types2.AccessList)
+
AddressInAccessList(addr libcommon.Address) bool
SlotInAccessList(addr libcommon.Address, slot libcommon.Hash) (addressOk bool, slotOk bool)
// AddAddressToAccessList adds the given address to the access list. This operation is safe to perform
diff --git a/core/vm/instructions_test.go b/core/vm/instructions_test.go
index 49d11fe6455..9be42471be1 100644
--- a/core/vm/instructions_test.go
+++ b/core/vm/instructions_test.go
@@ -25,11 +25,13 @@ import (
"testing"
"github.com/holiman/uint256"
- libcommon "github.com/ledgerwatch/erigon-lib/common"
- "github.com/ledgerwatch/erigon/core/vm/evmtypes"
+ libcommon "github.com/ledgerwatch/erigon-lib/common"
"github.com/ledgerwatch/erigon/common"
+ "github.com/ledgerwatch/erigon/common/u256"
+ "github.com/ledgerwatch/erigon/core/state"
+ "github.com/ledgerwatch/erigon/core/vm/evmtypes"
"github.com/ledgerwatch/erigon/core/vm/stack"
"github.com/ledgerwatch/erigon/crypto"
"github.com/ledgerwatch/erigon/params"
@@ -51,6 +53,14 @@ type twoOperandParams struct {
var commonParams []*twoOperandParams
var twoOpMethods map[string]executionFunc
+type contractRef struct {
+ addr libcommon.Address
+}
+
+func (c contractRef) Address() libcommon.Address {
+ return c.addr
+}
+
func init() {
// Params is a list of common edgecases that should be used for some common tests
@@ -561,6 +571,45 @@ func BenchmarkOpMstore(bench *testing.B) {
}
}
+func TestOpTstore(t *testing.T) {
+ var (
+ state = state.New(nil)
+ env = NewEVM(evmtypes.BlockContext{}, evmtypes.TxContext{}, state, params.TestChainConfig, Config{})
+ stack = stack.New()
+ mem = NewMemory()
+ evmInterpreter = NewEVMInterpreter(env, env.Config())
+ caller = libcommon.Address{}
+ to = libcommon.Address{1}
+ contractRef = contractRef{caller}
+ contract = NewContract(contractRef, AccountRef(to), u256.Num0, 0, false)
+ scopeContext = ScopeContext{mem, stack, contract}
+ value = common.Hex2Bytes("abcdef00000000000000abba000000000deaf000000c0de00100000000133700")
+ )
+
+ env.interpreter = evmInterpreter
+ pc := uint64(0)
+ // push the value to the stack
+ stack.Push(new(uint256.Int).SetBytes(value))
+ // push the location to the stack
+ stack.Push(new(uint256.Int))
+ opTstore(&pc, evmInterpreter, &scopeContext)
+ // there should be no elements on the stack after TSTORE
+ if stack.Len() != 0 {
+ t.Fatal("stack wrong size")
+ }
+ // push the location to the stack
+ stack.Push(new(uint256.Int))
+ opTload(&pc, evmInterpreter, &scopeContext)
+ // there should be one element on the stack after TLOAD
+ if stack.Len() != 1 {
+ t.Fatal("stack wrong size")
+ }
+ val := stack.Peek()
+ if !bytes.Equal(val.Bytes(), value) {
+ t.Fatal("incorrect element read from transient storage")
+ }
+}
+
func BenchmarkOpKeccak256(bench *testing.B) {
var (
env = NewEVM(evmtypes.BlockContext{}, evmtypes.TxContext{}, nil, params.TestChainConfig, Config{})
diff --git a/core/vm/opcodes.go b/core/vm/opcodes.go
index 7db520fd5e5..3d9728e569a 100644
--- a/core/vm/opcodes.go
+++ b/core/vm/opcodes.go
@@ -215,6 +215,12 @@ const (
SELFDESTRUCT OpCode = 0xff
)
+// 0xb0 range.
+const (
+ TLOAD OpCode = 0xb3
+ TSTORE OpCode = 0xb4
+)
+
// Since the opcodes aren't all in order we can't use a regular slice.
var opCodeToString = map[OpCode]string{
// 0x0 range - arithmetic ops.
@@ -370,6 +376,10 @@ var opCodeToString = map[OpCode]string{
LOG3: "LOG3",
LOG4: "LOG4",
+ // 0xb0 range.
+ TLOAD: "TLOAD",
+ TSTORE: "TSTORE",
+
// 0xf0 range.
CREATE: "CREATE",
CALL: "CALL",
@@ -461,6 +471,8 @@ var stringToOp = map[string]OpCode{
"GAS": GAS,
"JUMPDEST": JUMPDEST,
"PUSH0": PUSH0,
+ "TLOAD": TLOAD,
+ "TSTORE": TSTORE,
"PUSH1": PUSH1,
"PUSH2": PUSH2,
"PUSH3": PUSH3,
diff --git a/core/vm/runtime/runtime.go b/core/vm/runtime/runtime.go
index 8b9017a67d6..8dd6706448b 100644
--- a/core/vm/runtime/runtime.go
+++ b/core/vm/runtime/runtime.go
@@ -132,10 +132,9 @@ func Execute(code, input []byte, cfg *Config, bn uint64) ([]byte, *state.IntraBl
address = libcommon.BytesToAddress([]byte("contract"))
vmenv = NewEnv(cfg)
sender = vm.AccountRef(cfg.Origin)
+ rules = cfg.ChainConfig.Rules(vmenv.Context().BlockNumber, vmenv.Context().Time)
)
- if rules := cfg.ChainConfig.Rules(vmenv.Context().BlockNumber, vmenv.Context().Time); rules.IsBerlin {
- cfg.State.PrepareAccessList(cfg.Origin, &address, vm.ActivePrecompiles(rules), nil)
- }
+ cfg.State.Prepare(rules, cfg.Origin, cfg.Coinbase, &address, vm.ActivePrecompiles(rules), nil)
cfg.State.CreateAccount(address, true)
// set the receiver's (the executing contract) code for execution.
cfg.State.SetCode(address, code)
@@ -177,10 +176,9 @@ func Create(input []byte, cfg *Config, blockNr uint64) ([]byte, libcommon.Addres
var (
vmenv = NewEnv(cfg)
sender = vm.AccountRef(cfg.Origin)
+ rules = cfg.ChainConfig.Rules(vmenv.Context().BlockNumber, vmenv.Context().Time)
)
- if rules := cfg.ChainConfig.Rules(vmenv.Context().BlockNumber, vmenv.Context().Time); rules.IsBerlin {
- cfg.State.PrepareAccessList(cfg.Origin, nil, vm.ActivePrecompiles(rules), nil)
- }
+ cfg.State.Prepare(rules, cfg.Origin, cfg.Coinbase, nil, vm.ActivePrecompiles(rules), nil)
// Call the code with the given configuration.
code, address, leftOverGas, err := vmenv.Create(
@@ -204,9 +202,8 @@ func Call(address libcommon.Address, input []byte, cfg *Config) ([]byte, uint64,
sender := cfg.State.GetOrNewStateObject(cfg.Origin)
statedb := cfg.State
- if rules := cfg.ChainConfig.Rules(vmenv.Context().BlockNumber, vmenv.Context().Time); rules.IsBerlin {
- statedb.PrepareAccessList(cfg.Origin, &address, vm.ActivePrecompiles(rules), nil)
- }
+ rules := cfg.ChainConfig.Rules(vmenv.Context().BlockNumber, vmenv.Context().Time)
+ statedb.Prepare(rules, cfg.Origin, cfg.Coinbase, &address, vm.ActivePrecompiles(rules), nil)
// Call the code with the given configuration.
ret, leftOverGas, err := vmenv.Call(
diff --git a/eth/stagedsync/stage_mining_exec.go b/eth/stagedsync/stage_mining_exec.go
index a5525ad4135..337b3d96f39 100644
--- a/eth/stagedsync/stage_mining_exec.go
+++ b/eth/stagedsync/stage_mining_exec.go
@@ -383,7 +383,7 @@ func addTransactionsToMiningBlock(logPrefix string, current *MiningBlock, chainC
parentHeader := getHeader(header.ParentHash, header.Number.Uint64()-1)
var miningCommitTx = func(txn types.Transaction, coinbase libcommon.Address, vmConfig *vm.Config, chainConfig chain.Config, ibs *state.IntraBlockState, current *MiningBlock) ([]*types.Log, error) {
- ibs.Prepare(txn.Hash(), libcommon.Hash{}, tcount)
+ ibs.SetTxContext(txn.Hash(), libcommon.Hash{}, tcount)
gasSnap := gasPool.Gas()
snap := ibs.Snapshot()
log.Debug("addTransactionsToMiningBlock", "txn hash", txn.Hash())
diff --git a/eth/state_accessor.go b/eth/state_accessor.go
deleted file mode 100644
index 973ffd7ee9f..00000000000
--- a/eth/state_accessor.go
+++ /dev/null
@@ -1,17 +0,0 @@
-// Copyright 2021 The go-ethereum Authors
-// This file is part of the go-ethereum library.
-//
-// The go-ethereum library is free software: you can redistribute it and/or modify
-// it under the terms of the GNU Lesser General Public License as published by
-// the Free Software Foundation, either version 3 of the License, or
-// (at your option) any later version.
-//
-// The go-ethereum library is distributed in the hope that it will be useful,
-// but WITHOUT ANY WARRANTY; without even the implied warranty of
-// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
-// GNU Lesser General Public License for more details.
-//
-// You should have received a copy of the GNU Lesser General Public License
-// along with the go-ethereum library. If not, see .
-
-package eth
diff --git a/turbo/transactions/tracing.go b/turbo/transactions/tracing.go
index a1ac24d56a6..7a2a032523f 100644
--- a/turbo/transactions/tracing.go
+++ b/turbo/transactions/tracing.go
@@ -71,7 +71,7 @@ func ComputeTxEnv(ctx context.Context, engine consensus.EngineReader, block *typ
if historyV3 {
rules := cfg.Rules(BlockContext.BlockNumber, BlockContext.Time)
txn := block.Transactions()[txIndex]
- statedb.Prepare(txn.Hash(), block.Hash(), txIndex)
+ statedb.SetTxContext(txn.Hash(), block.Hash(), txIndex)
msg, _ := txn.AsMessage(*signer, block.BaseFee(), rules)
if msg.FeeCap().IsZero() && engine != nil {
syscall := func(contract libcommon.Address, data []byte) ([]byte, error) {
@@ -96,7 +96,7 @@ func ComputeTxEnv(ctx context.Context, engine consensus.EngineReader, block *typ
case <-ctx.Done():
return nil, evmtypes.BlockContext{}, evmtypes.TxContext{}, nil, nil, ctx.Err()
}
- statedb.Prepare(txn.Hash(), block.Hash(), idx)
+ statedb.SetTxContext(txn.Hash(), block.Hash(), idx)
// Assemble the transaction call message and return if the requested offset
msg, _ := txn.AsMessage(*signer, block.BaseFee(), rules)