From 1eca772a08dfa47b7a8c5d4c87ae96a225216c05 Mon Sep 17 00:00:00 2001 From: jangko Date: Sun, 9 Jul 2023 09:16:22 +0700 Subject: [PATCH] hive: pyspec simulator implementation --- hive_integration/nodocker/build_sims.sh | 4 +- .../nodocker/engine/engine_callsigs.nim | 3 + .../nodocker/engine/engine_client.nim | 26 ++- hive_integration/nodocker/pyspec/helpers.nim | 58 ++++++ .../nodocker/pyspec/pyspec_sim.nim | 167 ++++++++++++++++++ hive_integration/nodocker/pyspec/test_env.nim | 84 +++++++++ nimbus/rpc/engine_api.nim | 17 +- nimbus/rpc/merge/mergeutils.nim | 59 ++++--- tools/evmstate/helpers.nim | 17 +- 9 files changed, 395 insertions(+), 40 deletions(-) create mode 100644 hive_integration/nodocker/engine/engine_callsigs.nim create mode 100644 hive_integration/nodocker/pyspec/helpers.nim create mode 100644 hive_integration/nodocker/pyspec/pyspec_sim.nim create mode 100644 hive_integration/nodocker/pyspec/test_env.nim diff --git a/hive_integration/nodocker/build_sims.sh b/hive_integration/nodocker/build_sims.sh index 0827c91a8..c55946afb 100644 --- a/hive_integration/nodocker/build_sims.sh +++ b/hive_integration/nodocker/build_sims.sh @@ -13,11 +13,13 @@ ${ENV_SCRIPT} nim ${NIM_FLAGS} ${SIM_DIR}/engine/engine_sim ${ENV_SCRIPT} nim ${NIM_FLAGS} ${SIM_DIR}/consensus/consensus_sim ${ENV_SCRIPT} nim ${NIM_FLAGS} ${SIM_DIR}/graphql/graphql_sim ${ENV_SCRIPT} nim ${NIM_FLAGS} ${SIM_DIR}/rpc/rpc_sim +${ENV_SCRIPT} nim ${NIM_FLAGS} ${SIM_DIR}/pyspec/pyspec_sim ${SIM_DIR}/engine/engine_sim ${SIM_DIR}/consensus/consensus_sim ${SIM_DIR}/graphql/graphql_sim ${SIM_DIR}/rpc/rpc_sim +${SIM_DIR}/pyspec/pyspec_sim echo "## ${1}" > simulators.md -cat engine.md consensus.md graphql.md rpc.md >> simulators.md +cat engine.md consensus.md graphql.md rpc.md pyspec.md >> simulators.md diff --git a/hive_integration/nodocker/engine/engine_callsigs.nim b/hive_integration/nodocker/engine/engine_callsigs.nim new file mode 100644 index 000000000..01378eadc --- /dev/null +++ b/hive_integration/nodocker/engine/engine_callsigs.nim @@ -0,0 +1,3 @@ +import ethtypes, engine_api_types + +proc engine_newPayloadV2(payload: ExecutionPayloadV1OrV2): PayloadStatusV1 diff --git a/hive_integration/nodocker/engine/engine_client.nim b/hive_integration/nodocker/engine/engine_client.nim index f6ce41c21..f4609c437 100644 --- a/hive_integration/nodocker/engine/engine_client.nim +++ b/hive_integration/nodocker/engine/engine_client.nim @@ -3,7 +3,7 @@ import stew/byteutils, eth/[common, common/eth_types, rlp], chronos, web3/engine_api_types, - json_rpc/[rpcclient, errors], + json_rpc/[rpcclient, errors, jsonmarshal], ../../../tests/rpcclient/eth_api, ../../../premix/parser, ../../../nimbus/rpc/hexstrings, @@ -13,6 +13,12 @@ import web3/engine_api as web3_engine_api type Hash256 = eth_types.Hash256 +from os import DirSep, AltSep +const + sourceDir = currentSourcePath.rsplit({DirSep, AltSep}, 1)[0] + +createRpcSigs(RpcClient, sourceDir & "/engine_callsigs.nim") + template wrapTry(body: untyped) = try: body @@ -33,6 +39,13 @@ proc forkchoiceUpdatedV1*(client: RpcClient, wrapTrySimpleRes: client.engine_forkchoiceUpdatedV1(update, payloadAttributes) +proc forkchoiceUpdatedV2*(client: RpcClient, + update: ForkchoiceStateV1, + payloadAttributes = none(PayloadAttributesV2)): + Result[ForkchoiceUpdatedResponse, string] = + wrapTrySimpleRes: + client.engine_forkchoiceUpdatedV2(update, payloadAttributes) + proc getPayloadV1*(client: RpcClient, payloadId: PayloadID): Result[ExecutionPayloadV1, string] = wrapTrySimpleRes: client.engine_getPayloadV1(payloadId) @@ -49,6 +62,12 @@ proc newPayloadV2*(client: RpcClient, wrapTrySimpleRes: client.engine_newPayloadV2(payload) +proc newPayloadV2*(client: RpcClient, + payload: ExecutionPayloadV1OrV2): + Result[PayloadStatusV1, string] = + wrapTrySimpleRes: + client.engine_newPayloadV2(payload) + proc toBlockNumber(n: Option[HexQuantityStr]): common.BlockNumber = if n.isNone: return 0.toBlockNumber @@ -181,6 +200,11 @@ proc balanceAt*(client: RpcClient, address: EthAddress): Result[UInt256, string] let res = waitFor client.eth_getBalance(ethAddressStr(address), "latest") return ok(UInt256.fromHex(res.string)) +proc nonceAt*(client: RpcClient, address: EthAddress): Result[AccountNonce, string] = + wrapTry: + let res = waitFor client.eth_getTransactionCount(ethAddressStr(address), "latest") + return ok(fromHex[AccountNonce](res.string)) + proc txReceipt*(client: RpcClient, txHash: Hash256): Result[eth_api.ReceiptObject, string] = wrapTry: let res = waitFor client.eth_getTransactionReceipt(txHash) diff --git a/hive_integration/nodocker/pyspec/helpers.nim b/hive_integration/nodocker/pyspec/helpers.nim new file mode 100644 index 000000000..34e4a5ea3 --- /dev/null +++ b/hive_integration/nodocker/pyspec/helpers.nim @@ -0,0 +1,58 @@ +import + eth/[common], + json_rpc/[rpcclient], + web3/ethtypes, + ../../../nimbus/transaction + +import eth/common/eth_types as common_eth_types +type Hash256 = common_eth_types.Hash256 + +import web3/engine_api_types +from web3/ethtypes as web3types import nil + +type + Web3BlockHash* = web3types.BlockHash + Web3Address* = web3types.Address + Web3Bloom* = web3types.FixedBytes[256] + Web3Quantity* = web3types.Quantity + Web3PrevRandao* = web3types.FixedBytes[32] + Web3ExtraData* = web3types.DynamicBytes[0, 32] + +func toWdV1(wd: Withdrawal): WithdrawalV1 = + result = WithdrawalV1( + index: Web3Quantity wd.index, + validatorIndex: Web3Quantity wd.validatorIndex, + address: Web3Address wd.address, + amount: Web3Quantity wd.amount + ) + +func toPayloadV1OrV2*(blk: EthBlock): ExecutionPayloadV1OrV2 = + let header = blk.header + + # Return the new payload + result = ExecutionPayloadV1OrV2( + parentHash: Web3BlockHash header.parentHash.data, + feeRecipient: Web3Address header.coinbase, + stateRoot: Web3BlockHash header.stateRoot.data, + receiptsRoot: Web3BlockHash header.receiptRoot.data, + logsBloom: Web3Bloom header.bloom, + prevRandao: Web3PrevRandao header.mixDigest.data, + blockNumber: Web3Quantity header.blockNumber.truncate(uint64), + gasLimit: Web3Quantity header.gasLimit, + gasUsed: Web3Quantity header.gasUsed, + timestamp: Web3Quantity toUnix(header.timestamp), + extraData: Web3ExtraData header.extraData, + baseFeePerGas: header.baseFee, + blockHash: Web3BlockHash header.blockHash.data + ) + + for tx in blk.txs: + let txData = rlp.encode(tx) + result.transactions.add TypedTransaction(txData) + + if blk.withdrawals.isSome: + let withdrawals = blk.withdrawals.get + var wds = newSeqOfCap[WithdrawalV1](withdrawals.len) + for wd in withdrawals: + wds.add toWdV1(wd) + result.withdrawals = some(wds) diff --git a/hive_integration/nodocker/pyspec/pyspec_sim.nim b/hive_integration/nodocker/pyspec/pyspec_sim.nim new file mode 100644 index 000000000..569eac953 --- /dev/null +++ b/hive_integration/nodocker/pyspec/pyspec_sim.nim @@ -0,0 +1,167 @@ +# Nimbus +# Copyright (c) 2023 Status Research & Development GmbH +# Licensed under either of +# * Apache License, version 2.0, ([LICENSE-APACHE](LICENSE-APACHE)) +# * MIT license ([LICENSE-MIT](LICENSE-MIT)) +# at your option. +# This file may not be copied, modified, or distributed except according to +# those terms. + +import + std/[os, json, strutils, times, typetraits, options], + stew/[byteutils, results], + eth/common, + web3/engine_api_types, + ../sim_utils, + ../../../tools/common/helpers as chp, + ../../../tools/evmstate/helpers as ehp, + ../../../tests/test_helpers, + ../engine/engine_client, + ./test_env, + ./helpers + +const + baseFolder = "hive_integration/nodocker/pyspec" + caseFolder = baseFolder & "/testcases" + supportedNetwork = ["Merge", "Shanghai", "MergeToShanghaiAtTime15k"] + +type + Hash256 = common.Hash256 + +proc getPayload(node: JsonNode): ExecutionPayloadV1OrV2 = + let rlpBytes = hexToSeqByte(node.getStr) + toPayloadV1OrV2(rlp.decode(rlpBytes, EthBlock)) + +proc hash256(h: Web3BlockHash): Hash256 = + Hash256(data: distinctBase h) + +proc validatePostState(node: JsonNode, t: TestEnv): bool = + # check nonce, balance & storage of accounts in final block against fixture values + for account, genesisAccount in postState(node["postState"]): + # get nonce & balance from last block (end of test execution) + let nonceRes = t.rpcClient.nonceAt(account) + if nonceRes.isErr: + echo "unable to call nonce from account: " & account.toHex + echo nonceRes.error + return false + + let balanceRes = t.rpcClient.balanceAt(account) + if balanceRes.isErr: + echo "unable to call balance from account: " & account.toHex + echo balanceRes.error + return false + + # check final nonce & balance matches expected in fixture + if genesisAccount.nonce != nonceRes.value: + echo "nonce recieved from account 0x", + account.toHex, + " doesn't match expected ", + genesisAccount.nonce, + " got ", + nonceRes.value + return false + + if genesisAccount.balance != balanceRes.value: + echo "balance recieved from account 0x", + account.toHex, + " doesn't match expected ", + genesisAccount.balance, + " got ", + balanceRes.value + return false + + # check final storage + if genesisAccount.storage.len > 0: + for slot, val in genesisAccount.storage: + let sRes = t.rpcClient.storageAt(account, slot) + if sRes.isErr: + echo "unable to call storage from account: 0x", + account.toHex, + " at slot 0x", + slot.toHex + echo sRes.error + return false + + if val != sRes.value: + echo "storage recieved from account 0x", + account.toHex, + " at slot 0x", + slot.toHex, + " doesn't match expected 0x", + val.toHex, + " got 0x", + sRes.value.toHex + return false + + return true + +proc runTest(node: JsonNode, network: string): TestStatus = + let conf = getChainConfig(network) + var t = TestEnv(conf: makeTestConfig()) + t.setupELClient(conf, node) + + let blks = node["blocks"] + var latestValidHash = Hash256() + result = TestStatus.OK + for blkNode in blks: + let expectedStatus = if "expectException" in blkNode: + PayloadExecutionStatus.invalid + else: + PayloadExecutionStatus.valid + let payload = getPayload(blkNode["rlp"]) + let res = t.rpcClient.newPayloadV2(payload) + if res.isErr: + result = TestStatus.Failed + echo "unable to send block ", payload.blockNumber.uint64, ": ", res.error + break + + let pStatus = res.value + if pStatus.status == PayloadExecutionStatus.valid: + latestValidHash = hash256(pStatus.latestValidHash.get) + + if pStatus.status != expectedStatus: + result = TestStatus.Failed + echo "payload status mismatch for block ", payload.blockNumber.uint64, ", status: ", pStatus.status + if pStatus.validationError.isSome: + echo pStatus.validationError.get + break + + block: + # only update head of beacon chain if valid response occurred + if latestValidHash != Hash256(): + # update with latest valid response + let fcState = ForkchoiceStateV1(headBlockHash: BlockHash latestValidHash.data) + let res = t.rpcClient.forkchoiceUpdatedV2(fcState) + if res.isErr: + result = TestStatus.Failed + echo "unable to update head of beacon chain: ", res.error + break + + if not validatePostState(node, t): + result = TestStatus.Failed + break + + t.stopELClient() + +proc main() = + var stat: SimStat + let start = getTime() + + for fileName in walkDirRec(caseFolder): + if not fileName.endsWith(".json"): + continue + + let fixtureTests = json.parseFile(fileName) + for name, fixture in fixtureTests: + let network = fixture["network"].getStr + if network notin supportedNetwork: + # skip pre Merge tests + continue + + let status = runTest(fixture, network) + stat.inc(name, status) + + let elpd = getTime() - start + print(stat, elpd, "pyspec") + +main() diff --git a/hive_integration/nodocker/pyspec/test_env.nim b/hive_integration/nodocker/pyspec/test_env.nim new file mode 100644 index 000000000..5b93b0471 --- /dev/null +++ b/hive_integration/nodocker/pyspec/test_env.nim @@ -0,0 +1,84 @@ +import + std/[json], + eth/p2p as eth_p2p, + eth/trie/trie_defs, + stew/[byteutils], + json_rpc/[rpcserver, rpcclient], + ../../../nimbus/[ + config, + constants, + transaction, + db/accounts_cache, + core/sealer, + core/chain, + core/tx_pool, + rpc, + sync/protocol, + rpc/merge/merger, + common + ], + ../../../tests/test_helpers, + ../../../tools/evmstate/helpers + +type + TestEnv* = ref object + conf*: NimbusConf + ctx: EthContext + ethNode: EthereumNode + com: CommonRef + chainRef: ChainRef + rpcServer: RpcHttpServer + sealingEngine: SealingEngineRef + rpcClient*: RpcHttpClient + +const + engineSigner = hexToByteArray[20]("0x658bdf435d810c91414ec09147daa6db62406379") + +proc genesisHeader(node: JsonNode): BlockHeader = + let genesisRLP = hexToSeqByte(node["genesisRLP"].getStr) + rlp.decode(genesisRLP, EthBlock).header + +proc setupELClient*(t: TestEnv, conf: ChainConfig, node: JsonNode) = + let memDB = newMemoryDb() + t.ctx = newEthContext() + t.ethNode = setupEthNode(t.conf, t.ctx, eth) + t.com = CommonRef.new( + memDB, + conf, + t.conf.pruneMode == PruneMode.Full + ) + t.chainRef = newChain(t.com) + let + stateDB = AccountsCache.init(memDB, emptyRlpHash, t.conf.pruneMode == PruneMode.Full) + genesisHeader = node.genesisHeader + + setupStateDB(node["pre"], stateDB) + stateDB.persist() + + doAssert stateDB.rootHash == genesisHeader.stateRoot + + discard t.com.db.persistHeaderToDb(genesisHeader, + t.com.consensus == ConsensusType.POS) + doAssert(t.com.db.getCanonicalHead().blockHash == genesisHeader.blockHash) + + let txPool = TxPoolRef.new(t.com, engineSigner) + t.rpcServer = newRpcHttpServer(["localhost:8545"]) + t.sealingEngine = SealingEngineRef.new( + t.chainRef, t.ctx, engineSigner, + txPool, EngineStopped + ) + + let merger = MergerRef.new(t.com.db) + setupEthRpc(t.ethNode, t.ctx, t.com, txPool, t.rpcServer) + setupEngineAPI(t.sealingEngine, t.rpcServer, merger) + #setupDebugRpc(t.com, t.rpcServer) + + t.rpcServer.start() + + t.rpcClient = newRpcHttpClient() + waitFor t.rpcClient.connect("localhost", 8545.Port, false) + +proc stopELClient*(t: TestEnv) = + waitFor t.rpcClient.close() + waitFor t.sealingEngine.stop() + waitFor t.rpcServer.closeWait() diff --git a/nimbus/rpc/engine_api.nim b/nimbus/rpc/engine_api.nim index 7782b17df..f6e6f3859 100644 --- a/nimbus/rpc/engine_api.nim +++ b/nimbus/rpc/engine_api.nim @@ -108,7 +108,7 @@ proc handle_newPayload(sealingEngine: SealingEngineRef, api: EngineApiRef, com: else: raise invalidParams("if timestamp is earlier than Shanghai, " & "payload must be ExecutionPayloadV1") - + var header = toBlockHeader(payload) let blockHash = payload.blockHash.asEthHash var res = header.validateBlockHash(blockHash) @@ -170,10 +170,11 @@ proc handle_newPayload(sealingEngine: SealingEngineRef, api: EngineApiRef, com: td = db.getScore(header.parentHash) ttd = com.ttd.get(high(common.BlockNumber)) - if td < ttd: - warn "Ignoring pre-merge payload", - number = header.blockNumber, hash = blockHash, td, ttd - return invalidStatus() + when payload is ExecutionPayloadV1: + if (not com.forkGTE(MergeFork)) and td < ttd: + warn "Ignoring pre-merge payload", + number = header.blockNumber, hash = blockHash, td, ttd + return invalidStatus() if header.timestamp <= parent.timestamp: warn "Invalid timestamp", @@ -204,7 +205,7 @@ proc handle_newPayload(sealingEngine: SealingEngineRef, api: EngineApiRef, com: # TODO: cancel downloader return validStatus(blockHash) - + # https://github.com/ethereum/execution-apis/blob/main/src/engine/specification.md#engine_getpayloadv1 proc handle_getPayload(api: EngineApiRef, payloadId: PayloadID): GetPayloadV2Response {.raises: [CatchableError].} = trace "Engine API request received", @@ -215,7 +216,7 @@ proc handle_getPayload(api: EngineApiRef, payloadId: PayloadID): GetPayloadV2Res raise unknownPayload("Unknown payload") let blockValue = sumOfBlockPriorityFees(payload) - + return GetPayloadV2Response( executionPayload: payload, blockValue: blockValue @@ -426,7 +427,7 @@ proc handle_forkchoiceUpdated(sealingEngine: SealingEngineRef, com: CommonRef, a if res.isErr: error "Failed to create sealing payload", err = res.error raise invalidAttr(res.error) - + let payload = res.get let id = computePayloadId(blockHash, payloadAttrs) diff --git a/nimbus/rpc/merge/mergeutils.nim b/nimbus/rpc/merge/mergeutils.nim index c5113dbb2..07aa223a5 100644 --- a/nimbus/rpc/merge/mergeutils.nim +++ b/nimbus/rpc/merge/mergeutils.nim @@ -15,6 +15,7 @@ import eth/[trie, rlp, common, common/eth_types, trie/db], stew/[results, byteutils], ../../constants, + ../../utils/utils, ./mergetypes type Hash256 = eth_types.Hash256 @@ -55,19 +56,40 @@ proc calcRootHashRlp*(items: openArray[seq[byte]]): Hash256 = tr.put(rlp.encode(i), t) return tr.rootHash() -proc calcWithdrawalsRoot(withdrawals: seq[WithdrawalV1]): Hash256 = - calcRootHashRlp(withdrawals.map(writer.encode)) +proc toWithdrawal*(w: WithdrawalV1): Withdrawal = + Withdrawal( + index: uint64(w.index), + validatorIndex: uint64(w.validatorIndex), + address: distinctBase(w.address), + amount: uint64(w.amount) # AARDVARK: is this wei or gwei or what? + ) -func maybeWithdrawals*(payload: ExecutionPayloadV1 | ExecutionPayloadV2): Option[seq[WithdrawalV1]] = +proc toWithdrawalV1*(w: Withdrawal): WithdrawalV1 = + WithdrawalV1( + index: Quantity(w.index), + validatorIndex: Quantity(w.validatorIndex), + address: Address(w.address), + amount: Quantity(w.amount) # AARDVARK: is this wei or gwei or what? + ) + +proc maybeWithdrawalsRoot(payload: ExecutionPayloadV1 | ExecutionPayloadV2): Option[Hash256] = when payload is ExecutionPayloadV1: - none[seq[WithdrawalV1]]() + none(Hash256) else: - some(payload.withdrawals) + var wds = newSeqOfCap[Withdrawal](payload.withdrawals.len) + for wd in payload.withdrawals: + wds.add toWithdrawal(wd) + some(utils.calcWithdrawalsRoot(wds)) + +proc toWithdrawals(withdrawals: openArray[WithdrawalV1]): seq[WithDrawal] = + result = newSeqOfCap[Withdrawal](withdrawals.len) + for wd in withdrawals: + result.add toWithdrawal(wd) proc toBlockHeader*(payload: ExecutionPayloadV1 | ExecutionPayloadV2): EthBlockHeader = let transactions = seq[seq[byte]](payload.transactions) let txRoot = calcRootHashRlp(transactions) - + EthBlockHeader( parentHash : payload.parentHash.asEthHash, ommersHash : EMPTY_UNCLE_HASH, @@ -85,23 +107,7 @@ proc toBlockHeader*(payload: ExecutionPayloadV1 | ExecutionPayloadV2): EthBlockH mixDigest : payload.prevRandao.asEthHash, # EIP-4399 redefine `mixDigest` -> `prevRandao` nonce : default(BlockNonce), fee : some payload.baseFeePerGas, - withdrawalsRoot: payload.maybeWithdrawals.map(calcWithdrawalsRoot) # EIP-4895 - ) - -proc toWithdrawal*(w: WithdrawalV1): Withdrawal = - Withdrawal( - index: uint64(w.index), - validatorIndex: uint64(w.validatorIndex), - address: distinctBase(w.address), - amount: uint64(w.amount) # AARDVARK: is this wei or gwei or what? - ) - -proc toWithdrawalV1*(w: Withdrawal): WithdrawalV1 = - WithdrawalV1( - index: Quantity(w.index), - validatorIndex: Quantity(w.validatorIndex), - address: Address(w.address), - amount: Quantity(w.amount) # AARDVARK: is this wei or gwei or what? + withdrawalsRoot: payload.maybeWithdrawalsRoot # EIP-4895 ) proc toTypedTransaction*(tx: Transaction): TypedTransaction = @@ -112,12 +118,7 @@ proc toBlockBody*(payload: ExecutionPayloadV1 | ExecutionPayloadV2): BlockBody = for i, tx in payload.transactions: result.transactions[i] = rlp.decode(distinctBase tx, Transaction) when payload is ExecutionPayloadV2: - let ws = payload.maybeWithdrawals - result.withdrawals = - if ws.isSome: - some(ws.get.map(toWithdrawal)) - else: - none[seq[Withdrawal]]() + result.withdrawals = some(payload.withdrawals.toWithdrawals) proc `$`*(x: BlockHash): string = toHex(x) diff --git a/tools/evmstate/helpers.nim b/tools/evmstate/helpers.nim index 52dd287bb..390b867d9 100644 --- a/tools/evmstate/helpers.nim +++ b/tools/evmstate/helpers.nim @@ -15,7 +15,8 @@ import stint, stew/byteutils, ../../nimbus/transaction, - ../../nimbus/db/accounts_cache + ../../nimbus/db/accounts_cache, + ../../nimbus/common/chain_config template fromJson(T: type EthAddress, n: JsonNode): EthAddress = hexToByteArray(n.getStr, sizeof(T)) @@ -146,3 +147,17 @@ proc setupStateDB*(wantedState: JsonNode, stateDB: AccountsCache) = stateDB.setNonce(account, fromJson(AccountNonce, accountData["nonce"])) stateDB.setCode(account, fromJson(Blob, accountData["code"])) stateDB.setBalance(account, fromJson(UInt256, accountData["balance"])) + +iterator postState*(node: JsonNode): (EthAddress, GenesisAccount) = + for ac, accountData in node: + let account = hexToByteArray[20](ac) + var ga = GenesisAccount( + nonce : fromJson(AccountNonce, accountData["nonce"]), + code : fromJson(Blob, accountData["code"]), + balance: fromJson(UInt256, accountData["balance"]), + ) + + for slot, value in accountData{"storage"}: + ga.storage[fromHex(UInt256, slot)] = fromHex(UInt256, value.getStr) + + yield (account, ga)