Offload signature checking to taskpools

In block processing, depending on the complexity of a transaction and
hotness of caches etc, signature checking can actually make up the
majority of time needed to process a transaction (60% observed in some
randomly sampled block ranges).

Fortunately, this is a task that trivially can be offloaded to a task
pool similar to how nimbus-eth2 does it.

This PR introduces taskpools in the most simple way possible, by
performing signature checking concurrently with other TX processing,
assigning a taskpool task per TX effectively.

With this little trick, we're in gigagas land 🎉 on my laptop!

```
INF 2024-12-10 21:05:35.170+01:00 Imported blocks
blockNumber=3874817 b... mgps=1222.707 ...
```

Tests don't use the taskpool for now because it needs manual cleanup and
we don't have a good mechanism in place. Future PR:s should address this
by creating a common shutdown sequence that also closes and cleans up
other resources like the DB.
This commit is contained in:
Jacek Sieka 2024-12-11 10:25:26 +01:00
parent 674e65f359
commit f68a5d419f
No known key found for this signature in database
GPG Key ID: A1B09461ABB656B8
33 changed files with 142 additions and 51 deletions

View File

@ -23,6 +23,7 @@ proc processChainData(cd: ChainData): TestStatus =
let let
networkId = NetworkId(cd.params.config.chainId) networkId = NetworkId(cd.params.config.chainId)
com = CommonRef.new(newCoreDbRef DefaultDbMemory, com = CommonRef.new(newCoreDbRef DefaultDbMemory,
Taskpool.new(),
networkId, networkId,
cd.params cd.params
) )

View File

@ -59,6 +59,7 @@ const
proc makeCom*(conf: NimbusConf): CommonRef = proc makeCom*(conf: NimbusConf): CommonRef =
CommonRef.new( CommonRef.new(
newCoreDbRef DefaultDbMemory, newCoreDbRef DefaultDbMemory,
Taskpool.new(),
conf.networkId, conf.networkId,
conf.networkParams conf.networkParams
) )

View File

