Proper setup of test_rpc (#2969)

Instead of using ancient/dirty code to setup the rpc test, now using newest method from TxPool and ForkedChain.
Also fix some bugs in server_api discovered when using this new setup.
This commit is contained in:
andri lim 2024-12-22 16:18:46 +07:00 committed by GitHub
parent 557a9605cf
commit 487743fc2e
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
4 changed files with 271 additions and 260 deletions

View File

@ -682,11 +682,11 @@ func isInMemory*(c: ForkedChainRef, blockHash: Hash32): bool =
func memoryBlock*(c: ForkedChainRef, blockHash: Hash32): BlockDesc =
c.blocks.getOrDefault(blockHash)
func memoryTransaction*(c: ForkedChainRef, txHash: Hash32): Opt[Transaction] =
func memoryTransaction*(c: ForkedChainRef, txHash: Hash32): Opt[(Transaction, BlockNumber)] =
let (blockHash, index) = c.txRecords.getOrDefault(txHash, (Hash32.default, 0'u64))
c.blocks.withValue(blockHash, val) do:
return Opt.some(val.blk.transactions[index])
return Opt.none(Transaction)
return Opt.some( (val.blk.transactions[index], val.blk.header.number) )
return Opt.none((Transaction, BlockNumber))
proc latestBlock*(c: ForkedChainRef): Block =
c.blocks.withValue(c.cursorHash, val) do:
@ -740,6 +740,9 @@ proc blockByNumber*(c: ForkedChainRef, number: BlockNumber): Result[Block, strin
if number < c.baseHeader.number:
return c.db.getEthBlock(number)
if number == c.baseHeader.number:
return c.db.getEthBlock(c.baseHash)
shouldNotKeyError "blockByNumber":
var prevHash = c.cursorHash
while prevHash != c.baseHash:

View File

@ -45,7 +45,7 @@ proc setupCommonRpc*(node: EthereumNode, conf: NimbusConf, server: RpcServer) =
let numPeers = node.numPeers
result = numPeers < conf.maxPeers
server.rpc("net_peerCount") do() -> Web3Quantity:
server.rpc("net_peerCount") do() -> Quantity:
let peerCount = uint node.numPeers
result = w3Qty(peerCount)

View File

@ -50,7 +50,7 @@ proc getTotalDifficulty*(api: ServerAPIRef, blockHash: Hash32): UInt256 =
return totalDifficulty
proc getProof*(
accDB: LedgerRef, address: eth_types.Address, slots: seq[UInt256]
accDB: LedgerRef, address: Address, slots: seq[UInt256]
): ProofResponse =
let
acc = accDB.getEthAccount(address)
@ -142,7 +142,7 @@ proc setupServerAPI*(api: ServerAPIRef, server: RpcServer, ctx: EthContext) =
server.rpc("eth_getTransactionCount") do(
data: Address, blockTag: BlockTag
) -> Web3Quantity:
) -> Quantity:
## Returns the number of transactions ak.s. nonce sent from an address.
let
ledger = api.ledgerFromTag(blockTag).valueOr:
@ -151,11 +151,11 @@ proc setupServerAPI*(api: ServerAPIRef, server: RpcServer, ctx: EthContext) =
nonce = ledger.getNonce(address)
Quantity(nonce)
server.rpc("eth_blockNumber") do() -> Web3Quantity:
server.rpc("eth_blockNumber") do() -> Quantity:
## Returns integer of the current block number the client is on.
Quantity(api.chain.latestNumber)
server.rpc("eth_chainId") do() -> Web3Quantity:
server.rpc("eth_chainId") do() -> Quantity:
return Quantity(distinctBase(api.com.chainId))
server.rpc("eth_getCode") do(data: Address, blockTag: BlockTag) -> seq[byte]:
@ -376,7 +376,7 @@ proc setupServerAPI*(api: ServerAPIRef, server: RpcServer, ctx: EthContext) =
idx.inc
server.rpc("eth_estimateGas") do(args: TransactionArgs) -> Web3Quantity:
server.rpc("eth_estimateGas") do(args: TransactionArgs) -> Quantity:
## Generates and returns an estimate of how much gas is necessary to allow the transaction to complete.
## The transaction will not be added to the blockchain. Note that the estimate may be significantly more than
## the amount of gas actually used by the transaction, for a variety of reasons including EVM mechanics and node performance.
@ -392,17 +392,17 @@ proc setupServerAPI*(api: ServerAPIRef, server: RpcServer, ctx: EthContext) =
raise newException(ValueError, "rpcEstimateGas error: " & $error.code)
Quantity(gasUsed)
server.rpc("eth_gasPrice") do() -> Web3Quantity:
server.rpc("eth_gasPrice") do() -> Quantity:
## Returns an integer of the current gas price in wei.
w3Qty(calculateMedianGasPrice(api.chain).uint64)
server.rpc("eth_accounts") do() -> seq[eth_types.Address]:
server.rpc("eth_accounts") do() -> seq[Address]:
## Returns a list of addresses owned by client.
result = newSeqOfCap[eth_types.Address](ctx.am.numAccounts)
result = newSeqOfCap[Address](ctx.am.numAccounts)
for k in ctx.am.addresses:
result.add k
server.rpc("eth_getBlockTransactionCountByHash") do(data: Hash32) -> Web3Quantity:
server.rpc("eth_getBlockTransactionCountByHash") do(data: Hash32) -> Quantity:
## Returns the number of transactions in a block from a block matching the given block hash.
##
## data: hash of a block
@ -410,11 +410,11 @@ proc setupServerAPI*(api: ServerAPIRef, server: RpcServer, ctx: EthContext) =
let blk = api.chain.blockByHash(data).valueOr:
raise newException(ValueError, "Block not found")
Web3Quantity(blk.transactions.len)
Quantity(blk.transactions.len)
server.rpc("eth_getBlockTransactionCountByNumber") do(
blockTag: BlockTag
) -> Web3Quantity:
) -> Quantity:
## Returns the number of transactions in a block from a block matching the given block number.
##
## blockTag: integer of a block number, or the string "latest", "earliest" or "pending", see the default block parameter.
@ -422,9 +422,9 @@ proc setupServerAPI*(api: ServerAPIRef, server: RpcServer, ctx: EthContext) =
let blk = api.blockFromTag(blockTag).valueOr:
raise newException(ValueError, "Block not found")
Web3Quantity(blk.transactions.len)
Quantity(blk.transactions.len)
server.rpc("eth_getUncleCountByBlockHash") do(data: Hash32) -> Web3Quantity:
server.rpc("eth_getUncleCountByBlockHash") do(data: Hash32) -> Quantity:
## Returns the number of uncles in a block from a block matching the given block hash.
##
## data: hash of a block.
@ -432,9 +432,9 @@ proc setupServerAPI*(api: ServerAPIRef, server: RpcServer, ctx: EthContext) =
let blk = api.chain.blockByHash(data).valueOr:
raise newException(ValueError, "Block not found")
Web3Quantity(blk.uncles.len)
Quantity(blk.uncles.len)
server.rpc("eth_getUncleCountByBlockNumber") do(blockTag: BlockTag) -> Web3Quantity:
server.rpc("eth_getUncleCountByBlockNumber") do(blockTag: BlockTag) -> Quantity:
## Returns the number of uncles in a block from a block matching the given block number.
##
## blockTag: integer of a block number, or the string "latest", see the default block parameter.
@ -442,14 +442,14 @@ proc setupServerAPI*(api: ServerAPIRef, server: RpcServer, ctx: EthContext) =
let blk = api.blockFromTag(blockTag).valueOr:
raise newException(ValueError, "Block not found")
Web3Quantity(blk.uncles.len)
Quantity(blk.uncles.len)
template sign(privateKey: PrivateKey, message: string): seq[byte] =
# message length encoded as ASCII representation of decimal
let msgData = "\x19Ethereum Signed Message:\n" & $message.len & message
@(sign(privateKey, msgData.toBytes()).toRaw())
server.rpc("eth_sign") do(data: eth_types.Address, message: seq[byte]) -> seq[byte]:
server.rpc("eth_sign") do(data: Address, message: seq[byte]) -> seq[byte]:
## The sign method calculates an Ethereum specific signature with: sign(keccak256("\x19Ethereum Signed Message:\n" + len(message) + message))).
## By adding a prefix to the message makes the calculated signature recognisable as an Ethereum specific signature.
## This prevents misuse where a malicious DApp can sign arbitrary data (e.g. transaction) and use the signature to impersonate the victim.
@ -537,16 +537,15 @@ proc setupServerAPI*(api: ServerAPIRef, server: RpcServer, ctx: EthContext) =
if res.isOk:
return populateTransactionObject(res.get().tx, Opt.none(Hash32), Opt.none(uint64))
let txDetails = api.chain.db.getTransactionKey(txHash).valueOr:
return nil
if txDetails.index < 0:
block blockOne:
let
(blockHash, txid) = api.chain.txRecords(txHash)
tx = api.chain.memoryTransaction(txHash).valueOr:
return nil
return populateTransactionObject(tx, Opt.some(blockHash), Opt.some(txid))
# TODO: include block number
(tx, number) = api.chain.memoryTransaction(txHash).valueOr:
break blockOne
return populateTransactionObject(tx, Opt.some(blockHash), Opt.some(number), Opt.some(txid))
let txDetails = api.chain.db.getTransactionKey(txHash).valueOr:
return nil
let header = api.chain.db.getBlockHeader(txDetails.blockNumber).valueOr:
return nil
let tx = api.chain.db.getTransactionByIndex(header.txRoot, uint16(txDetails.index)).valueOr:
@ -559,7 +558,7 @@ proc setupServerAPI*(api: ServerAPIRef, server: RpcServer, ctx: EthContext) =
)
server.rpc("eth_getTransactionByBlockHashAndIndex") do(
data: Hash32, quantity: Web3Quantity
data: Hash32, quantity: Quantity
) -> TransactionObject:
## Returns information about a transaction by block hash and transaction index position.
##
@ -578,7 +577,7 @@ proc setupServerAPI*(api: ServerAPIRef, server: RpcServer, ctx: EthContext) =
)
server.rpc("eth_getTransactionByBlockNumberAndIndex") do(
quantityTag: BlockTag, quantity: Web3Quantity
quantityTag: BlockTag, quantity: Quantity
) -> TransactionObject:
## Returns information about a transaction by block number and transaction index position.
##
@ -597,7 +596,7 @@ proc setupServerAPI*(api: ServerAPIRef, server: RpcServer, ctx: EthContext) =
)
server.rpc("eth_getProof") do(
data: eth_types.Address, slots: seq[UInt256], quantityTag: BlockTag
data: Address, slots: seq[UInt256], quantityTag: BlockTag
) -> ProofResponse:
## Returns information about an account and storage slots (if the account is a contract
## and the slots are requested) along with account and storage proofs which prove the
@ -662,7 +661,7 @@ proc setupServerAPI*(api: ServerAPIRef, server: RpcServer, ctx: EthContext) =
except CatchableError as exc:
return AccessListResult(error: Opt.some("createAccessList error: " & exc.msg))
server.rpc("eth_blobBaseFee") do() -> Web3Quantity:
server.rpc("eth_blobBaseFee") do() -> Quantity:
## Returns the base fee per blob gas in wei.
let header = api.headerFromTag(blockId("latest")).valueOr:
raise newException(ValueError, "Block not found")
@ -677,7 +676,7 @@ proc setupServerAPI*(api: ServerAPIRef, server: RpcServer, ctx: EthContext) =
return w3Qty blobBaseFee.truncate(uint64)
server.rpc("eth_getUncleByBlockHashAndIndex") do(
data: Hash32, quantity: Web3Quantity
data: Hash32, quantity: Quantity
) -> BlockObject:
## Returns information about a uncle of a block by hash and uncle index position.
##
@ -701,7 +700,7 @@ proc setupServerAPI*(api: ServerAPIRef, server: RpcServer, ctx: EthContext) =
)
server.rpc("eth_getUncleByBlockNumberAndIndex") do(
quantityTag: BlockTag, quantity: Web3Quantity
quantityTag: BlockTag, quantity: Quantity
) -> BlockObject:
# Returns information about a uncle of a block by number and uncle index position.
##

