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:
web3-developer 2024-06-10 18:47:09 +08:00 committed by GitHub
parent f6be4bd0ec
commit c72d6aa5d6
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
11 changed files with 584 additions and 13 deletions

View File

@ -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)

View File

@ -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)

View File

@ -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)

View 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)

View File

@ -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():

View File

@ -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()

View File

@ -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")

View File

@ -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

View File

@ -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")

View File

@ -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,

View File

@ -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()