refactor touched accounts, selfdestruct, and log

simplify EVM and delegete those things to accounts cache.
also no more manual state clearing, accounts cache will be
responsible for both collecting touched account and perform
state clearing.
This commit is contained in:
jangko 2023-03-20 18:51:09 +07:00
parent 249cabd64e
commit 2a3c67b4be
No known key found for this signature in database
GPG Key ID: 31702AE10541E6B9
23 changed files with 125 additions and 116 deletions

View File

@ -83,7 +83,6 @@ proc procBlkPreamble(vmState: BaseVMState;
for withdrawal in body.withdrawals.get:
vmState.stateDB.addBalance(withdrawal.address, withdrawal.amount.gwei)
vmState.stateDB.deleteAccountIfEmpty(withdrawal.address)
else:
if header.withdrawalsRoot.isSome:
raise ValidationError.newException("Pre-Shanghai block header must not have withdrawalsRoot")
@ -111,7 +110,8 @@ proc procBlkEpilogue(vmState: BaseVMState;
vmState.mutateStateDB:
if vmState.generateWitness:
db.collectWitnessData()
db.persist(ClearCache in vmState.flags)
let clearEmptyAccount = vmState.determineFork >= FkSpurious
db.persist(clearEmptyAccount, ClearCache in vmState.flags)
let stateDb = vmState.stateDB
if header.stateRoot != stateDb.rootHash:

View File

@ -87,11 +87,11 @@ proc processTransactionImpl(
vmState.cumulativeGasUsed += gasBurned
result = ok(gasBurned)
vmState.clearSelfDestructsAndEmptyAccounts(fork, miner)
if vmState.generateWitness:
vmState.stateDB.collectWitnessData()
vmState.stateDB.persist(clearCache = false)
vmState.stateDB.persist(
clearEmptyAccount = fork >= FkSpurious,
clearCache = false)
# ------------------------------------------------------------------------------
# Public functions

View File

@ -71,7 +71,10 @@ proc persist(pst: TxPackerStateRef)
{.gcsafe,raises: [RlpError].} =
## Smart wrapper
if not pst.cleanState:
pst.xp.chain.vmState.stateDB.persist(clearCache = false)
let fork = pst.xp.chain.nextFork
pst.xp.chain.vmState.stateDB.persist(
clearEmptyAccount = fork >= FkSpurious,
clearCache = false)
pst.cleanState = true
# ------------------------------------------------------------------------------
@ -111,9 +114,6 @@ proc runTxCommit(pst: TxPackerStateRef; item: TxItemRef; gasBurned: GasInt)
let reward = gasBurned.u256 * gasTip.uint64.u256
vmState.stateDB.addBalance(xp.chain.feeRecipient, reward)
# Update account database
vmState.clearSelfDestructsAndEmptyAccounts(xp.chain.nextFork, xp.chain.feeRecipient)
if vmState.generateWitness:
vmState.stateDB.collectWitnessData()
@ -123,7 +123,7 @@ proc runTxCommit(pst: TxPackerStateRef; item: TxItemRef; gasBurned: GasInt)
# that the account cache has been saved, the `persist()` call is
# obligatory here.
if xp.chain.nextFork < FkByzantium:
pst.persist
pst.persist()
# Update receipts sequence
if vmState.receipts.len <= inx:
@ -188,7 +188,9 @@ proc vmExecGrabItem(pst: TxPackerStateRef; item: TxItemRef): Result[bool,void]
# Commit account state DB
vmState.stateDB.commit(accTx)
vmState.stateDB.persist(clearCache = false)
vmState.stateDB.persist(
clearEmptyAccount = xp.chain.nextFork >= FkSpurious,
clearCache = false)
# let midRoot = vmState.stateDB.rootHash -- notused
# Finish book-keeping and move item to `packed` bucket
@ -215,7 +217,9 @@ proc vmExecCommit(pst: TxPackerStateRef)
if vmState.generateWitness:
db.collectWitnessData()
# Finish up, then vmState.stateDB.rootHash may be accessed
db.persist(ClearCache in vmState.flags)
db.persist(
clearEmptyAccount = xp.chain.nextFork >= FkSpurious,
clearCache = ClearCache in vmState.flags)
# Update flexi-array, set proper length
let nItems = xp.txDB.byStatus.eq(txItemPacked).nItems

View File

@ -17,9 +17,13 @@ import
./pow/[difficulty, header],
./pow,
chronicles,
nimcrypto/utils,
stew/[objects, results]
# chronicles stuff
when loggingEnabled or enabledLogLevel >= NONE:
import
nimcrypto/utils
from stew/byteutils
import nil

View File

@ -6,11 +6,15 @@ import
./distinct_tries,
./access_list as ac_access_list
const
debugAccountsCache = false
type
AccountFlag = enum
Alive
IsNew
Dirty
Touched
CodeLoaded
CodeChanged
StorageChanged
@ -34,7 +38,7 @@ type
savePoint: SavePoint
witnessCache: Table[EthAddress, WitnessData]
isDirty: bool
touched: HashSet[EthAddress]
ripemdSpecial: bool
ReadOnlyStateDB* = distinct AccountsCache
@ -46,8 +50,12 @@ type
SavePoint* = ref object
parentSavepoint: SavePoint
cache: Table[EthAddress, RefAccount]
selfDestruct: HashSet[EthAddress]
logEntries: seq[Log]
accessList: ac_access_list.AccessList
state: TransactionState
when debugAccountsCache:
depth: int
const
emptyAcc = newAccount()
@ -55,19 +63,35 @@ const
resetFlags = {
Dirty,
IsNew,
Touched,
CodeChanged,
StorageChanged
}
proc beginSavepoint*(ac: var AccountsCache): SavePoint {.gcsafe.}
ripemdAddr* = block:
proc initAddress(x: int): EthAddress {.compileTime.} =
result[19] = x.byte
initAddress(3)
when debugAccountsCache:
import
stew/byteutils
proc inspectSavePoint(name: string, x: SavePoint) =
debugEcho "*** ", name, ": ", x.depth, " ***"
var sp = x
while sp != nil:
for address, acc in sp.cache:
debugEcho address.toHex, " ", acc.flags
sp = sp.parentSavepoint
proc beginSavepoint*(ac: var AccountsCache): SavePoint {.gcsafe.}
# FIXME-Adam: this is only necessary because of my sanity checks on the latest rootHash;
# take this out once those are gone.
proc rawTrie*(ac: AccountsCache): AccountsTrie = ac.trie
proc rawDb*(ac: AccountsCache): TrieDatabaseRef = ac.trie.db
# The AccountsCache is modeled after TrieDatabase for it's transaction style
proc init*(x: typedesc[AccountsCache], db: TrieDatabaseRef,
root: KeccakHash, pruneTrie: bool = true): AccountsCache =
@ -99,6 +123,11 @@ proc beginSavepoint*(ac: var AccountsCache): SavePoint =
result.parentSavepoint = ac.savePoint
ac.savePoint = result
when debugAccountsCache:
if not result.parentSavePoint.isNil:
result.depth = result.parentSavePoint.depth + 1
inspectSavePoint("snapshot", result)
proc rollback*(ac: var AccountsCache, sp: SavePoint) =
# Transactions should be handled in a strictly nested fashion.
# Any child transaction must be committed or rolled-back before
@ -107,6 +136,9 @@ proc rollback*(ac: var AccountsCache, sp: SavePoint) =
ac.savePoint = sp.parentSavepoint
sp.state = RolledBack
when debugAccountsCache:
inspectSavePoint("rollback", ac.savePoint)
proc commit*(ac: var AccountsCache, sp: SavePoint) =
# Transactions should be handled in a strictly nested fashion.
# Any child transaction must be committed or rolled-back before
@ -120,8 +152,13 @@ proc commit*(ac: var AccountsCache, sp: SavePoint) =
sp.parentSavepoint.cache[k] = v
ac.savePoint.accessList.merge(sp.accessList)
ac.savePoint.selfDestruct.incl sp.selfDestruct
ac.savePoint.logEntries.add sp.logEntries
sp.state = Committed
when debugAccountsCache:
inspectSavePoint("commit", ac.savePoint)
proc dispose*(ac: var AccountsCache, sp: SavePoint) {.inline.} =
if sp.state == Pending:
ac.rollback(sp)
@ -397,7 +434,7 @@ proc addBalance*(ac: AccountsCache, address: EthAddress, delta: UInt256) {.inlin
if delta == 0.u256:
let acc = ac.getAccount(address)
if acc.isEmpty:
ac.touched.incl address
ac.makeDirty(address).flags.incl Touched
return
ac.setBalance(address, ac.getBalance(address) + delta)
@ -445,8 +482,26 @@ proc deleteAccount*(ac: AccountsCache, address: EthAddress) =
let acc = ac.getAccount(address)
acc.kill()
proc deleteAccountIfEmpty*(ac: AccountsCache, address: EthAddress) =
# see https://github.com/ethereum/EIPs/blob/master/EIPS/eip-158.md
proc selfDestruct*(ac: AccountsCache, address: EthAddress) =
ac.savePoint.selfDestruct.incl address
proc selfDestructLen*(ac: AccountsCache): int =
ac.savePoint.selfDestruct.len
proc addLogEntry*(ac: AccountsCache, log: Log) =
ac.savePoint.logEntries.add log
proc logEntries*(ac: AccountsCache): seq[Log] =
ac.savePoint.logEntries
proc getAndClearLogEntries*(ac: AccountsCache): seq[Log] =
result = ac.savePoint.logEntries
ac.savePoint.logEntries.setLen(0)
proc ripemdSpecial*(ac: AccountsCache) =
ac.ripemdSpecial = true
proc deleteEmptyAccount(ac: AccountsCache, address: EthAddress) =
let acc = ac.getAccount(address, false)
if acc.isNil:
return
@ -454,14 +509,32 @@ proc deleteAccountIfEmpty*(ac: AccountsCache, address: EthAddress) =
return
if not acc.exists:
return
if address in ac.touched or Dirty in acc.flags:
ac.deleteAccount(address)
acc.kill()
proc persist*(ac: AccountsCache, clearCache: bool = true) =
proc clearEmptyAccounts(ac: AccountsCache) =
for address, acc in ac.savePoint.cache:
if Touched in acc.flags and
acc.isEmpty and acc.exists:
acc.kill()
# https://github.com/ethereum/EIPs/issues/716
if ac.ripemdSpecial:
ac.deleteEmptyAccount(ripemdAddr)
ac.ripemdSpecial = false
proc persist*(ac: AccountsCache,
clearEmptyAccount: bool = false,
clearCache: bool = true) =
# make sure all savepoint already committed
doAssert(ac.savePoint.parentSavepoint.isNil)
var cleanAccounts = initHashSet[EthAddress]()
if clearEmptyAccount:
ac.clearEmptyAccounts()
for address in ac.savePoint.selfDestruct:
ac.deleteAccount(address)
for address, acc in ac.savePoint.cache:
case acc.persistMode()
of Update:
@ -487,7 +560,7 @@ proc persist*(ac: AccountsCache, clearCache: bool = true) =
for x in cleanAccounts:
ac.savePoint.cache.del x
ac.touched.clear()
ac.savePoint.selfDestruct.clear()
# EIP2929
ac.savePoint.accessList.clear()

View File

@ -195,8 +195,6 @@ proc newComputation*(vmState: BaseVMState, message: Message,
result.stack = newStack()
result.returnStack = @[]
result.gasMeter.init(message.gas)
result.touchedAccounts = initHashSet[EthAddress]()
result.selfDestructs = initHashSet[EthAddress]()
if result.msg.isCreate():
result.msg.contractAddress = result.generateContractAddress(salt)
@ -214,8 +212,6 @@ proc newComputation*(vmState: BaseVMState, message: Message, code: seq[byte]): C
result.stack = newStack()
result.returnStack = @[]
result.gasMeter.init(message.gas)
result.touchedAccounts = initHashSet[EthAddress]()
result.selfDestructs = initHashSet[EthAddress]()
result.code = newCodeStream(code)
template gasCosts*(c: Computation): untyped =
@ -237,9 +233,6 @@ template isError*(c: Computation): bool =
func shouldBurnGas*(c: Computation): bool =
c.isError and c.error.burnsGas
proc isSelfDestructed*(c: Computation, address: EthAddress): bool =
result = address in c.selfDestructs
proc snapshot*(c: Computation) =
c.savePoint = c.vmState.stateDB.beginSavepoint()
@ -327,10 +320,7 @@ template asyncChainTo*(c: Computation, asyncOperation: Future[void], after: unty
after
proc merge*(c, child: Computation) =
c.logEntries.add child.logEntries
c.gasMeter.refundGas(child.gasMeter.gasRefunded)
c.selfDestructs.incl child.selfDestructs
c.touchedAccounts.incl child.touchedAccounts
proc execSelfDestruct*(c: Computation, beneficiary: EthAddress)
{.gcsafe, raises: [CatchableError].} =
@ -345,17 +335,16 @@ proc execSelfDestruct*(c: Computation, beneficiary: EthAddress)
# contract named itself as the beneficiary.
db.setBalance(c.msg.contractAddress, 0.u256)
# Register the account to be deleted
db.selfDestruct(c.msg.contractAddress)
trace "SELFDESTRUCT",
contractAddress = c.msg.contractAddress.toHex,
localBalance = localBalance.toString,
beneficiary = beneficiary.toHex
c.touchedAccounts.incl beneficiary
# Register the account to be deleted
c.selfDestructs.incl(c.msg.contractAddress)
proc addLogEntry*(c: Computation, log: Log) =
c.logEntries.add(log)
c.vmState.stateDB.addLogEntry(log)
proc getGasRefund*(c: Computation): GasInt =
if c.isSuccess:
@ -363,7 +352,8 @@ proc getGasRefund*(c: Computation): GasInt =
proc refundSelfDestruct*(c: Computation) =
let cost = gasFees[c.fork][RefundSelfDestruct]
c.gasMeter.refundGas(cost * c.selfDestructs.len)
let num = c.vmState.stateDB.selfDestructLen
c.gasMeter.refundGas(cost * num)
proc tracingEnabled*(c: Computation): bool =
TracerFlags.EnableTracing in c.vmState.tracer.flags

View File

@ -14,7 +14,7 @@ const
lowMemoryCompileTime {.used.} = lowmem > 0
import
std/[macros, sets, strformat],
std/[macros, strformat],
pkg/[chronicles, chronos, stew/byteutils],
".."/[constants, utils/utils, db/accounts_cache],
"."/[code_stream, computation],
@ -26,12 +26,6 @@ import
logScope:
topics = "vm opcode"
const
ripemdAddr = block:
proc initAddress(x: int): EthAddress {.compileTime.} =
result[19] = x.byte
initAddress(3)
# ------------------------------------------------------------------------------
# Private functions
# ------------------------------------------------------------------------------
@ -105,7 +99,6 @@ proc selectVM(c: Computation, fork: EVMFork, shouldPrepareTracer: bool)
genLowMemDispatcher(fork, c.instr, desc)
proc beforeExecCall(c: Computation) =
c.snapshot()
if c.msg.kind == evmcCall:
@ -121,15 +114,13 @@ proc afterExecCall(c: Computation) =
if c.isError or c.fork >= FkByzantium:
if c.msg.contractAddress == ripemdAddr:
# Special case to account for geth+parity bug
c.vmState.touchedAccounts.incl c.msg.contractAddress
c.vmState.stateDB.ripemdSpecial()
if c.isSuccess:
c.commit()
c.touchedAccounts.incl c.msg.contractAddress
else:
c.rollback()
proc beforeExecCreate(c: Computation): bool
{.gcsafe, raises: [ValueError].} =
c.vmState.mutateStateDB:
@ -179,7 +170,6 @@ proc afterExecCreate(c: Computation)
else:
c.rollback()
proc beforeExec(c: Computation): bool
{.gcsafe, raises: [ValueError].} =
if not c.msg.isCreate:

View File

@ -43,9 +43,7 @@ proc init(
self.blockDifficulty = difficulty
self.com = com
self.tracer = tracer
self.logEntries = @[]
self.stateDB = ac
self.touchedAccounts = initHashSet[EthAddress]()
self.minerAddress = miner
self.asyncFactory = AsyncOperationFactory(maybeDataSource: none[AsyncDataSource]())
@ -326,8 +324,7 @@ proc getTracingResult*(vmState: BaseVMState): JsonNode {.inline.} =
vmState.tracer.trace
proc getAndClearLogEntries*(vmState: BaseVMState): seq[Log] =
shallowCopy(result, vmState.logEntries)
vmState.logEntries = @[]
vmState.stateDB.getAndClearLogEntries()
proc enableTracing*(vmState: BaseVMState) =
vmState.tracer.flags.incl EnableTracing
@ -390,14 +387,3 @@ func forkDeterminationInfoForVMState*(vmState: BaseVMState): ForkDeterminationIn
func determineFork*(vmState: BaseVMState): EVMFork =
vmState.com.toEVMFork(vmState.forkDeterminationInfoForVMState)
proc clearSelfDestructsAndEmptyAccounts*(vmState: BaseVMState, fork: EVMFork, miner: EthAddress): void =
vmState.mutateStateDB:
for deletedAccount in vmState.selfDestructs:
db.deleteAccount(deletedAccount)
if fork >= FkSpurious:
vmState.touchedAccounts.incl(miner)
# EIP158/161 state clearing
for account in vmState.touchedAccounts:
db.deleteAccountIfEmpty(account)

View File

@ -52,10 +52,6 @@ proc postExecComputation(c: Computation) =
if c.fork < FkLondon:
# EIP-3529: Reduction in refunds
c.refundSelfDestruct()
shallowCopy(c.vmState.selfDestructs, c.selfDestructs)
shallowCopy(c.vmState.logEntries, c.logEntries)
c.vmState.touchedAccounts.incl c.touchedAccounts
c.vmState.status = c.isSuccess
proc execComputation*(c: Computation)

View File

@ -46,12 +46,9 @@ type
blockDifficulty*: UInt256
flags* : set[VMFlag]
tracer* : TransactionTracer
logEntries* : seq[Log]
receipts* : seq[Receipt]
stateDB* : AccountsCache
cumulativeGasUsed*: GasInt
touchedAccounts*: HashSet[EthAddress]
selfDestructs* : HashSet[EthAddress]
txOrigin* : EthAddress
txGasPrice* : GasInt
gasCosts* : GasCosts
@ -89,9 +86,6 @@ type
output*: seq[byte]
returnData*: seq[byte]
error*: Error
touchedAccounts*: HashSet[EthAddress]
selfDestructs*: HashSet[EthAddress]
logEntries*: seq[Log]
savePoint*: SavePoint
instr*: Op
opIndex*: int

View File

@ -264,7 +264,7 @@ proc finishRunningComputation(host: TransactionHost, call: CallParams): CallResu
shallowCopy(result.output, c.output)
result.contractAddress = if call.isCreate: c.msg.contractAddress
else: default(HostAddress)
shallowCopy(result.logEntries, c.logEntries)
result.logEntries = host.vmState.stateDB.logEntries()
result.stack = c.stack
result.memory = c.memory

View File

@ -220,15 +220,7 @@ proc selfDestruct(host: TransactionHost, address, beneficiary: HostAddress) {.sh
# This must come after sending to the beneficiary in case the
# contract named itself as the beneficiary.
db.setBalance(address, 0.u256)
# TODO: Calling via `computation` is necessary to make some tests pass.
# Here's one that passes only with this:
# tests/fixtures/eth_tests/GeneralStateTests/stRandom2/randomStatetest487.json
# We can't keep using `computation` though.
host.computation.touchedAccounts.incl(beneficiary)
host.computation.selfDestructs.incl(address)
#host.touchedAccounts.incl(beneficiary)
#host.selfDestructs.incl(address)
db.selfDestruct(address)
template call(host: TransactionHost, msg: EvmcMessage): EvmcResult =
# `call` is special. The C stack usage must be kept small for deeply nested
@ -266,13 +258,7 @@ proc emitLog(host: TransactionHost, address: HostAddress,
copyMem(log.data[0].addr, data, data_size.int)
log.address = address
# TODO: Calling via `computation` is necessary to makes some tests pass.
# Here's one that passes only with this:
# tests/fixtures/eth_tests/GeneralStateTests/stRandom2/randomStatetest583.json
# We can't keep using `computation` though.
host.computation.logEntries.add(log)
#host.logEntries.add(log)
host.vmState.stateDB.addlogEntry(log)
proc accessAccount(host: TransactionHost, address: HostAddress): EvmcAccessStatus {.show.} =
host.vmState.mutateStateDB:

View File

@ -59,9 +59,6 @@ type
code*: seq[byte]
cachedTxContext*: bool
txContext*: EvmcTxContext
logEntries*: seq[Log]
touchedAccounts*: HashSet[EthAddress]
selfDestructs*: HashSet[EthAddress]
depth*: int
saveComputation*: seq[Computation]
hostInterface*: ptr evmc_host_interface

View File

@ -9,7 +9,7 @@
# according to those terms.
import
std/[options, times, json, strutils, sets],
std/[options, times, json, strutils],
../common/common,
stew/byteutils,
../vm_state,
@ -98,12 +98,9 @@ proc debug*(vms: BaseVMState): string =
result.add "prevRandao : " & $vms.prevRandao & "\n"
result.add "blockDifficulty : " & $vms.blockDifficulty & "\n"
result.add "flags : " & $vms.flags & "\n"
result.add "logEntries.len : " & $vms.logEntries.len & "\n"
result.add "receipts.len : " & $vms.receipts.len & "\n"
result.add "stateDB.root : " & $vms.stateDB.rootHash & "\n"
result.add "cumulativeGasUsed: " & $vms.cumulativeGasUsed & "\n"
result.add "touchedAccs.len : " & $vms.touchedAccounts.len & "\n"
result.add "selfDestructs.len: " & $vms.selfDestructs.len & "\n"
result.add "txOrigin : " & $vms.txOrigin & "\n"
result.add "txGasPrice : " & $vms.txGasPrice & "\n"
result.add "fork : " & $vms.fork & "\n"

View File

@ -44,7 +44,6 @@ export
vmc.isError,
vmc.isOriginComputation,
vmc.isSuccess,
vmc.isSelfDestructed,
vmc.merge,
vmc.newComputation,
vmc.prepareTracer,

View File

@ -94,7 +94,6 @@ export
bChp.isError,
bChp.isOriginComputation,
bChp.isSuccess,
bChp.isSelfDestructed,
bChp.merge,
bChp.newComputation,
bChp.prepareTracer,

View File

@ -18,7 +18,6 @@ export
vms.`$`,
vms.blockNumber,
vms.buildWitness,
vms.clearSelfDestructsAndEmptyAccounts,
vms.coinbase,
vms.determineFork,
vms.difficulty,

View File

@ -350,7 +350,7 @@ proc verifyAsmResult(vmState: BaseVMState, com: CommonRef, boa: Assembler, asmRe
error "storage has different value", key=key, expected=val, actual=value
return false
let logs = vmState.logEntries
let logs = vmState.getAndClearLogEntries()
if logs.len != boa.logs.len:
error "different logs len", expected=boa.logs.len, actual=logs.len
return false

View File

@ -87,7 +87,7 @@ proc testFixtureIndexes(tester: Tester, testStatusIMPL: var TestStatus) =
let obtainedHash = vmState.readOnlyStateDB.rootHash
check obtainedHash == tester.expectedHash
let logEntries = vmState.getAndClearLogEntries()
let actualLogsHash = hashLogEntries(logEntries)
let actualLogsHash = rlpHash(logEntries)
check(tester.expectedLogs == actualLogsHash)
if tester.debugMode:
let success = tester.expectedLogs == actualLogsHash and obtainedHash == tester.expectedHash

View File

@ -7,7 +7,7 @@
import
std/[os, macros, json, strformat, strutils, tables],
stew/byteutils, net, eth/[keys, rlp, p2p], unittest2,
stew/byteutils, net, eth/[keys, p2p], unittest2,
testutils/markdown_reports,
../nimbus/[constants, config, transaction, errors],
../nimbus/db/accounts_cache,
@ -138,9 +138,6 @@ proc verifyStateDB*(wantedState: JsonNode, stateDB: ReadOnlyStateDB) =
if wantedNonce != actualNonce:
raise newException(ValidationError, &"{ac} nonceDiff {wantedNonce.toHex} != {actualNonce.toHex}")
proc hashLogEntries*(logs: seq[Log]): Hash256 =
keccakHash(rlp.encode(logs))
proc setupEthNode*(
conf: NimbusConf, ctx: EthContext,
capabilities: varargs[ProtocolInfo, `protocolInfo`]): EthereumNode =

View File

@ -31,9 +31,6 @@ proc coinbaseStateClearing*(vmState: BaseVMState,
if touched:
db.addBalance(miner, 0.u256)
if fork >= FkSpurious:
db.deleteAccountIfEmpty(miner)
# db.persist is an important step when using accounts_cache
# it will affect the account storage's location
# during the next call to `getComittedStorage`
@ -41,4 +38,6 @@ proc coinbaseStateClearing*(vmState: BaseVMState,
# do not clear cache, we need the cache when constructing
# post state
db.persist(clearCache = false)
db.persist(
clearEmptyAccount = fork >= FkSpurious,
clearCache = false)

View File

@ -215,7 +215,7 @@ proc runExecution(ctx: var StateContext, conf: StateConf, pre: JsonNode): StateR
vmState.mutateStateDB:
setupStateDB(pre, db)
db.persist() # settle accounts storage
db.persist(clearEmptyAccount = false) # settle accounts storage
defer:
ctx.verifyResult(vmState)

View File

@ -220,7 +220,6 @@ proc exec(ctx: var TransContext,
vmState.mutateStateDB:
db.addBalance(ctx.env.currentCoinbase, mainReward)
db.persist(clearCache = false)
let miner = ctx.env.currentCoinbase
let fork = vmState.com.toEVMFork
@ -411,7 +410,7 @@ proc transitionAction*(ctx: var TransContext, conf: T8NConf) =
vmState.mutateStateDB:
db.setupAlloc(ctx.alloc)
db.persist(clearCache = false)
db.persist(clearEmptyAccount = false, clearCache = false)
let res = exec(ctx, vmState, conf.stateReward, header)