diff --git a/Makefile b/Makefile index ec48b09f4..a9af6b17e 100644 --- a/Makefile +++ b/Makefile @@ -73,7 +73,6 @@ FLUFFY_TOOLS := \ portal_bridge \ beacon_lc_bridge \ eth_data_exporter \ - content_verifier \ blockwalk \ portalcli \ fcli_db diff --git a/fluffy/eth_data/history_data_seeding.nim b/fluffy/eth_data/history_data_seeding.nim index 76fc1c603..dd7e84f68 100644 --- a/fluffy/eth_data/history_data_seeding.nim +++ b/fluffy/eth_data/history_data_seeding.nim @@ -35,53 +35,6 @@ proc historyStore*( ok() -proc propagateEpochRecord*( - p: PortalProtocol, file: string -): Future[Result[void, string]] {.async.} = - ## Propagate a specific epoch accumulator into the network. - ## file holds the SSZ serialized epoch accumulator. - let epochRecordRes = readEpochRecord(file) - if epochRecordRes.isErr(): - return err(epochRecordRes.error) - else: - let - epochRecord = epochRecordRes.get() - rootHash = epochRecord.hash_tree_root() - key = ContentKey( - contentType: epochRecord, epochRecordKey: EpochRecordKey(epochHash: rootHash) - ) - encKey = history_content.encode(key) - # Note: The file actually holds the SSZ encoded accumulator, but we need - # to decode as we need the root for the content key. - encodedEpochRecord = SSZ.encode(epochRecord) - info "Gossiping epoch record", rootHash, contentKey = encKey - - p.storeContent(encKey, history_content.toContentId(encKey), encodedEpochRecord) - discard await p.neighborhoodGossip( - Opt.none(NodeId), ContentKeysList(@[encKey]), @[encodedEpochRecord] - ) - - return ok() - -proc propagateEpochRecords*( - p: PortalProtocol, path: string -): Future[Result[void, string]] {.async.} = - ## Propagate all epoch accumulators created when building the accumulator - ## from the block headers. - ## path is a directory that holds all SSZ encoded epoch accumulator files. - for i in 0 ..< preMergeEpochs: - let file = - try: - path / &"mainnet-epoch-record-{i.uint64:05}.ssz" - except ValueError as e: - raiseAssert e.msg - - let res = await p.propagateEpochRecord(file) - if res.isErr(): - return err(res.error) - - return ok() - proc historyPropagate*( p: PortalProtocol, dataFile: string, verify = false ): Future[Result[void, string]] {.async.} = diff --git a/fluffy/network/history/accumulator.nim b/fluffy/network/history/accumulator.nim index 125e85a1f..c639b3ff9 100644 --- a/fluffy/network/history/accumulator.nim +++ b/fluffy/network/history/accumulator.nim @@ -70,10 +70,6 @@ type historicalEpochs*: List[Bytes32, int(MAX_HISTORICAL_EPOCHS)] currentEpoch*: EpochRecord - BlockEpochData* = object - epochHash*: Bytes32 - blockRelativeIndex*: uint64 - func init*(T: type Accumulator): T = Accumulator( historicalEpochs: List[Bytes32, int(MAX_HISTORICAL_EPOCHS)].init(@[]), @@ -221,20 +217,3 @@ func buildHeaderWithProof*( proof: BlockHeaderProof.init(proof), ) ) - -func getBlockEpochDataForBlockNumber*( - a: FinishedAccumulator, bn: UInt256 -): Result[BlockEpochData, string] = - let blockNumber = bn.truncate(uint64) - - if blockNumber.isPreMerge: - let epochIndex = getEpochIndex(blockNumber) - - ok( - BlockEpochData( - epochHash: a.historicalEpochs[epochIndex], - blockRelativeIndex: getHeaderRecordIndex(blockNumber, epochIndex), - ) - ) - else: - err("Block number is post merge: " & $blockNumber) diff --git a/fluffy/network/history/content/content_deprecated.nim b/fluffy/network/history/content/content_deprecated.nim new file mode 100644 index 000000000..900aca228 --- /dev/null +++ b/fluffy/network/history/content/content_deprecated.nim @@ -0,0 +1,71 @@ +# fluffy +# Copyright (c) 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. + +## File do be deleted when pruneDeprecatedAccumulatorRecords has been active +## for long enough that most users have upgraded and as a result cleaned up +## their database. + +{.push raises: [].} + +import + nimcrypto/[sha2, hash], + stint, + chronicles, + ssz_serialization, + ../../../common/common_types, + ../../../database/content_db, + ../accumulator + +type + ContentType = enum + blockHeader = 0x00 + blockBody = 0x01 + receipts = 0x02 + epochRecordDeprecated = 0x03 + + BlockKey = object + blockHash: BlockHash + + EpochRecordKeyDeprecated = object + epochHash: Digest + + ContentKey = object + case contentType: ContentType + of blockHeader: + blockHeaderKey: BlockKey + of blockBody: + blockBodyKey: BlockKey + of receipts: + receiptsKey: BlockKey + of epochRecordDeprecated: + epochRecordKeyDeprecated: EpochRecordKeyDeprecated + +func encode(contentKey: ContentKey): ContentKeyByteList = + ContentKeyByteList.init(SSZ.encode(contentKey)) + +func toContentId(contentKey: ContentKeyByteList): ContentId = + let idHash = sha2.sha256.digest(contentKey.asSeq()) + readUintBE[256](idHash.data) + +proc pruneDeprecatedAccumulatorRecords*( + accumulator: FinishedAccumulator, contentDB: ContentDB +) = + info "Pruning deprecated accumulator records" + + for i, hash in accumulator.historicalEpochs: + let + root = Digest(data: hash) + epochRecordKey = ContentKey( + contentType: epochRecordDeprecated, + epochRecordKeyDeprecated: EpochRecordKeyDeprecated(epochHash: root), + ) + encodedKey = encode(epochRecordKey) + contentId = toContentId(encodedKey) + + contentDB.del(contentId) + + info "Pruning deprecated accumulator records finished" diff --git a/fluffy/network/history/content/content_keys.nim b/fluffy/network/history/content/content_keys.nim index 77e605a29..61209ea76 100644 --- a/fluffy/network/history/content/content_keys.nim +++ b/fluffy/network/history/content/content_keys.nim @@ -31,13 +31,13 @@ type blockHeader = 0x00 blockBody = 0x01 receipts = 0x02 - epochRecord = 0x03 + blockNumber = 0x03 BlockKey* = object blockHash*: BlockHash - EpochRecordKey* = object - epochHash*: Digest + BlockNumberKey* = object + blockNumber*: uint64 ContentKey* = object case contentType*: ContentType @@ -47,22 +47,23 @@ type blockBodyKey*: BlockKey of receipts: receiptsKey*: BlockKey - of epochRecord: - epochRecordKey*: EpochRecordKey + of blockNumber: + blockNumberKey*: BlockNumberKey -func init*(T: type ContentKey, contentType: ContentType, hash: BlockHash | Digest): T = - case contentType - of blockHeader: - ContentKey(contentType: contentType, blockHeaderKey: BlockKey(blockHash: hash)) - of blockBody: - ContentKey(contentType: contentType, blockBodyKey: BlockKey(blockHash: hash)) - of receipts: - ContentKey(contentType: contentType, receiptsKey: BlockKey(blockHash: hash)) - of epochRecord: +func blockHeaderContentKey*(id: BlockHash | uint64): ContentKey = + when id is BlockHash: + ContentKey(contentType: blockHeader, blockHeaderKey: BlockKey(blockHash: id)) + else: ContentKey( - contentType: contentType, epochRecordKey: EpochRecordKey(epochHash: hash) + contentType: blockNumber, blockNumberKey: BlockNumberKey(blockNumber: id) ) +func blockBodyContentKey*(hash: BlockHash): ContentKey = + ContentKey(contentType: blockBody, blockBodyKey: BlockKey(blockHash: hash)) + +func receiptsContentKey*(hash: BlockHash): ContentKey = + ContentKey(contentType: receipts, receiptsKey: BlockKey(blockHash: hash)) + func encode*(contentKey: ContentKey): ContentKeyByteList = ContentKeyByteList.init(SSZ.encode(contentKey)) @@ -96,9 +97,8 @@ func `$`*(x: ContentKey): string = res.add($x.blockBodyKey) of receipts: res.add($x.receiptsKey) - of epochRecord: - let key = x.epochRecordKey - res.add("epochHash: " & $key.epochHash) + of blockNumber: + res.add($x.blockNumberKey) res.add(")") diff --git a/fluffy/network/history/history_network.nim b/fluffy/network/history/history_network.nim index fe55ba3c5..1bbb5361d 100644 --- a/fluffy/network/history/history_network.nim +++ b/fluffy/network/history/history_network.nim @@ -17,7 +17,8 @@ import ../../database/content_db, ../../network_metadata, ../wire/[portal_protocol, portal_stream, portal_protocol_config], - "."/[history_content, accumulator, beacon_chain_historical_roots] + "."/[history_content, accumulator, beacon_chain_historical_roots], + ./content/content_deprecated logScope: topics = "portal_hist" @@ -178,8 +179,20 @@ template calcReceiptsRoot*(receipts: PortalReceipts): Hash256 = template calcWithdrawalsRoot*(receipts: Withdrawals): Hash256 = calcRootHash(receipts) +func validateBlockHeader*(header: BlockHeader, hash: BlockHash): Result[void, string] = + if not (header.blockHash() == hash): + err("Block header hash does not match") + else: + ok() + +func validateBlockHeader*(header: BlockHeader, number: uint64): Result[void, string] = + if not (header.number == number): + err("Block header number does not match") + else: + ok() + func validateBlockHeaderBytes*( - bytes: openArray[byte], hash: BlockHash + bytes: openArray[byte], id: uint64 | BlockHash ): Result[BlockHeader, string] = let header = ?decodeRlp(bytes, BlockHeader) @@ -191,11 +204,13 @@ func validateBlockHeaderBytes*( # pretty trivial to provide a non-canonical valid header. # It might be somewhat more useful if just done (temporarily) for the headers # post-merge which are currently provided without proof. + # For comparison by number this is obviously not sufficient as any other field + # could be manipulated and because of this a block header proof will always + # be needed. - if not (header.blockHash() == hash): - err("Block header hash does not match") - else: - ok(header) + ?header.validateBlockHeader(id) + + ok(header) proc validateBlockBody*( body: PortalBlockBodyLegacy, header: BlockHeader @@ -267,25 +282,25 @@ proc validateBlockBodyBytes*( # post merge, so the checks are still useful, for now. if isShanghai(chainConfig, timestamp): if header.withdrawalsRoot.isNone(): - return err("Expected withdrawalsRoot for Shanghai block") + err("Expected withdrawalsRoot for Shanghai block") elif header.ommersHash != EMPTY_UNCLE_HASH: - return err("Expected empty uncles for a Shanghai block") + err("Expected empty uncles for a Shanghai block") else: let body = ?decodeSsz(bytes, PortalBlockBodyShanghai) ?validateBlockBody(body, header) BlockBody.fromPortalBlockBody(body) elif isPoSBlock(chainConfig, header.number): if header.withdrawalsRoot.isSome(): - return err("Expected no withdrawalsRoot for pre Shanghai block") + err("Expected no withdrawalsRoot for pre Shanghai block") elif header.ommersHash != EMPTY_UNCLE_HASH: - return err("Expected empty uncles for a PoS block") + err("Expected empty uncles for a PoS block") else: let body = ?decodeSsz(bytes, PortalBlockBodyLegacy) ?validateBlockBody(body, header) BlockBody.fromPortalBlockBody(body) else: if header.withdrawalsRoot.isSome(): - return err("Expected no withdrawalsRoot for pre Shanghai block") + err("Expected no withdrawalsRoot for pre Shanghai block") else: let body = ?decodeSsz(bytes, PortalBlockBodyLegacy) ?validateBlockBody(body, header) @@ -294,12 +309,10 @@ proc validateBlockBodyBytes*( proc validateReceipts*( receipts: PortalReceipts, receiptsRoot: KeccakHash ): Result[void, string] = - let calculatedReceiptsRoot = calcReceiptsRoot(receipts) - - if calculatedReceiptsRoot != receiptsRoot: - return err("Unexpected receipt root") + if calcReceiptsRoot(receipts) != receiptsRoot: + err("Unexpected receipt root") else: - return ok() + ok() proc validateReceiptsBytes*( bytes: openArray[byte], receiptsRoot: KeccakHash @@ -393,14 +406,14 @@ func verifyHeader( verifyHeader(n.accumulator, header, proof) proc getVerifiedBlockHeader*( - n: HistoryNetwork, hash: BlockHash + n: HistoryNetwork, id: BlockHash | uint64 ): Future[Opt[BlockHeader]] {.async: (raises: [CancelledError]).} = let - contentKey = ContentKey.init(blockHeader, hash).encode() - contentId = contentKey.toContentId() + contentKey = blockHeaderContentKey(id).encode() + contentId = history_content.toContentId(contentKey) logScope: - hash + id contentKey # Note: This still requests a BlockHeaderWithProof from the database, as that @@ -421,7 +434,7 @@ proc getVerifiedBlockHeader*( warn "Failed decoding header with proof", error continue - header = validateBlockHeaderBytes(headerWithProof.header.asSeq(), hash).valueOr: + header = validateBlockHeaderBytes(headerWithProof.header.asSeq(), id).valueOr: warn "Validation of block header failed", error continue @@ -449,7 +462,7 @@ proc getBlockBody*( return Opt.some(BlockBody(transactions: @[], uncles: @[])) let - contentKey = ContentKey.init(blockBody, hash).encode() + contentKey = blockBodyContentKey(hash).encode() contentId = contentKey.toContentId() logScope: @@ -484,23 +497,36 @@ proc getBlockBody*( return Opt.none(BlockBody) proc getBlock*( - n: HistoryNetwork, hash: BlockHash + n: HistoryNetwork, id: BlockHash | uint64 ): Future[Opt[Block]] {.async: (raises: [CancelledError]).} = - debug "Trying to retrieve block with hash", hash + debug "Trying to retrieve block", id # Note: Using `getVerifiedBlockHeader` instead of getBlockHeader even though # proofs are not necessiarly needed, in order to avoid having to inject # also the original type into the network. let - header = (await n.getVerifiedBlockHeader(hash)).valueOr: - warn "Failed to get header when getting block", hash + header = (await n.getVerifiedBlockHeader(id)).valueOr: + warn "Failed to get header when getting block", id return Opt.none(Block) + hash = + when id is BlockHash: + id + else: + header.blockHash() body = (await n.getBlockBody(hash, header)).valueOr: warn "Failed to get body when getting block", hash return Opt.none(Block) return Opt.some((header, body)) +proc getBlockHashByNumber*( + n: HistoryNetwork, blockNumber: uint64 +): Future[Result[BlockHash, string]] {.async: (raises: [CancelledError]).} = + let header = (await n.getVerifiedBlockHeader(blockNumber)).valueOr: + return err("Cannot retrieve block header for given block number") + + ok(header.blockHash()) + proc getReceipts*( n: HistoryNetwork, hash: BlockHash, header: BlockHeader ): Future[Opt[seq[Receipt]]] {.async: (raises: [CancelledError]).} = @@ -509,7 +535,7 @@ proc getReceipts*( return Opt.some(newSeq[Receipt]()) let - contentKey = ContentKey.init(receipts, hash).encode() + contentKey = receiptsContentKey(hash).encode() contentId = contentKey.toContentId() logScope: @@ -539,70 +565,6 @@ proc getReceipts*( return Opt.some(receipts) -proc getEpochRecord( - n: HistoryNetwork, epochHash: Digest -): Future[Opt[EpochRecord]] {.async: (raises: [CancelledError]).} = - let - contentKey = ContentKey.init(epochRecord, epochHash).encode() - contentId = contentKey.toContentId() - - logScope: - epochHash - contentKey - - let accumulatorFromDb = n.getContentFromDb(EpochRecord, contentId) - if accumulatorFromDb.isSome(): - info "Fetched epoch accumulator from database" - return accumulatorFromDb - - for i in 0 ..< requestRetries: - let - accumulatorContent = (await n.portalProtocol.contentLookup(contentKey, contentId)).valueOr: - warn "Failed fetching epoch accumulator from the network" - return Opt.none(EpochRecord) - - epochRecord = - try: - SSZ.decode(accumulatorContent.content, EpochRecord) - except SerializationError: - continue - - let hash = hash_tree_root(epochRecord) - if hash == epochHash: - info "Fetched epoch accumulator from the network" - n.portalProtocol.storeContent(contentKey, contentId, accumulatorContent.content) - n.portalProtocol.triggerPoke( - accumulatorContent.nodesInterestedInContent, contentKey, - accumulatorContent.content, - ) - - return Opt.some(epochRecord) - else: - warn "Validation of epoch accumulator failed", resultedEpochHash = hash - - return Opt.none(EpochRecord) - -proc getBlockHashByNumber*( - n: HistoryNetwork, bn: UInt256 -): Future[Result[BlockHash, string]] {.async: (raises: [CancelledError]).} = - let - epochData = n.accumulator.getBlockEpochDataForBlockNumber(bn).valueOr: - return err(error) - digest = Digest(data: epochData.epochHash) - epoch = (await n.getEpochRecord(digest)).valueOr: - return err("Cannot retrieve epoch accumulator for given block number") - - ok(epoch[epochData.blockRelativeIndex].blockHash) - -proc getBlock*( - n: HistoryNetwork, bn: UInt256 -): Future[Result[Opt[Block], string]] {.async: (raises: [CancelledError]).} = - let - blockHash = ?(await n.getBlockHashByNumber(bn)) - maybeBlock = await n.getBlock(blockHash) - - return ok(maybeBlock) - proc validateContent( n: HistoryNetwork, content: seq[byte], contentKey: ContentKeyByteList ): Future[bool] {.async: (raises: [CancelledError]).} = @@ -649,24 +611,20 @@ proc validateContent( return false else: return true - of epochRecord: - # Check first if epochHash is part of master accumulator - let epochHash = key.epochRecordKey.epochHash - if not n.accumulator.historicalEpochs.contains(epochHash.data): - warn "Offered epoch accumulator is not part of master accumulator", epochHash - return false - - let epochRecord = - try: - SSZ.decode(content, EpochRecord) - except SerializationError: - warn "Failed decoding epoch accumulator" + of blockNumber: + let + headerWithProof = decodeSsz(content, BlockHeaderWithProof).valueOr: + warn "Failed decoding header with proof", error + return false + header = validateBlockHeaderBytes( + headerWithProof.header.asSeq(), key.blockNumberKey.blockNumber + ).valueOr: + warn "Invalid block header offered", error return false - # Next check the hash tree root, as this is probably more expensive - let hash = hash_tree_root(epochRecord) - if hash != epochHash: - warn "Epoch accumulator has invalid root hash" + let res = n.verifyHeader(header, headerWithProof.proof) + if res.isErr(): + warn "Failed on check if header is part of canonical chain", error = res.error return false else: return true @@ -771,6 +729,7 @@ proc start*(n: HistoryNetwork) = n.processContentLoop = processContentLoop(n) n.statusLogLoop = statusLogLoop(n) + pruneDeprecatedAccumulatorRecords(n.accumulator, n.contentDB) proc stop*(n: HistoryNetwork) = n.portalProtocol.stop() diff --git a/fluffy/rpc/rpc_eth_api.nim b/fluffy/rpc/rpc_eth_api.nim index 471597ab6..2172b242c 100644 --- a/fluffy/rpc/rpc_eth_api.nim +++ b/fluffy/rpc/rpc_eth_api.nim @@ -10,6 +10,7 @@ import std/[times, sequtils, strutils, typetraits], json_rpc/[rpcproxy, rpcserver], + chronicles, web3/conversions, # sigh, for FixedBytes marshalling web3/eth_api_types, web3/primitives as web3types, @@ -30,7 +31,7 @@ from ../../nimbus/beacon/web3_eth_conv import w3Addr, w3Hash, ethHash # Currently supported subset: # - eth_chainId # - eth_getBlockByHash -# - eth_getBlockByNumber - Partially: only by tags and block numbers before TheMerge +# - eth_getBlockByNumber # - eth_getBlockTransactionCountByHash # - eth_getLogs - Partially: only requests by block hash # @@ -279,15 +280,11 @@ proc installEthApiHandlers*( raise newException(ValueError, "Unsupported block tag " & tag) else: let - blockNumber = quantityTag.number.uint64.u256 - maybeBlock = (await historyNetwork.getBlock(blockNumber)).valueOr: - raise newException(ValueError, error) + blockNumber = quantityTag.number.uint64 + (header, body) = (await historyNetwork.getBlock(blockNumber)).valueOr: + return Opt.none(BlockObject) - if maybeBlock.isNone(): - return Opt.none(BlockObject) - else: - let (header, body) = maybeBlock.get() - return Opt.some(BlockObject.init(header, body, fullTransactions)) + return Opt.some(BlockObject.init(header, body, fullTransactions)) rpcServerWithProxy.rpc("eth_getBlockTransactionCountByHash") do( data: eth_api_types.Hash256 @@ -320,7 +317,7 @@ proc installEthApiHandlers*( ) -> seq[LogObject]: if filterOptions.blockHash.isNone(): # Currently only queries by blockhash are supported. - # To support range queries the Indicies network is required. + # TODO: Can impolement range queries by block number now. raise newException( ValueError, "Unsupported query: Only `blockHash` queries are currently supported", @@ -365,7 +362,7 @@ proc installEthApiHandlers*( raise newException(ValueError, "tag not yet implemented") else: let - blockNumber = quantityTag.number.uint64.u256 + blockNumber = quantityTag.number.uint64 blockHash = (await historyNetwork.getBlockHashByNumber(blockNumber)).valueOr: raise newException(ValueError, "Unable to get block hash") @@ -390,7 +387,7 @@ proc installEthApiHandlers*( raise newException(ValueError, "tag not yet implemented") else: let - blockNumber = quantityTag.number.uint64.u256 + blockNumber = quantityTag.number.uint64 blockHash = (await historyNetwork.getBlockHashByNumber(blockNumber)).valueOr: raise newException(ValueError, "Unable to get block hash") @@ -417,7 +414,7 @@ proc installEthApiHandlers*( raise newException(ValueError, "tag not yet implemented") else: let - blockNumber = quantityTag.number.uint64.u256 + blockNumber = quantityTag.number.uint64 blockHash = (await historyNetwork.getBlockHashByNumber(blockNumber)).valueOr: raise newException(ValueError, "Unable to get block hash") @@ -443,7 +440,7 @@ proc installEthApiHandlers*( raise newException(ValueError, "tag not yet implemented") else: let - blockNumber = quantityTag.number.uint64.u256 + blockNumber = quantityTag.number.uint64 blockHash = (await historyNetwork.getBlockHashByNumber(blockNumber)).valueOr: raise newException(ValueError, "Unable to get block hash") diff --git a/fluffy/rpc/rpc_portal_debug_api.nim b/fluffy/rpc/rpc_portal_debug_api.nim index c02a5ef86..c6c06b703 100644 --- a/fluffy/rpc/rpc_portal_debug_api.nim +++ b/fluffy/rpc/rpc_portal_debug_api.nim @@ -79,19 +79,3 @@ proc installPortalDebugApiHandlers*( return true else: raise newException(ValueError, $res.error) - - rpcServer.rpc("portal_" & network & "_propagateEpochRecord") do( - dataFile: string - ) -> bool: - let res = await p.propagateEpochRecord(dataFile) - if res.isOk(): - return true - else: - raise newException(ValueError, $res.error) - - rpcServer.rpc("portal_" & network & "_propagateEpochRecords") do(path: string) -> bool: - let res = await p.propagateEpochRecords(path) - if res.isOk(): - return true - else: - raise newException(ValueError, $res.error) diff --git a/fluffy/tests/portal_spec_tests/mainnet/test_history_content_keys.nim b/fluffy/tests/portal_spec_tests/mainnet/test_history_content_keys.nim index 0a85bb65e..8e143a262 100644 --- a/fluffy/tests/portal_spec_tests/mainnet/test_history_content_keys.nim +++ b/fluffy/tests/portal_spec_tests/mainnet/test_history_content_keys.nim @@ -35,9 +35,7 @@ suite "History ContentKey Encodings": contentIdHexBE = "3e86b3767b57402ea72e369ae0496ce47cc15be685bec3b4726b9f316e3895fe" - let contentKey = ContentKey( - contentType: blockHeader, blockHeaderKey: BlockKey(blockHash: blockHash) - ) + let contentKey = blockHeaderContentKey(blockHash) let encoded = encode(contentKey) check encoded.asSeq.toHex == contentKeyHex @@ -69,8 +67,7 @@ suite "History ContentKey Encodings": contentIdHexBE = "ebe414854629d60c58ddd5bf60fd72e41760a5f7a463fdcb169f13ee4a26786b" - let contentKey = - ContentKey(contentType: blockBody, blockBodyKey: BlockKey(blockHash: blockHash)) + let contentKey = blockBodyContentKey(blockHash) let encoded = encode(contentKey) check encoded.asSeq.toHex == contentKeyHex @@ -102,8 +99,7 @@ suite "History ContentKey Encodings": contentIdHexBE = "a888f4aafe9109d495ac4d4774a6277c1ada42035e3da5e10a04cc93247c04a4" - let contentKey = - ContentKey(contentType: receipts, receiptsKey: BlockKey(blockHash: blockHash)) + let contentKey = receiptsContentKey(blockHash) let encoded = encode(contentKey) check encoded.asSeq.toHex == contentKeyHex @@ -119,25 +115,20 @@ suite "History ContentKey Encodings": # In stint this does BE hex string toContentId(contentKey).toHex() == contentIdHexBE - test "Epoch Accumulator": + test "BlockHeader by Number": # Input - const epochHash = Digest.fromHex( - "0xe242814b90ed3950e13aac7e56ce116540c71b41d1516605aada26c6c07cc491" - ) + const blockNumber = 12345678'u64 # Output const - contentKeyHex = - "03e242814b90ed3950e13aac7e56ce116540c71b41d1516605aada26c6c07cc491" + contentKeyHex = "034e61bc0000000000" contentId = - "72232402989179419196382321898161638871438419016077939952896528930608027961710" + "14960950260935695396511307566164035182676768442501235074589175304147024756175" # or contentIdHexBE = - "9fb2175e76c6989e0fdac3ee10c40d2a81eb176af32e1c16193e3904fe56896e" + "2113990747a85ab39785d21342fa5db1f68acc0011605c0c73f68fc331643dcf" - let contentKey = ContentKey( - contentType: epochRecord, epochRecordKey: EpochRecordKey(epochHash: epochHash) - ) + let contentKey = blockHeaderContentKey(blockNumber) let encoded = encode(contentKey) check encoded.asSeq.toHex == contentKeyHex @@ -147,7 +138,7 @@ suite "History ContentKey Encodings": let contentKeyDecoded = decoded.get() check: contentKeyDecoded.contentType == contentKey.contentType - contentKeyDecoded.epochRecordKey == contentKey.epochRecordKey + contentKeyDecoded.blockNumberKey == contentKey.blockNumberKey toContentId(contentKey) == parse(contentId, StUint[256], 10) # In stint this does BE hex string diff --git a/fluffy/tests/state_network_tests/state_test_helpers.nim b/fluffy/tests/state_network_tests/state_test_helpers.nim index e75914c13..182d33a82 100644 --- a/fluffy/tests/state_network_tests/state_test_helpers.nim +++ b/fluffy/tests/state_network_tests/state_test_helpers.nim @@ -150,9 +150,7 @@ proc mockBlockHashToStateRoot*( blockHeaderWithProof = BlockHeaderWithProof( header: ByteList[2048].init(headerRlp), proof: BlockHeaderProof.init() ) - contentKeyBytes = history_content.ContentKey - .init(history_content.ContentType.blockHeader, blockHash) - .encode() + contentKeyBytes = blockHeaderContentKey(blockHash).encode() contentId = history_content.toContentId(contentKeyBytes) sn.portalProtocol().storeContent( diff --git a/fluffy/tests/test_accumulator.nim b/fluffy/tests/test_accumulator.nim index 34441e50d..743a49e0f 100644 --- a/fluffy/tests/test_accumulator.nim +++ b/fluffy/tests/test_accumulator.nim @@ -88,52 +88,3 @@ suite "Header Accumulator": let accumulatorRes = buildAccumulator(headers) check accumulatorRes.isErr() - - test "Header BlockNumber to EpochRecord Root": - # Note: This test assumes at least 3 epochs - const amount = mergeBlockNumber - - var - headerHashes: seq[Hash256] = @[] - headers: seq[BlockHeader] - - for i in 0 ..< amount: - let header = BlockHeader(number: i, difficulty: u256(1)) - headers.add(header) - headerHashes.add(header.blockHash()) - - let accumulatorRes = buildAccumulator(headers) - check accumulatorRes.isOk() - let accumulator = accumulatorRes.get() - - # Valid response for block numbers in epoch 0 - block: - for i in 0 ..< EPOCH_SIZE: - let res = accumulator.getBlockEpochDataForBlockNumber(u256(i)) - check: - res.isOk() - res.get().epochHash == accumulator.historicalEpochs[0] - - # Valid response for block numbers in epoch 1 - block: - for i in EPOCH_SIZE ..< (2 * EPOCH_SIZE): - let res = accumulator.getBlockEpochDataForBlockNumber(u256(i)) - check: - res.isOk() - res.get().epochHash == accumulator.historicalEpochs[1] - - # Valid response for block numbers in the incomplete (= last) epoch - block: - const startIndex = mergeBlockNumber - (mergeBlockNumber mod EPOCH_SIZE) - for i in startIndex ..< mergeBlockNumber: - let res = accumulator.getBlockEpochDataForBlockNumber(u256(i)) - check: - res.isOk() - res.get().epochHash == accumulator.historicalEpochs[preMergeEpochs - 1] - - # Error for block number at and past merge - block: - check: - accumulator.getBlockEpochDataForBlockNumber(u256(mergeBlockNumber)).isErr() - - accumulator.getBlockEpochDataForBlockNumber(u256(mergeBlockNumber + 1)).isErr() diff --git a/fluffy/tests/test_history_network.nim b/fluffy/tests/test_history_network.nim index 5c735029d..20b0eb7d1 100644 --- a/fluffy/tests/test_history_network.nim +++ b/fluffy/tests/test_history_network.nim @@ -114,34 +114,17 @@ procSuite "History Content Network": check headersWithProof.isOk() - # Only node 2 stores the headers and all epoch accumulators. + # Only node 2 stores the headers (by number) for headerWithProof in headersWithProof.get(): let header = rlp.decode(headerWithProof.header.asSeq(), BlockHeader) - headerHash = header.blockHash() - blockKey = BlockKey(blockHash: headerHash) - contentKey = ContentKey(contentType: blockHeader, blockHeaderKey: blockKey) + contentKey = blockHeaderContentKey(header.number) encKey = encode(contentKey) contentId = toContentId(contentKey) historyNode2.portalProtocol().storeContent( encKey, contentId, SSZ.encode(headerWithProof) ) - # Need to store the epoch accumulators to be able to do the block to hash - # mapping - for epochRecord in epochRecords: - let - rootHash = epochRecord.hash_tree_root() - contentKey = ContentKey( - contentType: ContentType.epochRecord, - epochRecordKey: EpochRecordKey(epochHash: rootHash), - ) - encKey = encode(contentKey) - contentId = toContentId(contentKey) - historyNode2.portalProtocol().storeContent( - encKey, contentId, SSZ.encode(epochRecord) - ) - check: historyNode1.portalProtocol().addNode(historyNode2.localNode()) == Added historyNode2.portalProtocol().addNode(historyNode1.localNode()) == Added @@ -150,15 +133,11 @@ procSuite "History Content Network": (await historyNode2.portalProtocol().ping(historyNode1.localNode())).isOk() for i in headersToTest: - let blockResponse = await historyNode1.historyNetwork.getBlock(u256(i)) + let blockResponse = await historyNode1.historyNetwork.getBlock(i.uint64) check blockResponse.isOk() - let blockOpt = blockResponse.get() - - check blockOpt.isSome() - - let (blockHeader, blockBody) = blockOpt.unsafeGet() + let (blockHeader, blockBody) = blockResponse.value() check blockHeader == headers[i] diff --git a/fluffy/tools/beacon_lc_bridge/beacon_lc_bridge.nim b/fluffy/tools/beacon_lc_bridge/beacon_lc_bridge.nim index 2310c52a2..61eb4ddf6 100644 --- a/fluffy/tools/beacon_lc_bridge/beacon_lc_bridge.nim +++ b/fluffy/tools/beacon_lc_bridge/beacon_lc_bridge.nim @@ -281,7 +281,7 @@ proc run(config: BeaconBridgeConf) {.raises: [CatchableError].} = blockhash = history_content.`$` hash block: # gossip header - let contentKey = history_content.ContentKey.init(blockHeader, hash) + let contentKey = blockHeaderContentKey(hash) let encodedContentKey = contentKey.encode.asSeq() try: @@ -304,7 +304,7 @@ proc run(config: BeaconBridgeConf) {.raises: [CatchableError].} = await sleepAsync(2.seconds) block: # gossip block - let contentKey = history_content.ContentKey.init(blockBody, hash) + let contentKey = blockBodyContentKey(hash) let encodedContentKey = contentKey.encode.asSeq() try: diff --git a/fluffy/tools/content_verifier.nim b/fluffy/tools/content_verifier.nim deleted file mode 100644 index 1d11c0563..000000000 --- a/fluffy/tools/content_verifier.nim +++ /dev/null @@ -1,91 +0,0 @@ -# Nimbus -# Copyright (c) 2022-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. - -# Tool to verify that certain Portal content is available on the network. -# Currently only supports checking `EpochRecord`s of the history network. - -{.push raises: [].} - -import - confutils, - chronicles, - chronicles/topics_registry, - stew/byteutils, - ../network_metadata, - ../network/history/[accumulator, history_content, history_network], - ../rpc/portal_rpc_client - -type ContentVerifierConf* = object - logLevel* {. - defaultValue: LogLevel.INFO, - defaultValueDesc: $LogLevel.INFO, - desc: "Sets the log level", - name: "log-level" - .}: LogLevel - - rpcAddress* {. - desc: "Address of the JSON-RPC service", - defaultValue: "127.0.0.1", - name: "rpc-address" - .}: string - - rpcPort* {. - defaultValue: 8545, desc: "Port of the JSON-RPC service", name: "rpc-port" - .}: uint16 - -proc checkAccumulators(client: RpcClient) {.async.} = - let accumulator = loadAccumulator() - - for i, hash in accumulator.historicalEpochs: - let root = Digest(data: hash) - let contentKey = ContentKey.init(epochRecord, root) - - try: - let contentInfo = await client.portal_historyRecursiveFindContent( - contentKey.encode.asSeq().toHex() - ) - - let res = decodeSsz(hexToSeqByte(contentInfo.content), EpochRecord) - if res.isErr(): - echo "[Invalid] EpochRecord number " & $i & ": " & $root & " error: " & res.error - else: - let epochRecord = res.get() - let resultingRoot = hash_tree_root(epochRecord) - if resultingRoot == root: - echo "[Available] EpochRecord number " & $i & ": " & $root - else: - echo "[Invalid] EpochRecord number " & $i & ": " & $root & - " error: Invalid root" - except RpcPostError as e: - # RpcPostError when for example timing out on the request. Could retry - # in this case. - fatal "Error occured on JSON-RPC request", error = e.msg - quit 1 - except ValueError as e: - # Either an error with the provided content key or the content was - # simply not available in the network - echo "[Not Available] EpochRecord number " & $i & ": " & $root & " error: " & e.msg - - # Using the http connection re-use seems to slow down these sequentual - # requests considerably. Force a new connection setup by doing a close after - # each request. - await client.close() - -proc run(config: ContentVerifierConf) {.async.} = - let client = newRpcHttpClient() - await client.connect(config.rpcAddress, Port(config.rpcPort), false) - - await checkAccumulators(client) - -when isMainModule: - {.pop.} - let config = ContentVerifierConf.load() - {.push raises: [].} - - setLogLevel(config.logLevel) - - waitFor run(config) diff --git a/fluffy/tools/portal_bridge/portal_bridge_history.nim b/fluffy/tools/portal_bridge/portal_bridge_history.nim index 3f62a9075..ffa84dc5e 100644 --- a/fluffy/tools/portal_bridge/portal_bridge_history.nim +++ b/fluffy/tools/portal_bridge/portal_bridge_history.nim @@ -143,7 +143,7 @@ proc gossipBlockHeader( headerWithProof: BlockHeaderWithProof, ): Future[Result[void, string]] {.async: (raises: []).} = let - contentKey = history_content.ContentKey.init(blockHeader, hash) + contentKey = blockHeaderContentKey(hash) encodedContentKeyHex = contentKey.encode.asSeq().toHex() peers = @@ -163,7 +163,7 @@ proc gossipBlockBody( body: PortalBlockBodyLegacy | PortalBlockBodyShanghai, ): Future[Result[void, string]] {.async: (raises: []).} = let - contentKey = history_content.ContentKey.init(blockBody, hash) + contentKey = blockBodyContentKey(hash) encodedContentKeyHex = contentKey.encode.asSeq().toHex() peers = @@ -181,8 +181,7 @@ proc gossipReceipts( client: RpcClient, hash: common_types.BlockHash, receipts: PortalReceipts ): Future[Result[void, string]] {.async: (raises: []).} = let - contentKey = - history_content.ContentKey.init(history_content.ContentType.receipts, hash) + contentKey = receiptsContentKey(hash) encodedContentKeyHex = contentKey.encode.asSeq().toHex() peers = @@ -416,7 +415,7 @@ proc runBackfillLoopAuditMode( # header block headerBlock: let - contentKey = ContentKey.init(blockHeader, blockHash) + contentKey = blockHeaderContentKey(blockHash) contentHex = try: ( @@ -448,7 +447,7 @@ proc runBackfillLoopAuditMode( # body block bodyBlock: let - contentKey = ContentKey.init(blockBody, blockHash) + contentKey = blockBodyContentKey(blockHash) contentHex = try: ( @@ -476,7 +475,7 @@ proc runBackfillLoopAuditMode( # receipts block receiptsBlock: let - contentKey = ContentKey.init(ContentType.receipts, blockHash) + contentKey = receiptsContentKey(blockHash) contentHex = try: (