Port p2p to server API (#2769)
* eth_gasPrice * signing endpoints * transaction by hash + temp fixes * fix CI * fix: state not persisted * decouple state access changes from this PR * rpc complete set * tests modified * tests temp modifications * add tests to CI + minor fixes * remove p2p * remove old dependency * fix suggestions * rework tests * rework kurtosis issue + comments * fix post bump issues * suggestions + logs * remove unused imports
This commit is contained in:
parent
58cde36656
commit
a45ac7e327
|
@ -83,7 +83,7 @@ jobs:
|
|||
cat kurtosis-network-params.yml | envsubst > assertoor.yaml
|
||||
sed -i "s/el_image: .*/el_image: localtestnet/" assertoor.yaml
|
||||
|
||||
kurtosis run github.com/ethpandaops/ethereum-package@4.3.0 --enclave assertoor-${{ github.run_id }} --args-file assertoor.yaml
|
||||
kurtosis run github.com/ethpandaops/ethereum-package --enclave assertoor-${{ github.run_id }} --args-file assertoor.yaml
|
||||
|
||||
enclave_dump=$(kurtosis enclave inspect assertoor-${{ github.run_id }})
|
||||
|
||||
|
|
|
@ -109,7 +109,7 @@ proc newEngineEnv*(conf: var NimbusConf, chainFile: string, enableAuth: bool): E
|
|||
beaconEngine = BeaconEngineRef.new(txPool, chain)
|
||||
serverApi = newServerAPI(chain, txPool)
|
||||
|
||||
setupServerAPI(serverApi, server)
|
||||
setupServerAPI(serverApi, server, ctx)
|
||||
setupEngineAPI(beaconEngine, server)
|
||||
# temporary disabled
|
||||
#setupDebugRpc(com, txPool, server)
|
||||
|
|
|
@ -59,7 +59,7 @@ proc setupELClient*(conf: ChainConfig, node: JsonNode): TestEnv =
|
|||
rpcServer = newRpcHttpServer(["127.0.0.1:0"])
|
||||
rpcClient = newRpcHttpClient()
|
||||
|
||||
setupServerAPI(serverApi, rpcServer)
|
||||
setupServerAPI(serverApi, rpcServer, newEthContext())
|
||||
setupEngineAPI(beaconEngine, rpcServer)
|
||||
|
||||
rpcServer.start()
|
||||
|
|
|
@ -16,8 +16,7 @@ import
|
|||
../../../nimbus/common,
|
||||
../../../nimbus/config,
|
||||
../../../nimbus/rpc,
|
||||
../../../nimbus/rpc/oracle,
|
||||
../../../nimbus/rpc/p2p,
|
||||
../../../nimbus/rpc/server_api,
|
||||
../../../nimbus/utils/utils,
|
||||
../../../nimbus/core/[chain, tx_pool],
|
||||
../../../tests/test_helpers,
|
||||
|
@ -46,11 +45,14 @@ proc manageAccounts(ctx: EthContext, conf: NimbusConf) =
|
|||
|
||||
proc setupRpcServer(ctx: EthContext, com: CommonRef,
|
||||
ethNode: EthereumNode, txPool: TxPoolRef,
|
||||
conf: NimbusConf): RpcServer =
|
||||
let rpcServer = newRpcHttpServer([initTAddress(conf.httpAddress, conf.httpPort)])
|
||||
let oracle = Oracle.new(com)
|
||||
conf: NimbusConf, chain: ForkedChainRef): RpcServer =
|
||||
let
|
||||
rpcServer = newRpcHttpServer([initTAddress(conf.httpAddress, conf.httpPort)])
|
||||
serverApi = newServerAPI(chain, txPool)
|
||||
|
||||
|
||||
setupCommonRpc(ethNode, conf, rpcServer)
|
||||
setupEthRpc(ethNode, ctx, com, txPool, oracle, rpcServer)
|
||||
setupServerAPI(serverApi, rpcServer, ctx)
|
||||
|
||||
rpcServer.start()
|
||||
rpcServer
|
||||
|
@ -92,7 +94,7 @@ proc setupEnv*(): TestEnv =
|
|||
# so it can know the latest account state
|
||||
doAssert txPool.smartHead(head, chainRef)
|
||||
|
||||
let rpcServer = setupRpcServer(ethCtx, com, ethNode, txPool, conf)
|
||||
let rpcServer = setupRpcServer(ethCtx, com, ethNode, txPool, conf, chainRef)
|
||||
let rpcClient = newRpcHttpClient()
|
||||
waitFor rpcClient.connect("127.0.0.1", Port(8545), false)
|
||||
let stopServer = stopRpcHttpServer
|
||||
|
|
|
@ -14,7 +14,7 @@ participants:
|
|||
el_extra_params: ["--log-level=DEBUG"]
|
||||
cl_type: nimbus
|
||||
cl_image: statusim/nimbus-eth2:multiarch-latest
|
||||
cl_extra_params: ["--log-level=DEBUG;INFO:gossip_eth2,attpool,libp2p,gossipsub,pubsubpeer,pubsub,switch,networking,sync,dialer,identify,syncman,connmanager,beacnde,lightcl,requman,gossip_lc,clearance,lpstream,mplexchannel,nodes-verification,tcptransport,chaindag,noise,eth,p2p,discv5,muxedupgrade,multistream,connection,secure,fee_recipient,mplex,syncpool,multiaddress,peer_proto;WARN:message_router"]
|
||||
cl_extra_params: ["--log-level=DEBUG"]
|
||||
use_separate_vc: false
|
||||
additional_services:
|
||||
- tx_spammer
|
||||
|
@ -23,7 +23,7 @@ additional_services:
|
|||
- blob_spammer
|
||||
mev_type: null
|
||||
assertoor_params:
|
||||
image: "ethpandaops/assertoor:latest"
|
||||
image: "ethpandaops/assertoor"
|
||||
run_stability_check: false
|
||||
run_block_proposal_check: true
|
||||
run_transaction_test: true
|
||||
|
|
|
@ -192,6 +192,11 @@ proc newPayload*(ben: BeaconEngineRef,
|
|||
hash = blockHash, number = header.number
|
||||
let vres = ben.chain.importBlock(blk)
|
||||
if vres.isErr:
|
||||
warn "Error importing block",
|
||||
number = header.number,
|
||||
hash = blockHash.short,
|
||||
parent = header.parentHash.short,
|
||||
error = vres.error()
|
||||
ben.setInvalidAncestor(header, blockHash)
|
||||
let blockHash = latestValidHash(db, parent, ttd)
|
||||
return invalidStatus(blockHash, vres.error())
|
||||
|
|
|
@ -164,7 +164,7 @@ proc validateBlock(c: ForkedChainRef,
|
|||
|
||||
ok()
|
||||
|
||||
proc replaySegment(c: ForkedChainRef, target: Hash32) =
|
||||
proc replaySegment*(c: ForkedChainRef, target: Hash32) =
|
||||
# Replay from base+1 to target block
|
||||
var
|
||||
prevHash = target
|
||||
|
@ -635,9 +635,18 @@ func baseHash*(c: ForkedChainRef): Hash32 =
|
|||
func txRecords*(c: ForkedChainRef, txHash: Hash32): (Hash32, uint64) =
|
||||
c.txRecords.getOrDefault(txHash, (Hash32.default, 0'u64))
|
||||
|
||||
func isInMemory*(c: ForkedChainRef, blockHash: Hash32): bool =
|
||||
c.blocks.hasKey(blockHash)
|
||||
|
||||
func memoryBlock*(c: ForkedChainRef, blockHash: Hash32): BlockDesc =
|
||||
c.blocks.getOrDefault(blockHash)
|
||||
|
||||
func memoryTransaction*(c: ForkedChainRef, txHash: Hash32): Opt[Transaction] =
|
||||
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)
|
||||
|
||||
proc latestBlock*(c: ForkedChainRef): Block =
|
||||
c.blocks.withValue(c.cursorHash, val) do:
|
||||
return val.blk
|
||||
|
|
|
@ -53,7 +53,7 @@ func installRPC(server: RpcServer,
|
|||
setupCommonRpc(nimbus.ethNode, conf, server)
|
||||
|
||||
if RpcFlag.Eth in flags:
|
||||
setupServerAPI(serverApi, server)
|
||||
setupServerAPI(serverApi, server, nimbus.ctx)
|
||||
|
||||
# # Tracer is currently disabled
|
||||
# if RpcFlag.Debug in flags:
|
||||
|
|
|
@ -1,607 +0,0 @@
|
|||
# Nimbus
|
||||
# Copyright (c) 2018-2024 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.
|
||||
|
||||
{.push raises: [].}
|
||||
|
||||
import
|
||||
std/[sequtils, times, tables, typetraits],
|
||||
json_rpc/rpcserver,
|
||||
stint,
|
||||
stew/byteutils,
|
||||
json_serialization,
|
||||
web3/conversions,
|
||||
json_serialization/stew/results,
|
||||
eth/common/eth_types_json_serialization,
|
||||
eth/[rlp, p2p],
|
||||
".."/[transaction, evm/state, constants],
|
||||
../db/ledger,
|
||||
./rpc_types, ./rpc_utils, ./oracle,
|
||||
../transaction/call_evm,
|
||||
../core/tx_pool,
|
||||
../core/eip4844,
|
||||
../common/[common, context],
|
||||
../utils/utils,
|
||||
../beacon/web3_eth_conv,
|
||||
../evm/evm_errors,
|
||||
./filters
|
||||
|
||||
type
|
||||
Header = eth_types.Header
|
||||
Hash32 = eth_types.Hash32
|
||||
|
||||
proc getProof*(
|
||||
accDB: LedgerRef,
|
||||
address: eth_types.Address,
|
||||
slots: seq[UInt256]): ProofResponse =
|
||||
let
|
||||
acc = accDB.getEthAccount(address)
|
||||
accExists = accDB.accountExists(address)
|
||||
accountProof = accDB.getAccountProof(address)
|
||||
slotProofs = accDB.getStorageProof(address, slots)
|
||||
|
||||
var storage = newSeqOfCap[StorageProof](slots.len)
|
||||
|
||||
for i, slotKey in slots:
|
||||
let slotValue = accDB.getStorage(address, slotKey)
|
||||
storage.add(StorageProof(
|
||||
key: slotKey,
|
||||
value: slotValue,
|
||||
proof: seq[RlpEncodedBytes](slotProofs[i])))
|
||||
|
||||
if accExists:
|
||||
ProofResponse(
|
||||
address: address,
|
||||
accountProof: seq[RlpEncodedBytes](accountProof),
|
||||
balance: acc.balance,
|
||||
nonce: w3Qty(acc.nonce),
|
||||
codeHash: acc.codeHash,
|
||||
storageHash: acc.storageRoot,
|
||||
storageProof: storage)
|
||||
else:
|
||||
ProofResponse(
|
||||
address: address,
|
||||
accountProof: seq[RlpEncodedBytes](accountProof),
|
||||
storageProof: storage)
|
||||
|
||||
proc setupEthRpc*(
|
||||
node: EthereumNode, ctx: EthContext, com: CommonRef,
|
||||
txPool: TxPoolRef, oracle: Oracle, server: RpcServer) =
|
||||
|
||||
let chainDB = com.db
|
||||
proc getStateDB(header:Header): LedgerRef =
|
||||
## Retrieves the account db from canonical head
|
||||
# we don't use accounst_cache here because it's only read operations
|
||||
LedgerRef.init(chainDB)
|
||||
|
||||
proc stateDBFromTag(quantityTag: BlockTag, readOnly = true): LedgerRef
|
||||
{.gcsafe, raises: [CatchableError].} =
|
||||
getStateDB(chainDB.headerFromTag(quantityTag))
|
||||
|
||||
server.rpc("eth_chainId") do() -> Web3Quantity:
|
||||
return w3Qty(distinctBase(com.chainId))
|
||||
|
||||
server.rpc("eth_syncing") do() -> SyncingStatus:
|
||||
## Returns SyncObject or false when not syncing.
|
||||
if com.syncState != Waiting:
|
||||
let sync = SyncObject(
|
||||
startingBlock: w3Qty com.syncStart,
|
||||
currentBlock : w3Qty com.syncCurrent,
|
||||
highestBlock : w3Qty com.syncHighest
|
||||
)
|
||||
return SyncingStatus(syncing: true, syncObject: sync)
|
||||
else:
|
||||
return SyncingStatus(syncing: false)
|
||||
|
||||
server.rpc("eth_gasPrice") do() -> Web3Quantity:
|
||||
## Returns an integer of the current gas price in wei.
|
||||
w3Qty(calculateMedianGasPrice(chainDB).uint64)
|
||||
|
||||
server.rpc("eth_accounts") do() -> seq[eth_types.Address]:
|
||||
## Returns a list of addresses owned by client.
|
||||
result = newSeqOfCap[eth_types.Address](ctx.am.numAccounts)
|
||||
for k in ctx.am.addresses:
|
||||
result.add k
|
||||
|
||||
server.rpc("eth_blockNumber") do() -> Web3Quantity:
|
||||
## Returns integer of the current block number the client is on.
|
||||
w3Qty(chainDB.getCanonicalHead().number)
|
||||
|
||||
server.rpc("eth_getBalance") do(data: eth_types.Address, quantityTag: BlockTag) -> UInt256:
|
||||
## Returns the balance of the account of given address.
|
||||
##
|
||||
## data: address to check for balance.
|
||||
## quantityTag: integer block number, or the string "latest", "earliest" or "pending", see the default block parameter.
|
||||
## Returns integer of the current balance in wei.
|
||||
let
|
||||
accDB = stateDBFromTag(quantityTag)
|
||||
address = data
|
||||
accDB.getBalance(address)
|
||||
|
||||
server.rpc("eth_getStorageAt") do(data: eth_types.Address, slot: UInt256, quantityTag: BlockTag) -> eth_types.FixedBytes[32]:
|
||||
## Returns the value from a storage position at a given address.
|
||||
##
|
||||
## data: address of the storage.
|
||||
## slot: integer of the position in the storage.
|
||||
## quantityTag: integer block number, or the string "latest", "earliest" or "pending", see the default block parameter.
|
||||
## Returns: the value at this storage position.
|
||||
let
|
||||
accDB = stateDBFromTag(quantityTag)
|
||||
address = data
|
||||
data = accDB.getStorage(address, slot)
|
||||
data.to(Bytes32)
|
||||
|
||||
server.rpc("eth_getTransactionCount") do(data: eth_types.Address, quantityTag: BlockTag) -> Web3Quantity:
|
||||
## Returns the number of transactions sent from an address.
|
||||
##
|
||||
## data: address.
|
||||
## quantityTag: integer block number, or the string "latest", "earliest" or "pending", see the default block parameter.
|
||||
## Returns integer of the number of transactions send from this address.
|
||||
let
|
||||
address = data
|
||||
accDB = stateDBFromTag(quantityTag)
|
||||
w3Qty(accDB.getNonce(address))
|
||||
|
||||
server.rpc("eth_getBlockTransactionCountByHash") do(data: Hash32) -> Web3Quantity:
|
||||
## Returns the number of transactions in a block from a block matching the given block hash.
|
||||
##
|
||||
## data: hash of a block
|
||||
## Returns integer of the number of transactions in this block.
|
||||
let
|
||||
blockHash = data
|
||||
header = chainDB.getBlockHeader(blockHash)
|
||||
txCount = chainDB.getTransactionCount(header.txRoot)
|
||||
Web3Quantity(txCount)
|
||||
|
||||
server.rpc("eth_getBlockTransactionCountByNumber") do(quantityTag: BlockTag) -> Web3Quantity:
|
||||
## Returns the number of transactions in a block matching the given block number.
|
||||
##
|
||||
## data: integer of a block number, or the string "earliest", "latest" or "pending", as in the default block parameter.
|
||||
## Returns integer of the number of transactions in this block.
|
||||
let
|
||||
header = chainDB.headerFromTag(quantityTag)
|
||||
txCount = chainDB.getTransactionCount(header.txRoot)
|
||||
Web3Quantity(txCount)
|
||||
|
||||
server.rpc("eth_getUncleCountByBlockHash") do(data: Hash32) -> Web3Quantity:
|
||||
## Returns the number of uncles in a block from a block matching the given block hash.
|
||||
##
|
||||
## data: hash of a block.
|
||||
## Returns integer of the number of uncles in this block.
|
||||
let
|
||||
blockHash = data
|
||||
header = chainDB.getBlockHeader(blockHash)
|
||||
unclesCount = chainDB.getUnclesCount(header.ommersHash)
|
||||
Web3Quantity(unclesCount)
|
||||
|
||||
server.rpc("eth_getUncleCountByBlockNumber") do(quantityTag: BlockTag) -> Web3Quantity:
|
||||
## Returns the number of uncles in a block from a block matching the given block number.
|
||||
##
|
||||
## quantityTag: integer of a block number, or the string "latest", "earliest" or "pending", see the default block parameter.
|
||||
## Returns integer of uncles in this block.
|
||||
let
|
||||
header = chainDB.headerFromTag(quantityTag)
|
||||
unclesCount = chainDB.getUnclesCount(header.ommersHash)
|
||||
Web3Quantity(unclesCount)
|
||||
|
||||
server.rpc("eth_getCode") do(data: eth_types.Address, quantityTag: BlockTag) -> seq[byte]:
|
||||
## Returns code at a given address.
|
||||
##
|
||||
## data: address
|
||||
## quantityTag: integer block number, or the string "latest", "earliest" or "pending", see the default block parameter.
|
||||
## Returns the code from the given address.
|
||||
let
|
||||
accDB = stateDBFromTag(quantityTag)
|
||||
address = data
|
||||
accDB.getCode(address).bytes()
|
||||
|
||||
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]:
|
||||
## 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.
|
||||
## Note the address to sign with must be unlocked.
|
||||
##
|
||||
## data: address.
|
||||
## message: message to sign.
|
||||
## Returns signature.
|
||||
let
|
||||
address = data
|
||||
acc = ctx.am.getAccount(address).tryGet()
|
||||
|
||||
if not acc.unlocked:
|
||||
raise newException(ValueError, "Account locked, please unlock it first")
|
||||
sign(acc.privateKey, cast[string](message))
|
||||
|
||||
server.rpc("eth_signTransaction") do(data: TransactionArgs) -> seq[byte]:
|
||||
## Signs a transaction that can be submitted to the network at a later time using with
|
||||
## eth_sendRawTransaction
|
||||
let
|
||||
address = data.`from`.get()
|
||||
acc = ctx.am.getAccount(address).tryGet()
|
||||
|
||||
if not acc.unlocked:
|
||||
raise newException(ValueError, "Account locked, please unlock it first")
|
||||
|
||||
let
|
||||
accDB = stateDBFromTag(blockId("latest"))
|
||||
tx = unsignedTx(data, chainDB, accDB.getNonce(address) + 1, com.chainId)
|
||||
eip155 = com.isEIP155(com.syncCurrent)
|
||||
signedTx = signTransaction(tx, acc.privateKey, eip155)
|
||||
result = rlp.encode(signedTx)
|
||||
|
||||
server.rpc("eth_sendTransaction") do(data: TransactionArgs) -> Hash32:
|
||||
## Creates new message call transaction or a contract creation, if the data field contains code.
|
||||
##
|
||||
## obj: the transaction object.
|
||||
## Returns the transaction hash, or the zero hash if the transaction is not yet available.
|
||||
## Note: Use eth_getTransactionReceipt to get the contract address, after the transaction was mined, when you created a contract.
|
||||
let
|
||||
address = data.`from`.get()
|
||||
acc = ctx.am.getAccount(address).tryGet()
|
||||
|
||||
if not acc.unlocked:
|
||||
raise newException(ValueError, "Account locked, please unlock it first")
|
||||
|
||||
let
|
||||
accDB = stateDBFromTag(blockId("latest"))
|
||||
tx = unsignedTx(data, chainDB, accDB.getNonce(address) + 1, com.chainId)
|
||||
eip155 = com.isEIP155(com.syncCurrent)
|
||||
signedTx = signTransaction(tx, acc.privateKey, eip155)
|
||||
networkPayload =
|
||||
if signedTx.txType == TxEip4844:
|
||||
if data.blobs.isNone or data.commitments.isNone or data.proofs.isNone:
|
||||
raise newException(ValueError, "EIP-4844 transaction needs blobs")
|
||||
if data.blobs.get.len != signedTx.versionedHashes.len:
|
||||
raise newException(ValueError, "Incorrect number of blobs")
|
||||
if data.commitments.get.len != signedTx.versionedHashes.len:
|
||||
raise newException(ValueError, "Incorrect number of commitments")
|
||||
if data.proofs.get.len != signedTx.versionedHashes.len:
|
||||
raise newException(ValueError, "Incorrect number of proofs")
|
||||
NetworkPayload(
|
||||
blobs: data.blobs.get.mapIt it.NetworkBlob,
|
||||
commitments: data.commitments.get,
|
||||
proofs: data.proofs.get)
|
||||
else:
|
||||
if data.blobs.isSome or data.commitments.isSome or data.proofs.isSome:
|
||||
raise newException(ValueError, "Blobs require EIP-4844 transaction")
|
||||
nil
|
||||
pooledTx = PooledTransaction(tx: signedTx, networkPayload: networkPayload)
|
||||
|
||||
txPool.add(pooledTx)
|
||||
rlpHash(signedTx)
|
||||
|
||||
server.rpc("eth_sendRawTransaction") do(txBytes: seq[byte]) -> Hash32:
|
||||
## Creates new message call transaction or a contract creation for signed transactions.
|
||||
##
|
||||
## data: the signed transaction data.
|
||||
## Returns the transaction hash, or the zero hash if the transaction is not yet available.
|
||||
## Note: Use eth_getTransactionReceipt to get the contract address, after the transaction was mined, when you created a contract.
|
||||
let
|
||||
pooledTx = decodePooledTx(txBytes)
|
||||
txHash = rlpHash(pooledTx)
|
||||
|
||||
txPool.add(pooledTx)
|
||||
let res = txPool.inPoolAndReason(txHash)
|
||||
if res.isErr:
|
||||
raise newException(ValueError, res.error)
|
||||
txHash
|
||||
|
||||
server.rpc("eth_call") do(args: TransactionArgs, quantityTag: BlockTag) -> seq[byte]:
|
||||
## Executes a new message call immediately without creating a transaction on the block chain.
|
||||
##
|
||||
## call: the transaction call object.
|
||||
## quantityTag: integer block number, or the string "latest", "earliest" or "pending", see the default block parameter.
|
||||
## Returns the return value of executed contract.
|
||||
let
|
||||
header = headerFromTag(chainDB, quantityTag)
|
||||
res = rpcCallEvm(args, header, com).valueOr:
|
||||
raise newException(ValueError, "rpcCallEvm error: " & $error.code)
|
||||
res.output
|
||||
|
||||
server.rpc("eth_estimateGas") do(args: TransactionArgs) -> Web3Quantity:
|
||||
## 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.
|
||||
##
|
||||
## args: the transaction call object.
|
||||
## quantityTag: integer block number, or the string "latest", "earliest" or "pending", see the default block parameter.
|
||||
## Returns the amount of gas used.
|
||||
let
|
||||
header = chainDB.headerFromTag(blockId("latest"))
|
||||
# TODO: DEFAULT_RPC_GAS_CAP should configurable
|
||||
gasUsed = rpcEstimateGas(args, header, com, DEFAULT_RPC_GAS_CAP).valueOr:
|
||||
raise newException(ValueError, "rpcEstimateGas error: " & $error.code)
|
||||
w3Qty(gasUsed)
|
||||
|
||||
server.rpc("eth_getBlockByHash") do(data: Hash32, fullTransactions: bool) -> BlockObject:
|
||||
## Returns information about a block by hash.
|
||||
##
|
||||
## data: Hash of a block.
|
||||
## fullTransactions: If true it returns the full transaction objects, if false only the hashes of the transactions.
|
||||
## Returns BlockObject or nil when no block was found.
|
||||
var
|
||||
header:Header
|
||||
hash = data
|
||||
|
||||
if chainDB.getBlockHeader(hash, header):
|
||||
populateBlockObject(header, chainDB, fullTransactions)
|
||||
else:
|
||||
nil
|
||||
|
||||
server.rpc("eth_getBlockByNumber") do(quantityTag: BlockTag, fullTransactions: bool) -> BlockObject:
|
||||
## Returns information about a block by block number.
|
||||
##
|
||||
## quantityTag: integer of a block number, or the string "earliest", "latest" or "pending", as in the default block parameter.
|
||||
## fullTransactions: If true it returns the full transaction objects, if false only the hashes of the transactions.
|
||||
## Returns BlockObject or nil when no block was found.
|
||||
try:
|
||||
let header = chainDB.headerFromTag(quantityTag)
|
||||
populateBlockObject(header, chainDB, fullTransactions)
|
||||
except CatchableError:
|
||||
nil
|
||||
|
||||
server.rpc("eth_getTransactionByHash") do(data: Hash32) -> TransactionObject:
|
||||
## Returns the information about a transaction requested by transaction hash.
|
||||
##
|
||||
## data: hash of a transaction.
|
||||
## Returns requested transaction information.
|
||||
let txHash = data
|
||||
let res = txPool.getItem(txHash)
|
||||
if res.isOk:
|
||||
return populateTransactionObject(res.get().tx)
|
||||
|
||||
let txDetails = chainDB.getTransactionKey(txHash)
|
||||
if txDetails.index < 0:
|
||||
return nil
|
||||
|
||||
let header = chainDB.getBlockHeader(txDetails.blockNumber)
|
||||
var tx: Transaction
|
||||
if chainDB.getTransactionByIndex(header.txRoot, uint16(txDetails.index), tx):
|
||||
result = populateTransactionObject(tx, Opt.some(header), Opt.some(txDetails.index))
|
||||
|
||||
server.rpc("eth_getTransactionByBlockHashAndIndex") do(data: Hash32, quantity: Web3Quantity) -> TransactionObject:
|
||||
## Returns information about a transaction by block hash and transaction index position.
|
||||
##
|
||||
## data: hash of a block.
|
||||
## quantity: integer of the transaction index position.
|
||||
## Returns requested transaction information.
|
||||
let index = uint64(quantity)
|
||||
var header:Header
|
||||
if not chainDB.getBlockHeader(data, header):
|
||||
return nil
|
||||
|
||||
var tx: Transaction
|
||||
if chainDB.getTransactionByIndex(header.txRoot, uint16(index), tx):
|
||||
populateTransactionObject(tx, Opt.some(header), Opt.some(index))
|
||||
else:
|
||||
nil
|
||||
|
||||
server.rpc("eth_getTransactionByBlockNumberAndIndex") do(quantityTag: BlockTag, quantity: Web3Quantity) -> TransactionObject:
|
||||
## Returns information about a transaction by block number and transaction index position.
|
||||
##
|
||||
## quantityTag: a block number, or the string "earliest", "latest" or "pending", as in the default block parameter.
|
||||
## quantity: the transaction index position.
|
||||
let
|
||||
header = chainDB.headerFromTag(quantityTag)
|
||||
index = uint64(quantity)
|
||||
|
||||
var tx: Transaction
|
||||
if chainDB.getTransactionByIndex(header.txRoot, uint16(index), tx):
|
||||
populateTransactionObject(tx, Opt.some(header), Opt.some(index))
|
||||
else:
|
||||
nil
|
||||
|
||||
server.rpc("eth_getTransactionReceipt") do(data: Hash32) -> ReceiptObject:
|
||||
## Returns the receipt of a transaction by transaction hash.
|
||||
##
|
||||
## data: hash of a transaction.
|
||||
## Returns transaction receipt.
|
||||
|
||||
let txDetails = chainDB.getTransactionKey(data)
|
||||
if txDetails.index < 0:
|
||||
return nil
|
||||
|
||||
let header = chainDB.getBlockHeader(txDetails.blockNumber)
|
||||
var tx: Transaction
|
||||
if not chainDB.getTransactionByIndex(header.txRoot, uint16(txDetails.index), tx):
|
||||
return nil
|
||||
|
||||
var
|
||||
idx = 0'u64
|
||||
prevGasUsed = GasInt(0)
|
||||
|
||||
for receipt in chainDB.getReceipts(header.receiptsRoot):
|
||||
let gasUsed = receipt.cumulativeGasUsed - prevGasUsed
|
||||
prevGasUsed = receipt.cumulativeGasUsed
|
||||
if idx == txDetails.index:
|
||||
return populateReceipt(receipt, gasUsed, tx, txDetails.index, header)
|
||||
idx.inc
|
||||
|
||||
server.rpc("eth_getUncleByBlockHashAndIndex") do(data: Hash32, quantity: Web3Quantity) -> BlockObject:
|
||||
## Returns information about a uncle of a block by hash and uncle index position.
|
||||
##
|
||||
## data: hash of block.
|
||||
## quantity: the uncle's index position.
|
||||
## Returns BlockObject or nil when no block was found.
|
||||
let index = uint64(quantity)
|
||||
var header:Header
|
||||
if not chainDB.getBlockHeader(data, header):
|
||||
return nil
|
||||
|
||||
let uncles = chainDB.getUncles(header.ommersHash)
|
||||
if index < 0 or index >= uncles.len.uint64:
|
||||
return nil
|
||||
|
||||
result = populateBlockObject(uncles[index], chainDB, false, true)
|
||||
result.totalDifficulty = chainDB.getScore(header.blockHash).valueOr(0.u256)
|
||||
|
||||
server.rpc("eth_getUncleByBlockNumberAndIndex") do(quantityTag: BlockTag, quantity: Web3Quantity) -> BlockObject:
|
||||
# Returns information about a uncle of a block by number and uncle index position.
|
||||
##
|
||||
## quantityTag: a block number, or the string "earliest", "latest" or "pending", as in the default block parameter.
|
||||
## quantity: the uncle's index position.
|
||||
## Returns BlockObject or nil when no block was found.
|
||||
let
|
||||
index = uint64(quantity)
|
||||
header = chainDB.headerFromTag(quantityTag)
|
||||
uncles = chainDB.getUncles(header.ommersHash)
|
||||
|
||||
if index < 0 or index >= uncles.len.uint64:
|
||||
return nil
|
||||
|
||||
result = populateBlockObject(uncles[index], chainDB, false, true)
|
||||
result.totalDifficulty = chainDB.getScore(header.blockHash).valueOr(0.u256)
|
||||
|
||||
proc getLogsForBlock(
|
||||
chain: CoreDbRef,
|
||||
hash: Hash32,
|
||||
header:Header,
|
||||
opts: FilterOptions): seq[FilterLog]
|
||||
{.gcsafe, raises: [RlpError,BlockNotFound].} =
|
||||
if headerBloomFilter(header, opts.address, opts.topics):
|
||||
let blockBody = chain.getBlockBody(hash)
|
||||
let receipts = chain.getReceipts(header.receiptsRoot)
|
||||
# Note: this will hit assertion error if number of block transactions
|
||||
# do not match block receipts.
|
||||
# Although this is fine as number of receipts should always match number
|
||||
# of transactions
|
||||
let logs = deriveLogs(header, blockBody.transactions, receipts)
|
||||
let filteredLogs = filterLogs(logs, opts.address, opts.topics)
|
||||
return filteredLogs
|
||||
else:
|
||||
return @[]
|
||||
|
||||
proc getLogsForRange(
|
||||
chain: CoreDbRef,
|
||||
start: common.BlockNumber,
|
||||
finish: common.BlockNumber,
|
||||
opts: FilterOptions): seq[FilterLog]
|
||||
{.gcsafe, raises: [RlpError,BlockNotFound].} =
|
||||
var logs = newSeq[FilterLog]()
|
||||
var i = start
|
||||
while i <= finish:
|
||||
let res = chain.getBlockHeaderWithHash(i)
|
||||
if res.isSome():
|
||||
let (hash, header)= res.unsafeGet()
|
||||
let filtered = chain.getLogsForBlock(header, hash, opts)
|
||||
logs.add(filtered)
|
||||
else:
|
||||
#
|
||||
return logs
|
||||
i = i + 1
|
||||
return logs
|
||||
|
||||
server.rpc("eth_getLogs") do(filterOptions: FilterOptions) -> seq[FilterLog]:
|
||||
## filterOptions: settings for this filter.
|
||||
## Returns a list of all logs matching a given filter object.
|
||||
## TODO: Current implementation is pretty naive and not efficient
|
||||
## as it requires to fetch all transactions and all receipts from database.
|
||||
## Other clients (Geth):
|
||||
## - Store logs related data in receipts.
|
||||
## - Have separate indexes for Logs in given block
|
||||
## Both of those changes require improvements to the way how we keep our data
|
||||
## in Nimbus.
|
||||
if filterOptions.blockHash.isSome():
|
||||
let hash = filterOptions.blockHash.unsafeGet()
|
||||
let header = chainDB.getBlockHeader(hash)
|
||||
return getLogsForBlock(chainDB, hash, header, filterOptions)
|
||||
else:
|
||||
# TODO: do something smarter with tags. It would be the best if
|
||||
# tag would be an enum (Earliest, Latest, Pending, Number), and all operations
|
||||
# would operate on this enum instead of raw strings. This change would need
|
||||
# to be done on every endpoint to be consistent.
|
||||
let fromHeader = chainDB.headerFromTag(filterOptions.fromBlock)
|
||||
let toHeader = chainDB.headerFromTag(filterOptions.toBlock)
|
||||
|
||||
# Note: if fromHeader.number > toHeader.number, no logs will be
|
||||
# returned. This is consistent with, what other ethereum clients return
|
||||
let logs = chainDB.getLogsForRange(
|
||||
fromHeader.number,
|
||||
toHeader.number,
|
||||
filterOptions
|
||||
)
|
||||
return logs
|
||||
|
||||
server.rpc("eth_getProof") do(data: eth_types.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
|
||||
## existence of the values in the state.
|
||||
## See spec here: https://eips.ethereum.org/EIPS/eip-1186
|
||||
##
|
||||
## data: address of the account.
|
||||
## slots: integers of the positions in the storage to return with storage proofs.
|
||||
## quantityTag: integer block number, or the string "latest", "earliest" or "pending", see the default block parameter.
|
||||
## Returns: the proof response containing the account, account proof and storage proof
|
||||
|
||||
let
|
||||
accDB = stateDBFromTag(quantityTag)
|
||||
address = data
|
||||
|
||||
getProof(accDB, address, slots)
|
||||
|
||||
server.rpc("eth_getBlockReceipts") do(quantityTag: BlockTag) -> Opt[seq[ReceiptObject]]:
|
||||
## Returns the receipts of a block.
|
||||
try:
|
||||
let header = chainDB.headerFromTag(quantityTag)
|
||||
var
|
||||
prevGasUsed = GasInt(0)
|
||||
recs: seq[ReceiptObject]
|
||||
txs: seq[Transaction]
|
||||
index = 0'u64
|
||||
|
||||
for tx in chainDB.getBlockTransactions(header):
|
||||
txs.add tx
|
||||
|
||||
for receipt in chainDB.getReceipts(header.receiptsRoot):
|
||||
let gasUsed = receipt.cumulativeGasUsed - prevGasUsed
|
||||
prevGasUsed = receipt.cumulativeGasUsed
|
||||
recs.add populateReceipt(receipt, gasUsed, txs[index], index, header)
|
||||
inc index
|
||||
|
||||
return Opt.some(recs)
|
||||
except CatchableError:
|
||||
return Opt.none(seq[ReceiptObject])
|
||||
|
||||
server.rpc("eth_createAccessList") do(args: TransactionArgs, quantityTag: BlockTag) -> AccessListResult:
|
||||
## Generates an access list for a transaction.
|
||||
try:
|
||||
let
|
||||
header = chainDB.headerFromTag(quantityTag)
|
||||
return createAccessList(header, com, args)
|
||||
except CatchableError as exc:
|
||||
return AccessListResult(
|
||||
error: Opt.some("createAccessList error: " & exc.msg),
|
||||
)
|
||||
|
||||
server.rpc("eth_blobBaseFee") do() -> Web3Quantity:
|
||||
## Returns the base fee per blob gas in wei.
|
||||
let header = chainDB.headerFromTag(blockId("latest"))
|
||||
if header.blobGasUsed.isNone:
|
||||
raise newException(ValueError, "blobGasUsed missing from latest header")
|
||||
if header.excessBlobGas.isNone:
|
||||
raise newException(ValueError, "excessBlobGas missing from latest header")
|
||||
let blobBaseFee = getBlobBaseFee(header.excessBlobGas.get) * header.blobGasUsed.get.u256
|
||||
if blobBaseFee > high(uint64).u256:
|
||||
raise newException(ValueError, "blobBaseFee is bigger than uint64.max")
|
||||
return w3Qty blobBaseFee.truncate(uint64)
|
||||
|
||||
server.rpc("eth_feeHistory") do(blockCount: Quantity,
|
||||
newestBlock: BlockTag,
|
||||
rewardPercentiles: Opt[seq[float64]]) -> FeeHistoryResult:
|
||||
let
|
||||
blocks = blockCount.uint64
|
||||
percentiles = rewardPercentiles.get(newSeq[float64]())
|
||||
res = feeHistory(oracle, blocks, newestBlock, percentiles)
|
||||
if res.isErr:
|
||||
raise newException(ValueError, res.error)
|
||||
return res.get
|
|
@ -10,7 +10,7 @@
|
|||
{.push raises: [].}
|
||||
|
||||
import
|
||||
std/[strutils, algorithm],
|
||||
std/[sequtils, algorithm],
|
||||
./rpc_types,
|
||||
./params,
|
||||
../db/core_db,
|
||||
|
@ -29,37 +29,6 @@ import
|
|||
../common/common,
|
||||
web3/eth_api_types
|
||||
|
||||
const
|
||||
defaultTag = blockId("latest")
|
||||
|
||||
proc headerFromTag*(chain: CoreDbRef, blockId: BlockTag): Header
|
||||
{.gcsafe, raises: [CatchableError].} =
|
||||
|
||||
if blockId.kind == bidAlias:
|
||||
let tag = blockId.alias.toLowerAscii
|
||||
case tag
|
||||
of "latest": result = chain.getCanonicalHead()
|
||||
of "earliest": result = chain.getBlockHeader(GENESIS_BLOCK_NUMBER)
|
||||
of "safe": result = chain.safeHeader()
|
||||
of "finalized": result = chain.finalizedHeader()
|
||||
of "pending":
|
||||
#TODO: Implement get pending block
|
||||
# We currently fall back to `latest` so that the `tx-spammer` in
|
||||
# `ethpandaops/ethereum-package` can make progress. A real
|
||||
# implementation is still required that takes into account any
|
||||
# pending transactions that have not yet been bundled into a block.
|
||||
result = chain.getCanonicalHead()
|
||||
else:
|
||||
raise newException(ValueError, "Unsupported block tag " & tag)
|
||||
else:
|
||||
let blockNum = blockId.number.uint64
|
||||
result = chain.getBlockHeader(blockNum)
|
||||
|
||||
proc headerFromTag*(chain: CoreDbRef, blockTag: Opt[BlockTag]): Header
|
||||
{.gcsafe, raises: [CatchableError].} =
|
||||
let blockId = blockTag.get(defaultTag)
|
||||
chain.headerFromTag(blockId)
|
||||
|
||||
proc calculateMedianGasPrice*(chain: CoreDbRef): GasInt
|
||||
{.gcsafe, raises: [CatchableError].} =
|
||||
var prices = newSeqOfCap[GasInt](64)
|
||||
|
@ -90,31 +59,36 @@ proc calculateMedianGasPrice*(chain: CoreDbRef): GasInt
|
|||
|
||||
proc unsignedTx*(tx: TransactionArgs, chain: CoreDbRef, defaultNonce: AccountNonce, chainId: ChainId): Transaction
|
||||
{.gcsafe, raises: [CatchableError].} =
|
||||
|
||||
var res: Transaction
|
||||
|
||||
if tx.to.isSome:
|
||||
result.to = Opt.some(tx.to.get)
|
||||
res.to = Opt.some(tx.to.get)
|
||||
|
||||
if tx.gas.isSome:
|
||||
result.gasLimit = tx.gas.get.GasInt
|
||||
res.gasLimit = tx.gas.get.GasInt
|
||||
else:
|
||||
result.gasLimit = 90000.GasInt
|
||||
res.gasLimit = 90000.GasInt
|
||||
|
||||
if tx.gasPrice.isSome:
|
||||
result.gasPrice = tx.gasPrice.get.GasInt
|
||||
res.gasPrice = tx.gasPrice.get.GasInt
|
||||
else:
|
||||
result.gasPrice = calculateMedianGasPrice(chain)
|
||||
res.gasPrice = calculateMedianGasPrice(chain)
|
||||
|
||||
if tx.value.isSome:
|
||||
result.value = tx.value.get
|
||||
res.value = tx.value.get
|
||||
else:
|
||||
result.value = 0.u256
|
||||
res.value = 0.u256
|
||||
|
||||
if tx.nonce.isSome:
|
||||
result.nonce = tx.nonce.get.AccountNonce
|
||||
res.nonce = tx.nonce.get.AccountNonce
|
||||
else:
|
||||
result.nonce = defaultNonce
|
||||
res.nonce = defaultNonce
|
||||
|
||||
result.payload = tx.payload
|
||||
result.chainId = chainId
|
||||
res.payload = tx.payload
|
||||
res.chainId = chainId
|
||||
|
||||
return res
|
||||
|
||||
proc toWd(wd: Withdrawal): WithdrawalObject =
|
||||
WithdrawalObject(
|
||||
|
@ -125,116 +99,123 @@ proc toWd(wd: Withdrawal): WithdrawalObject =
|
|||
)
|
||||
|
||||
proc toWdList(list: openArray[Withdrawal]): seq[WithdrawalObject] =
|
||||
result = newSeqOfCap[WithdrawalObject](list.len)
|
||||
var res = newSeqOfCap[WithdrawalObject](list.len)
|
||||
for x in list:
|
||||
result.add toWd(x)
|
||||
res.add toWd(x)
|
||||
return res
|
||||
|
||||
func toWdList(x: Opt[seq[eth_types.Withdrawal]]):
|
||||
Opt[seq[WithdrawalObject]] =
|
||||
if x.isNone: Opt.none(seq[WithdrawalObject])
|
||||
else: Opt.some(toWdList x.get)
|
||||
|
||||
proc populateTransactionObject*(tx: Transaction,
|
||||
optionalHeader: Opt[Header] = Opt.none(Header),
|
||||
optionalHash: Opt[eth_types.Hash32] = Opt.none(eth_types.Hash32),
|
||||
optionalNumber: Opt[eth_types.BlockNumber] = Opt.none(eth_types.BlockNumber),
|
||||
txIndex: Opt[uint64] = Opt.none(uint64)): TransactionObject =
|
||||
result = TransactionObject()
|
||||
result.`type` = Opt.some Quantity(tx.txType)
|
||||
if optionalHeader.isSome:
|
||||
let header = optionalHeader.get
|
||||
result.blockHash = Opt.some(header.blockHash)
|
||||
result.blockNumber = Opt.some(Quantity(header.number))
|
||||
var res = TransactionObject()
|
||||
res.`type` = Opt.some Quantity(tx.txType)
|
||||
res.blockHash = optionalHash
|
||||
res.blockNumber = w3Qty(optionalNumber)
|
||||
|
||||
if (let sender = tx.recoverSender(); sender.isOk):
|
||||
result.`from` = sender[]
|
||||
result.gas = Quantity(tx.gasLimit)
|
||||
result.gasPrice = Quantity(tx.gasPrice)
|
||||
result.hash = tx.rlpHash
|
||||
result.input = tx.payload
|
||||
result.nonce = Quantity(tx.nonce)
|
||||
result.to = Opt.some(tx.destination)
|
||||
res.`from` = sender[]
|
||||
res.gas = Quantity(tx.gasLimit)
|
||||
res.gasPrice = Quantity(tx.gasPrice)
|
||||
res.hash = tx.rlpHash
|
||||
res.input = tx.payload
|
||||
res.nonce = Quantity(tx.nonce)
|
||||
res.to = Opt.some(tx.destination)
|
||||
if txIndex.isSome:
|
||||
result.transactionIndex = Opt.some(Quantity(txIndex.get))
|
||||
result.value = tx.value
|
||||
result.v = Quantity(tx.V)
|
||||
result.r = tx.R
|
||||
result.s = tx.S
|
||||
result.maxFeePerGas = Opt.some Quantity(tx.maxFeePerGas)
|
||||
result.maxPriorityFeePerGas = Opt.some Quantity(tx.maxPriorityFeePerGas)
|
||||
res.transactionIndex = Opt.some(Quantity(txIndex.get))
|
||||
res.value = tx.value
|
||||
res.v = Quantity(tx.V)
|
||||
res.r = tx.R
|
||||
res.s = tx.S
|
||||
res.maxFeePerGas = Opt.some Quantity(tx.maxFeePerGas)
|
||||
res.maxPriorityFeePerGas = Opt.some Quantity(tx.maxPriorityFeePerGas)
|
||||
|
||||
if tx.txType >= TxEip2930:
|
||||
result.chainId = Opt.some(Quantity(tx.chainId))
|
||||
result.accessList = Opt.some(tx.accessList)
|
||||
res.chainId = Opt.some(Quantity(tx.chainId))
|
||||
res.accessList = Opt.some(tx.accessList)
|
||||
|
||||
if tx.txType >= TxEip4844:
|
||||
result.maxFeePerBlobGas = Opt.some(tx.maxFeePerBlobGas)
|
||||
result.blobVersionedHashes = Opt.some(tx.versionedHashes)
|
||||
res.maxFeePerBlobGas = Opt.some(tx.maxFeePerBlobGas)
|
||||
res.blobVersionedHashes = Opt.some(tx.versionedHashes)
|
||||
|
||||
proc populateBlockObject*(header: Header, chain: CoreDbRef, fullTx: bool, isUncle = false): BlockObject
|
||||
{.gcsafe, raises: [RlpError].} =
|
||||
let blockHash = header.blockHash
|
||||
result = BlockObject()
|
||||
return res
|
||||
|
||||
result.number = Quantity(header.number)
|
||||
result.hash = blockHash
|
||||
result.parentHash = header.parentHash
|
||||
result.nonce = Opt.some(FixedBytes[8] header.nonce)
|
||||
result.sha3Uncles = header.ommersHash
|
||||
result.logsBloom = FixedBytes[256] header.logsBloom
|
||||
result.transactionsRoot = header.txRoot
|
||||
result.stateRoot = header.stateRoot
|
||||
result.receiptsRoot = header.receiptsRoot
|
||||
result.miner = header.coinbase
|
||||
result.difficulty = header.difficulty
|
||||
result.extraData = HistoricExtraData header.extraData
|
||||
result.mixHash = Hash32 header.mixHash
|
||||
proc populateBlockObject*(blockHash: Hash32,
|
||||
blk: Block,
|
||||
totalDifficulty: UInt256,
|
||||
fullTx: bool,
|
||||
isUncle = false): BlockObject =
|
||||
template header: auto = blk.header
|
||||
|
||||
var res = BlockObject()
|
||||
res.number = Quantity(header.number)
|
||||
res.hash = blockHash
|
||||
res.parentHash = header.parentHash
|
||||
res.nonce = Opt.some(header.nonce)
|
||||
res.sha3Uncles = header.ommersHash
|
||||
res.logsBloom = header.logsBloom
|
||||
res.transactionsRoot = header.txRoot
|
||||
res.stateRoot = header.stateRoot
|
||||
res.receiptsRoot = header.receiptsRoot
|
||||
res.miner = header.coinbase
|
||||
res.difficulty = header.difficulty
|
||||
res.extraData = HistoricExtraData header.extraData
|
||||
res.mixHash = Hash32 header.mixHash
|
||||
|
||||
# discard sizeof(seq[byte]) of extraData and use actual length
|
||||
let size = sizeof(Header) - sizeof(seq[byte]) + header.extraData.len
|
||||
result.size = Quantity(size)
|
||||
res.size = Quantity(size)
|
||||
|
||||
result.gasLimit = Quantity(header.gasLimit)
|
||||
result.gasUsed = Quantity(header.gasUsed)
|
||||
result.timestamp = Quantity(header.timestamp)
|
||||
result.baseFeePerGas = header.baseFeePerGas
|
||||
res.gasLimit = Quantity(header.gasLimit)
|
||||
res.gasUsed = Quantity(header.gasUsed)
|
||||
res.timestamp = Quantity(header.timestamp)
|
||||
res.baseFeePerGas = header.baseFeePerGas
|
||||
res.totalDifficulty = totalDifficulty
|
||||
|
||||
if not isUncle:
|
||||
result.totalDifficulty = chain.getScore(blockHash).valueOr(0.u256)
|
||||
result.uncles = chain.getUncleHashes(header)
|
||||
res.uncles = blk.uncles.mapIt(it.blockHash)
|
||||
|
||||
if fullTx:
|
||||
var i = 0'u64
|
||||
for tx in chain.getBlockTransactions(header):
|
||||
result.transactions.add txOrHash(populateTransactionObject(tx, Opt.some(header), Opt.some(i)))
|
||||
inc i
|
||||
for i, tx in blk.transactions:
|
||||
let txObj = populateTransactionObject(tx,
|
||||
Opt.some(blockHash),
|
||||
Opt.some(header.number), Opt.some(i.uint64))
|
||||
res.transactions.add txOrHash(txObj)
|
||||
else:
|
||||
for x in chain.getBlockTransactionHashes(header):
|
||||
result.transactions.add txOrHash(x)
|
||||
for i, tx in blk.transactions:
|
||||
let txHash = rlpHash(tx)
|
||||
res.transactions.add txOrHash(txHash)
|
||||
|
||||
if header.withdrawalsRoot.isSome:
|
||||
result.withdrawalsRoot = Opt.some(header.withdrawalsRoot.get)
|
||||
result.withdrawals = Opt.some(toWdList(chain.getWithdrawals(header.withdrawalsRoot.get)))
|
||||
res.withdrawalsRoot = header.withdrawalsRoot
|
||||
res.withdrawals = toWdList blk.withdrawals
|
||||
res.parentBeaconBlockRoot = header.parentBeaconBlockRoot
|
||||
res.blobGasUsed = w3Qty(header.blobGasUsed)
|
||||
res.excessBlobGas = w3Qty(header.excessBlobGas)
|
||||
|
||||
if header.blobGasUsed.isSome:
|
||||
result.blobGasUsed = Opt.some(Quantity(header.blobGasUsed.get))
|
||||
|
||||
if header.excessBlobGas.isSome:
|
||||
result.excessBlobGas = Opt.some(Quantity(header.excessBlobGas.get))
|
||||
|
||||
if header.parentBeaconBlockRoot.isSome:
|
||||
result.parentBeaconBlockRoot = Opt.some(header.parentBeaconBlockRoot.get)
|
||||
return res
|
||||
|
||||
proc populateReceipt*(receipt: Receipt, gasUsed: GasInt, tx: Transaction,
|
||||
txIndex: uint64, header: Header): ReceiptObject =
|
||||
let sender = tx.recoverSender()
|
||||
result = ReceiptObject()
|
||||
result.transactionHash = tx.rlpHash
|
||||
result.transactionIndex = Quantity(txIndex)
|
||||
result.blockHash = header.blockHash
|
||||
result.blockNumber = Quantity(header.number)
|
||||
var res = ReceiptObject()
|
||||
res.transactionHash = tx.rlpHash
|
||||
res.transactionIndex = Quantity(txIndex)
|
||||
res.blockHash = header.blockHash
|
||||
res.blockNumber = Quantity(header.number)
|
||||
if sender.isSome():
|
||||
result.`from` = sender.get()
|
||||
result.to = Opt.some(tx.destination)
|
||||
result.cumulativeGasUsed = Quantity(receipt.cumulativeGasUsed)
|
||||
result.gasUsed = Quantity(gasUsed)
|
||||
result.`type` = Opt.some Quantity(receipt.receiptType)
|
||||
res.`from` = sender.get()
|
||||
res.to = Opt.some(tx.destination)
|
||||
res.cumulativeGasUsed = Quantity(receipt.cumulativeGasUsed)
|
||||
res.gasUsed = Quantity(gasUsed)
|
||||
res.`type` = Opt.some Quantity(receipt.receiptType)
|
||||
|
||||
if tx.contractCreation and sender.isSome:
|
||||
result.contractAddress = Opt.some(tx.creationAddress(sender[]))
|
||||
res.contractAddress = Opt.some(tx.creationAddress(sender[]))
|
||||
|
||||
for log in receipt.logs:
|
||||
# TODO: Work everywhere with either `Hash32` as topic or `array[32, byte]`
|
||||
|
@ -246,36 +227,38 @@ proc populateReceipt*(receipt: Receipt, gasUsed: GasInt, tx: Transaction,
|
|||
removed: false,
|
||||
# TODO: Not sure what is difference between logIndex and TxIndex and how
|
||||
# to calculate it.
|
||||
logIndex: Opt.some(result.transactionIndex),
|
||||
logIndex: Opt.some(res.transactionIndex),
|
||||
# Note: the next 4 fields cause a lot of duplication of data, but the spec
|
||||
# is what it is. Not sure if other clients actually add this.
|
||||
transactionIndex: Opt.some(result.transactionIndex),
|
||||
transactionHash: Opt.some(result.transactionHash),
|
||||
blockHash: Opt.some(result.blockHash),
|
||||
blockNumber: Opt.some(result.blockNumber),
|
||||
transactionIndex: Opt.some(res.transactionIndex),
|
||||
transactionHash: Opt.some(res.transactionHash),
|
||||
blockHash: Opt.some(res.blockHash),
|
||||
blockNumber: Opt.some(res.blockNumber),
|
||||
# The actual fields
|
||||
address: log.address,
|
||||
data: log.data,
|
||||
topics: topics
|
||||
)
|
||||
result.logs.add(logObject)
|
||||
res.logs.add(logObject)
|
||||
|
||||
result.logsBloom = FixedBytes[256] receipt.logsBloom
|
||||
res.logsBloom = FixedBytes[256] receipt.logsBloom
|
||||
|
||||
# post-transaction stateroot (pre Byzantium).
|
||||
if receipt.hasStateRoot:
|
||||
result.root = Opt.some(receipt.stateRoot)
|
||||
res.root = Opt.some(receipt.stateRoot)
|
||||
else:
|
||||
# 1 = success, 0 = failure.
|
||||
result.status = Opt.some(Quantity(receipt.status.uint64))
|
||||
res.status = Opt.some(Quantity(receipt.status.uint64))
|
||||
|
||||
let baseFeePerGas = header.baseFeePerGas.get(0.u256)
|
||||
let normTx = eip1559TxNormalization(tx, baseFeePerGas.truncate(GasInt))
|
||||
result.effectiveGasPrice = Quantity(normTx.gasPrice)
|
||||
res.effectiveGasPrice = Quantity(normTx.gasPrice)
|
||||
|
||||
if tx.txType == TxEip4844:
|
||||
result.blobGasUsed = Opt.some(Quantity(tx.versionedHashes.len.uint64 * GAS_PER_BLOB.uint64))
|
||||
result.blobGasPrice = Opt.some(getBlobBaseFee(header.excessBlobGas.get(0'u64)))
|
||||
res.blobGasUsed = Opt.some(Quantity(tx.versionedHashes.len.uint64 * GAS_PER_BLOB.uint64))
|
||||
res.blobGasPrice = Opt.some(getBlobBaseFee(header.excessBlobGas.get(0'u64)))
|
||||
|
||||
return res
|
||||
|
||||
proc createAccessList*(header: Header,
|
||||
com: CommonRef,
|
||||
|
|
|
@ -10,9 +10,12 @@
|
|||
{.push raises: [].}
|
||||
|
||||
import
|
||||
chronicles,
|
||||
std/[sequtils, strutils],
|
||||
stint,
|
||||
web3/[conversions, eth_api_types],
|
||||
eth/common/base,
|
||||
stew/byteutils,
|
||||
../common/common,
|
||||
json_rpc/rpcserver,
|
||||
../db/ledger,
|
||||
|
@ -22,32 +25,70 @@ import
|
|||
../transaction,
|
||||
../transaction/call_evm,
|
||||
../evm/evm_errors,
|
||||
../core/eip4844,
|
||||
./rpc_types,
|
||||
./rpc_utils,
|
||||
./filters,
|
||||
./server_api_helpers
|
||||
./filters
|
||||
|
||||
type
|
||||
ServerAPIRef* = ref object
|
||||
type ServerAPIRef* = ref object
|
||||
com: CommonRef
|
||||
chain: ForkedChainRef
|
||||
txPool: TxPoolRef
|
||||
|
||||
const
|
||||
defaultTag = blockId("latest")
|
||||
const defaultTag = blockId("latest")
|
||||
|
||||
func newServerAPI*(c: ForkedChainRef, t: TxPoolRef): ServerAPIRef =
|
||||
ServerAPIRef(
|
||||
com: c.com,
|
||||
chain: c,
|
||||
txPool: t
|
||||
ServerAPIRef(com: c.com, chain: c, txPool: t)
|
||||
|
||||
proc getTotalDifficulty*(api: ServerAPIRef, blockHash: Hash32): UInt256 =
|
||||
var totalDifficulty: UInt256
|
||||
if api.com.db.getTd(blockHash, totalDifficulty):
|
||||
return totalDifficulty
|
||||
else:
|
||||
return api.com.db.headTotalDifficulty()
|
||||
|
||||
proc getProof*(
|
||||
accDB: LedgerRef, address: eth_types.Address, slots: seq[UInt256]
|
||||
): ProofResponse =
|
||||
let
|
||||
acc = accDB.getEthAccount(address)
|
||||
accExists = accDB.accountExists(address)
|
||||
accountProof = accDB.getAccountProof(address)
|
||||
slotProofs = accDB.getStorageProof(address, slots)
|
||||
|
||||
var storage = newSeqOfCap[StorageProof](slots.len)
|
||||
|
||||
for i, slotKey in slots:
|
||||
let slotValue = accDB.getStorage(address, slotKey)
|
||||
storage.add(
|
||||
StorageProof(
|
||||
key: slotKey, value: slotValue, proof: seq[RlpEncodedBytes](slotProofs[i])
|
||||
)
|
||||
)
|
||||
|
||||
if accExists:
|
||||
ProofResponse(
|
||||
address: address,
|
||||
accountProof: seq[RlpEncodedBytes](accountProof),
|
||||
balance: acc.balance,
|
||||
nonce: w3Qty(acc.nonce),
|
||||
codeHash: acc.codeHash,
|
||||
storageHash: acc.storageRoot,
|
||||
storageProof: storage,
|
||||
)
|
||||
else:
|
||||
ProofResponse(
|
||||
address: address,
|
||||
accountProof: seq[RlpEncodedBytes](accountProof),
|
||||
storageProof: storage,
|
||||
)
|
||||
|
||||
proc headerFromTag(api: ServerAPIRef, blockTag: BlockTag): Result[Header, string] =
|
||||
if blockTag.kind == bidAlias:
|
||||
let tag = blockTag.alias.toLowerAscii
|
||||
case tag
|
||||
of "latest": return ok(api.chain.latestHeader)
|
||||
of "latest":
|
||||
return ok(api.chain.latestHeader)
|
||||
else:
|
||||
return err("Unsupported block tag " & tag)
|
||||
else:
|
||||
|
@ -78,7 +119,7 @@ proc blockFromTag(api: ServerAPIRef, blockTag: BlockTag): Result[Block, string]
|
|||
let blockNum = base.BlockNumber blockTag.number
|
||||
return api.chain.blockByNumber(blockNum)
|
||||
|
||||
proc setupServerAPI*(api: ServerAPIRef, server: RpcServer) =
|
||||
proc setupServerAPI*(api: ServerAPIRef, server: RpcServer, ctx: EthContext) =
|
||||
server.rpc("eth_getBalance") do(data: Address, blockTag: BlockTag) -> UInt256:
|
||||
## Returns the balance of the account of given address.
|
||||
let
|
||||
|
@ -87,7 +128,9 @@ proc setupServerAPI*(api: ServerAPIRef, server: RpcServer) =
|
|||
address = data
|
||||
ledger.getBalance(address)
|
||||
|
||||
server.rpc("eth_getStorageAt") do(data: Address, slot: UInt256, blockTag: BlockTag) -> FixedBytes[32]:
|
||||
server.rpc("eth_getStorageAt") do(
|
||||
data: Address, slot: UInt256, blockTag: BlockTag
|
||||
) -> FixedBytes[32]:
|
||||
## Returns the value from a storage position at a given address.
|
||||
let
|
||||
ledger = api.ledgerFromTag(blockTag).valueOr:
|
||||
|
@ -96,7 +139,9 @@ proc setupServerAPI*(api: ServerAPIRef, server: RpcServer) =
|
|||
value = ledger.getStorage(address, slot)
|
||||
value.to(Bytes32)
|
||||
|
||||
server.rpc("eth_getTransactionCount") do(data: Address, blockTag: BlockTag) -> Web3Quantity:
|
||||
server.rpc("eth_getTransactionCount") do(
|
||||
data: Address, blockTag: BlockTag
|
||||
) -> Web3Quantity:
|
||||
## Returns the number of transactions ak.s. nonce sent from an address.
|
||||
let
|
||||
ledger = api.ledgerFromTag(blockTag).valueOr:
|
||||
|
@ -124,7 +169,9 @@ proc setupServerAPI*(api: ServerAPIRef, server: RpcServer) =
|
|||
address = data
|
||||
ledger.getCode(address).bytes()
|
||||
|
||||
server.rpc("eth_getBlockByHash") do(data: Hash32, fullTransactions: bool) -> BlockObject:
|
||||
server.rpc("eth_getBlockByHash") do(
|
||||
data: Hash32, fullTransactions: bool
|
||||
) -> BlockObject:
|
||||
## Returns information about a block by hash.
|
||||
##
|
||||
## data: Hash of a block.
|
||||
|
@ -135,9 +182,13 @@ proc setupServerAPI*(api: ServerAPIRef, server: RpcServer) =
|
|||
let blk = api.chain.blockByHash(blockHash).valueOr:
|
||||
return nil
|
||||
|
||||
return populateBlockObject(blockHash, blk, fullTransactions)
|
||||
return populateBlockObject(
|
||||
blockHash, blk, api.getTotalDifficulty(blockHash), fullTransactions
|
||||
)
|
||||
|
||||
server.rpc("eth_getBlockByNumber") do(blockTag: BlockTag, fullTransactions: bool) -> BlockObject:
|
||||
server.rpc("eth_getBlockByNumber") do(
|
||||
blockTag: BlockTag, fullTransactions: bool
|
||||
) -> BlockObject:
|
||||
## Returns information about a block by block number.
|
||||
##
|
||||
## blockTag: integer of a block number, or the string "earliest", "latest" or "pending", as in the default block parameter.
|
||||
|
@ -147,7 +198,9 @@ proc setupServerAPI*(api: ServerAPIRef, server: RpcServer) =
|
|||
return nil
|
||||
|
||||
let blockHash = blk.header.blockHash
|
||||
return populateBlockObject(blockHash, blk, fullTransactions)
|
||||
return populateBlockObject(
|
||||
blockHash, blk, api.getTotalDifficulty(blockHash), fullTransactions
|
||||
)
|
||||
|
||||
server.rpc("eth_syncing") do() -> SyncingStatus:
|
||||
## Returns SyncObject or false when not syncing.
|
||||
|
@ -155,21 +208,25 @@ proc setupServerAPI*(api: ServerAPIRef, server: RpcServer) =
|
|||
let sync = SyncObject(
|
||||
startingBlock: Quantity(api.com.syncStart),
|
||||
currentBlock: Quantity(api.com.syncCurrent),
|
||||
highestBlock : Quantity(api.com.syncHighest)
|
||||
highestBlock: Quantity(api.com.syncHighest),
|
||||
)
|
||||
return SyncingStatus(syncing: true, syncObject: sync)
|
||||
else:
|
||||
return SyncingStatus(syncing: false)
|
||||
|
||||
proc getLogsForBlock(
|
||||
chain: ForkedChainRef,
|
||||
header: Header,
|
||||
opts: FilterOptions): seq[FilterLog]
|
||||
{.gcsafe, raises: [RlpError].} =
|
||||
chain: ForkedChainRef, header: Header, opts: FilterOptions
|
||||
): seq[FilterLog] {.gcsafe, raises: [RlpError].} =
|
||||
if headerBloomFilter(header, opts.address, opts.topics):
|
||||
let
|
||||
receipts = chain.db.getReceipts(header.receiptsRoot)
|
||||
txs = chain.db.getTransactions(header.txRoot)
|
||||
let (receipts, txs) =
|
||||
if api.chain.isInMemory(header.blockHash):
|
||||
let blk = api.chain.memoryBlock(header.blockHash)
|
||||
(blk.receipts, blk.blk.transactions)
|
||||
else:
|
||||
(
|
||||
chain.db.getReceipts(header.receiptsRoot),
|
||||
chain.db.getTransactions(header.txRoot),
|
||||
)
|
||||
# Note: this will hit assertion error if number of block transactions
|
||||
# do not match block receipts.
|
||||
# Although this is fine as number of receipts should always match number
|
||||
|
@ -184,8 +241,8 @@ proc setupServerAPI*(api: ServerAPIRef, server: RpcServer) =
|
|||
chain: ForkedChainRef,
|
||||
start: base.BlockNumber,
|
||||
finish: base.BlockNumber,
|
||||
opts: FilterOptions): seq[FilterLog]
|
||||
{.gcsafe, raises: [RlpError].} =
|
||||
opts: FilterOptions,
|
||||
): seq[FilterLog] {.gcsafe, raises: [RlpError].} =
|
||||
var
|
||||
logs = newSeq[FilterLog]()
|
||||
blockNum = start
|
||||
|
@ -228,11 +285,7 @@ proc setupServerAPI*(api: ServerAPIRef, server: RpcServer) =
|
|||
|
||||
# Note: if fromHeader.number > toHeader.number, no logs will be
|
||||
# returned. This is consistent with, what other ethereum clients return
|
||||
return api.chain.getLogsForRange(
|
||||
blockFrom.number,
|
||||
blockTo.number,
|
||||
filterOptions
|
||||
)
|
||||
return api.chain.getLogsForRange(blockFrom.number, blockTo.number, filterOptions)
|
||||
|
||||
server.rpc("eth_sendRawTransaction") do(txBytes: seq[byte]) -> Hash32:
|
||||
## Creates new message call transaction or a contract creation for signed transactions.
|
||||
|
@ -278,14 +331,16 @@ proc setupServerAPI*(api: ServerAPIRef, server: RpcServer) =
|
|||
|
||||
if blockhash == zeroHash32:
|
||||
# Receipt in database
|
||||
let txDetails = api.chain.db.getTransactionKey(txHash)
|
||||
let txDetails = api.chain.db.getTransactionKey(data)
|
||||
if txDetails.index < 0:
|
||||
return nil
|
||||
|
||||
let header = api.chain.headerByNumber(txDetails.blockNumber).valueOr:
|
||||
raise newException(ValueError, "Block not found")
|
||||
var tx: Transaction
|
||||
if not api.chain.db.getTransactionByIndex(header.txRoot, uint16(txDetails.index), tx):
|
||||
if not api.chain.db.getTransactionByIndex(
|
||||
header.txRoot, uint16(txDetails.index), tx
|
||||
):
|
||||
return nil
|
||||
|
||||
for receipt in api.chain.db.getReceipts(header.receiptsRoot):
|
||||
|
@ -304,7 +359,9 @@ proc setupServerAPI*(api: ServerAPIRef, server: RpcServer) =
|
|||
prevGasUsed = receipt.cumulativeGasUsed
|
||||
|
||||
if txid == idx:
|
||||
return populateReceipt(receipt, gasUsed, blkdesc.blk.transactions[txid], txid, blkdesc.blk.header)
|
||||
return populateReceipt(
|
||||
receipt, gasUsed, blkdesc.blk.transactions[txid], txid, blkdesc.blk.header
|
||||
)
|
||||
|
||||
idx.inc
|
||||
|
||||
|
@ -319,6 +376,334 @@ proc setupServerAPI*(api: ServerAPIRef, server: RpcServer) =
|
|||
let
|
||||
header = api.headerFromTag(blockId("latest")).valueOr:
|
||||
raise newException(ValueError, "Block not found")
|
||||
#TODO: change 0 to configureable gas cap
|
||||
gasUsed = rpcEstimateGas(args, header, api.chain.com, DEFAULT_RPC_GAS_CAP).valueOr:
|
||||
raise newException(ValueError, "rpcEstimateGas error: " & $error.code)
|
||||
Quantity(gasUsed)
|
||||
|
||||
server.rpc("eth_gasPrice") do() -> Web3Quantity:
|
||||
## Returns an integer of the current gas price in wei.
|
||||
w3Qty(calculateMedianGasPrice(api.com.db).uint64)
|
||||
|
||||
server.rpc("eth_accounts") do() -> seq[eth_types.Address]:
|
||||
## Returns a list of addresses owned by client.
|
||||
result = newSeqOfCap[eth_types.Address](ctx.am.numAccounts)
|
||||
for k in ctx.am.addresses:
|
||||
result.add k
|
||||
|
||||
server.rpc("eth_getBlockTransactionCountByHash") do(data: Hash32) -> Web3Quantity:
|
||||
## Returns the number of transactions in a block from a block matching the given block hash.
|
||||
##
|
||||
## data: hash of a block
|
||||
## Returns integer of the number of transactions in this block.
|
||||
let blk = api.chain.blockByHash(data).valueOr:
|
||||
raise newException(ValueError, "Block not found")
|
||||
|
||||
Web3Quantity(blk.transactions.len)
|
||||
|
||||
server.rpc("eth_getBlockTransactionCountByNumber") do(
|
||||
blockTag: BlockTag
|
||||
) -> Web3Quantity:
|
||||
## 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.
|
||||
## Returns integer of the number of transactions in this block.
|
||||
let blk = api.blockFromTag(blockTag).valueOr:
|
||||
raise newException(ValueError, "Block not found")
|
||||
|
||||
Web3Quantity(blk.transactions.len)
|
||||
|
||||
server.rpc("eth_getUncleCountByBlockHash") do(data: Hash32) -> Web3Quantity:
|
||||
## Returns the number of uncles in a block from a block matching the given block hash.
|
||||
##
|
||||
## data: hash of a block.
|
||||
## Returns integer of the number of uncles in this block.
|
||||
let blk = api.chain.blockByHash(data).valueOr:
|
||||
raise newException(ValueError, "Block not found")
|
||||
|
||||
Web3Quantity(blk.uncles.len)
|
||||
|
||||
server.rpc("eth_getUncleCountByBlockNumber") do(blockTag: BlockTag) -> Web3Quantity:
|
||||
## 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.
|
||||
## Returns integer of the number of uncles in this block.
|
||||
let blk = api.blockFromTag(blockTag).valueOr:
|
||||
raise newException(ValueError, "Block not found")
|
||||
|
||||
Web3Quantity(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]:
|
||||
## 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.
|
||||
## Note the address to sign with must be unlocked.
|
||||
##
|
||||
## data: address.
|
||||
## message: message to sign.
|
||||
## Returns signature.
|
||||
let
|
||||
address = data
|
||||
acc = ctx.am.getAccount(address).tryGet()
|
||||
|
||||
if not acc.unlocked:
|
||||
raise newException(ValueError, "Account locked, please unlock it first")
|
||||
sign(acc.privateKey, cast[string](message))
|
||||
|
||||
server.rpc("eth_signTransaction") do(data: TransactionArgs) -> seq[byte]:
|
||||
## Signs a transaction that can be submitted to the network at a later time using with
|
||||
## eth_sendRawTransaction
|
||||
let
|
||||
address = data.`from`.get()
|
||||
acc = ctx.am.getAccount(address).tryGet()
|
||||
|
||||
if not acc.unlocked:
|
||||
raise newException(ValueError, "Account locked, please unlock it first")
|
||||
|
||||
let
|
||||
accDB = api.ledgerFromTag(blockId("latest")).valueOr:
|
||||
raise newException(ValueError, "Latest Block not found")
|
||||
tx = unsignedTx(data, api.chain.db, accDB.getNonce(address) + 1, api.com.chainId)
|
||||
eip155 = api.com.isEIP155(api.chain.latestNumber)
|
||||
signedTx = signTransaction(tx, acc.privateKey, eip155)
|
||||
return rlp.encode(signedTx)
|
||||
|
||||
server.rpc("eth_sendTransaction") do(data: TransactionArgs) -> Hash32:
|
||||
## Creates new message call transaction or a contract creation, if the data field contains code.
|
||||
##
|
||||
## obj: the transaction object.
|
||||
## Returns the transaction hash, or the zero hash if the transaction is not yet available.
|
||||
## Note: Use eth_getTransactionReceipt to get the contract address, after the transaction was mined, when you created a contract.
|
||||
let
|
||||
address = data.`from`.get()
|
||||
acc = ctx.am.getAccount(address).tryGet()
|
||||
|
||||
if not acc.unlocked:
|
||||
raise newException(ValueError, "Account locked, please unlock it first")
|
||||
|
||||
let
|
||||
accDB = api.ledgerFromTag(blockId("latest")).valueOr:
|
||||
raise newException(ValueError, "Latest Block not found")
|
||||
tx = unsignedTx(data, api.chain.db, accDB.getNonce(address) + 1, api.com.chainId)
|
||||
eip155 = api.com.isEIP155(api.chain.latestNumber)
|
||||
signedTx = signTransaction(tx, acc.privateKey, eip155)
|
||||
networkPayload =
|
||||
if signedTx.txType == TxEip4844:
|
||||
if data.blobs.isNone or data.commitments.isNone or data.proofs.isNone:
|
||||
raise newException(ValueError, "EIP-4844 transaction needs blobs")
|
||||
if data.blobs.get.len != signedTx.versionedHashes.len:
|
||||
raise newException(ValueError, "Incorrect number of blobs")
|
||||
if data.commitments.get.len != signedTx.versionedHashes.len:
|
||||
raise newException(ValueError, "Incorrect number of commitments")
|
||||
if data.proofs.get.len != signedTx.versionedHashes.len:
|
||||
raise newException(ValueError, "Incorrect number of proofs")
|
||||
NetworkPayload(
|
||||
blobs: data.blobs.get.mapIt it.NetworkBlob,
|
||||
commitments: data.commitments.get,
|
||||
proofs: data.proofs.get,
|
||||
)
|
||||
else:
|
||||
if data.blobs.isSome or data.commitments.isSome or data.proofs.isSome:
|
||||
raise newException(ValueError, "Blobs require EIP-4844 transaction")
|
||||
nil
|
||||
pooledTx = PooledTransaction(tx: signedTx, networkPayload: networkPayload)
|
||||
|
||||
api.txPool.add(pooledTx)
|
||||
rlpHash(signedTx)
|
||||
|
||||
server.rpc("eth_getTransactionByHash") do(data: Hash32) -> TransactionObject:
|
||||
## Returns the information about a transaction requested by transaction hash.
|
||||
##
|
||||
## data: hash of a transaction.
|
||||
## Returns requested transaction information.
|
||||
let txHash = data
|
||||
let res = api.txPool.getItem(txHash)
|
||||
if res.isOk:
|
||||
return populateTransactionObject(res.get().tx, Opt.none(Hash32), Opt.none(uint64))
|
||||
|
||||
let txDetails = api.chain.db.getTransactionKey(txHash)
|
||||
if txDetails.index < 0:
|
||||
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
|
||||
|
||||
let header = api.chain.db.getBlockHeader(txDetails.blockNumber)
|
||||
var tx: Transaction
|
||||
if api.chain.db.getTransactionByIndex(header.txRoot, uint16(txDetails.index), tx):
|
||||
return populateTransactionObject(
|
||||
tx,
|
||||
Opt.some(header.blockHash),
|
||||
Opt.some(header.number),
|
||||
Opt.some(txDetails.index),
|
||||
)
|
||||
|
||||
server.rpc("eth_getTransactionByBlockHashAndIndex") do(
|
||||
data: Hash32, quantity: Web3Quantity
|
||||
) -> TransactionObject:
|
||||
## Returns information about a transaction by block hash and transaction index position.
|
||||
##
|
||||
## data: hash of a block.
|
||||
## quantity: integer of the transaction index position.
|
||||
## Returns requested transaction information.
|
||||
let index = uint64(quantity)
|
||||
let blk = api.chain.blockByHash(data).valueOr:
|
||||
return nil
|
||||
|
||||
if index >= uint64(blk.transactions.len):
|
||||
return nil
|
||||
|
||||
populateTransactionObject(
|
||||
blk.transactions[index], Opt.some(data), Opt.some(blk.header.number), Opt.some(index)
|
||||
)
|
||||
|
||||
server.rpc("eth_getTransactionByBlockNumberAndIndex") do(
|
||||
quantityTag: BlockTag, quantity: Web3Quantity
|
||||
) -> TransactionObject:
|
||||
## Returns information about a transaction by block number and transaction index position.
|
||||
##
|
||||
## quantityTag: a block number, or the string "earliest", "latest" or "pending", as in the default block parameter.
|
||||
## quantity: the transaction index position.
|
||||
## NOTE : "pending" blockTag is not supported.
|
||||
let index = uint64(quantity)
|
||||
let blk = api.blockFromTag(quantityTag).valueOr:
|
||||
return nil
|
||||
|
||||
if index >= uint64(blk.transactions.len):
|
||||
return nil
|
||||
|
||||
populateTransactionObject(
|
||||
blk.transactions[index], Opt.some(blk.header.blockHash), Opt.some(blk.header.number), Opt.some(index)
|
||||
)
|
||||
|
||||
server.rpc("eth_getProof") do(
|
||||
data: eth_types.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
|
||||
## existence of the values in the state.
|
||||
## See spec here: https://eips.ethereum.org/EIPS/eip-1186
|
||||
##
|
||||
## data: address of the account.
|
||||
## slots: integers of the positions in the storage to return with storage proofs.
|
||||
## quantityTag: integer block number, or the string "latest", "earliest" or "pending", see the default block parameter.
|
||||
## Returns: the proof response containing the account, account proof and storage proof
|
||||
let accDB = api.ledgerFromTag(quantityTag).valueOr:
|
||||
raise newException(ValueError, "Block not found")
|
||||
|
||||
getProof(accDB, data, slots)
|
||||
|
||||
server.rpc("eth_getBlockReceipts") do(
|
||||
quantityTag: BlockTag
|
||||
) -> Opt[seq[ReceiptObject]]:
|
||||
## Returns the receipts of a block.
|
||||
let
|
||||
header = api.headerFromTag(quantityTag).valueOr:
|
||||
raise newException(ValueError, "Block not found")
|
||||
blkHash = header.blockHash
|
||||
|
||||
var
|
||||
prevGasUsed = GasInt(0)
|
||||
receipts: seq[Receipt]
|
||||
recs: seq[ReceiptObject]
|
||||
txs: seq[Transaction]
|
||||
index = 0'u64
|
||||
|
||||
if api.chain.haveBlockAndState(blkHash):
|
||||
let blkdesc = api.chain.memoryBlock(blkHash)
|
||||
receipts = blkdesc.receipts
|
||||
txs = blkdesc.blk.transactions
|
||||
else:
|
||||
for receipt in api.chain.db.getReceipts(header.receiptsRoot):
|
||||
receipts.add receipt
|
||||
txs = api.chain.db.getTransactions(header.txRoot)
|
||||
|
||||
try:
|
||||
for receipt in receipts:
|
||||
let gasUsed = receipt.cumulativeGasUsed - prevGasUsed
|
||||
prevGasUsed = receipt.cumulativeGasUsed
|
||||
recs.add populateReceipt(receipt, gasUsed, txs[index], index, header)
|
||||
inc index
|
||||
return Opt.some(recs)
|
||||
except CatchableError:
|
||||
return Opt.none(seq[ReceiptObject])
|
||||
|
||||
server.rpc("eth_createAccessList") do(
|
||||
args: TransactionArgs, quantityTag: BlockTag
|
||||
) -> AccessListResult:
|
||||
## Generates an access list for a transaction.
|
||||
try:
|
||||
let header = api.headerFromTag(quantityTag).valueOr:
|
||||
raise newException(ValueError, "Block not found")
|
||||
return createAccessList(header, api.com, args)
|
||||
except CatchableError as exc:
|
||||
return AccessListResult(error: Opt.some("createAccessList error: " & exc.msg))
|
||||
|
||||
server.rpc("eth_blobBaseFee") do() -> Web3Quantity:
|
||||
## Returns the base fee per blob gas in wei.
|
||||
let header = api.headerFromTag(blockId("latest")).valueOr:
|
||||
raise newException(ValueError, "Block not found")
|
||||
if header.blobGasUsed.isNone:
|
||||
raise newException(ValueError, "blobGasUsed missing from latest header")
|
||||
if header.excessBlobGas.isNone:
|
||||
raise newException(ValueError, "excessBlobGas missing from latest header")
|
||||
let blobBaseFee =
|
||||
getBlobBaseFee(header.excessBlobGas.get) * header.blobGasUsed.get.u256
|
||||
if blobBaseFee > high(uint64).u256:
|
||||
raise newException(ValueError, "blobBaseFee is bigger than uint64.max")
|
||||
return w3Qty blobBaseFee.truncate(uint64)
|
||||
|
||||
server.rpc("eth_getUncleByBlockHashAndIndex") do(
|
||||
data: Hash32, quantity: Web3Quantity
|
||||
) -> BlockObject:
|
||||
## Returns information about a uncle of a block by hash and uncle index position.
|
||||
##
|
||||
## data: hash of block.
|
||||
## quantity: the uncle's index position.
|
||||
## Returns BlockObject or nil when no block was found.
|
||||
let index = uint64(quantity)
|
||||
let blk = api.chain.blockByHash(data).valueOr:
|
||||
return nil
|
||||
|
||||
if index < 0 or index >= blk.uncles.len.uint64:
|
||||
return nil
|
||||
|
||||
let
|
||||
uncle = api.chain.blockByHash(blk.uncles[index].blockHash).valueOr:
|
||||
return nil
|
||||
uncleHash = uncle.header.blockHash
|
||||
|
||||
return populateBlockObject(
|
||||
uncleHash, uncle, api.getTotalDifficulty(uncleHash), false, true
|
||||
)
|
||||
|
||||
server.rpc("eth_getUncleByBlockNumberAndIndex") do(
|
||||
quantityTag: BlockTag, quantity: Web3Quantity
|
||||
) -> BlockObject:
|
||||
# Returns information about a uncle of a block by number and uncle index position.
|
||||
##
|
||||
## quantityTag: a block number, or the string "earliest", "latest" or "pending", as in the default block parameter.
|
||||
## quantity: the uncle's index position.
|
||||
## Returns BlockObject or nil when no block was found.
|
||||
let index = uint64(quantity)
|
||||
let blk = api.blockFromTag(quantityTag).valueOr:
|
||||
return nil
|
||||
|
||||
if index < 0 or index >= blk.uncles.len.uint64:
|
||||
return nil
|
||||
|
||||
let
|
||||
uncle = api.chain.blockByHash(blk.uncles[index].blockHash).valueOr:
|
||||
return nil
|
||||
uncleHash = uncle.header.blockHash
|
||||
|
||||
return populateBlockObject(
|
||||
uncleHash, uncle, api.getTotalDifficulty(uncleHash), false, true
|
||||
)
|
||||
|
|
|
@ -87,7 +87,7 @@ sed -i "s/el_image: .*/el_image: $new_el_image/" assertoor.yaml
|
|||
|
||||
sudo kurtosis run \
|
||||
--enclave nimbus-localtestnet \
|
||||
github.com/ethpandaops/ethereum-package@4.3.0 \
|
||||
github.com/ethpandaops/ethereum-package \
|
||||
--args-file assertoor.yaml
|
||||
|
||||
enclave_dump=$(kurtosis enclave inspect nimbus-localtestnet)
|
||||
|
|
|
@ -15,10 +15,10 @@ cliBuilder:
|
|||
./test_evm_support,
|
||||
./test_genesis,
|
||||
./test_precompiles,
|
||||
./test_rpc,
|
||||
./test_generalstate_json,
|
||||
./test_tracer_json,
|
||||
#./test_persistblock_json, -- fails
|
||||
#./test_rpc, -- fails
|
||||
./test_filters,
|
||||
./test_op_arith,
|
||||
./test_op_bit,
|
||||
|
|
|
@ -87,7 +87,7 @@ proc setupEnv(envFork: HardFork = MergeFork): TestEnv =
|
|||
beaconEngine = BeaconEngineRef.new(txPool, chain)
|
||||
serverApi = newServerAPI(chain, txPool)
|
||||
|
||||
setupServerAPI(serverApi, server)
|
||||
setupServerAPI(serverApi, server, newEthContext())
|
||||
setupEngineAPI(beaconEngine, server)
|
||||
|
||||
server.start()
|
||||
|
|
|
@ -15,7 +15,7 @@ import
|
|||
eth/[rlp, trie/trie_defs, trie/hexary_proof_verification],
|
||||
../nimbus/db/[ledger, core_db],
|
||||
../nimbus/common/chain_config,
|
||||
../nimbus/rpc/p2p
|
||||
../nimbus/rpc/server_api
|
||||
|
||||
type
|
||||
Hash32 = eth_types.Hash32
|
||||
|
|
|
@ -6,13 +6,14 @@
|
|||
# at your option. This file may not be copied, modified, or distributed except according to those terms.
|
||||
|
||||
import
|
||||
chronicles,
|
||||
std/[json, os, typetraits, times, sequtils],
|
||||
asynctest, web3/eth_api,
|
||||
stew/byteutils,
|
||||
json_rpc/[rpcserver, rpcclient],
|
||||
nimcrypto/[keccak, hash],
|
||||
eth/[rlp, keys, trie/hexary_proof_verification],
|
||||
eth/common/transaction_utils,
|
||||
eth/[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/db/[ledger, storage_types],
|
||||
../nimbus/sync/protocol,
|
||||
|
@ -26,11 +27,6 @@ import
|
|||
./macro_assembler,
|
||||
./test_block_fixture
|
||||
|
||||
const
|
||||
zeroAddress = block:
|
||||
var rc: Address
|
||||
rc
|
||||
|
||||
type
|
||||
Hash32 = common.Hash32
|
||||
Header = common.Header
|
||||
|
@ -50,7 +46,7 @@ func emptyStorageHash(): Hash32 =
|
|||
|
||||
proc verifyAccountProof(trustedStateRoot: Hash32, res: ProofResponse): MptProofVerificationResult =
|
||||
let
|
||||
key = toSeq(keccak256(res.address).data)
|
||||
key = toSeq(keccak256(res.address.data).data)
|
||||
value = rlp.encode(Account(
|
||||
nonce: res.nonce.uint64,
|
||||
balance: res.balance,
|
||||
|
@ -78,18 +74,34 @@ proc persistFixtureBlock(chainDB: CoreDbRef) =
|
|||
let header = getBlockHeader4514995()
|
||||
# Manually inserting header to avoid any parent checks
|
||||
discard chainDB.ctx.getKvt.put(genericHashKey(header.blockHash).toOpenArray, rlp.encode(header))
|
||||
chainDB.addBlockNumberToHashLookup(header)
|
||||
chainDB.addBlockNumberToHashLookup(header.number, header.blockHash)
|
||||
chainDB.persistTransactions(header.number, header.txRoot, getBlockBody4514995().transactions)
|
||||
chainDB.persistReceipts(header.receiptsRoot, getReceipts4514995())
|
||||
|
||||
proc setupEnv(com: CommonRef, signer, ks2: Address, ctx: EthContext): TestEnv =
|
||||
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()
|
||||
|
||||
|
||||
# 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
|
||||
parent = com.db.getCanonicalHead()
|
||||
acc = ctx.am.getAccount(signer).tryGet()
|
||||
blockNumber = 1.BlockNumber
|
||||
blockNumber = 1'u64
|
||||
parent = com.db.getCanonicalHead()
|
||||
parentHash = parent.blockHash
|
||||
|
||||
const code = evmByteCode:
|
||||
let code = evmByteCode:
|
||||
Push4 "0xDEADBEEF" # PUSH
|
||||
Push1 "0x00" # MSTORE AT 0x00
|
||||
Mstore
|
||||
|
@ -99,28 +111,26 @@ proc setupEnv(com: CommonRef, signer, ks2: Address, ctx: EthContext): TestEnv =
|
|||
|
||||
let
|
||||
vmHeader = Header(parentHash: parentHash, gasLimit: 5_000_000)
|
||||
vmState = BaseVMState.new(
|
||||
parent = Header(stateRoot: parent.stateRoot),
|
||||
header = vmHeader,
|
||||
com = com)
|
||||
vmState = BaseVMState()
|
||||
vmState.init(parent, vmHeader, com)
|
||||
|
||||
vmState.stateDB.setCode(ks2, code)
|
||||
vmState.stateDB.addBalance(
|
||||
signer, 1.u256 * 1_000_000_000.u256 * 1_000_000_000.u256) # 1 ETH
|
||||
|
||||
# Test data created for eth_getProof tests
|
||||
let regularAcc = Hash32.fromHex("0x0000000000000000000000000000000000000001")
|
||||
let regularAcc = Address.fromHex("0x0000000000000000000000000000000000000001")
|
||||
vmState.stateDB.addBalance(regularAcc, 2_000_000_000.u256)
|
||||
vmState.stateDB.setNonce(regularAcc, 1.uint64)
|
||||
|
||||
let contractAccWithStorage = Hash32.fromHex("0x0000000000000000000000000000000000000002")
|
||||
let contractAccWithStorage = Address.fromHex("0x0000000000000000000000000000000000000002")
|
||||
vmState.stateDB.addBalance(contractAccWithStorage, 1_000_000_000.u256)
|
||||
vmState.stateDB.setNonce(contractAccWithStorage, 2.uint64)
|
||||
vmState.stateDB.setCode(contractAccWithStorage, code)
|
||||
vmState.stateDB.setStorage(contractAccWithStorage, u256(0), u256(1234))
|
||||
vmState.stateDB.setStorage(contractAccWithStorage, u256(1), u256(2345))
|
||||
|
||||
let contractAccNoStorage = Hash32.fromHex("0x0000000000000000000000000000000000000003")
|
||||
let contractAccNoStorage = Address.fromHex("0x0000000000000000000000000000000000000003")
|
||||
vmState.stateDB.setCode(contractAccNoStorage, code)
|
||||
|
||||
|
||||
|
@ -128,19 +138,19 @@ proc setupEnv(com: CommonRef, signer, ks2: Address, ctx: EthContext): TestEnv =
|
|||
unsignedTx1 = Transaction(
|
||||
txType : TxLegacy,
|
||||
nonce : 0,
|
||||
gasPrice: 30_000_000_000,
|
||||
gasPrice: uint64(30_000_000_000),
|
||||
gasLimit: 70_000,
|
||||
value : 1.u256,
|
||||
to : some(zeroAddress),
|
||||
to : Opt.some(zeroAddress),
|
||||
chainId : com.chainId,
|
||||
)
|
||||
unsignedTx2 = Transaction(
|
||||
txType : TxLegacy,
|
||||
nonce : 1,
|
||||
gasPrice: 30_000_000_100,
|
||||
gasPrice: uint64(30_000_000_100),
|
||||
gasLimit: 70_000,
|
||||
value : 2.u256,
|
||||
to : some(zeroAddress),
|
||||
to : Opt.some(zeroAddress),
|
||||
chainId : com.chainId,
|
||||
)
|
||||
eip155 = com.isEIP155(com.syncCurrent)
|
||||
|
@ -159,13 +169,10 @@ proc setupEnv(com: CommonRef, signer, ks2: Address, ctx: EthContext): TestEnv =
|
|||
doAssert(rc.isOk, "Invalid transaction: " & rc.error)
|
||||
vmState.receipts[txIndex] = makeReceipt(vmState, tx.txType)
|
||||
|
||||
com.db.persistReceipts(vmState.receipts)
|
||||
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`
|
||||
#
|
||||
receiptRoot = com.db.ctx.getColumn(CtReceipts).state(updateOk=true).valueOr(EMPTY_ROOT_HASH)
|
||||
date = dateTime(2017, mMar, 30)
|
||||
timeStamp = date.toTime.toUnix.EthTime
|
||||
difficulty = com.calcDifficulty(timeStamp, parent)
|
||||
|
@ -175,37 +182,43 @@ proc setupEnv(com: CommonRef, signer, ks2: Address, ctx: EthContext): TestEnv =
|
|||
|
||||
var header = Header(
|
||||
parentHash : parentHash,
|
||||
#coinbase*: Address
|
||||
stateRoot : vmState.stateDB.getStateRoot(),
|
||||
txRoot : txRoot,
|
||||
receiptsRoot : receiptsRoot,
|
||||
bloom : createBloom(vmState.receipts),
|
||||
stateRoot : vmState.stateDB.getStateRoot,
|
||||
transactionsRoot : txRoot,
|
||||
receiptsRoot : calcReceiptsRoot(vmState.receipts),
|
||||
logsBloom : createBloom(vmState.receipts),
|
||||
difficulty : difficulty,
|
||||
blockNumber : blockNumber,
|
||||
number : blockNumber,
|
||||
gasLimit : vmState.cumulativeGasUsed + 1_000_000,
|
||||
gasUsed : vmState.cumulativeGasUsed,
|
||||
timestamp : timeStamp
|
||||
#extraData: Blob
|
||||
#mixHash: Hash32
|
||||
#nonce: BlockNonce
|
||||
)
|
||||
|
||||
doAssert com.db.persistHeader(header,
|
||||
com.pos.isNil, com.startOfHistory)
|
||||
|
||||
let uncles = [header]
|
||||
header.ommersHash = com.db.persistUncles(uncles)
|
||||
|
||||
doAssert com.db.persistHeader(header,
|
||||
com.consensus == ConsensusType.POS)
|
||||
com.pos.isNil, com.startOfHistory)
|
||||
|
||||
com.db.persistFixtureBlock()
|
||||
|
||||
com.db.persistent(header.number).isOkOr:
|
||||
echo "Failed to save state: ", $error
|
||||
quit(QuitFailure)
|
||||
|
||||
result = TestEnv(
|
||||
txHash: signedTx1.rlpHash,
|
||||
blockHash: header.hash
|
||||
blockHash: header.blockHash
|
||||
)
|
||||
|
||||
|
||||
proc rpcMain*() =
|
||||
suite "Remote Procedure Calls":
|
||||
# TODO: Include other transports such as Http
|
||||
let
|
||||
conf = makeTestConfig()
|
||||
conf = makeConfig(@[])
|
||||
ctx = newEthContext()
|
||||
ethNode = setupEthNode(conf, ctx, eth)
|
||||
com = CommonRef.new(
|
||||
|
@ -213,12 +226,9 @@ proc rpcMain*() =
|
|||
conf.networkId,
|
||||
conf.networkParams
|
||||
)
|
||||
signer = Hash32 bytes32"0x0e69cde81b1aa07a45c32c6cd85d67229d36bb1b"
|
||||
ks2 = Hash32 bytes32"0xa3b2222afa5c987da6ef773fde8d01b9f23d481f"
|
||||
ks3 = Hash32 bytes32"0x597176e9a64aad0845d83afdaf698fbeff77703b"
|
||||
|
||||
# disable POS/post Merge feature
|
||||
com.setTTD none(DifficultyInt)
|
||||
signer = Address.fromHex "0x0e69cde81b1aa07a45c32c6cd85d67229d36bb1b"
|
||||
ks2 = Address.fromHex "0xa3b2222afa5c987da6ef773fde8d01b9f23d481f"
|
||||
ks3 = Address.fromHex "0x597176e9a64aad0845d83afdaf698fbeff77703b"
|
||||
|
||||
let keyStore = "tests" / "keystore"
|
||||
let res = ctx.am.loadKeystores(keyStore)
|
||||
|
@ -232,24 +242,30 @@ proc rpcMain*() =
|
|||
debugEcho unlock.error
|
||||
doAssert(unlock.isOk)
|
||||
|
||||
let env = setupEnv(com, signer, ks2, ctx)
|
||||
let
|
||||
env = setupEnv(signer, ks2, ctx, com)
|
||||
chain = ForkedChainRef.init(com)
|
||||
txPool = TxPoolRef.new(com)
|
||||
|
||||
# Create Ethereum RPCs
|
||||
let RPC_PORT = 0 # let the OS choose a port
|
||||
var
|
||||
rpcServer = newRpcSocketServer(["127.0.0.1:" & $RPC_PORT])
|
||||
client = newRpcSocketClient()
|
||||
txPool = TxPoolRef.new(com, conf.engineSigner)
|
||||
oracle = Oracle.new(com)
|
||||
# txPool must be informed of active head
|
||||
# so it can know the latest account state
|
||||
doAssert txPool.smartHead(chain.latestHeader, chain)
|
||||
|
||||
setupCommonRpc(ethNode, conf, rpcServer)
|
||||
setupEthRpc(ethNode, ctx, com, txPool, oracle, rpcServer)
|
||||
let
|
||||
server = newRpcHttpServerWithParams("127.0.0.1:0").valueOr:
|
||||
quit(QuitFailure)
|
||||
serverApi = newServerAPI(chain, 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)
|
||||
|
||||
# Begin tests
|
||||
rpcServer.start()
|
||||
waitFor client.connect("127.0.0.1", rpcServer.localAddress[0].port)
|
||||
|
||||
# TODO: add more tests here
|
||||
test "web3_clientVersion":
|
||||
let res = await client.web3_clientVersion()
|
||||
check res == ClientId
|
||||
|
@ -274,15 +290,6 @@ proc rpcMain*() =
|
|||
let peerCount = ethNode.peerPool.connectedNodes.len
|
||||
check res == w3Qty(peerCount)
|
||||
|
||||
test "eth_protocolVersion":
|
||||
let res = await client.eth_protocolVersion()
|
||||
# Use a hard-coded number instead of the same expression as the client,
|
||||
# so that bugs introduced via that expression are detected. Using the
|
||||
# same expression as the client can hide issues when the value is wrong
|
||||
# in both places. When the expected value genuinely changes, it'll be
|
||||
# obvious. Just change this number.
|
||||
check res == $ethVersion
|
||||
|
||||
test "eth_chainId":
|
||||
let res = await client.eth_chainId()
|
||||
check res == w3Qty(distinctBase(com.chainId))
|
||||
|
@ -293,24 +300,9 @@ proc rpcMain*() =
|
|||
let syncing = ethNode.peerPool.connectedNodes.len > 0
|
||||
check syncing == false
|
||||
else:
|
||||
check com.syncStart == res.syncObject.startingBlock.uint64.u256
|
||||
check com.syncCurrent == res.syncObject.currentBlock.uint64.u256
|
||||
check com.syncHighest == res.syncObject.highestBlock.uint64.u256
|
||||
|
||||
test "eth_coinbase":
|
||||
let res = await client.eth_coinbase()
|
||||
# currently we don't have miner
|
||||
check res == default(Address)
|
||||
|
||||
test "eth_mining":
|
||||
let res = await client.eth_mining()
|
||||
# currently we don't have miner
|
||||
check res == false
|
||||
|
||||
test "eth_hashrate":
|
||||
let res = await client.eth_hashrate()
|
||||
# currently we don't have miner
|
||||
check res == w3Qty(0'u64)
|
||||
check com.syncStart == res.syncObject.startingBlock.uint64
|
||||
check com.syncCurrent == res.syncObject.currentBlock.uint64
|
||||
check com.syncHighest == res.syncObject.highestBlock.uint64
|
||||
|
||||
test "eth_gasPrice":
|
||||
let res = await client.eth_gasPrice()
|
||||
|
@ -327,23 +319,23 @@ proc rpcMain*() =
|
|||
check res == w3Qty(0x1'u64)
|
||||
|
||||
test "eth_getBalance":
|
||||
let a = await client.eth_getBalance(Hash32.fromHex("0xfff33a3bd36abdbd412707b8e310d6011454a7ae"), blockId(0'u64))
|
||||
let a = await client.eth_getBalance(Address.fromHex("0xfff33a3bd36abdbd412707b8e310d6011454a7ae"), blockId(1'u64))
|
||||
check a == UInt256.fromHex("0x1b1ae4d6e2ef5000000")
|
||||
let b = await client.eth_getBalance(Hash32.fromHex("0xfff4bad596633479a2a29f9a8b3f78eefd07e6ee"), blockId(0'u64))
|
||||
let b = await client.eth_getBalance(Address.fromHex("0xfff4bad596633479a2a29f9a8b3f78eefd07e6ee"), blockId(1'u64))
|
||||
check b == UInt256.fromHex("0x56bc75e2d63100000")
|
||||
let c = await client.eth_getBalance(Hash32.fromHex("0xfff7ac99c8e4feb60c9750054bdc14ce1857f181"), blockId(0'u64))
|
||||
let c = await client.eth_getBalance(Address.fromHex("0xfff7ac99c8e4feb60c9750054bdc14ce1857f181"), blockId(1'u64))
|
||||
check c == UInt256.fromHex("0x3635c9adc5dea00000")
|
||||
|
||||
test "eth_getStorageAt":
|
||||
let res = await client.eth_getStorageAt(Hash32.fromHex("0xfff33a3bd36abdbd412707b8e310d6011454a7ae"), 0.u256, blockId(0'u64))
|
||||
check default(Hash32) == res
|
||||
let res = await client.eth_getStorageAt(Address.fromHex("0xfff33a3bd36abdbd412707b8e310d6011454a7ae"), 0.u256, blockId(1'u64))
|
||||
check FixedBytes[32](zeroHash32.data) == res
|
||||
|
||||
test "eth_getTransactionCount":
|
||||
let res = await client.eth_getTransactionCount(Hash32.fromHex("0xfff7ac99c8e4feb60c9750054bdc14ce1857f181"), blockId(0'u64))
|
||||
let res = await client.eth_getTransactionCount(Address.fromHex("0xfff7ac99c8e4feb60c9750054bdc14ce1857f181"), blockId(1'u64))
|
||||
check res == w3Qty(0'u64)
|
||||
|
||||
test "eth_getBlockTransactionCountByHash":
|
||||
let hash = com.db.getBlockHash(0.BlockNumber)
|
||||
let hash = com.db.getBlockHash(0'u64)
|
||||
let res = await client.eth_getBlockTransactionCountByHash(hash)
|
||||
check res == w3Qty(0'u64)
|
||||
|
||||
|
@ -352,7 +344,7 @@ proc rpcMain*() =
|
|||
check res == w3Qty(0'u64)
|
||||
|
||||
test "eth_getUncleCountByBlockHash":
|
||||
let hash = com.db.getBlockHash(0.BlockNumber)
|
||||
let hash = com.db.getBlockHash(0'u64)
|
||||
let res = await client.eth_getUncleCountByBlockHash(hash)
|
||||
check res == w3Qty(0'u64)
|
||||
|
||||
|
@ -361,7 +353,7 @@ proc rpcMain*() =
|
|||
check res == w3Qty(0'u64)
|
||||
|
||||
test "eth_getCode":
|
||||
let res = await client.eth_getCode(Hash32.fromHex("0xfff7ac99c8e4feb60c9750054bdc14ce1857f181"), blockId(0'u64))
|
||||
let res = await client.eth_getCode(Address.fromHex("0xfff7ac99c8e4feb60c9750054bdc14ce1857f181"), blockId(1'u64))
|
||||
check res.len == 0
|
||||
|
||||
test "eth_sign":
|
||||
|
@ -384,12 +376,12 @@ proc rpcMain*() =
|
|||
|
||||
test "eth_signTransaction, eth_sendTransaction, eth_sendRawTransaction":
|
||||
var unsignedTx = TransactionArgs(
|
||||
`from`: signer.some,
|
||||
to: ks2.some,
|
||||
gas: w3Qty(100000'u).some,
|
||||
gasPrice: none(Quantity),
|
||||
value: some 100.u256,
|
||||
nonce: none(Quantity)
|
||||
`from`: Opt.some(signer),
|
||||
to: Opt.some(ks2),
|
||||
gas: Opt.some(w3Qty(100000'u)),
|
||||
gasPrice: Opt.none(Quantity),
|
||||
value: Opt.some(100.u256),
|
||||
nonce: Opt.none(Quantity)
|
||||
)
|
||||
|
||||
let signedTxBytes = await client.eth_signTransaction(unsignedTx)
|
||||
|
@ -402,11 +394,11 @@ proc rpcMain*() =
|
|||
|
||||
test "eth_call":
|
||||
var ec = TransactionArgs(
|
||||
`from`: signer.some,
|
||||
to: ks2.some,
|
||||
gas: w3Qty(100000'u).some,
|
||||
gasPrice: none(Quantity),
|
||||
value: some 100.u256
|
||||
`from`: Opt.some(signer),
|
||||
to: Opt.some(ks2),
|
||||
gas: Opt.some(w3Qty(100000'u)),
|
||||
gasPrice: Opt.none(Quantity),
|
||||
value: Opt.some(100.u256)
|
||||
)
|
||||
|
||||
let res = await client.eth_call(ec, "latest")
|
||||
|
@ -414,11 +406,11 @@ proc rpcMain*() =
|
|||
|
||||
test "eth_estimateGas":
|
||||
var ec = TransactionArgs(
|
||||
`from`: signer.some,
|
||||
to: ks3.some,
|
||||
gas: w3Qty(42000'u).some,
|
||||
gasPrice: w3Qty(100'u).some,
|
||||
value: some 100.u256
|
||||
`from`: Opt.some(signer),
|
||||
to: Opt.some(ks3),
|
||||
gas: Opt.some(w3Qty(42000'u)),
|
||||
gasPrice: Opt.some(w3Qty(100'u)),
|
||||
value: Opt.some(100.u256)
|
||||
)
|
||||
|
||||
let res = await client.eth_estimateGas(ec)
|
||||
|
@ -441,14 +433,14 @@ proc rpcMain*() =
|
|||
test "eth_getTransactionByHash":
|
||||
let res = await client.eth_getTransactionByHash(env.txHash)
|
||||
check res.isNil.not
|
||||
check res.number.get() == w3BlockNumber(1'u64)
|
||||
check res.blockNumber.get() == w3BlockNumber(1'u64)
|
||||
let res2 = await client.eth_getTransactionByHash(env.blockHash)
|
||||
check res2.isNil
|
||||
|
||||
test "eth_getTransactionByBlockHashAndIndex":
|
||||
let res = await client.eth_getTransactionByBlockHashAndIndex(env.blockHash, w3Qty(0'u64))
|
||||
check res.isNil.not
|
||||
check res.number.get() == w3BlockNumber(1'u64)
|
||||
check res.blockNumber.get() == w3BlockNumber(1'u64)
|
||||
|
||||
let res2 = await client.eth_getTransactionByBlockHashAndIndex(env.blockHash, w3Qty(3'u64))
|
||||
check res2.isNil
|
||||
|
@ -459,18 +451,29 @@ proc rpcMain*() =
|
|||
test "eth_getTransactionByBlockNumberAndIndex":
|
||||
let res = await client.eth_getTransactionByBlockNumberAndIndex("latest", w3Qty(1'u64))
|
||||
check res.isNil.not
|
||||
check res.number.get() == w3BlockNumber(1'u64)
|
||||
check res.blockNumber.get() == w3BlockNumber(1'u64)
|
||||
|
||||
let res2 = await client.eth_getTransactionByBlockNumberAndIndex("latest", w3Qty(3'u64))
|
||||
check res2.isNil
|
||||
|
||||
test "eth_getTransactionReceipt":
|
||||
let res = await client.eth_getTransactionReceipt(env.txHash)
|
||||
check res.isNil.not
|
||||
check res.number == w3BlockNumber(1'u64)
|
||||
# TODO: Solved with Issue #2700
|
||||
|
||||
let res2 = await client.eth_getTransactionReceipt(env.blockHash)
|
||||
check res2.isNil
|
||||
# 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 == w3BlockNumber(1'u64)
|
||||
|
||||
# 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))
|
||||
|
@ -495,7 +498,7 @@ proc rpcMain*() =
|
|||
let testHeader = getBlockHeader4514995()
|
||||
let testHash = testHeader.blockHash
|
||||
let filterOptions = FilterOptions(
|
||||
blockHash: some(testHash),
|
||||
blockHash: Opt.some(testHash),
|
||||
topics: @[]
|
||||
)
|
||||
let logs = await client.eth_getLogs(filterOptions)
|
||||
|
@ -507,41 +510,19 @@ proc rpcMain*() =
|
|||
for l in logs:
|
||||
check:
|
||||
l.blockHash.isSome()
|
||||
l.blockHash.unsafeGet() == testHash
|
||||
l.logIndex.unsafeGet() == w3Qty(i.uint64)
|
||||
inc i
|
||||
|
||||
test "eth_getLogs by blockNumber, no filters":
|
||||
let testHeader = getBlockHeader4514995()
|
||||
let testHash = testHeader.blockHash
|
||||
let fBlock = blockId(testHeader.number)
|
||||
let tBlock = blockId(testHeader.number)
|
||||
let filterOptions = FilterOptions(
|
||||
fromBlock: some(fBlock),
|
||||
toBlock: some(tBlock)
|
||||
)
|
||||
let logs = await client.eth_getLogs(filterOptions)
|
||||
|
||||
check:
|
||||
len(logs) == 54
|
||||
|
||||
var i = 0
|
||||
for l in logs:
|
||||
check:
|
||||
l.blockHash.isSome()
|
||||
l.blockHash.unsafeGet() == testHash
|
||||
l.logIndex.unsafeGet() == w3Qty(i.uint64)
|
||||
l.blockHash.get() == testHash
|
||||
l.logIndex.get() == w3Qty(i.uint64)
|
||||
inc i
|
||||
|
||||
test "eth_getLogs by blockhash, filter logs at specific positions":
|
||||
let testHeader = getBlockHeader4514995()
|
||||
let testHash = testHeader.blockHash
|
||||
|
||||
let topic = Hash32.fromHex("0xddf252ad1be2c89b69c2b068fc378daa952ba7f163c4a11628f55a4df523b3ef")
|
||||
let topic1 = Hash32.fromHex("0x000000000000000000000000fdc183d01a793613736cd40a5a578f49add1772b")
|
||||
let topic = Bytes32.fromHex("0xddf252ad1be2c89b69c2b068fc378daa952ba7f163c4a11628f55a4df523b3ef")
|
||||
let topic1 = Bytes32.fromHex("0x000000000000000000000000fdc183d01a793613736cd40a5a578f49add1772b")
|
||||
|
||||
let filterOptions = FilterOptions(
|
||||
blockHash: some(testHash),
|
||||
blockHash: Opt.some(testHash),
|
||||
topics: @[
|
||||
TopicOrList(kind: slkList, list: @[topic]),
|
||||
TopicOrList(kind: slkNull),
|
||||
|
@ -559,16 +540,16 @@ proc rpcMain*() =
|
|||
let testHeader = getBlockHeader4514995()
|
||||
let testHash = testHeader.blockHash
|
||||
|
||||
let topic = Hash32.fromHex("0xddf252ad1be2c89b69c2b068fc378daa952ba7f163c4a11628f55a4df523b3ef")
|
||||
let topic1 = Hash32.fromHex("0xa64da754fccf55aa65a1f0128a648633fade3884b236e879ee9f64c78df5d5d7")
|
||||
let topic = Bytes32.fromHex("0xddf252ad1be2c89b69c2b068fc378daa952ba7f163c4a11628f55a4df523b3ef")
|
||||
let topic1 = Bytes32.fromHex("0xa64da754fccf55aa65a1f0128a648633fade3884b236e879ee9f64c78df5d5d7")
|
||||
|
||||
let topic2 = Hash32.fromHex("0x000000000000000000000000e16c02eac87920033ac72fc55ee1df3151c75786")
|
||||
let topic3 = Hash32.fromHex("0x000000000000000000000000b626a5facc4de1c813f5293ec3be31979f1d1c78")
|
||||
let topic2 = Bytes32.fromHex("0x000000000000000000000000e16c02eac87920033ac72fc55ee1df3151c75786")
|
||||
let topic3 = Bytes32.fromHex("0x000000000000000000000000b626a5facc4de1c813f5293ec3be31979f1d1c78")
|
||||
|
||||
|
||||
|
||||
let filterOptions = FilterOptions(
|
||||
blockHash: some(testHash),
|
||||
blockHash: Opt.some(testHash),
|
||||
topics: @[
|
||||
TopicOrList(kind: slkList, list: @[topic, topic1]),
|
||||
TopicOrList(kind: slkList, list: @[topic2, topic3])
|
||||
|
@ -586,7 +567,7 @@ proc rpcMain*() =
|
|||
block:
|
||||
# account doesn't exist
|
||||
let
|
||||
address = Hash32.fromHex("0x0000000000000000000000000000000000000004")
|
||||
address = Address.fromHex("0x0000000000000000000000000000000000000004")
|
||||
proofResponse = await client.eth_getProof(address, @[], blockId(1'u64))
|
||||
storageProof = proofResponse.storageProof
|
||||
|
||||
|
@ -602,7 +583,7 @@ proc rpcMain*() =
|
|||
block:
|
||||
# account exists but requested slots don't exist
|
||||
let
|
||||
address = Hash32.fromHex("0x0000000000000000000000000000000000000001")
|
||||
address = Address.fromHex("0x0000000000000000000000000000000000000001")
|
||||
slot1Key = 0.u256
|
||||
slot2Key = 1.u256
|
||||
proofResponse = await client.eth_getProof(address, @[slot1Key, slot2Key], blockId(1'u64))
|
||||
|
@ -626,7 +607,7 @@ proc rpcMain*() =
|
|||
block:
|
||||
# contract account with no storage slots
|
||||
let
|
||||
address = Hash32.fromHex("0x0000000000000000000000000000000000000003")
|
||||
address = Address.fromHex("0x0000000000000000000000000000000000000003")
|
||||
slot1Key = 0.u256 # Doesn't exist
|
||||
proofResponse = await client.eth_getProof(address, @[slot1Key], blockId(1'u64))
|
||||
storageProof = proofResponse.storageProof
|
||||
|
@ -649,7 +630,7 @@ proc rpcMain*() =
|
|||
block:
|
||||
# contract account with storage slots
|
||||
let
|
||||
address = Hash32.fromHex("0x0000000000000000000000000000000000000002")
|
||||
address = Address.fromHex("0x0000000000000000000000000000000000000002")
|
||||
slot1Key = 0.u256
|
||||
slot2Key = 1.u256
|
||||
slot3Key = 2.u256 # Doesn't exist
|
||||
|
@ -680,7 +661,7 @@ proc rpcMain*() =
|
|||
block:
|
||||
# externally owned account
|
||||
let
|
||||
address = Hash32.fromHex("0x0000000000000000000000000000000000000001")
|
||||
address = Address.fromHex("0x0000000000000000000000000000000000000001")
|
||||
proofResponse = await client.eth_getProof(address, @[], blockId(1'u64))
|
||||
storageProof = proofResponse.storageProof
|
||||
|
||||
|
@ -696,28 +677,10 @@ proc rpcMain*() =
|
|||
test "eth_getProof - Multiple blocks":
|
||||
let blockData = await client.eth_getBlockByNumber("latest", true)
|
||||
|
||||
block:
|
||||
# block 0 - account doesn't exist yet
|
||||
let
|
||||
address = Hash32.fromHex("0x0000000000000000000000000000000000000002")
|
||||
slot1Key = 100.u256
|
||||
proofResponse = await client.eth_getProof(address, @[slot1Key], blockId(0'u64))
|
||||
storageProof = proofResponse.storageProof
|
||||
|
||||
check:
|
||||
proofResponse.address == address
|
||||
verifyAccountProof(blockData.stateRoot, proofResponse).kind == InvalidProof
|
||||
proofResponse.balance == 0.u256
|
||||
proofResponse.codeHash == zeroHash()
|
||||
proofResponse.nonce == w3Qty(0.uint64)
|
||||
proofResponse.storageHash == zeroHash()
|
||||
storageProof.len() == 1
|
||||
verifySlotProof(proofResponse.storageHash, storageProof[0]).kind == InvalidProof
|
||||
|
||||
block:
|
||||
# block 1 - account has balance, code and storage
|
||||
let
|
||||
address = Hash32.fromHex("0x0000000000000000000000000000000000000002")
|
||||
address = Address.fromHex("0x0000000000000000000000000000000000000002")
|
||||
slot2Key = 1.u256
|
||||
proofResponse = await client.eth_getProof(address, @[slot2Key], blockId(1'u64))
|
||||
storageProof = proofResponse.storageProof
|
||||
|
@ -732,17 +695,13 @@ proc rpcMain*() =
|
|||
storageProof.len() == 1
|
||||
verifySlotProof(proofResponse.storageHash, storageProof[0]).isValid()
|
||||
|
||||
test "eth_getBlockReceipts":
|
||||
let recs = await client.eth_getBlockReceipts(blockId("latest"))
|
||||
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
|
||||
close(client, server)
|
||||
|
||||
rpcServer.stop()
|
||||
rpcServer.close()
|
||||
proc setErrorLevel* =
|
||||
discard
|
||||
when defined(chronicles_runtime_filtering) and loggingEnabled:
|
||||
setLogLevel(LogLevel.ERROR)
|
||||
|
||||
when isMainModule:
|
||||
setErrorLevel()
|
||||
rpcMain()
|
||||
|
|
Loading…
Reference in New Issue