@ -54,7 +54,7 @@ proc processBlock(
if header.parentBeaconBlockRoot.isSome: if header.parentBeaconBlockRoot.isSome:
? vmState.processBeaconBlockRoot(header.parentBeaconBlockRoot.get) ? vmState.processBeaconBlockRoot(header.parentBeaconBlockRoot.get)
? processTransactions(vmState, header, blk.transactions) ? processTransactions(vmState, header, blk.transactions, taskpool = com.taskpool)
if com.isShanghaiOrLater(header.timestamp): if com.isShanghaiOrLater(header.timestamp):
for withdrawal in blk.withdrawals.get: for withdrawal in blk.withdrawals.get:

View File

@ -79,7 +79,8 @@ proc main() =
ethCtx = newEthContext() ethCtx = newEthContext()
ethNode = setupEthNode(conf, ethCtx, eth) ethNode = setupEthNode(conf, ethCtx, eth)
com = CommonRef.new(newCoreDbRef DefaultDbMemory, com = CommonRef.new(newCoreDbRef DefaultDbMemory,
conf.networkId, Taskpool.new(),
conf.networkId,
conf.networkParams conf.networkParams
) )

View File

@ -38,7 +38,7 @@ proc setupELClient*(conf: ChainConfig, node: JsonNode): TestEnv =
let let
memDB = newCoreDbRef DefaultDbMemory memDB = newCoreDbRef DefaultDbMemory
genesisHeader = node.genesisHeader genesisHeader = node.genesisHeader
com = CommonRef.new(memDB, conf) com = CommonRef.new(memDB, Taskpool.new(), conf)
stateDB = LedgerRef.init(memDB) stateDB = LedgerRef.init(memDB)
chain = newForkedChain(com, genesisHeader) chain = newForkedChain(com, genesisHeader)

View File

@ -46,11 +46,11 @@ proc manageAccounts(ctx: EthContext, conf: NimbusConf) =
proc setupRpcServer(ctx: EthContext, com: CommonRef, proc setupRpcServer(ctx: EthContext, com: CommonRef,
ethNode: EthereumNode, txPool: TxPoolRef, ethNode: EthereumNode, txPool: TxPoolRef,
conf: NimbusConf, chain: ForkedChainRef): RpcServer = conf: NimbusConf, chain: ForkedChainRef): RpcServer =
let let
rpcServer = newRpcHttpServer([initTAddress(conf.httpAddress, conf.httpPort)]) rpcServer = newRpcHttpServer([initTAddress(conf.httpAddress, conf.httpPort)])
serverApi = newServerAPI(chain, txPool) serverApi = newServerAPI(chain, txPool)
setupCommonRpc(ethNode, conf, rpcServer) setupCommonRpc(ethNode, conf, rpcServer)
setupServerAPI(serverApi, rpcServer, ctx) setupServerAPI(serverApi, rpcServer, ctx)
@ -80,6 +80,7 @@ proc setupEnv*(): TestEnv =
ethCtx = newEthContext() ethCtx = newEthContext()
ethNode = setupEthNode(conf, ethCtx, eth) ethNode = setupEthNode(conf, ethCtx, eth)
com = CommonRef.new(newCoreDbRef DefaultDbMemory, com = CommonRef.new(newCoreDbRef DefaultDbMemory,
Taskpool.new(),
conf.networkId, conf.networkId,
conf.networkParams conf.networkParams
) )

View File

@ -15,7 +15,8 @@ import
../db/[core_db, ledger, storage_types], ../db/[core_db, ledger, storage_types],
../utils/[utils, ec_recover], ../utils/[utils, ec_recover],
".."/[constants, errors, version], ".."/[constants, errors, version],
"."/[chain_config, evmforks, genesis, hardforks] "."/[chain_config, evmforks, genesis, hardforks],
taskpools
export export
chain_config, chain_config,
@ -25,7 +26,8 @@ export
evmforks, evmforks,
hardforks, hardforks,
genesis, genesis,
utils utils,
taskpools
type type
SyncProgress = object SyncProgress = object
@ -97,6 +99,9 @@ type
extraData: string extraData: string
## Value of extraData field when building block ## Value of extraData field when building block
taskpool*: Taskpool
## Shared task pool for offloading computation to other threads
# ------------------------------------------------------------------------------ # ------------------------------------------------------------------------------
# Forward declarations # Forward declarations
# ------------------------------------------------------------------------------ # ------------------------------------------------------------------------------
@ -165,6 +170,7 @@ proc initializeDb(com: CommonRef) =
proc init(com : CommonRef, proc init(com : CommonRef,
db : CoreDbRef, db : CoreDbRef,
taskpool : Taskpool,
networkId : NetworkId, networkId : NetworkId,
config : ChainConfig, config : ChainConfig,
genesis : Genesis, genesis : Genesis,
@ -181,6 +187,7 @@ proc init(com : CommonRef,
com.pruneHistory= pruneHistory com.pruneHistory= pruneHistory
com.pos = CasperRef.new com.pos = CasperRef.new
com.extraData = ShortClientId com.extraData = ShortClientId
com.taskpool = taskpool
# com.forkIdCalculator and com.genesisHash are set # com.forkIdCalculator and com.genesisHash are set
# by setForkId # by setForkId
@ -223,6 +230,7 @@ proc isBlockAfterTtd(com: CommonRef, header: Header): bool =
proc new*( proc new*(
_: type CommonRef; _: type CommonRef;
db: CoreDbRef; db: CoreDbRef;
taskpool: Taskpool;
networkId: NetworkId = MainNet; networkId: NetworkId = MainNet;
params = networkParams(MainNet); params = networkParams(MainNet);
pruneHistory = false; pruneHistory = false;
@ -233,6 +241,7 @@ proc new*(
new(result) new(result)
result.init( result.init(
db, db,
taskpool,
networkId, networkId,
params.config, params.config,
params.genesis, params.genesis,
@ -241,6 +250,7 @@ proc new*(
proc new*( proc new*(
_: type CommonRef; _: type CommonRef;
db: CoreDbRef; db: CoreDbRef;
taskpool: Taskpool;
config: ChainConfig; config: ChainConfig;
networkId: NetworkId = MainNet; networkId: NetworkId = MainNet;
pruneHistory = false; pruneHistory = false;
@ -251,6 +261,7 @@ proc new*(
new(result) new(result)
result.init( result.init(
db, db,
taskpool,
networkId, networkId,
config, config,
nil, nil,

View File

@ -356,6 +356,12 @@ type
defaultValueDesc: $ClientId defaultValueDesc: $ClientId
name: "agent-string" .}: string name: "agent-string" .}: string
numThreads* {.
separator: "\pPERFORMANCE OPTIONS",
defaultValue: 0,
desc: "Number of worker threads (\"0\" = use as many threads as there are CPU cores available)"
name: "num-threads" .}: int
beaconChunkSize* {. beaconChunkSize* {.
hidden hidden
desc: "Number of blocks per database transaction for beacon sync" desc: "Number of blocks per database transaction for beacon sync"

View File

@ -149,6 +149,7 @@ proc persistBlocksImpl(
skipValidation, skipValidation,
skipReceipts = skipValidation and NoPersistReceipts in flags, skipReceipts = skipValidation and NoPersistReceipts in flags,
skipUncles = NoPersistUncles in flags, skipUncles = NoPersistUncles in flags,
taskpool = c.com.taskpool,
) )
let blockHash = header.blockHash() let blockHash = header.blockHash()

View File

@ -8,6 +8,8 @@
# at your option. This file may not be copied, modified, or distributed except # at your option. This file may not be copied, modified, or distributed except
# according to those terms. # according to those terms.
{.push raises: [], gcsafe.}
import import
../../common/common, ../../common/common,
../../utils/utils, ../../utils/utils,
@ -21,11 +23,50 @@ import
./calculate_reward, ./calculate_reward,
./executor_helpers, ./executor_helpers,
./process_transaction, ./process_transaction,
eth/common/transaction_utils, eth/common/[keys, transaction_utils],
chronicles, chronicles,
results results,
taskpools
{.push raises: [].} template withSender(txs: openArray[Transaction], body: untyped) =
# Execute transactions offloading the signature checking to the task pool if
# it's available
if taskpool == nil:
for txIndex {.inject.}, tx {.inject.} in txs:
let sender {.inject.} = tx.recoverSender().valueOr(default(Address))
body
else:
type Entry = (Signature, Hash32, Flowvar[Address])
proc recoverTask(e: ptr Entry): Address {.nimcall.} =
let pk = recover(e[][0], SkMessage(e[][1].data))
if pk.isOk():
pk[].to(Address)
else:
default(Address)
var entries = newSeq[Entry](txs.len)
# Prepare signature recovery tasks for each transaction - for simplicity,
# we use `default(Address)` to signal sig check failure
for i, e in entries.mpairs():
e[0] = txs[i].signature().valueOr(default(Signature))
e[1] = txs[i].rlpHashForSigning(txs[i].isEip155)
let a = addr e
# Spawning the task here allows it to start early, while we still haven't
# hashed subsequent txs
e[2] = taskpool.spawn recoverTask(a)
for txIndex {.inject.}, e in entries.mpairs():
template tx(): untyped =
txs[txIndex]
# Sync blocks until the sender is available from the task pool - as soon
# as we have it, we can process this transaction while the senders of the
# other transactions are being computed
let sender {.inject.} = sync(e[2])
body
# Factored this out of procBlkPreamble so that it can be used directly for # Factored this out of procBlkPreamble so that it can be used directly for
# stateless execution of specific transactions. # stateless execution of specific transactions.
@ -34,15 +75,17 @@ proc processTransactions*(
header: Header, header: Header,
transactions: seq[Transaction], transactions: seq[Transaction],
skipReceipts = false, skipReceipts = false,
collectLogs = false collectLogs = false,
taskpool: Taskpool = nil,
): Result[void, string] = ): Result[void, string] =
vmState.receipts.setLen(if skipReceipts: 0 else: transactions.len) vmState.receipts.setLen(if skipReceipts: 0 else: transactions.len)
vmState.cumulativeGasUsed = 0 vmState.cumulativeGasUsed = 0
vmState.allLogs = @[] vmState.allLogs = @[]
for txIndex, tx in transactions: withSender(transactions):
let sender = tx.recoverSender().valueOr: if sender == default(Address):
return err("Could not get sender for tx with index " & $(txIndex)) return err("Could not get sender for tx with index " & $(txIndex))
let rc = vmState.processTransaction(tx, sender, header) let rc = vmState.processTransaction(tx, sender, header)
if rc.isErr: if rc.isErr:
return err("Error processing tx with index " & $(txIndex) & ":" & rc.error) return err("Error processing tx with index " & $(txIndex) & ":" & rc.error)
@ -60,7 +103,10 @@ proc processTransactions*(
ok() ok()
proc procBlkPreamble( proc procBlkPreamble(
vmState: BaseVMState, blk: Block, skipValidation, skipReceipts, skipUncles: bool vmState: BaseVMState,
blk: Block,
skipValidation, skipReceipts, skipUncles: bool,
taskpool: Taskpool,
): Result[void, string] = ): Result[void, string] =
template header(): Header = template header(): Header =
blk.header blk.header
@ -97,7 +143,9 @@ proc procBlkPreamble(
return err("Transactions missing from body") return err("Transactions missing from body")
let collectLogs = header.requestsHash.isSome and not skipValidation let collectLogs = header.requestsHash.isSome and not skipValidation
?processTransactions(vmState, header, blk.transactions, skipReceipts, collectLogs) ?processTransactions(
vmState, header, blk.transactions, skipReceipts, collectLogs, taskpool
)
elif blk.transactions.len > 0: elif blk.transactions.len > 0:
return err("Transactions in block with empty txRoot") return err("Transactions in block with empty txRoot")
@ -150,7 +198,8 @@ proc procBlkEpilogue(
# large ranges of blocks, implicitly limiting its size using the gas limit # large ranges of blocks, implicitly limiting its size using the gas limit
db.persist( db.persist(
clearEmptyAccount = vmState.com.isSpuriousOrLater(header.number), clearEmptyAccount = vmState.com.isSpuriousOrLater(header.number),
clearCache = true) clearCache = true,
)
var var
withdrawalReqs: seq[byte] withdrawalReqs: seq[byte]
@ -173,17 +222,15 @@ proc procBlkEpilogue(
expected = header.stateRoot, expected = header.stateRoot,
actual = stateRoot, actual = stateRoot,
arrivedFrom = vmState.parent.stateRoot arrivedFrom = vmState.parent.stateRoot
return err("stateRoot mismatch, expect: " & return
$header.stateRoot & ", got: " & $stateRoot) err("stateRoot mismatch, expect: " & $header.stateRoot & ", got: " & $stateRoot)
if not skipReceipts: if not skipReceipts:
let bloom = createBloom(vmState.receipts) let bloom = createBloom(vmState.receipts)
if header.logsBloom != bloom: if header.logsBloom != bloom:
debug "wrong logsBloom in block", debug "wrong logsBloom in block",
blockNumber = header.number, blockNumber = header.number, actual = bloom, expected = header.logsBloom
actual = bloom,
expected = header.logsBloom
return err("bloom mismatch") return err("bloom mismatch")
let receiptsRoot = calcReceiptsRoot(vmState.receipts) let receiptsRoot = calcReceiptsRoot(vmState.receipts)
@ -199,7 +246,8 @@ proc procBlkEpilogue(
if header.requestsHash.isSome: if header.requestsHash.isSome:
let let
depositReqs = ?parseDepositLogs(vmState.allLogs, vmState.com.depositContractAddress) depositReqs =
?parseDepositLogs(vmState.allLogs, vmState.com.depositContractAddress)
requestsHash = calcRequestsHash(depositReqs, withdrawalReqs, consolidationReqs) requestsHash = calcRequestsHash(depositReqs, withdrawalReqs, consolidationReqs)
if header.requestsHash.get != requestsHash: if header.requestsHash.get != requestsHash:
@ -223,9 +271,10 @@ proc processBlock*(
skipValidation: bool = false, skipValidation: bool = false,
skipReceipts: bool = false, skipReceipts: bool = false,
skipUncles: bool = false, skipUncles: bool = false,
taskpool: Taskpool = nil,
): Result[void, string] = ): Result[void, string] =
## Generalised function to processes `blk` for any network. ## Generalised function to processes `blk` for any network.
?vmState.procBlkPreamble(blk, skipValidation, skipReceipts, skipUncles) ?vmState.procBlkPreamble(blk, skipValidation, skipReceipts, skipUncles, taskpool)
# EIP-3675: no reward for miner in POA/POS # EIP-3675: no reward for miner in POA/POS
if not vmState.com.proofOfStake(blk.header): if not vmState.com.proofOfStake(blk.header):

View File

@ -11,7 +11,7 @@ import
../nimbus/compile_info ../nimbus/compile_info
import import
std/[os, strutils, net], std/[os, osproc, strutils, net],
chronicles, chronicles,
eth/net/nat, eth/net/nat,
metrics, metrics,
@ -217,8 +217,24 @@ proc run(nimbus: NimbusNode, conf: NimbusConf) =
preventLoadingDataDirForTheWrongNetwork(coreDB, conf) preventLoadingDataDirForTheWrongNetwork(coreDB, conf)
setupMetrics(nimbus, conf) setupMetrics(nimbus, conf)
let taskpool =
try:
if conf.numThreads < 0:
fatal "The number of threads --num-threads cannot be negative."
quit 1
elif conf.numThreads == 0:
Taskpool.new(numThreads = min(countProcessors(), 16))
else:
Taskpool.new(numThreads = conf.numThreads)
except CatchableError as e:
fatal "Cannot start taskpool", err = e.msg
quit 1
info "Threadpool started", numThreads = taskpool.numThreads
let com = CommonRef.new( let com = CommonRef.new(
db = coreDB, db = coreDB,
taskpool = taskpool,
pruneHistory = (conf.chainDbMode == AriPrune), pruneHistory = (conf.chainDbMode == AriPrune),
networkId = conf.networkId, networkId = conf.networkId,
params = conf.networkParams) params = conf.networkParams)

View File

@ -265,7 +265,7 @@ proc initVMEnv*(network: string): BaseVMState =
cdb = DefaultDbMemory.newCoreDbRef() cdb = DefaultDbMemory.newCoreDbRef()
com = CommonRef.new( com = CommonRef.new(
cdb, cdb,
conf, nil, conf,
conf.chainId.NetworkId) conf.chainId.NetworkId)
parent = Header(stateRoot: EMPTY_ROOT_HASH) parent = Header(stateRoot: EMPTY_ROOT_HASH)
parentHash = rlpHash(parent) parentHash = rlpHash(parent)

View File

@ -57,7 +57,7 @@ proc main() {.used.} =
var conf = makeConfig() var conf = makeConfig()
let db = newCoreDbRef( let db = newCoreDbRef(
DefaultDbPersistent, string conf.dataDir, DbOptions.init()) DefaultDbPersistent, string conf.dataDir, DbOptions.init())
let com = CommonRef.new(db) let com = CommonRef.new(db, nil)
com.dumpTest(97) com.dumpTest(97)
com.dumpTest(98) # no uncles and no tx com.dumpTest(98) # no uncles and no tx

View File

@ -69,7 +69,7 @@ proc executeCase(node: JsonNode): bool =
memDB = newCoreDbRef DefaultDbMemory memDB = newCoreDbRef DefaultDbMemory
stateDB = LedgerRef.init(memDB) stateDB = LedgerRef.init(memDB)
config = getChainConfig(env.network) config = getChainConfig(env.network)
com = CommonRef.new(memDB, config) com = CommonRef.new(memDB, nil, config)
setupStateDB(env.pre, stateDB) setupStateDB(env.pre, stateDB)
stateDB.persist() stateDB.persist()

View File

@ -181,6 +181,7 @@ proc initRunnerDB(
result = CommonRef.new( result = CommonRef.new(
db = coreDB, db = coreDB,
taskpool = nil,
networkId = networkId, networkId = networkId,
params = params, params = params,
pruneHistory = pruneHistory) pruneHistory = pruneHistory)

View File

@ -61,6 +61,7 @@ proc setupConfig(genesisFile: string): NimbusConf =
proc setupCom(conf: NimbusConf): CommonRef = proc setupCom(conf: NimbusConf): CommonRef =
CommonRef.new( CommonRef.new(
newCoreDbRef DefaultDbMemory, newCoreDbRef DefaultDbMemory,
nil,
conf.networkId, conf.networkId,
conf.networkParams conf.networkParams
) )

View File

@ -347,6 +347,7 @@ proc runTestOverflow() =
let com = CommonRef.new( let com = CommonRef.new(
newCoreDbRef(DefaultDbMemory), newCoreDbRef(DefaultDbMemory),
nil,
config = chainConfigForNetwork(MainNet) config = chainConfigForNetwork(MainNet)
) )

View File

@ -37,6 +37,7 @@ proc setupEnv(): TestEnv =
proc newCom(env: TestEnv): CommonRef = proc newCom(env: TestEnv): CommonRef =
CommonRef.new( CommonRef.new(
newCoreDbRef DefaultDbMemory, newCoreDbRef DefaultDbMemory,
nil,
env.conf.networkId, env.conf.networkId,
env.conf.networkParams env.conf.networkParams
) )

View File

@ -72,7 +72,7 @@ template runTest(network: untyped, name: string) =
test name: test name:
var var
params = networkParams(network) params = networkParams(network)
com = CommonRef.new(newCoreDbRef DefaultDbMemory, network, params) com = CommonRef.new(newCoreDbRef DefaultDbMemory, nil, network, params)
for i, x in `network IDs`: for i, x in `network IDs`:
let id = com.forkId(x.number, x.time) let id = com.forkId(x.number, x.time)

View File

@ -77,7 +77,7 @@ proc dumpDebugData(ctx: TestCtx, vmState: BaseVMState, gasUsed: GasInt, success:
proc testFixtureIndexes(ctx: var TestCtx, testStatusIMPL: var TestStatus) = proc testFixtureIndexes(ctx: var TestCtx, testStatusIMPL: var TestStatus) =
let let
com = CommonRef.new(newCoreDbRef DefaultDbMemory, ctx.chainConfig) com = CommonRef.new(newCoreDbRef DefaultDbMemory, nil, ctx.chainConfig)
parent = Header(stateRoot: emptyRoot) parent = Header(stateRoot: emptyRoot)
tracer = if ctx.trace: tracer = if ctx.trace:
newLegacyTracer({}) newLegacyTracer({})

View File

@ -27,11 +27,11 @@ proc findFilePath(file: string): string =
return path return path
proc makeGenesis(networkId: NetworkId): Header = proc makeGenesis(networkId: NetworkId): Header =
let com = CommonRef.new(newCoreDbRef DefaultDbMemory, params = networkParams(networkId)) let com = CommonRef.new(newCoreDbRef DefaultDbMemory, taskpool = nil, params = networkParams(networkId))
com.genesisHeader com.genesisHeader
proc proofOfStake(params: NetworkParams): bool = proc proofOfStake(params: NetworkParams): bool =
let com = CommonRef.new(newCoreDbRef DefaultDbMemory, let com = CommonRef.new(newCoreDbRef DefaultDbMemory, taskpool = nil,
networkId = params.config.chainId.NetworkId, networkId = params.config.chainId.NetworkId,
params = params) params = params)
let header = com.genesisHeader let header = com.genesisHeader
@ -66,7 +66,7 @@ proc customGenesisTest() =
test "Devnet4.json (aka Kintsugi in all but chainId)": test "Devnet4.json (aka Kintsugi in all but chainId)":
var cg: NetworkParams var cg: NetworkParams
check loadNetworkParams("devnet4.json".findFilePath, cg) check loadNetworkParams("devnet4.json".findFilePath, cg)
let com = CommonRef.new(newCoreDbRef DefaultDbMemory, params = cg) let com = CommonRef.new(newCoreDbRef DefaultDbMemory, taskpool = nil, params = cg)
let stateRoot = hash32"3b84f313bfd49c03cc94729ade2e0de220688f813c0c895a99bd46ecc9f45e1e" let stateRoot = hash32"3b84f313bfd49c03cc94729ade2e0de220688f813c0c895a99bd46ecc9f45e1e"
let genesisHash = hash32"a28d8d73e087a01d09d8cb806f60863652f30b6b6dfa4e0157501ff07d422399" let genesisHash = hash32"a28d8d73e087a01d09d8cb806f60863652f30b6b6dfa4e0157501ff07d422399"
check com.genesisHeader.stateRoot == stateRoot check com.genesisHeader.stateRoot == stateRoot
@ -76,7 +76,7 @@ proc customGenesisTest() =
test "Devnet5.json (aka Kiln in all but chainId and TTD)": test "Devnet5.json (aka Kiln in all but chainId and TTD)":
var cg: NetworkParams var cg: NetworkParams
check loadNetworkParams("devnet5.json".findFilePath, cg) check loadNetworkParams("devnet5.json".findFilePath, cg)
let com = CommonRef.new(newCoreDbRef DefaultDbMemory, params = cg) let com = CommonRef.new(newCoreDbRef DefaultDbMemory, taskpool = nil, params = cg)
let stateRoot = hash32"52e628c7f35996ba5a0402d02b34535993c89ff7fc4c430b2763ada8554bee62" let stateRoot = hash32"52e628c7f35996ba5a0402d02b34535993c89ff7fc4c430b2763ada8554bee62"
let genesisHash = hash32"51c7fe41be669f69c45c33a56982cbde405313342d9e2b00d7c91a7b284dd4f8" let genesisHash = hash32"51c7fe41be669f69c45c33a56982cbde405313342d9e2b00d7c91a7b284dd4f8"
check com.genesisHeader.stateRoot == stateRoot check com.genesisHeader.stateRoot == stateRoot
@ -86,7 +86,7 @@ proc customGenesisTest() =
test "Mainnet shadow fork 1": test "Mainnet shadow fork 1":
var cg: NetworkParams var cg: NetworkParams
check loadNetworkParams("mainshadow1.json".findFilePath, cg) check loadNetworkParams("mainshadow1.json".findFilePath, cg)
let com = CommonRef.new(newCoreDbRef DefaultDbMemory, params = cg) let com = CommonRef.new(newCoreDbRef DefaultDbMemory, taskpool = nil, params = cg)
let stateRoot = hash32"d7f8974fb5ac78d9ac099b9ad5018bedc2ce0a72dad1827a1709da30580f0544" let stateRoot = hash32"d7f8974fb5ac78d9ac099b9ad5018bedc2ce0a72dad1827a1709da30580f0544"
let genesisHash = hash32"d4e56740f876aef8c010b86a40d5f56745a118d0906a34e69aec8c0db1cb8fa3" let genesisHash = hash32"d4e56740f876aef8c010b86a40d5f56745a118d0906a34e69aec8c0db1cb8fa3"
let ttd = "46_089_003_871_917_200_000_000".parse(UInt256) let ttd = "46_089_003_871_917_200_000_000".parse(UInt256)
@ -99,7 +99,7 @@ proc customGenesisTest() =
# parse using geth format should produce the same result with nimbus format # parse using geth format should produce the same result with nimbus format
var cg: NetworkParams var cg: NetworkParams
check loadNetworkParams("geth_mainshadow1.json".findFilePath, cg) check loadNetworkParams("geth_mainshadow1.json".findFilePath, cg)
let com = CommonRef.new(newCoreDbRef DefaultDbMemory, params = cg) let com = CommonRef.new(newCoreDbRef DefaultDbMemory, taskpool = nil, params = cg)
let stateRoot = hash32"d7f8974fb5ac78d9ac099b9ad5018bedc2ce0a72dad1827a1709da30580f0544" let stateRoot = hash32"d7f8974fb5ac78d9ac099b9ad5018bedc2ce0a72dad1827a1709da30580f0544"
let genesisHash = hash32"d4e56740f876aef8c010b86a40d5f56745a118d0906a34e69aec8c0db1cb8fa3" let genesisHash = hash32"d4e56740f876aef8c010b86a40d5f56745a118d0906a34e69aec8c0db1cb8fa3"
let ttd = "46_089_003_871_917_200_000_000".parse(UInt256) let ttd = "46_089_003_871_917_200_000_000".parse(UInt256)
@ -114,7 +114,7 @@ proc customGenesisTest() =
test "Holesky": test "Holesky":
var cg: NetworkParams var cg: NetworkParams
check loadNetworkParams("holesky.json".findFilePath, cg) check loadNetworkParams("holesky.json".findFilePath, cg)
let com = CommonRef.new(newCoreDbRef DefaultDbMemory, params = cg) let com = CommonRef.new(newCoreDbRef DefaultDbMemory, taskpool = nil, params = cg)
let stateRoot = hash32"69D8C9D72F6FA4AD42D4702B433707212F90DB395EB54DC20BC85DE253788783" let stateRoot = hash32"69D8C9D72F6FA4AD42D4702B433707212F90DB395EB54DC20BC85DE253788783"
let genesisHash = hash32"b5f7f912443c940f21fd611f12828d75b534364ed9e95ca4e307729a4661bde4" let genesisHash = hash32"b5f7f912443c940f21fd611f12828d75b534364ed9e95ca4e307729a4661bde4"
check com.genesisHeader.stateRoot == stateRoot check com.genesisHeader.stateRoot == stateRoot
@ -125,7 +125,7 @@ proc customGenesisTest() =
# parse using geth format should produce the same result with nimbus format # parse using geth format should produce the same result with nimbus format
var cg: NetworkParams var cg: NetworkParams
check loadNetworkParams("geth_holesky.json".findFilePath, cg) check loadNetworkParams("geth_holesky.json".findFilePath, cg)
let com = CommonRef.new(newCoreDbRef DefaultDbMemory, params = cg) let com = CommonRef.new(newCoreDbRef DefaultDbMemory, taskpool = nil, params = cg)
let stateRoot = hash32"69D8C9D72F6FA4AD42D4702B433707212F90DB395EB54DC20BC85DE253788783" let stateRoot = hash32"69D8C9D72F6FA4AD42D4702B433707212F90DB395EB54DC20BC85DE253788783"
let genesisHash = hash32"b5f7f912443c940f21fd611f12828d75b534364ed9e95ca4e307729a4661bde4" let genesisHash = hash32"b5f7f912443c940f21fd611f12828d75b534364ed9e95ca4e307729a4661bde4"
check com.genesisHeader.stateRoot == stateRoot check com.genesisHeader.stateRoot == stateRoot

View File

@ -69,7 +69,7 @@ proc setupChain(): CommonRef =
) )
let com = CommonRef.new( let com = CommonRef.new(
newCoreDbRef DefaultDbMemory, newCoreDbRef DefaultDbMemory, nil,
CustomNet, CustomNet,
customNetwork customNetwork
) )

View File

@ -97,7 +97,7 @@ proc initEnv(): TestEnv =
let let
com = CommonRef.new( com = CommonRef.new(
newCoreDbRef DefaultDbMemory, newCoreDbRef DefaultDbMemory, nil,
conf.networkId, conf.networkId,
conf.networkParams conf.networkParams
) )

View File

@ -19,7 +19,7 @@ proc testFixture(node: JsonNode, testStatusIMPL: var TestStatus) =
blockNumber = UInt256.fromHex(node["blockNumber"].getStr()) blockNumber = UInt256.fromHex(node["blockNumber"].getStr())
memoryDB = newCoreDbRef DefaultDbMemory memoryDB = newCoreDbRef DefaultDbMemory
config = chainConfigForNetwork(MainNet) config = chainConfigForNetwork(MainNet)
com = CommonRef.new(memoryDB, config) com = CommonRef.new(memoryDB, nil, config)
state = node["state"] state = node["state"]
for k, v in state: for k, v in state:

View File

@ -71,7 +71,7 @@ proc testFixture(fixtures: JsonNode, testStatusIMPL: var TestStatus) =
conf = getChainConfig(parseFork(fixtures["fork"].getStr)) conf = getChainConfig(parseFork(fixtures["fork"].getStr))
data = fixtures["data"] data = fixtures["data"]
privateKey = PrivateKey.fromHex("7a28b5ba57c53603b0b07b56bba752f7784bf506fa95edc395f5cf6c7514fe9d")[] privateKey = PrivateKey.fromHex("7a28b5ba57c53603b0b07b56bba752f7784bf506fa95edc395f5cf6c7514fe9d")[]
com = CommonRef.new(newCoreDbRef DefaultDbMemory, config = conf) com = CommonRef.new(newCoreDbRef DefaultDbMemory, nil, config = conf)
vmState = BaseVMState.new( vmState = BaseVMState.new(
Header(number: 1'u64, stateRoot: emptyRlpHash), Header(number: 1'u64, stateRoot: emptyRlpHash),
Header(), Header(),

View File

@ -222,7 +222,7 @@ proc rpcMain*() =
ctx = newEthContext() ctx = newEthContext()
ethNode = setupEthNode(conf, ctx, eth) ethNode = setupEthNode(conf, ctx, eth)
com = CommonRef.new( com = CommonRef.new(
newCoreDbRef DefaultDbMemory, newCoreDbRef DefaultDbMemory, nil,
conf.networkId, conf.networkId,
conf.networkParams conf.networkParams
) )

View File

@ -130,7 +130,7 @@ proc testFixtureImpl(node: JsonNode, testStatusIMPL: var TestStatus, memoryDB: C
var var
blockNumberHex = node["blockNumber"].getStr() blockNumberHex = node["blockNumber"].getStr()
blockNumber = parseHexInt(blockNumberHex).uint64 blockNumber = parseHexInt(blockNumberHex).uint64
com = CommonRef.new(memoryDB, chainConfigForNetwork(MainNet)) com = CommonRef.new(memoryDB, nil, chainConfigForNetwork(MainNet))
state = node["state"] state = node["state"]
receipts = node["receipts"] receipts = node["receipts"]

View File

@ -92,7 +92,7 @@ proc setupTxPool*(getStatus: proc(): TxItemStatus): (CommonRef, TxPoolRef, int)
txEnv.fillGenesis(conf.networkParams) txEnv.fillGenesis(conf.networkParams)
let com = CommonRef.new( let com = CommonRef.new(
newCoreDbRef DefaultDbMemory, newCoreDbRef DefaultDbMemory, nil,
conf.networkId, conf.networkId,
conf.networkParams conf.networkParams
) )

View File

@ -124,7 +124,7 @@ proc initEnv(envFork: HardFork): TestEnv =
let let
com = com =
CommonRef.new(newCoreDbRef DefaultDbMemory, conf.networkId, conf.networkParams) CommonRef.new(newCoreDbRef DefaultDbMemory, nil, conf.networkId, conf.networkParams)
chain = newForkedChain(com, com.genesisHeader) chain = newForkedChain(com, com.genesisHeader)
result = TestEnv( result = TestEnv(

View File

@ -56,7 +56,7 @@ proc main() {.used.} =
var conf = makeConfig() var conf = makeConfig()
let db = newCoreDbRef( let db = newCoreDbRef(
DefaultDbPersistent, string conf.dataDir, DbOptions.init()) DefaultDbPersistent, string conf.dataDir, DbOptions.init())
let com = CommonRef.new(db) let com = CommonRef.new(db, nil)
com.dumpTest(97) com.dumpTest(97)
com.dumpTest(46147) com.dumpTest(46147)

View File

@ -105,7 +105,7 @@ proc writeRootHashToStderr(stateRoot: Hash32) =
proc runExecution(ctx: var StateContext, conf: StateConf, pre: JsonNode): StateResult = proc runExecution(ctx: var StateContext, conf: StateConf, pre: JsonNode): StateResult =
let let
com = CommonRef.new(newCoreDbRef DefaultDbMemory, ctx.chainConfig) com = CommonRef.new(newCoreDbRef DefaultDbMemory, nil, ctx.chainConfig)
stream = newFileStream(stderr) stream = newFileStream(stderr)
tracer = if conf.jsonEnabled: tracer = if conf.jsonEnabled:
newJsonTracer(stream, ctx.tracerFlags, conf.pretty) newJsonTracer(stream, ctx.tracerFlags, conf.pretty)

View File

@ -349,11 +349,11 @@ proc exec(ctx: TransContext,
excessBlobGas = ctx.env.currentExcessBlobGas excessBlobGas = ctx.env.currentExcessBlobGas
elif ctx.env.parentExcessBlobGas.isSome and ctx.env.parentBlobGasUsed.isSome: elif ctx.env.parentExcessBlobGas.isSome and ctx.env.parentBlobGasUsed.isSome:
excessBlobGas = Opt.some calcExcessBlobGas(vmState.parent) excessBlobGas = Opt.some calcExcessBlobGas(vmState.parent)
if excessBlobGas.isSome: if excessBlobGas.isSome:
result.result.blobGasUsed = Opt.some vmState.blobGasUsed result.result.blobGasUsed = Opt.some vmState.blobGasUsed
result.result.currentExcessBlobGas = excessBlobGas result.result.currentExcessBlobGas = excessBlobGas
if vmState.com.isPragueOrLater(ctx.env.currentTimestamp): if vmState.com.isPragueOrLater(ctx.env.currentTimestamp):
var allLogs: seq[Log] var allLogs: seq[Log]
for rec in result.result.receipts: for rec in result.result.receipts:
@ -469,7 +469,7 @@ proc transitionAction*(ctx: var TransContext, conf: T8NConf) =
config.depositContractAddress = ctx.env.depositContractAddress config.depositContractAddress = ctx.env.depositContractAddress
config.chainId = conf.stateChainId.ChainId config.chainId = conf.stateChainId.ChainId
let com = CommonRef.new(newCoreDbRef DefaultDbMemory, config) let com = CommonRef.new(newCoreDbRef DefaultDbMemory, Taskpool.new(), config)
# Sanity check, to not `panic` in state_transition # Sanity check, to not `panic` in state_transition
if com.isLondonOrLater(ctx.env.currentNumber): if com.isLondonOrLater(ctx.env.currentNumber):

@ -1 +1 @@
Subproject commit d4c43137c0590cb47f893a66ca8cb027fa6c217e Subproject commit f0b386933ffea89098bc232359f8524393c2c1ef