View File

@ -7,20 +7,20 @@
import
chronicles,
std/[json, os, typetraits, times, sequtils],
asynctest, web3/eth_api,
std/[json, typetraits, sequtils],
asynctest,
web3/eth_api,
stew/byteutils,
json_rpc/[rpcserver, rpcclient],
eth/[rlp, trie/hexary_proof_verification],
eth/[p2p, rlp, trie/hexary_proof_verification],
eth/common/[transaction_utils, addresses],
../hive_integration/nodocker/engine/engine_client,
../nimbus/[constants, transaction, config, evm/state, evm/types, version],
../nimbus/[constants, transaction, config, version],
../nimbus/db/[ledger, storage_types],
../nimbus/sync/protocol,
../nimbus/core/[tx_pool, chain, executor, executor/executor_helpers, pow/difficulty],
../nimbus/core/[tx_pool, chain, pow/difficulty, casper],
../nimbus/utils/utils,
../nimbus/common,
../nimbus/rpc,
../nimbus/[common, rpc],
../nimbus/rpc/rpc_types,
../nimbus/beacon/web3_eth_conv,
./test_helpers,
@ -29,16 +29,43 @@ import
type
Hash32 = common.Hash32
Header = common.Header
TestEnv = object
txHash: Hash32
conf : NimbusConf
com : CommonRef
txPool : TxPoolRef
server : RpcHttpServer
client : RpcHttpClient
chain : ForkedChainRef
ctx : EthContext
node : EthereumNode
txHash : Hash32
blockHash: Hash32
nonce : uint64
chainId : ChainId
const
zeroHash = hash32"0x0000000000000000000000000000000000000000000000000000000000000000"
emptyCodeHash = hash32"0xc5d2460186f7233c927e7db2dcc703c0e500b653ca82273b7bfad8045d85a470"
emptyStorageHash = hash32"0x56e81f171bcc55a6ff8345e692c0f86e5b48e01b996cadc001622fb5e363b421"
genesisFile = "tests/customgenesis/cancun123.json"
contractCode = evmByteCode:
Push4 "0xDEADBEEF" # PUSH
Push1 "0x00" # MSTORE AT 0x00
Mstore
Push1 "0x04" # RETURN LEN
Push1 "0x1C" # RETURN OFFSET at 28
Return
keyStore = "tests/keystore"
signer = address"0x0e69cde81b1aa07a45c32c6cd85d67229d36bb1b"
contractAddress = address"0xa3b2222afa5c987da6ef773fde8d01b9f23d481f"
extraAddress = address"0x597176e9a64aad0845d83afdaf698fbeff77703b"
regularAcc = address"0x0000000000000000000000000000000000000001"
contractAccWithStorage = address"0x0000000000000000000000000000000000000002"
contractAccNoStorage = address"0x0000000000000000000000000000000000000003"
feeRecipient = address"0000000000000000000000000000000000000212"
prevRandao = Bytes32 EMPTY_UNCLE_HASH # it can be any valid hash
oneETH = 1.u256 * 1_000_000_000.u256 * 1_000_000_000.u256
proc verifyAccountProof(trustedStateRoot: Hash32, res: ProofResponse): MptProofVerificationResult =
let
@ -74,193 +101,187 @@ proc persistFixtureBlock(chainDB: CoreDbRef) =
chainDB.persistTransactions(header.number, header.txRoot, getBlockBody4514995().transactions)
chainDB.persistReceipts(header.receiptsRoot, getReceipts4514995())
proc setupConfig(): NimbusConf =
makeConfig(@[
"--custom-network:" & genesisFile
])
proc setupCom(conf: NimbusConf): CommonRef =
CommonRef.new(
newCoreDbRef DefaultDbMemory,
nil,
conf.networkId,
conf.networkParams
)
proc setupClient(port: Port): RpcHttpClient =
let client = newRpcHttpClient()
waitFor client.connect("127.0.0.1", port, false)
return client
proc close(client: RpcHttpClient, server: RpcHttpServer) =
waitFor client.close()
waitFor server.closeWait()
proc close(env: TestEnv) =
waitFor env.client.close()
waitFor env.server.closeWait()
func makeTx(
env: var TestEnv,
signerKey: PrivateKey,
recipient: addresses.Address,
amount: UInt256,
gasPrice: GasInt,
payload: openArray[byte] = []
): Transaction =
const
gasLimit = 70000.GasInt
# NOTE : The setup of the environment should have been done through the
# `ForkedChainRef`, however the `ForkedChainRef` is does not persist blocks to the db
# unless the base distance is reached. This is not the case for the tests, so we
# have to manually persist the blocks to the db.
# Main goal of the tests to check the RPC calls, can serve data persisted in the db
# as data from memory blocks are easily tested via kurtosis or other tests
proc setupEnv(signer, ks2: Address, ctx: EthContext, com: CommonRef): TestEnv =
var
acc = ctx.am.getAccount(signer).tryGet()
blockNumber = 1'u64
parent = com.db.getCanonicalHead().expect("canonicalHead exists")
parentHash = parent.blockHash
let tx = Transaction(
txType: TxLegacy,
chainId: env.chainId,
nonce: AccountNonce(env.nonce),
gasPrice: gasPrice,
gasLimit: gasLimit,
to: Opt.some(recipient),
value: amount,
payload: @payload,
)
let code = evmByteCode:
Push4 "0xDEADBEEF" # PUSH
Push1 "0x00" # MSTORE AT 0x00
Mstore
Push1 "0x04" # RETURN LEN
Push1 "0x1C" # RETURN OFFSET at 28
Return
inc env.nonce
signTransaction(tx, signerKey, eip155 = true)
proc setupEnv(envFork: HardFork = MergeFork): TestEnv =
doAssert(envFork >= MergeFork)
let
vmHeader = Header(parentHash: parentHash, gasLimit: 5_000_000)
vmState = BaseVMState()
vmState.init(parent, vmHeader, com)
conf = setupConfig()
vmState.ledger.setCode(ks2, code)
vmState.ledger.addBalance(
signer, 1.u256 * 1_000_000_000.u256 * 1_000_000_000.u256) # 1 ETH
conf.networkParams.genesis.alloc[contractAddress] = GenesisAccount(code: contractCode)
conf.networkParams.genesis.alloc[signer] = GenesisAccount(balance: oneETH)
# Test data created for eth_getProof tests
let regularAcc = address"0x0000000000000000000000000000000000000001"
vmState.ledger.addBalance(regularAcc, 2_000_000_000.u256)
vmState.ledger.setNonce(regularAcc, 1.uint64)
conf.networkParams.genesis.alloc[regularAcc] = GenesisAccount(
balance: 2_000_000_000.u256,
nonce: 1.uint64)
let contractAccWithStorage = address"0x0000000000000000000000000000000000000002"
vmState.ledger.addBalance(contractAccWithStorage, 1_000_000_000.u256)
vmState.ledger.setNonce(contractAccWithStorage, 2.uint64)
vmState.ledger.setCode(contractAccWithStorage, code)
vmState.ledger.setStorage(contractAccWithStorage, u256(0), u256(1234))
vmState.ledger.setStorage(contractAccWithStorage, u256(1), u256(2345))
conf.networkParams.genesis.alloc[contractAccWithStorage] = GenesisAccount(
balance: 1_000_000_000.u256,
nonce: 2.uint64,
code: contractCode,
storage: {
0.u256: 1234.u256,
1.u256: 2345.u256,
}.toTable)
let contractAccNoStorage = address"0x0000000000000000000000000000000000000003"
vmState.ledger.setCode(contractAccNoStorage, code)
conf.networkParams.genesis.alloc[contractAccNoStorage] = GenesisAccount(code: contractCode)
if envFork >= Shanghai:
conf.networkParams.config.shanghaiTime = Opt.some(0.EthTime)
if envFork >= Cancun:
conf.networkParams.config.cancunTime = Opt.some(0.EthTime)
if envFork >= Prague:
conf.networkParams.config.pragueTime = Opt.some(0.EthTime)
let
unsignedTx1 = Transaction(
txType : TxLegacy,
nonce : 0,
gasPrice: uint64(30_000_000_000),
gasLimit: 70_000,
value : 1.u256,
to : Opt.some(zeroAddress),
chainId : com.chainId,
)
unsignedTx2 = Transaction(
txType : TxLegacy,
nonce : 1,
gasPrice: uint64(30_000_000_100),
gasLimit: 70_000,
value : 2.u256,
to : Opt.some(zeroAddress),
chainId : com.chainId,
)
eip155 = com.isEIP155(com.syncCurrent)
signedTx1 = signTransaction(unsignedTx1, acc.privateKey, eip155)
signedTx2 = signTransaction(unsignedTx2, acc.privateKey, eip155)
txs = [signedTx1, signedTx2]
com = setupCom(conf)
chain = ForkedChainRef.init(com)
txPool = TxPoolRef.new(chain)
let txRoot = calcTxRoot(txs)
com.db.persistTransactions(blockNumber, txRoot, txs)
vmState.receipts = newSeq[Receipt](txs.len)
vmState.cumulativeGasUsed = 0
for txIndex, tx in txs:
let sender = tx.recoverSender().expect("valid signature")
let rc = vmState.processTransaction(tx, sender, vmHeader)
doAssert(rc.isOk, "Invalid transaction: " & rc.error)
vmState.receipts[txIndex] = makeReceipt(vmState, tx.txType)
# txPool must be informed of active head
# so it can know the latest account state
doAssert txPool.smartHead(chain.latestHeader)
let
# TODO: `getColumn(CtReceipts)` does not exists anymore. There s only the
# generic `MPT` left that can be retrieved with `getGeneric()`,
# optionally with argument `clearData=true`
date = dateTime(2017, mMar, 30)
timeStamp = date.toTime.toUnix.EthTime
difficulty = com.calcDifficulty(timeStamp, parent)
server = newRpcHttpServerWithParams("127.0.0.1:0").valueOr:
echo "Failed to create rpc server: ", error
quit(QuitFailure)
serverApi = newServerAPI(txPool)
client = setupClient(server.localAddress[0].port)
ctx = newEthContext()
node = setupEthNode(conf, ctx, eth)
# call persist() before we get the stateRoot
vmState.ledger.persist()
ctx.am.loadKeystores(keyStore).isOkOr:
debugEcho error
quit(QuitFailure)
var header = Header(
parentHash : parentHash,
stateRoot : vmState.ledger.getStateRoot,
transactionsRoot: txRoot,
receiptsRoot : calcReceiptsRoot(vmState.receipts),
logsBloom : createBloom(vmState.receipts),
difficulty : difficulty,
number : blockNumber,
gasLimit : vmState.cumulativeGasUsed + 1_000_000,
gasUsed : vmState.cumulativeGasUsed,
timestamp : timeStamp
)
let acc1 = ctx.am.getAccount(signer).tryGet()
ctx.am.unlockAccount(signer, acc1.keystore["password"].getStr()).isOkOr:
debugEcho error
quit(QuitFailure)
com.db.persistHeaderAndSetHead(header,
com.startOfHistory).expect("persistHeader not error")
setupServerAPI(serverApi, server, ctx)
setupCommonRpc(node, conf, server)
server.start()
let uncles = [header]
header.ommersHash = com.db.persistUncles(uncles)
TestEnv(
conf : conf,
com : com,
txPool : txPool,
server : server,
client : client,
chain : chain,
ctx : ctx,
node : node,
chainId: conf.networkParams.config.chainId,
)
com.db.persistHeaderAndSetHead(header,
com.startOfHistory).expect("persistHeader not error")
proc generateBlock(env: var TestEnv) =
let
com = env.com
xp = env.txPool
ctx = env.ctx
acc = ctx.am.getAccount(signer).tryGet()
tx1 = env.makeTx(acc.privateKey, zeroAddress, 1.u256, 30_000_000_000'u64)
tx2 = env.makeTx(acc.privateKey, zeroAddress, 2.u256, 30_000_000_100'u64)
chain = env.chain
# txPool must be informed of active head
# so it can know the latest account state
doAssert xp.smartHead(chain.latestHeader)
xp.add(PooledTransaction(tx: tx1))
xp.add(PooledTransaction(tx: tx2))
doAssert(xp.nItems.total == 2)
# generate block
com.pos.prevRandao = prevRandao
com.pos.feeRecipient = feeRecipient
com.pos.timestamp = EthTime.now()
let bundle = xp.assembleBlock().valueOr:
debugEcho error
quit(QuitFailure)
let blk = bundle.blk
doAssert(blk.transactions.len == 2)
# import block
chain.importBlock(blk).isOkOr:
debugEcho error
quit(QuitFailure)
doAssert xp.smartHead(chain.latestHeader)
com.db.persistFixtureBlock()
com.db.persistent(header.number).isOkOr:
echo "Failed to save state: ", $error
quit(QuitFailure)
result = TestEnv(
txHash: signedTx1.rlpHash,
blockHash: header.blockHash
)
env.txHash = tx1.rlpHash
env.blockHash = blk.header.blockHash
createRpcSigsFromNim(RpcClient):
proc web3_clientVersion(): string
proc web3_sha3(data: seq[byte]): Hash32
proc net_version(): string
proc net_listening(): bool
proc net_peerCount(): Quantity
proc rpcMain*() =
suite "Remote Procedure Calls":
# TODO: Include other transports such as Http
var env = setupEnv()
env.generateBlock()
let
conf = makeConfig(@[])
ctx = newEthContext()
ethNode = setupEthNode(conf, ctx, eth)
com = CommonRef.new(
newCoreDbRef DefaultDbMemory, nil,
conf.networkId,
conf.networkParams
)
signer = address"0x0e69cde81b1aa07a45c32c6cd85d67229d36bb1b"
ks2 = address"0xa3b2222afa5c987da6ef773fde8d01b9f23d481f"
ks3 = address"0x597176e9a64aad0845d83afdaf698fbeff77703b"
let keyStore = "tests" / "keystore"
let res = ctx.am.loadKeystores(keyStore)
if res.isErr:
debugEcho res.error
doAssert(res.isOk)
let acc1 = ctx.am.getAccount(signer).tryGet()
let unlock = ctx.am.unlockAccount(signer, acc1.keystore["password"].getStr())
if unlock.isErr:
debugEcho unlock.error
doAssert(unlock.isOk)
let
env = setupEnv(signer, ks2, ctx, com)
chain = ForkedChainRef.init(com)
txPool = TxPoolRef.new(chain)
# txPool must be informed of active head
# so it can know the latest account state
doAssert txPool.smartHead(chain.latestHeader)
let
server = newRpcHttpServerWithParams("127.0.0.1:0").valueOr:
quit(QuitFailure)
serverApi = newServerAPI(txPool)
setupServerAPI(serverApi, server, ctx)
setupCommonRpc(ethNode, conf, server)
server.start()
let client = setupClient(server.localAddress[0].port)
# disable POS/post Merge feature
com.setTTD Opt.none(DifficultyInt)
client = env.client
node = env.node
com = env.com
test "web3_clientVersion":
let res = await client.web3_clientVersion()
@ -274,16 +295,16 @@ proc rpcMain*() =
test "net_version":
let res = await client.net_version()
check res == $conf.networkId
check res == $env.conf.networkId
test "net_listening":
let res = await client.net_listening()
let listening = ethNode.peerPool.connectedNodes.len < conf.maxPeers
let listening = node.peerPool.connectedNodes.len < env.conf.maxPeers
check res == listening
test "net_peerCount":
let res = await client.net_peerCount()
let peerCount = ethNode.peerPool.connectedNodes.len
let peerCount = node.peerPool.connectedNodes.len
check res == w3Qty(peerCount)
test "eth_chainId":
@ -293,7 +314,7 @@ proc rpcMain*() =
test "eth_syncing":
let res = await client.eth_syncing()
if res.syncing == false:
let syncing = ethNode.peerPool.connectedNodes.len > 0
let syncing = node.peerPool.connectedNodes.len > 0
check syncing == false
else:
check com.syncStart == res.syncObject.startingBlock.uint64
@ -307,41 +328,39 @@ proc rpcMain*() =
test "eth_accounts":
let res = await client.eth_accounts()
check signer in res
check ks2 in res
check ks3 in res
check contractAddress in res
check extraAddress in res
test "eth_blockNumber":
let res = await client.eth_blockNumber()
check res == w3Qty(0x1'u64)
test "eth_getBalance":
let a = await client.eth_getBalance(address"0xfff33a3bd36abdbd412707b8e310d6011454a7ae", blockId(1'u64))
check a == UInt256.fromHex("0x1b1ae4d6e2ef5000000")
let b = await client.eth_getBalance(address"0xfff4bad596633479a2a29f9a8b3f78eefd07e6ee", blockId(1'u64))
check b == UInt256.fromHex("0x56bc75e2d63100000")
let c = await client.eth_getBalance(address"0xfff7ac99c8e4feb60c9750054bdc14ce1857f181", blockId(1'u64))
check c == UInt256.fromHex("0x3635c9adc5dea00000")
let a = await client.eth_getBalance(signer, blockId(1'u64))
check a == 998739999997899997'u256
let b = await client.eth_getBalance(regularAcc, blockId(1'u64))
check b == 2_000_000_000.u256
let c = await client.eth_getBalance(contractAccWithStorage, blockId(1'u64))
check c == 1_000_000_000.u256
test "eth_getStorageAt":
let res = await client.eth_getStorageAt(address"0xfff33a3bd36abdbd412707b8e310d6011454a7ae", 0.u256, blockId(1'u64))
check FixedBytes[32](zeroHash32.data) == res
let res = await client.eth_getStorageAt(contractAccWithStorage, 1.u256, blockId(1'u64))
check FixedBytes[32](2345.u256.toBytesBE) == res
test "eth_getTransactionCount":
let res = await client.eth_getTransactionCount(address"0xfff7ac99c8e4feb60c9750054bdc14ce1857f181", blockId(1'u64))
check res == w3Qty(0'u64)
let res = await client.eth_getTransactionCount(signer, blockId(1'u64))
check res == w3Qty(2'u64)
test "eth_getBlockTransactionCountByHash":
let hash = com.db.getBlockHash(0'u64).expect("block hash exists")
let res = await client.eth_getBlockTransactionCountByHash(hash)
check res == w3Qty(0'u64)
let res = await client.eth_getBlockTransactionCountByHash(env.blockHash)
check res == w3Qty(2'u64)
test "eth_getBlockTransactionCountByNumber":
let res = await client.eth_getBlockTransactionCountByNumber(blockId(0'u64))
check res == w3Qty(0'u64)
let res = await client.eth_getBlockTransactionCountByNumber(blockId(1'u64))
check res == w3Qty(2'u64)
test "eth_getUncleCountByBlockHash":
let hash = com.db.getBlockHash(0'u64).expect("block hash exists")
let res = await client.eth_getUncleCountByBlockHash(hash)
let res = await client.eth_getUncleCountByBlockHash(env.blockHash)
check res == w3Qty(0'u64)
test "eth_getUncleCountByBlockNumber":
@ -349,15 +368,15 @@ proc rpcMain*() =
check res == w3Qty(0'u64)
test "eth_getCode":
let res = await client.eth_getCode(address"0xfff7ac99c8e4feb60c9750054bdc14ce1857f181", blockId(1'u64))
check res.len == 0
let res = await client.eth_getCode(contractAddress, blockId(1'u64))
check res.len == contractCode.len
test "eth_sign":
let msg = "hello world"
let msgBytes = @(msg.toOpenArrayByte(0, msg.len-1))
expect JsonRpcError:
discard await client.eth_sign(ks2, msgBytes)
discard await client.eth_sign(contractAddress, msgBytes)
let res = await client.eth_sign(signer, msgBytes)
let sig = Signature.fromRaw(res).tryGet()
@ -371,9 +390,9 @@ proc rpcMain*() =
check recoveredAddr == signer # verified
test "eth_signTransaction, eth_sendTransaction, eth_sendRawTransaction":
var unsignedTx = TransactionArgs(
let unsignedTx = TransactionArgs(
`from`: Opt.some(signer),
to: Opt.some(ks2),
to: Opt.some(contractAddress),
gas: Opt.some(w3Qty(100000'u)),
gasPrice: Opt.none(Quantity),
value: Opt.some(100.u256),
@ -389,9 +408,9 @@ proc rpcMain*() =
check hashAhex == hashBhex
test "eth_call":
var ec = TransactionArgs(
let ec = TransactionArgs(
`from`: Opt.some(signer),
to: Opt.some(ks2),
to: Opt.some(contractAddress),
gas: Opt.some(w3Qty(100000'u)),
gasPrice: Opt.none(Quantity),
value: Opt.some(100.u256)
@ -401,9 +420,9 @@ proc rpcMain*() =
check res == hexToSeqByte("deadbeef")
test "eth_estimateGas":
var ec = TransactionArgs(
let ec = TransactionArgs(
`from`: Opt.some(signer),
to: Opt.some(ks3),
to: Opt.some(extraAddress),
gas: Opt.some(w3Qty(42000'u)),
gasPrice: Opt.some(w3Qty(100'u)),
value: Opt.some(100.u256)
@ -452,29 +471,26 @@ proc rpcMain*() =
let res2 = await client.eth_getTransactionByBlockNumberAndIndex("latest", w3Qty(3'u64))
check res2.isNil
# TODO: Solved with Issue #2700
test "eth_getBlockReceipts":
let recs = await client.eth_getBlockReceipts(blockId(1'u64))
check recs.isSome
if recs.isSome:
let receipts = recs.get
check receipts.len == 2
check receipts[0].transactionIndex == 0.Quantity
check receipts[1].transactionIndex == 1.Quantity
# test "eth_getBlockReceipts":
# let recs = await client.eth_getBlockReceipts(blockId(1'u64))
# check recs.isSome
# if recs.isSome:
# let receipts = recs.get
# check receipts.len == 2
# check receipts[0].transactionIndex == 0.Quantity
# check receipts[1].transactionIndex == 1.Quantity
test "eth_getTransactionReceipt":
let res = await client.eth_getTransactionReceipt(env.txHash)
check res.isNil.not
check res.blockNumber == w3Qty(1'u64)
# test "eth_getTransactionReceipt":
# let res = await client.eth_getTransactionReceipt(env.txHash)
# check res.isNil.not
# check res.blockNumber == w3Qty(1'u64)
# let res2 = await client.eth_getTransactionReceipt(env.blockHash)
# check res2.isNil
let res2 = await client.eth_getTransactionReceipt(env.blockHash)
check res2.isNil
test "eth_getUncleByBlockHashAndIndex":
let res = await client.eth_getUncleByBlockHashAndIndex(env.blockHash, w3Qty(0'u64))
check res.isNil.not
check res.number == w3Qty(1'u64)
check res.isNil
let res2 = await client.eth_getUncleByBlockHashAndIndex(env.blockHash, w3Qty(1'u64))
check res2.isNil
@ -484,8 +500,7 @@ proc rpcMain*() =
test "eth_getUncleByBlockNumberAndIndex":
let res = await client.eth_getUncleByBlockNumberAndIndex("latest", w3Qty(0'u64))
check res.isNil.not
check res.number == w3Qty(1'u64)
check res.isNil
let res2 = await client.eth_getUncleByBlockNumberAndIndex("latest", w3Qty(1'u64))
check res2.isNil
@ -579,7 +594,7 @@ proc rpcMain*() =
block:
# account exists but requested slots don't exist
let
address = address"0x0000000000000000000000000000000000000001"
address = regularAcc
slot1Key = 0.u256
slot2Key = 1.u256
proofResponse = await client.eth_getProof(address, @[slot1Key, slot2Key], blockId(1'u64))
@ -603,7 +618,7 @@ proc rpcMain*() =
block:
# contract account with no storage slots
let
address = address"0x0000000000000000000000000000000000000003"
address = contractAccNoStorage
slot1Key = 0.u256 # Doesn't exist
proofResponse = await client.eth_getProof(address, @[slot1Key], blockId(1'u64))
storageProof = proofResponse.storageProof
@ -626,7 +641,7 @@ proc rpcMain*() =
block:
# contract account with storage slots
let
address = address"0x0000000000000000000000000000000000000002"
address = contractAccWithStorage
slot1Key = 0.u256
slot2Key = 1.u256
slot3Key = 2.u256 # Doesn't exist
@ -657,7 +672,7 @@ proc rpcMain*() =
block:
# externally owned account
let
address = address"0x0000000000000000000000000000000000000001"
address = regularAcc
proofResponse = await client.eth_getProof(address, @[], blockId(1'u64))
storageProof = proofResponse.storageProof
@ -676,7 +691,7 @@ proc rpcMain*() =
block:
# block 1 - account has balance, code and storage
let
address = address"0x0000000000000000000000000000000000000002"
address = contractAccWithStorage
slot2Key = 1.u256
proofResponse = await client.eth_getProof(address, @[slot2Key], blockId(1'u64))
storageProof = proofResponse.storageProof
@ -691,13 +706,7 @@ proc rpcMain*() =
storageProof.len() == 1
verifySlotProof(proofResponse.storageHash, storageProof[0]).isValid()
close(client, server)
proc setErrorLevel* =
discard
when defined(chronicles_runtime_filtering) and loggingEnabled:
setLogLevel(LogLevel.ERROR)
env.close()
when isMainModule:
setErrorLevel()
rpcMain()