Fluffy state network tests (#2245)

* Add state network get content test.

* Completed state network get content tests.

* Completed state gossip offer content test.

* Improve state network offer content test.

* Completed state gossip offer test.

* Rename tests.
This commit is contained in:
web3-developer 2024-05-30 17:00:57 +08:00 committed by GitHub
parent 4b8219a0dd
commit 1eb170e686
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
12 changed files with 1201 additions and 301 deletions

View File

@ -122,17 +122,6 @@ proc getContractCode*(
): Future[Opt[ContractCodeRetrieval]] {.inline.} =
n.getContent(key, ContractCodeRetrieval)
# High level endpoints
# eth_getBalance
# eth_getStorageAt
# eth_getCode
func decodeKey(contentKey: ByteList): Opt[ContentKey] =
let key = ContentKey.decode(contentKey).valueOr:
return Opt.none(ContentKey)
Opt.some(key)
proc getStateRootByBlockHash(
n: StateNetwork, hash: BlockHash
): Future[Opt[KeccakHash]] {.async.} =
@ -146,7 +135,7 @@ proc getStateRootByBlockHash(
Opt.some(header.stateRoot)
proc processOffer(
proc processOffer*(
n: StateNetwork,
maybeSrcNodeId: Opt[NodeId],
contentKeyBytes: ByteList,
@ -183,10 +172,11 @@ proc processContentLoop(n: StateNetwork) {.async.} =
try:
while true:
let (srcNodeId, contentKeys, contentValues) = await n.contentQueue.popFirst()
for i, contentValueBytes in contentValues:
let
contentKeyBytes = contentKeys[i]
contentKey = decodeKey(contentKeyBytes).valueOr:
contentKey = ContentKey.decode(contentKeyBytes).valueOr:
error "Unable to decode offered content key", contentKeyBytes
continue
@ -219,8 +209,7 @@ proc processContentLoop(n: StateNetwork) {.async.} =
trace "processContentLoop canceled"
proc start*(n: StateNetwork) =
info "Starting Portal execution state network",
protocolId = n.portalProtocol.protocolId
info "Starting Portal State Network", protocolId = n.portalProtocol.protocolId
n.portalProtocol.start()
n.processContentLoop = processContentLoop(n)

View File

@ -8,13 +8,14 @@
{.warning[UnusedImport]: off.}
import
./test_state_content_keys,
./test_state_content_keys_vectors,
./test_state_content_nibbles,
./test_state_content_values,
#./test_state_network_gossip,
./test_state_network,
./test_state_recursivegossip_genesis,
./test_state_recursivegossip_vectors,
./test_state_content_values_vectors,
./test_state_gossip_getparent_genesis,
./test_state_gossip_getparent_vectors,
./test_state_gossip_gossipoffer_vectors,
./test_state_network_getcontent_vectors,
./test_state_network_offercontent_vectors,
./test_state_validation_genesis,
./test_state_validation_trieproof,
./test_state_validation_vectors

View File

@ -9,10 +9,17 @@
import
std/[sugar, sequtils],
chronos,
eth/[common, trie, trie/db],
eth/p2p/discoveryv5/protocol as discv5_protocol,
eth/p2p/discoveryv5/routing_table,
../../network/wire/[portal_protocol, portal_stream, portal_protocol_config],
../../nimbus/common/chain_config,
../../network/state/[state_content, state_utils],
../../eth_data/yaml_utils
../../network/history/[history_content, history_network],
../../network/state/[state_content, state_utils, state_network],
../../eth_data/yaml_utils,
../../database/content_db,
../test_helpers
export yaml_utils
@ -129,3 +136,53 @@ proc toState*(
accountTrie.put(key, value)
(accountTrie, storageStates)
type StateNode* = ref object
discoveryProtocol*: discv5_protocol.Protocol
stateNetwork*: StateNetwork
proc newStateNode*(
rng: ref HmacDrbgContext, port: int
): StateNode {.raises: [CatchableError].} =
let
node = initDiscoveryNode(rng, PrivateKey.random(rng[]), localAddress(port))
db = ContentDB.new("", uint32.high, inMemory = true)
sm = StreamManager.new(node)
hn = HistoryNetwork.new(node, db, sm, FinishedAccumulator())
sn = StateNetwork.new(node, db, sm, historyNetwork = Opt.some(hn))
return StateNode(discoveryProtocol: node, stateNetwork: sn)
proc portalProtocol*(sn: StateNode): PortalProtocol =
sn.stateNetwork.portalProtocol
proc localNode*(sn: StateNode): Node =
sn.discoveryProtocol.localNode
proc start*(sn: StateNode) =
sn.stateNetwork.start()
proc stop*(sn: StateNode) {.async.} =
sn.stateNetwork.stop()
await sn.discoveryProtocol.closeWait()
proc containsId*(sn: StateNode, contentId: ContentId): bool =
return sn.stateNetwork.contentDB.get(contentId).isSome()
proc mockBlockHashToStateRoot*(
sn: StateNode, blockHash: BlockHash, stateRoot: KeccakHash
) =
let
blockHeader = BlockHeader(stateRoot: stateRoot)
headerRlp = rlp.encode(blockHeader)
blockHeaderWithProof = BlockHeaderWithProof(
header: ByteList.init(headerRlp), proof: BlockHeaderProof.init()
)
contentKeyBytes = history_content.ContentKey
.init(history_content.ContentType.blockHeader, blockHash)
.encode()
contentId = history_content.toContentId(contentKeyBytes)
sn.portalProtocol().storeContent(
contentKeyBytes, contentId, SSZ.encode(blockHeaderWithProof)
)

View File

@ -16,7 +16,7 @@ import
../../network/state/[state_content, state_validation, state_gossip, state_utils],
./state_test_helpers
suite "State Recursive Gossip - Genesis JSON Files":
suite "State Gossip getParent - Genesis JSON Files":
let genesisFiles = [
"berlin2000.json", "calaveras.json", "chainid1.json", "chainid7.json",
"devnet4.json", "devnet5.json", "holesky.json", "mainshadow1.json", "merge.json",

View File

@ -15,7 +15,7 @@ import
../../network/state/[state_content, state_gossip],
./state_test_helpers
suite "State Recursive Gossip - Test Vectors":
suite "State Gossip getParent - Test Vectors":
test "Check account trie node parent matches expected recursive gossip":
const file = testVectorDir / "account_trie_node.yaml"

View File

@ -0,0 +1,389 @@
# 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],
../../database/content_db,
./state_test_helpers
procSuite "State Gossip - Gossip Offer":
const STATE_NODE1_PORT = 20602
const STATE_NODE2_PORT = 20603
let rng = newRng()
asyncTest "Gossip account trie nodes":
const file = testVectorDir / "account_trie_node.yaml"
let
testCase = YamlTrieNodeKVs.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
(await stateNode1.portalProtocol().ping(stateNode2.localNode())).isOk()
for i, testData in testCase:
if i == 1:
continue # skip scenario with no parent
let
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 = AccountTrieNodeOffer.decode(contentValueBytes).get()
parentContentKeyBytes =
testData.recursive_gossip.content_key.hexToSeqByte().ByteList
parentContentKey = ContentKey.decode(parentContentKeyBytes).get()
parentContentId = toContentId(parentContentKeyBytes)
parentContentValueBytes =
testData.recursive_gossip.content_value_offer.hexToSeqByte()
parentContentValue = AccountTrieNodeOffer.decode(parentContentValueBytes).get()
# set valid state root
stateNode1.mockBlockHashToStateRoot(contentValue.blockHash, stateRoot)
stateNode2.mockBlockHashToStateRoot(contentValue.blockHash, stateRoot)
check not stateNode2.containsId(contentId)
await stateNode1.portalProtocol.gossipOffer(
Opt.none(NodeId),
contentKeyBytes,
contentValueBytes,
contentKey.accountTrieNodeKey,
contentValue,
)
# wait for offer to be processed by state node 2
while not stateNode2.stateNetwork.contentQueue.empty():
await sleepAsync(1.milliseconds)
await sleepAsync(100.milliseconds)
# check that both the offer and parent were received by the second state instance
let res1 =
await stateNode2.stateNetwork.getAccountTrieNode(contentKey.accountTrieNodeKey)
check:
stateNode2.containsId(contentId)
res1.isOk()
res1.get() == contentValue.toRetrievalValue()
res1.get().node == contentValue.toRetrievalValue().node
let res2 = await stateNode2.stateNetwork.getAccountTrieNode(
parentContentKey.accountTrieNodeKey
)
check:
stateNode2.containsId(parentContentId)
res2.isOk()
res2.get() == parentContentValue.toRetrievalValue()
res2.get().node == parentContentValue.toRetrievalValue().node
await stateNode1.stop()
await stateNode2.stop()
asyncTest "Gossip contract trie nodes":
const file = testVectorDir / "contract_storage_trie_node.yaml"
let
testCase = YamlTrieNodeKVs.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
(await stateNode1.portalProtocol().ping(stateNode2.localNode())).isOk()
for i, testData in testCase:
if i == 1:
continue # skip scenario with no parent
let
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 = ContractTrieNodeOffer.decode(contentValueBytes).get()
parentContentKeyBytes =
testData.recursive_gossip.content_key.hexToSeqByte().ByteList
parentContentKey = ContentKey.decode(parentContentKeyBytes).get()
parentContentId = toContentId(parentContentKeyBytes)
parentContentValueBytes =
testData.recursive_gossip.content_value_offer.hexToSeqByte()
parentContentValue = ContractTrieNodeOffer.decode(parentContentValueBytes).get()
# set valid state root
stateNode1.mockBlockHashToStateRoot(contentValue.blockHash, stateRoot)
stateNode2.mockBlockHashToStateRoot(contentValue.blockHash, stateRoot)
check not stateNode2.containsId(contentId)
await stateNode1.portalProtocol.gossipOffer(
Opt.none(NodeId),
contentKeyBytes,
contentValueBytes,
contentKey.contractTrieNodeKey,
contentValue,
)
# wait for offer to be processed by state node 2
while not stateNode2.stateNetwork.contentQueue.empty():
await sleepAsync(1.milliseconds)
await sleepAsync(100.milliseconds)
# check that both the offer and parent were received by the second state instance
let res1 = await stateNode2.stateNetwork.getContractTrieNode(
contentKey.contractTrieNodeKey
)
check:
stateNode2.containsId(contentId)
res1.isOk()
res1.get() == contentValue.toRetrievalValue()
res1.get().node == contentValue.toRetrievalValue().node
let res2 = await stateNode2.stateNetwork.getContractTrieNode(
parentContentKey.contractTrieNodeKey
)
check:
stateNode2.containsId(parentContentId)
res2.isOk()
res2.get() == parentContentValue.toRetrievalValue()
res2.get().node == parentContentValue.toRetrievalValue().node
await stateNode1.stop()
await stateNode2.stop()
asyncTest "Gossip contract bytecode":
const file = testVectorDir / "contract_bytecode.yaml"
let
testCase = YamlContractBytecodeKVs.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
(await stateNode1.portalProtocol().ping(stateNode2.localNode())).isOk()
for i, testData in testCase:
let
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)
check not stateNode2.containsId(contentId)
await stateNode1.portalProtocol.gossipOffer(
Opt.none(NodeId),
contentKeyBytes,
contentValueBytes,
contentKey.contractCodeKey,
contentValue,
)
# wait for offer to be processed by state node 2
while not stateNode2.stateNetwork.contentQueue.empty():
await sleepAsync(1.milliseconds)
await sleepAsync(100.milliseconds)
# check that both the offer and parent were received by the second state instance
let res1 =
await stateNode2.stateNetwork.getContractCode(contentKey.contractCodeKey)
check:
stateNode2.containsId(contentId)
res1.isOk()
res1.get() == contentValue.toRetrievalValue()
res1.get().code == contentValue.toRetrievalValue().code
await stateNode1.stop()
await stateNode2.stop()
asyncTest "Recursive gossip account trie nodes":
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)
check not stateNode1.containsId(contentId)
check not stateNode2.containsId(contentId)
# offer the leaf node
await stateNode1.portalProtocol.gossipOffer(
Opt.none(NodeId),
contentKeyBytes,
contentValueBytes,
contentKey.accountTrieNodeKey,
contentValue,
)
# wait for recursive gossip to complete
await sleepAsync(1000.milliseconds)
# check that all nodes were received by both state instances
for kv in testData.recursive_gossip:
let
expectedKeyBytes = kv.content_key.hexToSeqByte().ByteList
expectedKey = ContentKey.decode(expectedKeyBytes).get()
expectedId = toContentId(expectedKeyBytes)
expectedValue =
AccountTrieNodeOffer.decode(kv.content_value.hexToSeqByte()).get()
res1 = await stateNode1.stateNetwork.getAccountTrieNode(
expectedKey.accountTrieNodeKey
)
res2 = await stateNode2.stateNetwork.getAccountTrieNode(
expectedKey.accountTrieNodeKey
)
check:
stateNode1.containsId(expectedId)
stateNode2.containsId(expectedId)
res1.isOk()
res1.get() == expectedValue.toRetrievalValue()
res1.get().node == expectedValue.toRetrievalValue().node
res2.isOk()
res2.get() == expectedValue.toRetrievalValue()
res2.get().node == expectedValue.toRetrievalValue().node
await stateNode1.stop()
await stateNode2.stop()
asyncTest "Recursive gossip contract trie nodes":
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 = ContractTrieNodeOffer.decode(contentValueBytes).get()
# set valid state root
stateNode1.mockBlockHashToStateRoot(contentValue.blockHash, stateRoot)
stateNode2.mockBlockHashToStateRoot(contentValue.blockHash, stateRoot)
check not stateNode1.containsId(contentId)
check not stateNode2.containsId(contentId)
# 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)
# check that all nodes were received by both state instances
for kv in testData.recursive_gossip:
let
expectedKeyBytes = kv.content_key.hexToSeqByte().ByteList
expectedKey = ContentKey.decode(expectedKeyBytes).get()
expectedId = toContentId(expectedKeyBytes)
expectedValue =
ContractTrieNodeOffer.decode(kv.content_value.hexToSeqByte()).get()
res1 = await stateNode1.stateNetwork.getContractTrieNode(
expectedKey.contractTrieNodeKey
)
res2 = await stateNode2.stateNetwork.getContractTrieNode(
expectedKey.contractTrieNodeKey
)
check:
stateNode1.containsId(expectedId)
stateNode2.containsId(expectedId)
res1.isOk()
res1.get() == expectedValue.toRetrievalValue()
res1.get().node == expectedValue.toRetrievalValue().node
res2.isOk()
res2.get() == expectedValue.toRetrievalValue()
res2.get().node == expectedValue.toRetrievalValue().node
await stateNode1.stop()
await stateNode2.stop()

View File

@ -1,158 +0,0 @@
# 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,
nimcrypto/hash,
testutils/unittests,
chronos,
eth/keys,
eth/trie,
eth/common/[eth_types, eth_hash],
eth/p2p/discoveryv5/protocol as discv5_protocol,
eth/p2p/discoveryv5/routing_table,
../../network/wire/[portal_protocol, portal_stream],
../../network/state/[state_content, state_network],
../../database/content_db,
../test_helpers,
./state_test_helpers
procSuite "State Network":
let rng = newRng()
asyncTest "Test Share Full State":
let
accounts =
getGenesisAlloc("fluffy" / "tests" / "custom_genesis" / "chainid7.json")
(trie, _) = accounts.toState()
node1 = initDiscoveryNode(rng, PrivateKey.random(rng[]), localAddress(20312))
sm1 = StreamManager.new(node1)
node2 = initDiscoveryNode(rng, PrivateKey.random(rng[]), localAddress(20313))
sm2 = StreamManager.new(node2)
proto1 =
StateNetwork.new(node1, ContentDB.new("", uint32.high, inMemory = true), sm1)
proto2 =
StateNetwork.new(node2, ContentDB.new("", uint32.high, inMemory = true), sm2)
check proto2.portalProtocol.addNode(node1.localNode) == Added
var keys: seq[seq[byte]]
for k, v in trie.replicate:
keys.add(k)
var nodeHash: NodeHash
copyMem(nodeHash.data.addr, unsafeAddr k[0], sizeof(nodeHash.data))
let
# TODO: add stateRoot, and path eventually
accountTrieNodeKey = AccountTrieNodeKey(nodeHash: nodeHash)
contentKey = ContentKey(
contentType: accountTrieNode, accountTrieNodeKey: accountTrieNodeKey
)
contentId = toContentId(contentKey.encode())
value = AccountTrieNodeRetrieval(node: TrieNode.init(v))
discard proto1.contentDB.put(
contentId, value.encode(), proto1.portalProtocol.localNode.id
)
for key in keys:
var nodeHash: NodeHash
copyMem(nodeHash.data.addr, unsafeAddr key[0], sizeof(nodeHash.data))
let
accountTrieNodeKey = AccountTrieNodeKey(nodeHash: nodeHash)
contentKey = ContentKey(
contentType: accountTrieNode, accountTrieNodeKey: accountTrieNodeKey
)
contentId = toContentId(contentKey.encode())
# Note: GetContent and thus the lookup here is not really needed, as we
# only have to request data to one node.
let accTrieNode = await proto2.getAccountTrieNode(accountTrieNodeKey)
check accTrieNode.isSome()
let hash = keccakHash(accTrieNode.get().node.asSeq())
check hash.data == key
proto1.stop()
proto2.stop()
await node1.closeWait()
await node2.closeWait()
asyncTest "Find content in the network via content lookup":
# TODO: Improve this test so it actually need to go through several
# findNodes request, to properly test the lookup call.
let
accounts =
getGenesisAlloc("fluffy" / "tests" / "custom_genesis" / "chainid7.json")
(trie, _) = accounts.toState()
node1 = initDiscoveryNode(rng, PrivateKey.random(rng[]), localAddress(20312))
sm1 = StreamManager.new(node1)
node2 = initDiscoveryNode(rng, PrivateKey.random(rng[]), localAddress(20313))
sm2 = StreamManager.new(node2)
node3 = initDiscoveryNode(rng, PrivateKey.random(rng[]), localAddress(20314))
sm3 = StreamManager.new(node3)
proto1 =
StateNetwork.new(node1, ContentDB.new("", uint32.high, inMemory = true), sm1)
proto2 =
StateNetwork.new(node2, ContentDB.new("", uint32.high, inMemory = true), sm2)
proto3 =
StateNetwork.new(node3, ContentDB.new("", uint32.high, inMemory = true), sm3)
# Node1 knows about Node2, and Node2 knows about Node3 which hold all content
check proto1.portalProtocol.addNode(node2.localNode) == Added
check proto2.portalProtocol.addNode(node3.localNode) == Added
check (await proto2.portalProtocol.ping(node3.localNode)).isOk()
var keys: seq[seq[byte]]
for k, v in trie.replicate:
keys.add(k)
var nodeHash: NodeHash
copyMem(nodeHash.data.addr, unsafeAddr k[0], sizeof(nodeHash.data))
let
accountTrieNodeKey = AccountTrieNodeKey(nodeHash: nodeHash)
contentKey = ContentKey(
contentType: accountTrieNode, accountTrieNodeKey: accountTrieNodeKey
)
contentId = toContentId(contentKey.encode())
value = AccountTrieNodeRetrieval(node: TrieNode.init(v))
discard proto2.contentDB.put(
contentId, value.encode(), proto2.portalProtocol.localNode.id
)
# Not needed right now as 1 node is enough considering node 1 is connected
# to both.
discard proto3.contentDB.put(
contentId, value.encode(), proto3.portalProtocol.localNode.id
)
# Get first key
var nodeHash: NodeHash
let firstKey = keys[0]
copyMem(nodeHash.data.addr, unsafeAddr firstKey[0], sizeof(nodeHash.data))
let
accountTrieNodeKey = AccountTrieNodeKey(nodeHash: nodeHash)
contentKey =
ContentKey(contentType: accountTrieNode, accountTrieNodeKey: accountTrieNodeKey)
let accTrieNode = await proto1.getAccountTrieNode(accountTrieNodeKey)
check accTrieNode.isSome()
let hash = keccakHash(accTrieNode.get().node.asSeq())
check hash.data == firstKey
proto1.stop()
proto2.stop()
await node1.closeWait()
await node2.closeWait()
await node3.closeWait()

View File

@ -0,0 +1,350 @@
# 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/p2p/discoveryv5/protocol as discv5_protocol,
eth/p2p/discoveryv5/routing_table,
../../network/wire/[portal_protocol, portal_stream],
../../network/state/[state_content, state_network],
../../database/content_db,
./state_test_helpers
procSuite "State Network - Get Content":
const STATE_NODE1_PORT = 20402
const STATE_NODE2_PORT = 20403
let rng = newRng()
# Single state instance tests
asyncTest "Single state instance - Get existing account trie node":
const file = testVectorDir / "account_trie_node.yaml"
let
testCase = YamlTrieNodeKVs.loadFromYaml(file).valueOr:
raiseAssert "Cannot read test vector: " & error
stateNode1 = newStateNode(rng, STATE_NODE1_PORT)
stateNode1.start()
for testData in testCase:
let
contentKeyBytes = testData.content_key.hexToSeqByte().ByteList
contentKey = ContentKey.decode(contentKeyBytes).get()
contentId = toContentId(contentKeyBytes)
contentValueBytes = testData.content_value_retrieval.hexToSeqByte()
expectedContentValue = AccountTrieNodeRetrieval.decode(contentValueBytes).get()
stateNode1.portalProtocol().storeContent(
contentKeyBytes, contentId, contentValueBytes
)
let res =
await stateNode1.stateNetwork.getAccountTrieNode(contentKey.accountTrieNodeKey)
check:
res.isOk()
res.get() == expectedContentValue
res.get().node == expectedContentValue.node
await stateNode1.stop()
asyncTest "Single state instance - Get missing account trie node":
const file = testVectorDir / "account_trie_node.yaml"
let
testCase = YamlTrieNodeKVs.loadFromYaml(file).valueOr:
raiseAssert "Cannot read test vector: " & error
stateNode1 = newStateNode(rng, STATE_NODE1_PORT)
stateNode1.start()
for testData in testCase:
let
contentKeyBytes = testData.content_key.hexToSeqByte().ByteList
contentKey = ContentKey.decode(contentKeyBytes).get()
let res =
await stateNode1.stateNetwork.getAccountTrieNode(contentKey.accountTrieNodeKey)
check:
res.isNone()
await stateNode1.stop()
asyncTest "Single state instance - Get existing contract trie node":
const file = testVectorDir / "contract_storage_trie_node.yaml"
let
testCase = YamlTrieNodeKVs.loadFromYaml(file).valueOr:
raiseAssert "Cannot read test vector: " & error
stateNode1 = newStateNode(rng, STATE_NODE1_PORT)
stateNode1.start()
for testData in testCase:
let
contentKeyBytes = testData.content_key.hexToSeqByte().ByteList
contentKey = ContentKey.decode(contentKeyBytes).get()
contentId = toContentId(contentKeyBytes)
contentValueBytes = testData.content_value_retrieval.hexToSeqByte()
expectedContentValue = ContractTrieNodeRetrieval.decode(contentValueBytes).get()
stateNode1.portalProtocol().storeContent(
contentKeyBytes, contentId, contentValueBytes
)
let res = await stateNode1.stateNetwork.getContractTrieNode(
contentKey.contractTrieNodeKey
)
check:
res.isOk()
res.get() == expectedContentValue
res.get().node == expectedContentValue.node
await stateNode1.stop()
asyncTest "Single state instance - Get missing contract trie node":
const file = testVectorDir / "contract_storage_trie_node.yaml"
let
testCase = YamlTrieNodeKVs.loadFromYaml(file).valueOr:
raiseAssert "Cannot read test vector: " & error
stateNode1 = newStateNode(rng, STATE_NODE1_PORT)
stateNode1.start()
for testData in testCase:
let
contentKeyBytes = testData.content_key.hexToSeqByte().ByteList
contentKey = ContentKey.decode(contentKeyBytes).get()
let res = await stateNode1.stateNetwork.getContractTrieNode(
contentKey.contractTrieNodeKey
)
check:
res.isNone()
await stateNode1.stop()
asyncTest "Single state instance - Get existing contract bytecode":
const file = testVectorDir / "contract_bytecode.yaml"
let
testCase = YamlContractBytecodeKVs.loadFromYaml(file).valueOr:
raiseAssert "Cannot read test vector: " & error
stateNode1 = newStateNode(rng, STATE_NODE1_PORT)
stateNode1.start()
for testData in testCase:
let
contentKeyBytes = testData.content_key.hexToSeqByte().ByteList
contentKey = ContentKey.decode(contentKeyBytes).get()
contentId = toContentId(contentKeyBytes)
contentValueBytes = testData.content_value_retrieval.hexToSeqByte()
expectedContentValue = ContractCodeRetrieval.decode(contentValueBytes).get()
stateNode1.portalProtocol().storeContent(
contentKeyBytes, contentId, contentValueBytes
)
let res =
await stateNode1.stateNetwork.getContractCode(contentKey.contractCodeKey)
check:
res.isOk()
res.get() == expectedContentValue
res.get().code == expectedContentValue.code
await stateNode1.stop()
asyncTest "Single state instance - Get missing contract bytecode":
const file = testVectorDir / "contract_bytecode.yaml"
let
testCase = YamlContractBytecodeKVs.loadFromYaml(file).valueOr:
raiseAssert "Cannot read test vector: " & error
stateNode1 = newStateNode(rng, STATE_NODE1_PORT)
stateNode1.start()
for testData in testCase:
let
contentKeyBytes = testData.content_key.hexToSeqByte().ByteList
contentKey = ContentKey.decode(contentKeyBytes).get()
let res =
await stateNode1.stateNetwork.getContractCode(contentKey.contractCodeKey)
check:
res.isNone()
await stateNode1.stop()
# Two state instances tests
asyncTest "Two state instances - Get existing account trie node":
const file = testVectorDir / "account_trie_node.yaml"
let
testCase = YamlTrieNodeKVs.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 testData in testCase:
let
contentKeyBytes = testData.content_key.hexToSeqByte().ByteList
contentKey = ContentKey.decode(contentKeyBytes).get()
contentId = toContentId(contentKeyBytes)
contentValueBytes = testData.content_value_retrieval.hexToSeqByte()
expectedContentValue = AccountTrieNodeRetrieval.decode(contentValueBytes).get()
# only store the content in the first state instance
stateNode1.portalProtocol().storeContent(
contentKeyBytes, contentId, contentValueBytes
)
check:
stateNode1.containsId(contentId)
not stateNode2.containsId(contentId)
let
res1 = await stateNode1.stateNetwork.getAccountTrieNode(
contentKey.accountTrieNodeKey
)
res2 = await stateNode2.stateNetwork.getAccountTrieNode(
contentKey.accountTrieNodeKey
)
check:
stateNode1.containsId(contentId)
stateNode2.containsId(contentId)
res1.isOk()
res1.get() == expectedContentValue
res1.get().node == expectedContentValue.node
res2.isOk()
res2.get() == expectedContentValue
res2.get().node == expectedContentValue.node
await stateNode1.stop()
await stateNode2.stop()
asyncTest "Two state instances - Get existing contract trie node":
const file = testVectorDir / "contract_storage_trie_node.yaml"
let
testCase = YamlTrieNodeKVs.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 testData in testCase:
let
contentKeyBytes = testData.content_key.hexToSeqByte().ByteList
contentKey = ContentKey.decode(contentKeyBytes).get()
contentId = toContentId(contentKeyBytes)
contentValueBytes = testData.content_value_retrieval.hexToSeqByte()
expectedContentValue = ContractTrieNodeRetrieval.decode(contentValueBytes).get()
# only store the content in the first state instance
stateNode1.portalProtocol().storeContent(
contentKeyBytes, contentId, contentValueBytes
)
check:
stateNode1.containsId(contentId)
not stateNode2.containsId(contentId)
let
res1 = await stateNode1.stateNetwork.getContractTrieNode(
contentKey.contractTrieNodeKey
)
res2 = await stateNode2.stateNetwork.getContractTrieNode(
contentKey.contractTrieNodeKey
)
check:
stateNode1.containsId(contentId)
stateNode2.containsId(contentId)
res1.isOk()
res1.get() == expectedContentValue
res1.get().node == expectedContentValue.node
res2.isOk()
res2.get() == expectedContentValue
res2.get().node == expectedContentValue.node
await stateNode1.stop()
await stateNode2.stop()
asyncTest "Two state instances - Get existing contract bytecode":
const file = testVectorDir / "contract_bytecode.yaml"
let
testCase = YamlContractBytecodeKVs.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 testData in testCase:
let
contentKeyBytes = testData.content_key.hexToSeqByte().ByteList
contentKey = ContentKey.decode(contentKeyBytes).get()
contentId = toContentId(contentKeyBytes)
contentValueBytes = testData.content_value_retrieval.hexToSeqByte()
expectedContentValue = ContractCodeRetrieval.decode(contentValueBytes).get()
# only store the content in the first state instance
stateNode1.portalProtocol().storeContent(
contentKeyBytes, contentId, contentValueBytes
)
check:
stateNode1.containsId(contentId)
not stateNode2.containsId(contentId)
let
res1 = await stateNode1.stateNetwork.getContractCode(contentKey.contractCodeKey)
res2 = await stateNode2.stateNetwork.getContractCode(contentKey.contractCodeKey)
check:
stateNode1.containsId(contentId)
stateNode2.containsId(contentId)
res1.isOk()
res1.get() == expectedContentValue
res1.get().code == expectedContentValue.code
res2.isOk()
res2.get() == expectedContentValue
res2.get().code == expectedContentValue.code
await stateNode1.stop()
await stateNode2.stop()

View File

@ -1,118 +0,0 @@
# Fluffy
# Copyright (c) 2023-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,
chronos,
testutils/unittests,
stew/[byteutils, results],
eth/p2p/discoveryv5/protocol as discv5_protocol,
../../network/wire/[portal_protocol, portal_stream],
../../network/history/[history_content, history_network],
../../network/state/[state_network, state_gossip],
../../database/content_db,
.././test_helpers,
../../eth_data/yaml_utils
const testVectorDir = "./vendor/portal-spec-tests/tests/mainnet/state/validation/"
procSuite "State Network Gossip":
let rng = newRng()
asyncTest "Test Gossip of Account Trie Node Offer":
const file = testVectorDir / "recursive_gossip.yaml"
type YamlRecursiveGossipKV = object
content_key: string
content_value: string
type YamlRecursiveGossipData = object
state_root: string
recursive_gossip: seq[YamlRecursiveGossipKV]
type YamlRecursiveGossipKVs = seq[YamlRecursiveGossipData]
let
testCase = YamlRecursiveGossipKVs.loadFromYaml(file).valueOr:
raiseAssert "Cannot read test vector: " & error
recursiveGossipSteps = testCase[0].recursive_gossip
numOfClients = recursiveGossipSteps.len() - 1
var clients: seq[StateNetwork]
for i in 0 .. numOfClients:
let
node = initDiscoveryNode(rng, PrivateKey.random(rng[]), localAddress(20400 + i))
db = ContentDB.new("", uint32.high, inMemory = true)
sm = StreamManager.new(node)
hn = HistoryNetwork.new(node, db, sm, FinishedAccumulator())
proto = StateNetwork.new(node, db, sm, historyNetwork = Opt.some(hn))
proto.start()
clients.add(proto)
for i in 0 .. numOfClients - 1:
let
currentNode = clients[i]
nextNode = clients[i + 1]
check:
currentNode.portalProtocol.addNode(nextNode.portalProtocol.localNode) == Added
(await currentNode.portalProtocol.ping(nextNode.portalProtocol.localNode)).isOk()
let
blockHeader = BlockHeader(
stateRoot: Hash256.fromHex(
"0x1ad7b80af0c28bc1489513346d2706885be90abb07f23ca28e50482adb392d61"
)
)
headerRlp = rlp.encode(blockHeader)
blockHeaderWithProof = BlockHeaderWithProof(
header: ByteList.init(headerRlp), proof: BlockHeaderProof.init()
)
value = recursiveGossipSteps[0].content_value.hexToSeqByte()
decodedValue = AccountTrieNodeOffer.decode(value).get()
contentKey = history_content.ContentKey
.init(history_content.ContentType.blockHeader, decodedValue.blockHash)
.encode()
contentId = history_content.toContentId(contentKey)
clients[i].contentDB.put(contentId, SSZ.encode(blockHeaderWithProof))
for i in 0 .. numOfClients - 1:
let
pair = recursiveGossipSteps[i]
currentNode = clients[i]
nextNode = clients[i + 1]
key = ByteList.init(pair.content_key.hexToSeqByte())
decodedKey = state_content.ContentKey.decode(key).valueOr:
raiseAssert "Cannot decode key"
nextKey = ByteList.init(recursiveGossipSteps[1].content_key.hexToSeqByte())
decodedNextKey = state_content.ContentKey.decode(nextKey).valueOr:
raiseAssert "Cannot decode key"
value = pair.content_value.hexToSeqByte()
decodedValue = AccountTrieNodeOffer.decode(value).get()
nextValue = recursiveGossipSteps[1].content_value.hexToSeqByte()
nextDecodedValue = AccountTrieNodeOffer.decode(nextValue).get()
nextRetrievalValue = nextDecodedValue.toRetrievalValue()
if i == 0:
await currentNode.portalProtocol.gossipOffer(
Opt.none(NodeId), key, value, decodedKey.accountTrieNodeKey, decodedValue
)
await sleepAsync(100.milliseconds) #TODO figure out how to get rid of this sleep
check (await nextNode.getAccountTrieNode(decodedNextKey.accountTrieNodeKey)) ==
Opt.some(nextRetrievalValue)
for i in 0 .. numOfClients:
await clients[i].portalProtocol.baseProtocol.closeWait()
# TODO Add tests for Contract Trie Node Offer & Contract Code Offer

View File

@ -0,0 +1,390 @@
# 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],
../../database/content_db,
./state_test_helpers
procSuite "State Network - Offer Content":
const
STATE_NODE1_PORT = 20502
STATE_NODE2_PORT = 20503
let rng = newRng()
# Single state instance tests
asyncTest "Single state instance - Offer account trie nodes":
const file = testVectorDir / "account_trie_node.yaml"
let testCase = YamlTrieNodeKVs.loadFromYaml(file).valueOr:
raiseAssert "Cannot read test vector: " & error
for testData in testCase:
let
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 = AccountTrieNodeOffer.decode(contentValueBytes).get()
stateNode1 = newStateNode(rng, STATE_NODE1_PORT)
stateNode1.start()
# no state root yet
check (
await stateNode1.stateNetwork.processOffer(
Opt.none(NodeId),
contentKeyBytes,
contentValueBytes,
contentKey.accountTrieNodeKey,
AccountTrieNodeOffer,
)
).isErr()
# set bad state root
let badStateRoot = KeccakHash.fromBytes(
"0xBAD7b80af0c28bc1489513346d2706885be90abb07f23ca28e50482adb392d61".hexToSeqByte()
)
stateNode1.mockBlockHashToStateRoot(contentValue.blockHash, badStateRoot)
check (
await stateNode1.stateNetwork.processOffer(
Opt.none(NodeId),
contentKeyBytes,
contentValueBytes,
contentKey.accountTrieNodeKey,
AccountTrieNodeOffer,
)
).isErr()
# set valid state root
stateNode1.mockBlockHashToStateRoot(contentValue.blockHash, stateRoot)
check not stateNode1.containsId(contentId)
let processRes = await stateNode1.stateNetwork.processOffer(
Opt.none(NodeId),
contentKeyBytes,
contentValueBytes,
contentKey.accountTrieNodeKey,
AccountTrieNodeOffer,
)
check processRes.isOk()
let getRes =
await stateNode1.stateNetwork.getAccountTrieNode(contentKey.accountTrieNodeKey)
check:
stateNode1.containsId(contentId)
getRes.isOk()
getRes.get() == contentValue.toRetrievalValue()
getRes.get().node == contentValue.toRetrievalValue().node
await stateNode1.stop()
asyncTest "Single state instance - Offer contract trie nodes":
const file = testVectorDir / "contract_storage_trie_node.yaml"
let testCase = YamlTrieNodeKVs.loadFromYaml(file).valueOr:
raiseAssert "Cannot read test vector: " & error
for testData in testCase:
let
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 = ContractTrieNodeOffer.decode(contentValueBytes).get()
stateNode1 = newStateNode(rng, STATE_NODE1_PORT)
stateNode1.start()
# no state root yet
check (
await stateNode1.stateNetwork.processOffer(
Opt.none(NodeId),
contentKeyBytes,
contentValueBytes,
contentKey.contractTrieNodeKey,
ContractTrieNodeOffer,
)
).isErr()
# set bad state root
let badStateRoot = KeccakHash.fromBytes(
"0xBAD7b80af0c28bc1489513346d2706885be90abb07f23ca28e50482adb392d61".hexToSeqByte()
)
stateNode1.mockBlockHashToStateRoot(contentValue.blockHash, badStateRoot)
check (
await stateNode1.stateNetwork.processOffer(
Opt.none(NodeId),
contentKeyBytes,
contentValueBytes,
contentKey.contractTrieNodeKey,
ContractTrieNodeOffer,
)
).isErr()
# set valid state root
stateNode1.mockBlockHashToStateRoot(contentValue.blockHash, stateRoot)
check not stateNode1.containsId(contentId)
let processRes = await stateNode1.stateNetwork.processOffer(
Opt.none(NodeId),
contentKeyBytes,
contentValueBytes,
contentKey.contractTrieNodeKey,
ContractTrieNodeOffer,
)
check processRes.isOk()
let getRes = await stateNode1.stateNetwork.getContractTrieNode(
contentKey.contractTrieNodeKey
)
check:
stateNode1.containsId(contentId)
getRes.isOk()
getRes.get() == contentValue.toRetrievalValue()
getRes.get().node == contentValue.toRetrievalValue().node
await stateNode1.stop()
asyncTest "Single state instance - Offer contract bytecode":
const file = testVectorDir / "contract_bytecode.yaml"
let testCase = YamlContractBytecodeKVs.loadFromYaml(file).valueOr:
raiseAssert "Cannot read test vector: " & error
for testData in testCase:
let
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()
stateNode1 = newStateNode(rng, STATE_NODE1_PORT)
stateNode1.start()
# no state root yet
check (
await stateNode1.stateNetwork.processOffer(
Opt.none(NodeId),
contentKeyBytes,
contentValueBytes,
contentKey.contractCodeKey,
ContractCodeOffer,
)
).isErr()
# set bad state root
let badStateRoot = KeccakHash.fromBytes(
"0xBAD7b80af0c28bc1489513346d2706885be90abb07f23ca28e50482adb392d61".hexToSeqByte()
)
stateNode1.mockBlockHashToStateRoot(contentValue.blockHash, badStateRoot)
check (
await stateNode1.stateNetwork.processOffer(
Opt.none(NodeId),
contentKeyBytes,
contentValueBytes,
contentKey.contractCodeKey,
ContractCodeOffer,
)
).isErr()
# set valid state root
stateNode1.mockBlockHashToStateRoot(contentValue.blockHash, stateRoot)
check not stateNode1.containsId(contentId)
let processRes = await stateNode1.stateNetwork.processOffer(
Opt.none(NodeId),
contentKeyBytes,
contentValueBytes,
contentKey.contractCodeKey,
ContractCodeOffer,
)
check processRes.isOk()
let getRes =
await stateNode1.stateNetwork.getContractCode(contentKey.contractCodeKey)
check:
stateNode1.containsId(contentId)
getRes.isOk()
getRes.get() == contentValue.toRetrievalValue()
getRes.get().code == contentValue.toRetrievalValue().code
await stateNode1.stop()
# Two state instances tests - State node 1 offers content to state node 2
asyncTest "Two state instances - Offer account trie nodes":
const file = testVectorDir / "account_trie_node.yaml"
let testCase = YamlTrieNodeKVs.loadFromYaml(file).valueOr:
raiseAssert "Cannot read test vector: " & error
for testData in testCase:
let
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 = AccountTrieNodeOffer.decode(contentValueBytes).get()
contentKV = ContentKV(contentKey: contentKeyBytes, content: contentValueBytes)
stateNode1 = newStateNode(rng, STATE_NODE1_PORT)
stateNode2 = newStateNode(rng, STATE_NODE2_PORT)
stateNode1.start()
stateNode2.start()
check:
stateNode1.portalProtocol().addNode(stateNode2.localNode()) == Added
(await stateNode1.portalProtocol().ping(stateNode2.localNode())).isOk()
# set valid state root
stateNode1.mockBlockHashToStateRoot(contentValue.blockHash, stateRoot)
stateNode2.mockBlockHashToStateRoot(contentValue.blockHash, stateRoot)
check not stateNode2.containsId(contentId)
let offerResult =
await stateNode1.portalProtocol.offer(stateNode2.localNode(), @[contentKV])
check offerResult.isOk()
# wait for offer to be processed by state node 2
while not stateNode2.stateNetwork.contentQueue.empty():
await sleepAsync(1.milliseconds)
await sleepAsync(100.milliseconds)
let getRes =
await stateNode2.stateNetwork.getAccountTrieNode(contentKey.accountTrieNodeKey)
check:
stateNode2.containsId(contentId)
getRes.isOk()
getRes.get() == contentValue.toRetrievalValue()
getRes.get().node == contentValue.toRetrievalValue().node
await stateNode1.stop()
await stateNode2.stop()
asyncTest "Two state instances - Offer contract trie nodes":
const file = testVectorDir / "contract_storage_trie_node.yaml"
let testCase = YamlTrieNodeKVs.loadFromYaml(file).valueOr:
raiseAssert "Cannot read test vector: " & error
for testData in testCase:
let
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 = ContractTrieNodeOffer.decode(contentValueBytes).get()
contentKV = ContentKV(contentKey: contentKeyBytes, content: contentValueBytes)
stateNode1 = newStateNode(rng, STATE_NODE1_PORT)
stateNode2 = newStateNode(rng, STATE_NODE2_PORT)
stateNode1.start()
stateNode2.start()
check:
stateNode1.portalProtocol().addNode(stateNode2.localNode()) == Added
(await stateNode1.portalProtocol().ping(stateNode2.localNode())).isOk()
# set valid state root
stateNode1.mockBlockHashToStateRoot(contentValue.blockHash, stateRoot)
stateNode2.mockBlockHashToStateRoot(contentValue.blockHash, stateRoot)
check not stateNode2.containsId(contentId)
let offerResult =
await stateNode1.portalProtocol.offer(stateNode2.localNode(), @[contentKV])
check offerResult.isOk()
# wait for offer to be processed by state node 2
while not stateNode2.stateNetwork.contentQueue.empty():
await sleepAsync(1.milliseconds)
await sleepAsync(100.milliseconds)
let getRes = await stateNode2.stateNetwork.getContractTrieNode(
contentKey.contractTrieNodeKey
)
check:
stateNode2.containsId(contentId)
getRes.isOk()
getRes.get() == contentValue.toRetrievalValue()
getRes.get().node == contentValue.toRetrievalValue().node
await stateNode1.stop()
await stateNode2.stop()
asyncTest "Two state instances - Offer contract bytecode":
const file = testVectorDir / "contract_bytecode.yaml"
let testCase = YamlContractBytecodeKVs.loadFromYaml(file).valueOr:
raiseAssert "Cannot read test vector: " & error
for testData in testCase:
let
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()
contentKV = ContentKV(contentKey: contentKeyBytes, content: contentValueBytes)
stateNode1 = newStateNode(rng, STATE_NODE1_PORT)
stateNode2 = newStateNode(rng, STATE_NODE2_PORT)
stateNode1.start()
stateNode2.start()
check:
stateNode1.portalProtocol().addNode(stateNode2.localNode()) == Added
(await stateNode1.portalProtocol().ping(stateNode2.localNode())).isOk()
# set valid state root
stateNode1.mockBlockHashToStateRoot(contentValue.blockHash, stateRoot)
stateNode2.mockBlockHashToStateRoot(contentValue.blockHash, stateRoot)
check not stateNode2.containsId(contentId)
let offerResult =
await stateNode1.portalProtocol.offer(stateNode2.localNode(), @[contentKV])
check offerResult.isOk()
# wait for offer to be processed by state node 2
while not stateNode2.stateNetwork.contentQueue.empty():
await sleepAsync(1.milliseconds)
await sleepAsync(100.milliseconds)
let getRes =
await stateNode2.stateNetwork.getContractCode(contentKey.contractCodeKey)
check:
stateNode2.containsId(contentId)
getRes.isOk()
getRes.get() == contentValue.toRetrievalValue()
getRes.get().code == contentValue.toRetrievalValue().code
await stateNode1.stop()
await stateNode2.stop()