mirror of
https://github.com/status-im/nimbus-eth1.git
synced 2025-01-12 05:14:14 +00:00
Implementation of procs to support Fluffy state JSON-RPC endpoints. (#2318)
* Started implementation of state endpoints. * Add rpc calls and server stubs. * Initial implementation of getAccountProof and getStorageProof. * Refactor validation to use toAccount utils functions. * Add state endpoints tests.
This commit is contained in:
parent
f6be4bd0ec
commit
c72d6aa5d6
@ -610,17 +610,23 @@ proc getEpochAccumulator(
|
||||
|
||||
return Opt.none(EpochAccumulator)
|
||||
|
||||
proc getBlock*(
|
||||
proc getBlockHashByNumber*(
|
||||
n: HistoryNetwork, bn: UInt256
|
||||
): Future[Result[Opt[Block], string]] {.async.} =
|
||||
): Future[Result[BlockHash, string]] {.async.} =
|
||||
let
|
||||
epochData = n.accumulator.getBlockEpochDataForBlockNumber(bn).valueOr:
|
||||
return err(error)
|
||||
digest = Digest(data: epochData.epochHash)
|
||||
epoch = (await n.getEpochAccumulator(digest)).valueOr:
|
||||
return err("Cannot retrieve epoch accumulator for given block number")
|
||||
blockHash = epoch[epochData.blockRelativeIndex].blockHash
|
||||
|
||||
ok(epoch[epochData.blockRelativeIndex].blockHash)
|
||||
|
||||
proc getBlock*(
|
||||
n: HistoryNetwork, bn: UInt256
|
||||
): Future[Result[Opt[Block], string]] {.async.} =
|
||||
let
|
||||
blockHash = ?(await n.getBlockHashByNumber(bn))
|
||||
maybeBlock = await n.getBlock(blockHash)
|
||||
|
||||
return ok(maybeBlock)
|
||||
|
@ -91,6 +91,9 @@ func toRetrievalValue*(offer: ContractTrieNodeOffer): ContractTrieNodeRetrieval
|
||||
func toRetrievalValue*(offer: ContractCodeOffer): ContractCodeRetrieval =
|
||||
ContractCodeRetrieval.init(offer.code)
|
||||
|
||||
func empty*(T: type TrieProof): T =
|
||||
TrieProof.init(@[])
|
||||
|
||||
func encode*(value: ContentValueType): seq[byte] =
|
||||
SSZ.encode(value)
|
||||
|
||||
|
@ -44,6 +44,9 @@ func init*(T: type Nibbles, packed: openArray[byte], isEven: bool): T =
|
||||
|
||||
Nibbles(output)
|
||||
|
||||
func empty*(T: type Nibbles): T =
|
||||
Nibbles.init(@[], true)
|
||||
|
||||
func encode*(nibbles: Nibbles): seq[byte] =
|
||||
SSZ.encode(nibbles)
|
||||
|
||||
|
184
fluffy/network/state/state_endpoints.nim
Normal file
184
fluffy/network/state/state_endpoints.nim
Normal file
@ -0,0 +1,184 @@
|
||||
# Fluffy
|
||||
# Copyright (c) 2021-2024 Status Research & Development GmbH
|
||||
# Licensed and distributed under either of
|
||||
# * MIT license (license terms in the root directory or at https://opensource.org/licenses/MIT).
|
||||
# * Apache v2 license (license terms in the root directory or at https://www.apache.org/licenses/LICENSE-2.0).
|
||||
# at your option. This file may not be copied, modified, or distributed except according to those terms.
|
||||
|
||||
import
|
||||
results,
|
||||
chronos,
|
||||
chronicles,
|
||||
eth/common/eth_hash,
|
||||
eth/common/eth_types,
|
||||
../../common/common_utils,
|
||||
./state_network,
|
||||
./state_utils
|
||||
|
||||
export results, state_network
|
||||
|
||||
logScope:
|
||||
topics = "portal_state"
|
||||
|
||||
proc getNextNodeHash(
|
||||
trieNode: TrieNode, nibbles: UnpackedNibbles, nibbleIdx: var int
|
||||
): Opt[(Nibbles, NodeHash)] =
|
||||
doAssert(nibbles.len() > 0)
|
||||
doAssert(nibbleIdx < nibbles.len())
|
||||
|
||||
let trieNodeRlp = rlpFromBytes(trieNode.asSeq())
|
||||
# the trie node should have already been validated
|
||||
doAssert(not trieNodeRlp.isEmpty())
|
||||
doAssert(trieNodeRlp.listLen() == 2 or trieNodeRlp.listLen() == 17)
|
||||
|
||||
if trieNodeRlp.listLen() == 17:
|
||||
let nextNibble = nibbles[nibbleIdx]
|
||||
doAssert(nextNibble < 16)
|
||||
|
||||
let nextHashBytes = trieNodeRlp.listElem(nextNibble.int)
|
||||
doAssert(not nextHashBytes.isEmpty())
|
||||
|
||||
nibbleIdx += 1
|
||||
return Opt.some(
|
||||
(
|
||||
nibbles[0 ..< nibbleIdx].packNibbles(),
|
||||
KeccakHash.fromBytes(nextHashBytes.toBytes()),
|
||||
)
|
||||
)
|
||||
|
||||
# leaf or extension node
|
||||
let (_, isLeaf, prefix) = decodePrefix(trieNodeRlp.listElem(0))
|
||||
if isLeaf:
|
||||
return Opt.none((Nibbles, NodeHash))
|
||||
|
||||
# extension node
|
||||
nibbleIdx += prefix.unpackNibbles().len()
|
||||
|
||||
let nextHashBytes = trieNodeRlp.listElem(1)
|
||||
doAssert(not nextHashBytes.isEmpty())
|
||||
|
||||
Opt.some(
|
||||
(
|
||||
nibbles[0 ..< nibbleIdx].packNibbles(),
|
||||
KeccakHash.fromBytes(nextHashBytes.toBytes()),
|
||||
)
|
||||
)
|
||||
|
||||
proc getAccountProof(
|
||||
n: StateNetwork, stateRoot: KeccakHash, address: Address
|
||||
): Future[Opt[TrieProof]] {.async.} =
|
||||
let nibbles = address.toPath().unpackNibbles()
|
||||
|
||||
var
|
||||
nibblesIdx = 0
|
||||
key = AccountTrieNodeKey.init(Nibbles.empty(), stateRoot)
|
||||
proof = TrieProof.empty()
|
||||
|
||||
while nibblesIdx < nibbles.len():
|
||||
let
|
||||
accountTrieNode = (await n.getAccountTrieNode(key)).valueOr:
|
||||
# log something here
|
||||
return Opt.none(TrieProof)
|
||||
trieNode = accountTrieNode.node
|
||||
|
||||
let added = proof.add(trieNode)
|
||||
doAssert(added)
|
||||
|
||||
let (nextPath, nextNodeHash) = trieNode.getNextNodeHash(nibbles, nibblesIdx).valueOr:
|
||||
break
|
||||
|
||||
key = AccountTrieNodeKey.init(nextPath, nextNodeHash)
|
||||
|
||||
Opt.some(proof)
|
||||
|
||||
proc getStorageProof(
|
||||
n: StateNetwork, storageRoot: KeccakHash, address: Address, storageKey: UInt256
|
||||
): Future[Opt[TrieProof]] {.async.} =
|
||||
let nibbles = storageKey.toPath().unpackNibbles()
|
||||
|
||||
var
|
||||
nibblesIdx = 0
|
||||
key = ContractTrieNodeKey.init(address, Nibbles.empty(), storageRoot)
|
||||
proof = TrieProof.empty()
|
||||
|
||||
while nibblesIdx < nibbles.len():
|
||||
let
|
||||
contractTrieNode = (await n.getContractTrieNode(key)).valueOr:
|
||||
# log something here
|
||||
return Opt.none(TrieProof)
|
||||
trieNode = contractTrieNode.node
|
||||
|
||||
let added = proof.add(trieNode)
|
||||
doAssert(added)
|
||||
|
||||
let (nextPath, nextNodeHash) = trieNode.getNextNodeHash(nibbles, nibblesIdx).valueOr:
|
||||
break
|
||||
|
||||
key = ContractTrieNodeKey.init(address, nextPath, nextNodeHash)
|
||||
|
||||
Opt.some(proof)
|
||||
|
||||
proc getAccount(
|
||||
n: StateNetwork, blockHash: BlockHash, address: Address
|
||||
): Future[Opt[Account]] {.async.} =
|
||||
let
|
||||
stateRoot = (await n.getStateRootByBlockHash(blockHash)).valueOr:
|
||||
warn "Failed to get state root by block hash"
|
||||
return Opt.none(Account)
|
||||
accountProof = (await n.getAccountProof(stateRoot, address)).valueOr:
|
||||
warn "Failed to get account proof"
|
||||
return Opt.none(Account)
|
||||
account = accountProof.toAccount().valueOr:
|
||||
warn "Failed to get account from accountProof"
|
||||
return Opt.none(Account)
|
||||
|
||||
Opt.some(account)
|
||||
|
||||
# Used by: eth_getBalance,
|
||||
proc getBalance*(
|
||||
n: StateNetwork, blockHash: BlockHash, address: Address
|
||||
): Future[Opt[UInt256]] {.async.} =
|
||||
let account = (await n.getAccount(blockHash, address)).valueOr:
|
||||
return Opt.none(UInt256)
|
||||
|
||||
Opt.some(account.balance)
|
||||
|
||||
# Used by: eth_getTransactionCount
|
||||
proc getTransactionCount*(
|
||||
n: StateNetwork, blockHash: BlockHash, address: Address
|
||||
): Future[Opt[AccountNonce]] {.async.} =
|
||||
let account = (await n.getAccount(blockHash, address)).valueOr:
|
||||
return Opt.none(AccountNonce)
|
||||
|
||||
Opt.some(account.nonce)
|
||||
|
||||
# Used by: eth_getStorageAt
|
||||
proc getStorageAt*(
|
||||
n: StateNetwork, blockHash: BlockHash, address: Address, slotKey: UInt256
|
||||
): Future[Opt[UInt256]] {.async.} =
|
||||
let
|
||||
account = (await n.getAccount(blockHash, address)).valueOr:
|
||||
return Opt.none(UInt256)
|
||||
storageProof = (await n.getStorageProof(account.storageRoot, address, slotKey)).valueOr:
|
||||
warn "Failed to get storage proof"
|
||||
return Opt.none(UInt256)
|
||||
slotValue = storageProof.toSlot().valueOr:
|
||||
warn "Failed to get slot from storageProof"
|
||||
return Opt.none(UInt256)
|
||||
|
||||
Opt.some(slotValue)
|
||||
|
||||
# Used by: eth_getCode
|
||||
proc getCode*(
|
||||
n: StateNetwork, blockHash: BlockHash, address: Address
|
||||
): Future[Opt[Bytecode]] {.async.} =
|
||||
let
|
||||
account = (await n.getAccount(blockHash, address)).valueOr:
|
||||
return Opt.none(Bytecode)
|
||||
contractCodeKey = ContractCodeKey.init(address, account.codeHash)
|
||||
|
||||
let contractCodeRetrieval = (await n.getContractCode(contractCodeKey)).valueOr:
|
||||
warn "Failed to get contract code"
|
||||
return Opt.none(Bytecode)
|
||||
|
||||
Opt.some(contractCodeRetrieval.code)
|
@ -122,7 +122,7 @@ proc getContractCode*(
|
||||
): Future[Opt[ContractCodeRetrieval]] {.inline.} =
|
||||
n.getContent(key, ContractCodeRetrieval)
|
||||
|
||||
proc getStateRootByBlockHash(
|
||||
proc getStateRootByBlockHash*(
|
||||
n: StateNetwork, hash: BlockHash
|
||||
): Future[Opt[KeccakHash]] {.async.} =
|
||||
if n.historyNetwork.isNone():
|
||||
|
@ -22,8 +22,8 @@ func decodePrefix*(nodePrefixRlp: Rlp): (byte, bool, Nibbles) =
|
||||
|
||||
(firstNibble.byte, isLeaf, nibbles)
|
||||
|
||||
func rlpDecodeAccountTrieNode*(accountNode: TrieNode): Result[Account, string] =
|
||||
let accNodeRlp = rlpFromBytes(accountNode.asSeq())
|
||||
func rlpDecodeAccountTrieNode*(accountTrieNode: TrieNode): Result[Account, string] =
|
||||
let accNodeRlp = rlpFromBytes(accountTrieNode.asSeq())
|
||||
if accNodeRlp.isEmpty() or accNodeRlp.listLen() != 2:
|
||||
return err("invalid account trie node - malformed")
|
||||
|
||||
@ -36,3 +36,39 @@ func rlpDecodeAccountTrieNode*(accountNode: TrieNode): Result[Account, string] =
|
||||
return err("invalid account trie node - leaf prefix expected")
|
||||
|
||||
decodeRlp(accNodeRlp.listElem(1).toBytes(), Account)
|
||||
|
||||
# TODO: test the below functions
|
||||
|
||||
func rlpDecodeContractTrieNode*(contractTrieNode: TrieNode): Result[UInt256, string] =
|
||||
let storageNodeRlp = rlpFromBytes(contractTrieNode.asSeq())
|
||||
if storageNodeRlp.isEmpty() or storageNodeRlp.listLen() != 2:
|
||||
return err("invalid contract trie node - malformed")
|
||||
|
||||
let storageNodePrefixRlp = storageNodeRlp.listElem(0)
|
||||
if storageNodePrefixRlp.isEmpty():
|
||||
return err("invalid contract trie node - empty prefix")
|
||||
|
||||
let (_, isLeaf, _) = decodePrefix(storageNodePrefixRlp)
|
||||
if not isLeaf:
|
||||
return err("invalid contract trie node - leaf prefix expected")
|
||||
|
||||
decodeRlp(storageNodeRlp.listElem(1).toBytes(), UInt256)
|
||||
|
||||
func toAccount*(accountProof: TrieProof): Result[Account, string] {.inline.} =
|
||||
doAssert(accountProof.len() > 1)
|
||||
|
||||
rlpDecodeAccountTrieNode(accountProof[^1])
|
||||
|
||||
func toSlot*(storageProof: TrieProof): Result[UInt256, string] {.inline.} =
|
||||
doAssert(storageProof.len() > 1)
|
||||
|
||||
rlpDecodeContractTrieNode(storageProof[^1])
|
||||
|
||||
func toPath*(hash: KeccakHash): Nibbles {.inline.} =
|
||||
Nibbles.init(hash.data, isEven = true)
|
||||
|
||||
func toPath*(address: Address): Nibbles =
|
||||
keccakHash(address).toPath()
|
||||
|
||||
func toPath*(slotKey: UInt256): Nibbles =
|
||||
keccakHash(toBytesBE(slotKey)).toPath()
|
||||
|
@ -140,15 +140,14 @@ proc validateOffer*(
|
||||
proc validateOffer*(
|
||||
trustedStateRoot: KeccakHash, key: ContractTrieNodeKey, offer: ContractTrieNodeOffer
|
||||
): Result[void, string] =
|
||||
let addressHash = keccakHash(key.address).data
|
||||
?validateTrieProof(
|
||||
trustedStateRoot,
|
||||
Nibbles.init(addressHash, true),
|
||||
key.address.toPath(),
|
||||
offer.accountProof,
|
||||
allowKeyEndInPathForLeafs = true,
|
||||
)
|
||||
|
||||
let account = ?rlpDecodeAccountTrieNode(offer.accountProof[^1])
|
||||
let account = ?offer.accountProof.toAccount()
|
||||
|
||||
?validateTrieProof(account.storageRoot, key.path, offer.storageProof)
|
||||
|
||||
@ -157,15 +156,14 @@ proc validateOffer*(
|
||||
proc validateOffer*(
|
||||
trustedStateRoot: KeccakHash, key: ContractCodeKey, offer: ContractCodeOffer
|
||||
): Result[void, string] =
|
||||
let addressHash = keccakHash(key.address).data
|
||||
?validateTrieProof(
|
||||
trustedStateRoot,
|
||||
Nibbles.init(addressHash, true),
|
||||
key.address.toPath(),
|
||||
offer.accountProof,
|
||||
allowKeyEndInPathForLeafs = true,
|
||||
)
|
||||
|
||||
let account = ?rlpDecodeAccountTrieNode(offer.accountProof[^1])
|
||||
let account = ?offer.accountProof.toAccount()
|
||||
if not offer.code.hashEquals(account.codeHash):
|
||||
return err("hash of bytecode doesn't match the code hash in the account proof")
|
||||
|
||||
|
@ -9,6 +9,7 @@
|
||||
|
||||
import
|
||||
std/json,
|
||||
stint,
|
||||
json_serialization/stew/results,
|
||||
json_rpc/[client, jsonmarshal],
|
||||
web3/conversions,
|
||||
@ -30,3 +31,14 @@ createRpcSigsFromNim(RpcClient):
|
||||
proc eth_getBlockReceipts(blockId: string): Opt[seq[ReceiptObject]]
|
||||
proc eth_getBlockReceipts(blockId: BlockNumber): Opt[seq[ReceiptObject]]
|
||||
proc eth_getBlockReceipts(blockId: RtBlockIdentifier): Opt[seq[ReceiptObject]]
|
||||
|
||||
proc eth_getBalance(data: Address, blockId: BlockIdentifier): UInt256
|
||||
proc eth_getTransactionCount(data: Address, blockId: BlockIdentifier): Quantity
|
||||
proc eth_getStorageAt(
|
||||
data: Address, slot: UInt256, blockId: BlockIdentifier
|
||||
): FixedBytes[32]
|
||||
|
||||
proc eth_getCode(data: Address, blockId: BlockIdentifier): seq[byte]
|
||||
proc eth_getProof(
|
||||
address: Address, slots: seq[UInt256], blockId: BlockIdentifier
|
||||
): ProofResponse
|
||||
|
@ -343,4 +343,65 @@ proc installEthApiHandlers*(
|
||||
return filteredLogs
|
||||
else:
|
||||
# bloomfilter returned false, there are no logs matching the criteria
|
||||
return @[]
|
||||
return
|
||||
@[]
|
||||
|
||||
# rpcServerWithProxy.rpc("eth_getBalance") do(
|
||||
# data: Address, quantityTag: RtBlockIdentifier
|
||||
# ) -> 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.
|
||||
# # TODO
|
||||
# raiseAssert("Not implemented")
|
||||
|
||||
# rpcServerWithProxy.rpc("eth_getTransactionCount") do(
|
||||
# data: Address, quantityTag: RtBlockIdentifier
|
||||
# ) -> Quantity:
|
||||
# ## 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.
|
||||
# # TODO
|
||||
# raiseAssert("Not implemented")
|
||||
|
||||
# rpcServerWithProxy.rpc("eth_getStorageAt") do(
|
||||
# data: Address, slot: UInt256, quantityTag: RtBlockIdentifier
|
||||
# ) -> 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.
|
||||
# # TODO
|
||||
# raiseAssert("Not implemented")
|
||||
|
||||
# rpcServerWithProxy.rpc("eth_getCode") do(
|
||||
# data: Address, quantityTag: RtBlockIdentifier
|
||||
# ) -> 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.
|
||||
# # TODO
|
||||
# raiseAssert("Not implemented")
|
||||
|
||||
# rpcServerWithProxy.rpc("eth_getProof") do(
|
||||
# address: Address, slots: seq[UInt256], quantityTag: RtBlockIdentifier
|
||||
# ) -> 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
|
||||
# # TODO
|
||||
# raiseAssert("Not implemented")
|
||||
|
@ -11,6 +11,7 @@ import
|
||||
./test_state_content_keys_vectors,
|
||||
./test_state_content_nibbles,
|
||||
./test_state_content_values_vectors,
|
||||
./test_state_endpoints_vectors,
|
||||
./test_state_gossip_getparent_genesis,
|
||||
./test_state_gossip_getparent_vectors,
|
||||
./test_state_gossip_gossipoffer_vectors,
|
||||
|
@ -0,0 +1,267 @@
|
||||
# Fluffy
|
||||
# Copyright (c) 2021-2024 Status Research & Development GmbH
|
||||
# Licensed and distributed under either of
|
||||
# * MIT license (license terms in the root directory or at https://opensource.org/licenses/MIT).
|
||||
# * Apache v2 license (license terms in the root directory or at https://www.apache.org/licenses/LICENSE-2.0).
|
||||
# at your option. This file may not be copied, modified, or distributed except according to those terms.
|
||||
|
||||
import
|
||||
std/os,
|
||||
testutils/unittests,
|
||||
chronos,
|
||||
stew/byteutils,
|
||||
eth/common,
|
||||
eth/p2p/discoveryv5/protocol as discv5_protocol,
|
||||
eth/p2p/discoveryv5/routing_table,
|
||||
../../common/common_utils,
|
||||
../../network/wire/[portal_protocol, portal_stream],
|
||||
../../network/state/
|
||||
[state_content, state_network, state_gossip, state_endpoints, state_utils],
|
||||
../../database/content_db,
|
||||
./state_test_helpers
|
||||
|
||||
procSuite "State Endpoints":
|
||||
const STATE_NODE1_PORT = 20602
|
||||
const STATE_NODE2_PORT = 20603
|
||||
|
||||
let rng = newRng()
|
||||
|
||||
asyncTest "Gossip then query getBalance and getTransactionCount":
|
||||
const file = testVectorDir / "recursive_gossip.yaml"
|
||||
|
||||
let
|
||||
testCase = YamlRecursiveGossipKVs.loadFromYaml(file).valueOr:
|
||||
raiseAssert "Cannot read test vector: " & error
|
||||
stateNode1 = newStateNode(rng, STATE_NODE1_PORT)
|
||||
stateNode2 = newStateNode(rng, STATE_NODE2_PORT)
|
||||
|
||||
stateNode1.start()
|
||||
stateNode2.start()
|
||||
|
||||
check:
|
||||
stateNode1.portalProtocol().addNode(stateNode2.localNode()) == Added
|
||||
stateNode2.portalProtocol().addNode(stateNode1.localNode()) == Added
|
||||
(await stateNode1.portalProtocol().ping(stateNode2.localNode())).isOk()
|
||||
(await stateNode2.portalProtocol().ping(stateNode1.localNode())).isOk()
|
||||
|
||||
for i, testData in testCase:
|
||||
if i == 1:
|
||||
continue
|
||||
|
||||
let
|
||||
stateRoot = KeccakHash.fromBytes(testData.state_root.hexToSeqByte())
|
||||
leafData = testData.recursive_gossip[0]
|
||||
contentKeyBytes = leafData.content_key.hexToSeqByte().ByteList
|
||||
contentKey = ContentKey.decode(contentKeyBytes).get()
|
||||
contentId = toContentId(contentKeyBytes)
|
||||
contentValueBytes = leafData.content_value.hexToSeqByte()
|
||||
contentValue = AccountTrieNodeOffer.decode(contentValueBytes).get()
|
||||
|
||||
# set valid state root
|
||||
stateNode1.mockBlockHashToStateRoot(contentValue.blockHash, stateRoot)
|
||||
stateNode2.mockBlockHashToStateRoot(contentValue.blockHash, stateRoot)
|
||||
|
||||
# offer the leaf node
|
||||
await stateNode1.portalProtocol.gossipOffer(
|
||||
Opt.none(NodeId),
|
||||
contentKeyBytes,
|
||||
contentValueBytes,
|
||||
contentKey.accountTrieNodeKey,
|
||||
contentValue,
|
||||
)
|
||||
|
||||
# wait for recursive gossip to complete
|
||||
await sleepAsync(2000.milliseconds)
|
||||
|
||||
let
|
||||
address =
|
||||
if i == 0:
|
||||
EthAddress.fromHex("0xc02aaa39b223fe8d0a0e5c4f27ead9083c756cc2")
|
||||
elif i == 2:
|
||||
EthAddress.fromHex("0x1584a2c066b7a455dbd6ae2807a7334e83c35fa5")
|
||||
else:
|
||||
raiseAssert("Invalid test case")
|
||||
expectedAccount = contentValue.proof.toAccount().get()
|
||||
|
||||
block:
|
||||
# check stateNode1
|
||||
let
|
||||
balanceRes =
|
||||
await stateNode1.stateNetwork.getBalance(contentValue.blockHash, address)
|
||||
nonceRes = await stateNode1.stateNetwork.getTransactionCount(
|
||||
contentValue.blockHash, address
|
||||
)
|
||||
|
||||
check:
|
||||
balanceRes.isOk()
|
||||
balanceRes.get() == expectedAccount.balance
|
||||
nonceRes.isOk()
|
||||
nonceRes.get() == expectedAccount.nonce
|
||||
|
||||
block:
|
||||
# check stateNode2
|
||||
let
|
||||
balanceRes =
|
||||
await stateNode2.stateNetwork.getBalance(contentValue.blockHash, address)
|
||||
nonceRes = await stateNode2.stateNetwork.getTransactionCount(
|
||||
contentValue.blockHash, address
|
||||
)
|
||||
|
||||
check:
|
||||
balanceRes.isOk()
|
||||
balanceRes.get() == expectedAccount.balance
|
||||
nonceRes.isOk()
|
||||
nonceRes.get() == expectedAccount.nonce
|
||||
|
||||
block:
|
||||
# test non-existant account
|
||||
let
|
||||
badAddress = EthAddress.fromHex("0xbadaaa39b223fe8d0a0e5c4f27ead9083c756cc2")
|
||||
balanceRes =
|
||||
await stateNode2.stateNetwork.getBalance(contentValue.blockHash, badAddress)
|
||||
nonceRes = await stateNode2.stateNetwork.getTransactionCount(
|
||||
contentValue.blockHash, badAddress
|
||||
)
|
||||
|
||||
check:
|
||||
balanceRes.isNone()
|
||||
nonceRes.isNone()
|
||||
|
||||
await stateNode1.stop()
|
||||
await stateNode2.stop()
|
||||
|
||||
asyncTest "Gossip then query getStorageAt and getCode":
|
||||
const
|
||||
file = testVectorDir / "recursive_gossip.yaml"
|
||||
bytecodeFile = testVectorDir / "contract_bytecode.yaml"
|
||||
|
||||
let
|
||||
testCase = YamlRecursiveGossipKVs.loadFromYaml(file).valueOr:
|
||||
raiseAssert "Cannot read test vector: " & error
|
||||
stateNode1 = newStateNode(rng, STATE_NODE1_PORT)
|
||||
stateNode2 = newStateNode(rng, STATE_NODE2_PORT)
|
||||
|
||||
stateNode1.start()
|
||||
stateNode2.start()
|
||||
|
||||
check:
|
||||
stateNode1.portalProtocol().addNode(stateNode2.localNode()) == Added
|
||||
stateNode2.portalProtocol().addNode(stateNode1.localNode()) == Added
|
||||
(await stateNode1.portalProtocol().ping(stateNode2.localNode())).isOk()
|
||||
(await stateNode2.portalProtocol().ping(stateNode1.localNode())).isOk()
|
||||
|
||||
block:
|
||||
# seed the account data
|
||||
let
|
||||
testData = testCase[0]
|
||||
stateRoot = KeccakHash.fromBytes(testData.state_root.hexToSeqByte())
|
||||
leafData = testData.recursive_gossip[0]
|
||||
contentKeyBytes = leafData.content_key.hexToSeqByte().ByteList
|
||||
contentKey = ContentKey.decode(contentKeyBytes).get()
|
||||
contentId = toContentId(contentKeyBytes)
|
||||
contentValueBytes = leafData.content_value.hexToSeqByte()
|
||||
contentValue = AccountTrieNodeOffer.decode(contentValueBytes).get()
|
||||
|
||||
# set valid state root
|
||||
stateNode1.mockBlockHashToStateRoot(contentValue.blockHash, stateRoot)
|
||||
stateNode2.mockBlockHashToStateRoot(contentValue.blockHash, stateRoot)
|
||||
|
||||
# offer the leaf node
|
||||
await stateNode1.portalProtocol.gossipOffer(
|
||||
Opt.none(NodeId),
|
||||
contentKeyBytes,
|
||||
contentValueBytes,
|
||||
contentKey.accountTrieNodeKey,
|
||||
contentValue,
|
||||
)
|
||||
|
||||
block:
|
||||
# seed the storage data
|
||||
let
|
||||
testData = testCase[1]
|
||||
stateRoot = KeccakHash.fromBytes(testData.state_root.hexToSeqByte())
|
||||
leafData = testData.recursive_gossip[0]
|
||||
contentKeyBytes = leafData.content_key.hexToSeqByte().ByteList
|
||||
contentKey = ContentKey.decode(contentKeyBytes).get()
|
||||
contentId = toContentId(contentKeyBytes)
|
||||
contentValueBytes = leafData.content_value.hexToSeqByte()
|
||||
contentValue = ContractTrieNodeOffer.decode(contentValueBytes).get()
|
||||
|
||||
# set valid state root
|
||||
stateNode1.mockBlockHashToStateRoot(contentValue.blockHash, stateRoot)
|
||||
stateNode2.mockBlockHashToStateRoot(contentValue.blockHash, stateRoot)
|
||||
|
||||
# offer the leaf node
|
||||
await stateNode1.portalProtocol.gossipOffer(
|
||||
Opt.none(NodeId),
|
||||
contentKeyBytes,
|
||||
contentValueBytes,
|
||||
contentKey.contractTrieNodeKey,
|
||||
contentValue,
|
||||
)
|
||||
|
||||
# wait for recursive gossip to complete
|
||||
await sleepAsync(1000.milliseconds)
|
||||
|
||||
let
|
||||
address = EthAddress.fromHex("0xc02aaa39b223fe8d0a0e5c4f27ead9083c756cc2")
|
||||
slot = 2.u256
|
||||
badSlot = 3.u256
|
||||
expectedSlot = contentValue.storageProof.toSlot().get()
|
||||
|
||||
slotRes = await stateNode2.stateNetwork.getStorageAt(
|
||||
contentValue.blockHash, address, slot
|
||||
)
|
||||
badSlotRes = await stateNode2.stateNetwork.getStorageAt(
|
||||
contentValue.blockHash, address, badSlot
|
||||
)
|
||||
|
||||
check:
|
||||
slotRes.isOk()
|
||||
slotRes.get() == expectedSlot
|
||||
badSlotRes.isNone()
|
||||
|
||||
block:
|
||||
# seed the contract bytecode
|
||||
let
|
||||
testCase = YamlContractBytecodeKVs.loadFromYaml(bytecodeFile).valueOr:
|
||||
raiseAssert "Cannot read test vector: " & error
|
||||
testData = testCase[0]
|
||||
stateRoot = KeccakHash.fromBytes(testData.state_root.hexToSeqByte())
|
||||
contentKeyBytes = testData.content_key.hexToSeqByte().ByteList
|
||||
contentKey = ContentKey.decode(contentKeyBytes).get()
|
||||
contentId = toContentId(contentKeyBytes)
|
||||
contentValueBytes = testData.content_value_offer.hexToSeqByte()
|
||||
contentValue = ContractCodeOffer.decode(contentValueBytes).get()
|
||||
|
||||
# set valid state root
|
||||
stateNode1.mockBlockHashToStateRoot(contentValue.blockHash, stateRoot)
|
||||
stateNode2.mockBlockHashToStateRoot(contentValue.blockHash, stateRoot)
|
||||
|
||||
await stateNode1.portalProtocol.gossipOffer(
|
||||
Opt.none(NodeId),
|
||||
contentKeyBytes,
|
||||
contentValueBytes,
|
||||
contentKey.contractCodeKey,
|
||||
contentValue,
|
||||
)
|
||||
|
||||
# wait for recursive gossip to complete
|
||||
await sleepAsync(1000.milliseconds)
|
||||
|
||||
let
|
||||
address = EthAddress.fromHex("0xc02aaa39b223fe8d0a0e5c4f27ead9083c756cc2")
|
||||
badAddress = EthAddress.fromHex("0xbadaaa39b223fe8d0a0e5c4f27ead9083c756cc2")
|
||||
expectedCode = contentValue.code
|
||||
|
||||
codeRes = await stateNode2.stateNetwork.getCode(contentValue.blockHash, address)
|
||||
badCodeRes =
|
||||
await stateNode2.stateNetwork.getCode(contentValue.blockHash, badAddress)
|
||||
|
||||
check:
|
||||
codeRes.isOk()
|
||||
codeRes.get() == expectedCode
|
||||
badCodeRes.isNone()
|
||||
|
||||
await stateNode1.stop()
|
||||
await stateNode2.stop()
|
Loading…
x
Reference in New Issue
Block a user