Refactor/clean-up/prepping history network code (#2836)

- Move any validation related code to new file
- Move any type conversion related code to new file
- Prepare validation code for adding different type of canonical
proofs
- Prepare for validation code for json-rpc api
- several other clean-ups and renames

Sort of a first pass as the validation code specifically can
use some further changes.
This commit is contained in:
Kim De Mey 2024-11-06 08:49:55 +01:00 committed by GitHub
parent 39e6b63138
commit 6374bfb39c
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
11 changed files with 416 additions and 425 deletions

View File

@ -36,6 +36,12 @@ func decodeRlp*(input: openArray[byte], T: type): Result[T, string] =
except RlpError as e: except RlpError as e:
err(e.msg) err(e.msg)
func decodeRlpOrRaise*(input: openArray[byte], T: type): T =
try:
rlp.decode(input, T)
except RlpError as e:
raiseAssert(e.msg)
func decodeSsz*(input: openArray[byte], T: type): Result[T, string] = func decodeSsz*(input: openArray[byte], T: type): Result[T, string] =
try: try:
ok(SSZ.decode(input, T)) ok(SSZ.decode(input, T))

View File

@ -18,7 +18,7 @@ import
../../database/content_db, ../../database/content_db,
../../network_metadata, ../../network_metadata,
../wire/[portal_protocol, portal_stream, portal_protocol_config], ../wire/[portal_protocol, portal_stream, portal_protocol_config],
"."/[history_content, validation/historical_hashes_accumulator], "."/[history_content, history_validation, history_type_conversions],
../beacon/beacon_chain_historical_roots, ../beacon/beacon_chain_historical_roots,
./content/content_deprecated ./content/content_deprecated
@ -28,7 +28,7 @@ from eth/common/accounts import EMPTY_ROOT_HASH
logScope: logScope:
topics = "portal_hist" topics = "portal_hist"
export historical_hashes_accumulator, blocks_rlp export blocks_rlp
type type
HistoryNetwork* = ref object HistoryNetwork* = ref object
@ -46,303 +46,25 @@ type
func toContentIdHandler(contentKey: ContentKeyByteList): results.Opt[ContentId] = func toContentIdHandler(contentKey: ContentKeyByteList): results.Opt[ContentId] =
ok(toContentId(contentKey)) ok(toContentId(contentKey))
## Calls to go from SSZ decoded Portal types to RLP fully decoded EL types ## Get local content calls
func fromPortalBlockBody*( proc getLocalContent(
T: type BlockBody, body: PortalBlockBodyLegacy
): Result[T, string] =
## Get the EL BlockBody from the SSZ-decoded `PortalBlockBodyLegacy`.
try:
var transactions: seq[Transaction]
for tx in body.transactions:
transactions.add(rlp.decode(tx.asSeq(), Transaction))
let uncles = rlp.decode(body.uncles.asSeq(), seq[Header])
ok(BlockBody(transactions: transactions, uncles: uncles))
except RlpError as e:
err("RLP decoding failed: " & e.msg)
func fromPortalBlockBody*(
T: type BlockBody, body: PortalBlockBodyShanghai
): Result[T, string] =
## Get the EL BlockBody from the SSZ-decoded `PortalBlockBodyShanghai`.
try:
var transactions: seq[Transaction]
for tx in body.transactions:
transactions.add(rlp.decode(tx.asSeq(), Transaction))
var withdrawals: seq[Withdrawal]
for w in body.withdrawals:
withdrawals.add(rlp.decode(w.asSeq(), Withdrawal))
ok(
BlockBody(
transactions: transactions,
uncles: @[], # Uncles must be empty, this is verified in `validateBlockBody`
withdrawals: Opt.some(withdrawals),
)
)
except RlpError as e:
err("RLP decoding failed: " & e.msg)
func fromPortalBlockBodyOrRaise*(
T: type BlockBody, body: PortalBlockBodyLegacy | PortalBlockBodyShanghai
): T =
## Get the EL BlockBody from one of the SSZ-decoded Portal BlockBody types.
## Will raise Assertion in case of invalid RLP encodings. Only use of data
## has been validated before!
let res = BlockBody.fromPortalBlockBody(body)
if res.isOk():
res.get()
else:
raiseAssert(res.error)
func fromPortalReceipts*(
T: type seq[Receipt], receipts: PortalReceipts
): Result[T, string] =
## Get the full decoded EL seq[Receipt] from the SSZ-decoded `PortalReceipts`.
try:
var res: seq[Receipt]
for receipt in receipts:
res.add(rlp.decode(receipt.asSeq(), Receipt))
ok(res)
except RlpError as e:
err("RLP decoding failed: " & e.msg)
## Calls to encode EL block types to the SSZ encoded Portal types.
# TODO: The fact that we have different Portal BlockBody types for the different
# forks but not for the EL BlockBody (usage of Option) does not play so well
# together.
func fromBlockBody*(T: type PortalBlockBodyLegacy, body: BlockBody): T =
var transactions: Transactions
for tx in body.transactions:
discard transactions.add(TransactionByteList(rlp.encode(tx)))
let uncles = Uncles(rlp.encode(body.uncles))
PortalBlockBodyLegacy(transactions: transactions, uncles: uncles)
func fromBlockBody*(T: type PortalBlockBodyShanghai, body: BlockBody): T =
var transactions: Transactions
for tx in body.transactions:
discard transactions.add(TransactionByteList(rlp.encode(tx)))
let uncles = Uncles(rlp.encode(body.uncles))
doAssert(body.withdrawals.isSome())
var withdrawals: Withdrawals
for w in body.withdrawals.get():
discard withdrawals.add(WithdrawalByteList(rlp.encode(w)))
PortalBlockBodyShanghai(
transactions: transactions, uncles: uncles, withdrawals: withdrawals
)
func fromReceipts*(T: type PortalReceipts, receipts: seq[Receipt]): T =
var portalReceipts: PortalReceipts
for receipt in receipts:
discard portalReceipts.add(ReceiptByteList(rlp.encode(receipt)))
portalReceipts
func encode*(blockBody: BlockBody): seq[byte] =
if blockBody.withdrawals.isSome():
SSZ.encode(PortalBlockBodyShanghai.fromBlockBody(blockBody))
else:
SSZ.encode(PortalBlockBodyLegacy.fromBlockBody(blockBody))
func encode*(receipts: seq[Receipt]): seq[byte] =
let portalReceipts = PortalReceipts.fromReceipts(receipts)
SSZ.encode(portalReceipts)
## Calls and helper calls to do validation of block header, body and receipts
# TODO: Failures on validation and perhaps deserialisation should be punished
# for if/when peer scoring/banning is added.
func validateBlockHeader*(header: Header, blockHash: Hash32): Result[void, string] =
if not (header.rlpHash() == blockHash):
err("Block header hash does not match")
else:
ok()
func validateBlockHeader*(header: Header, number: uint64): Result[void, string] =
if not (header.number == number):
err("Block header number does not match")
else:
ok()
func validateBlockHeaderBytes*(
bytes: openArray[byte], id: uint64 | Hash32
): Result[Header, string] =
let header = ?decodeRlp(bytes, Header)
# Note:
# One could do additional quick-checks here such as timestamp vs the optional
# (later forks) added fields. E.g. Shanghai field, Cancun fields,
# zero ommersHash, etc.
# However, the block hash comparison will obviously catch these and it is
# 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.
?header.validateBlockHeader(id)
ok(header)
template append*(w: var RlpWriter, v: TransactionByteList) =
w.appendRawBytes(v.asSeq)
template append*(w: var RlpWriter, v: WithdrawalByteList) =
w.appendRawBytes(v.asSeq)
template append*(w: var RlpWriter, v: ReceiptByteList) =
w.appendRawBytes(v.asSeq)
proc validateBlockBody*(
body: PortalBlockBodyLegacy, header: Header
): Result[void, string] =
## Validate the block body against the txRoot and ommersHash from the header.
let calculatedOmmersHash = keccak256(body.uncles.asSeq())
if calculatedOmmersHash != header.ommersHash:
return err("Invalid ommers hash")
let calculatedTxsRoot = orderedTrieRoot(body.transactions.asSeq)
if calculatedTxsRoot != header.txRoot:
return err(
"Invalid transactions root: expected " & $header.txRoot & " - got " &
$calculatedTxsRoot
)
ok()
proc validateBlockBody*(
body: PortalBlockBodyShanghai, header: Header
): Result[void, string] =
## Validate the block body against the txRoot, ommersHash and withdrawalsRoot
## from the header.
# Shortcut the ommersHash calculation as uncles must be an RLP encoded
# empty list
if body.uncles.asSeq() != @[byte 0xc0]:
return err("Invalid ommers hash, uncles list is not empty")
let calculatedTxsRoot = orderedTrieRoot(body.transactions.asSeq)
if calculatedTxsRoot != header.txRoot:
return err(
"Invalid transactions root: expected " & $header.txRoot & " - got " &
$calculatedTxsRoot
)
# TODO: This check is done higher up but perhaps this can become cleaner with
# some refactor.
doAssert(header.withdrawalsRoot.isSome())
let
calculatedWithdrawalsRoot = orderedTrieRoot(body.withdrawals.asSeq)
headerWithdrawalsRoot = header.withdrawalsRoot.get()
if calculatedWithdrawalsRoot != headerWithdrawalsRoot:
return err(
"Invalid withdrawals root: expected " & $headerWithdrawalsRoot & " - got " &
$calculatedWithdrawalsRoot
)
ok()
proc decodeBlockBodyBytes*(bytes: openArray[byte]): Result[BlockBody, string] =
if (let body = decodeSsz(bytes, PortalBlockBodyShanghai); body.isOk()):
BlockBody.fromPortalBlockBody(body.get())
elif (let body = decodeSsz(bytes, PortalBlockBodyLegacy); body.isOk()):
BlockBody.fromPortalBlockBody(body.get())
else:
err("All Portal block body decodings failed")
proc validateBlockBodyBytes*(
bytes: openArray[byte], header: Header
): Result[BlockBody, string] =
## Fully decode the SSZ encoded Portal Block Body and validate it against the
## header.
## TODO: improve this decoding in combination with the block body validation
## calls.
let timestamp = Moment.init(header.timestamp.int64, Second)
# TODO: The additional header checks are not needed as header is implicitly
# verified by means of the accumulator? Except that we don't use this yet
# post merge, so the checks are still useful, for now.
if isShanghai(chainConfig, timestamp):
if header.withdrawalsRoot.isNone():
err("Expected withdrawalsRoot for Shanghai block")
elif header.ommersHash != EMPTY_UNCLE_HASH:
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():
err("Expected no withdrawalsRoot for pre Shanghai block")
elif header.ommersHash != EMPTY_UNCLE_HASH:
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():
err("Expected no withdrawalsRoot for pre Shanghai block")
else:
let body = ?decodeSsz(bytes, PortalBlockBodyLegacy)
?validateBlockBody(body, header)
BlockBody.fromPortalBlockBody(body)
proc validateReceipts*(
receipts: PortalReceipts, receiptsRoot: Hash32
): Result[void, string] =
if orderedTrieRoot(receipts.asSeq) != receiptsRoot:
err("Unexpected receipt root")
else:
ok()
proc validateReceiptsBytes*(
bytes: openArray[byte], receiptsRoot: Hash32
): Result[seq[Receipt], string] =
## Fully decode the SSZ encoded receipts and validate it against the header's
## receipts root.
let receipts = ?decodeSsz(bytes, PortalReceipts)
?validateReceipts(receipts, receiptsRoot)
seq[Receipt].fromPortalReceipts(receipts)
## Content helper calls for specific history network types
proc getContent(
n: HistoryNetwork, n: HistoryNetwork,
T: type Header, T: type Header,
contentKey: ContentKeyByteList, contentKey: ContentKeyByteList,
contentId: ContentId, contentId: ContentId,
): Opt[T] = ): Opt[T] =
let localContent = n.portalProtocol.getLocalContent(contentKey, contentId).valueOr: let
localContent = n.portalProtocol.getLocalContent(contentKey, contentId).valueOr:
return Opt.none(T) return Opt.none(T)
let headerWithProof = # Stored data should always be serialized correctly
try: headerWithProof = decodeSszOrRaise(localContent, BlockHeaderWithProof)
SSZ.decode(localContent, BlockHeaderWithProof) header = decodeRlpOrRaise(headerWithProof.header.asSeq(), T)
except SerializationError as e:
raiseAssert(e.msg)
let res = decodeRlp(headerWithProof.header.asSeq(), T) Opt.some(header)
if res.isErr():
raiseAssert(res.error)
else:
Opt.some(res.get())
proc getContent( proc getLocalContent(
n: HistoryNetwork, n: HistoryNetwork,
T: type BlockBody, T: type BlockBody,
contentKey: ContentKeyByteList, contentKey: ContentKeyByteList,
@ -370,26 +92,22 @@ proc getContent(
Opt.some(body) Opt.some(body)
proc getContent( proc getLocalContent(
n: HistoryNetwork, n: HistoryNetwork,
T: type seq[Receipt], T: type seq[Receipt],
contentKey: ContentKeyByteList, contentKey: ContentKeyByteList,
contentId: ContentId, contentId: ContentId,
): Opt[T] = ): Opt[T] =
let localContent = n.portalProtocol.getLocalContent(contentKey, contentId).valueOr: let
localContent = n.portalProtocol.getLocalContent(contentKey, contentId).valueOr:
return Opt.none(T) return Opt.none(T)
let portalReceipts = # Stored data should always be serialized correctly
try: portalReceipts = decodeSszOrRaise(localContent, PortalReceipts)
SSZ.decode(localContent, PortalReceipts) receipts = T.fromPortalReceipts(portalReceipts).valueOr:
except SerializationError: raiseAssert(error)
raiseAssert("Stored data should always be serialized correctly")
let res = T.fromPortalReceipts(portalReceipts) Opt.some(receipts)
if res.isErr():
raiseAssert(res.error)
else:
Opt.some(res.get())
## Public API to get the history network specific types, either from database ## Public API to get the history network specific types, either from database
## or through a lookup on the Portal Network ## or through a lookup on the Portal Network
@ -402,11 +120,6 @@ proc getContent(
# ongoing requests are cancelled after the receival of the first response, # ongoing requests are cancelled after the receival of the first response,
# however that response is not yet validated at that moment. # however that response is not yet validated at that moment.
func verifyHeader(
n: HistoryNetwork, header: Header, proof: BlockHeaderProof
): Result[void, string] =
verifyHeader(n.accumulator, header, proof)
proc getVerifiedBlockHeader*( proc getVerifiedBlockHeader*(
n: HistoryNetwork, id: Hash32 | uint64 n: HistoryNetwork, id: Hash32 | uint64
): Future[Opt[Header]] {.async: (raises: [CancelledError]).} = ): Future[Opt[Header]] {.async: (raises: [CancelledError]).} =
@ -421,10 +134,10 @@ proc getVerifiedBlockHeader*(
# Note: This still requests a BlockHeaderWithProof from the database, as that # Note: This still requests a BlockHeaderWithProof from the database, as that
# is what is stored. But the proof doesn't need to be verified as it gets # is what is stored. But the proof doesn't need to be verified as it gets
# gets verified before storing. # gets verified before storing.
let headerFromDb = n.getContent(Header, contentKey, contentId) let localContent = n.getLocalContent(Header, contentKey, contentId)
if headerFromDb.isSome(): if localContent.isSome():
info "Fetched block header from database" info "Fetched block header locally"
return headerFromDb return localContent
for i in 0 ..< (1 + n.contentRequestRetries): for i in 0 ..< (1 + n.contentRequestRetries):
let let
@ -432,18 +145,10 @@ proc getVerifiedBlockHeader*(
warn "Failed fetching block header with proof from the network" warn "Failed fetching block header with proof from the network"
return Opt.none(Header) return Opt.none(Header)
headerWithProof = decodeSsz(headerContent.content, BlockHeaderWithProof).valueOr: header = validateCanonicalHeaderBytes(headerContent.content, id, n.accumulator).valueOr:
warn "Failed decoding header with proof", error = error
continue
header = validateBlockHeaderBytes(headerWithProof.header.asSeq(), id).valueOr:
warn "Validation of block header failed", error = error warn "Validation of block header failed", error = error
continue continue
if (let r = n.verifyHeader(header, headerWithProof.proof); r.isErr):
warn "Verification of block header failed", error = r.error
continue
info "Fetched valid block header from the network" info "Fetched valid block header from the network"
# Content is valid, it can be stored and propagated to interested peers # Content is valid, it can be stored and propagated to interested peers
n.portalProtocol.storeContent( n.portalProtocol.storeContent(
@ -456,7 +161,7 @@ proc getVerifiedBlockHeader*(
return Opt.some(header) return Opt.some(header)
# Headers were requested `1 + requestRetries` times and all failed on validation # Headers were requested `1 + requestRetries` times and all failed on validation
return Opt.none(Header) Opt.none(Header)
proc getBlockBody*( proc getBlockBody*(
n: HistoryNetwork, blockHash: Hash32, header: Header n: HistoryNetwork, blockHash: Hash32, header: Header
@ -473,10 +178,10 @@ proc getBlockBody*(
blockHash blockHash
contentKey contentKey
let bodyFromDb = n.getContent(BlockBody, contentKey, contentId, header) let localContent = n.getLocalContent(BlockBody, contentKey, contentId, header)
if bodyFromDb.isSome(): if localContent.isSome():
info "Fetched block body from database" info "Fetched block body locally"
return bodyFromDb return localContent
for i in 0 ..< (1 + n.contentRequestRetries): for i in 0 ..< (1 + n.contentRequestRetries):
let let
@ -500,7 +205,7 @@ proc getBlockBody*(
return Opt.some(body) return Opt.some(body)
# Bodies were requested `1 + requestRetries` times and all failed on validation # Bodies were requested `1 + requestRetries` times and all failed on validation
return Opt.none(BlockBody) Opt.none(BlockBody)
proc getBlock*( proc getBlock*(
n: HistoryNetwork, id: Hash32 | uint64 n: HistoryNetwork, id: Hash32 | uint64
@ -523,7 +228,7 @@ proc getBlock*(
warn "Failed to get body when getting block", hash warn "Failed to get body when getting block", hash
return Opt.none(Block) return Opt.none(Block)
return Opt.some((header, body)) Opt.some((header, body))
proc getBlockHashByNumber*( proc getBlockHashByNumber*(
n: HistoryNetwork, blockNumber: uint64 n: HistoryNetwork, blockNumber: uint64
@ -548,10 +253,10 @@ proc getReceipts*(
blockHash blockHash
contentKey contentKey
let receiptsFromDb = n.getContent(seq[Receipt], contentKey, contentId) let localContent = n.getLocalContent(seq[Receipt], contentKey, contentId)
if receiptsFromDb.isSome(): if localContent.isSome():
info "Fetched receipts from database" info "Fetched receipts locally"
return receiptsFromDb return localContent
for i in 0 ..< (1 + n.contentRequestRetries): for i in 0 ..< (1 + n.contentRequestRetries):
let let
@ -574,71 +279,45 @@ proc getReceipts*(
return Opt.some(receipts) return Opt.some(receipts)
# Receipts were requested `1 + requestRetries` times and all failed on validation # Receipts were requested `1 + requestRetries` times and all failed on validation
return Opt.none(seq[Receipt]) Opt.none(seq[Receipt])
proc validateContent( proc validateContent(
n: HistoryNetwork, content: seq[byte], contentKey: ContentKeyByteList n: HistoryNetwork, content: seq[byte], contentKeyBytes: ContentKeyByteList
): Future[bool] {.async: (raises: [CancelledError]).} = ): Future[Result[void, string]] {.async: (raises: [CancelledError]).} =
let key = contentKey.decode().valueOr: let contentKey = contentKeyBytes.decode().valueOr:
return false return err("Error decoding content key")
case key.contentType case contentKey.contentType
of blockHeader: of blockHeader:
let let _ = validateCanonicalHeaderBytes(
headerWithProof = decodeSsz(content, BlockHeaderWithProof).valueOr: content, contentKey.blockHeaderKey.blockHash, n.accumulator
warn "Failed decoding header with proof", error
return false
header = validateBlockHeaderBytes(
headerWithProof.header.asSeq(), key.blockHeaderKey.blockHash
).valueOr: ).valueOr:
warn "Invalid block header offered", error return err("Failed validating block header: " & error)
return false
let res = n.verifyHeader(header, headerWithProof.proof) ok()
if res.isErr():
warn "Failed on check if header is part of canonical chain", error = res.error
return false
else:
return true
of blockBody: of blockBody:
let header = (await n.getVerifiedBlockHeader(key.blockBodyKey.blockHash)).valueOr:
warn "Failed getting canonical header for block"
return false
let res = validateBlockBodyBytes(content, header)
if res.isErr():
warn "Failed validating block body", error = res.error
return false
else:
return true
of receipts:
let header = (await n.getVerifiedBlockHeader(key.receiptsKey.blockHash)).valueOr:
warn "Failed getting canonical header for receipts"
return false
let res = validateReceiptsBytes(content, header.receiptsRoot)
if res.isErr():
warn "Failed validating receipts", error = res.error
return false
else:
return true
of blockNumber:
let let
headerWithProof = decodeSsz(content, BlockHeaderWithProof).valueOr: header = (await n.getVerifiedBlockHeader(contentKey.blockBodyKey.blockHash)).valueOr:
warn "Failed decoding header with proof", error return err("Failed getting canonical header for block")
return false _ = validateBlockBodyBytes(content, header).valueOr:
header = validateBlockHeaderBytes( return err("Failed validating block body: " & error)
headerWithProof.header.asSeq(), key.blockNumberKey.blockNumber
).valueOr:
warn "Invalid block header offered", error
return false
let res = n.verifyHeader(header, headerWithProof.proof) ok()
if res.isErr(): of receipts:
warn "Failed on check if header is part of canonical chain", error = res.error let
return false header = (await n.getVerifiedBlockHeader(contentKey.receiptsKey.blockHash)).valueOr:
else: return err("Failed getting canonical header for receipts")
return true _ = validateReceiptsBytes(content, header.receiptsRoot).valueOr:
return err("Failed validating receipts: " & error)
ok()
of blockNumber:
let _ = validateCanonicalHeaderBytes(
content, contentKey.blockNumberKey.blockNumber, n.accumulator
).valueOr:
return err("Failed validating block header: " & error)
ok()
proc new*( proc new*(
T: type HistoryNetwork, T: type HistoryNetwork,
@ -685,7 +364,8 @@ proc validateContent(
# content passed here can have less items then contentKeys, but not more. # content passed here can have less items then contentKeys, but not more.
for i, contentItem in contentItems: for i, contentItem in contentItems:
let contentKey = contentKeys[i] let contentKey = contentKeys[i]
if await n.validateContent(contentItem, contentKey): let res = await n.validateContent(contentItem, contentKey)
if res.isOk():
let contentId = n.portalProtocol.toContentId(contentKey).valueOr: let contentId = n.portalProtocol.toContentId(contentKey).valueOr:
error "Received offered content with invalid content key", contentKey error "Received offered content with invalid content key", contentKey
return false return false
@ -694,7 +374,7 @@ proc validateContent(
debug "Received offered content validated successfully", contentKey debug "Received offered content validated successfully", contentKey
else: else:
error "Received offered content failed validation", contentKey error "Received offered content failed validation", contentKey, error = res.error
return false return false
return true return true

View File

@ -0,0 +1,145 @@
# 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.
{.push raises: [].}
import
eth/common/[headers_rlp, blocks_rlp, receipts_rlp, transactions_rlp],
./history_content
export history_content, headers_rlp, blocks_rlp, receipts_rlp
## Calls to go from SSZ decoded Portal types to RLP fully decoded EL types
func fromPortalBlockBody*(
T: type BlockBody, body: PortalBlockBodyLegacy
): Result[T, string] =
## Get the EL BlockBody from the SSZ-decoded `PortalBlockBodyLegacy`.
try:
var transactions: seq[Transaction]
for tx in body.transactions:
transactions.add(rlp.decode(tx.asSeq(), Transaction))
let uncles = rlp.decode(body.uncles.asSeq(), seq[Header])
ok(BlockBody(transactions: transactions, uncles: uncles))
except RlpError as e:
err("RLP decoding failed: " & e.msg)
func fromPortalBlockBody*(
T: type BlockBody, body: PortalBlockBodyShanghai
): Result[T, string] =
## Get the EL BlockBody from the SSZ-decoded `PortalBlockBodyShanghai`.
try:
var transactions: seq[Transaction]
for tx in body.transactions:
transactions.add(rlp.decode(tx.asSeq(), Transaction))
var withdrawals: seq[Withdrawal]
for w in body.withdrawals:
withdrawals.add(rlp.decode(w.asSeq(), Withdrawal))
ok(
BlockBody(
transactions: transactions,
uncles: @[], # Uncles must be empty, this is verified in `validateBlockBody`
withdrawals: Opt.some(withdrawals),
)
)
except RlpError as e:
err("RLP decoding failed: " & e.msg)
func fromPortalBlockBodyBytes*(bytes: openArray[byte]): Result[BlockBody, string] =
if (let res = decodeSsz(bytes, PortalBlockBodyLegacy); res.isOk()):
BlockBody.fromPortalBlockBody(res.value())
elif (let res = decodeSsz(bytes, PortalBlockBodyShanghai); res.isOk()):
BlockBody.fromPortalBlockBody(res.value())
else:
err("Invalid Portal BlockBody encoding")
func fromPortalBlockBodyOrRaise*(
T: type BlockBody, body: PortalBlockBodyLegacy | PortalBlockBodyShanghai
): T =
## Get the EL BlockBody from one of the SSZ-decoded Portal BlockBody types.
## Will raise Assertion in case of invalid RLP encodings. Only use for data
## has been validated before!
let res = BlockBody.fromPortalBlockBody(body)
if res.isOk():
res.get()
else:
raiseAssert(res.error)
func fromPortalReceipts*(
T: type seq[Receipt], receipts: PortalReceipts
): Result[T, string] =
## Get the full decoded EL seq[Receipt] from the SSZ-decoded `PortalReceipts`.
try:
var res: seq[Receipt]
for receipt in receipts:
res.add(rlp.decode(receipt.asSeq(), Receipt))
ok(res)
except RlpError as e:
err("RLP decoding failed: " & e.msg)
## Calls to convert EL block types to the Portal types.
func fromBlockBody*(T: type PortalBlockBodyLegacy, body: BlockBody): T =
var transactions: Transactions
for tx in body.transactions:
discard transactions.add(TransactionByteList(rlp.encode(tx)))
let uncles = Uncles(rlp.encode(body.uncles))
PortalBlockBodyLegacy(transactions: transactions, uncles: uncles)
func fromBlockBody*(T: type PortalBlockBodyShanghai, body: BlockBody): T =
var transactions: Transactions
for tx in body.transactions:
discard transactions.add(TransactionByteList(rlp.encode(tx)))
let uncles = Uncles(rlp.encode(body.uncles))
doAssert(body.withdrawals.isSome())
var withdrawals: Withdrawals
for w in body.withdrawals.get():
discard withdrawals.add(WithdrawalByteList(rlp.encode(w)))
PortalBlockBodyShanghai(
transactions: transactions, uncles: uncles, withdrawals: withdrawals
)
func fromReceipts*(T: type PortalReceipts, receipts: seq[Receipt]): T =
var portalReceipts: PortalReceipts
for receipt in receipts:
discard portalReceipts.add(ReceiptByteList(rlp.encode(receipt)))
portalReceipts
## Calls to encode EL types to the SSZ encoded Portal types.
func encode*(blockBody: BlockBody): seq[byte] =
if blockBody.withdrawals.isSome():
SSZ.encode(PortalBlockBodyShanghai.fromBlockBody(blockBody))
else:
SSZ.encode(PortalBlockBodyLegacy.fromBlockBody(blockBody))
func encode*(receipts: seq[Receipt]): seq[byte] =
let portalReceipts = PortalReceipts.fromReceipts(receipts)
SSZ.encode(portalReceipts)
## RLP writer append calls for the Portal/SSZ types
template append*(w: var RlpWriter, v: TransactionByteList) =
w.appendRawBytes(v.asSeq)
template append*(w: var RlpWriter, v: WithdrawalByteList) =
w.appendRawBytes(v.asSeq)
template append*(w: var RlpWriter, v: ReceiptByteList) =
w.appendRawBytes(v.asSeq)

View File

@ -0,0 +1,177 @@
# 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.
{.push raises: [].}
import
chronos/timer,
eth/trie/ordered_trie,
../../network_metadata,
./history_type_conversions,
./validation/historical_hashes_accumulator
from eth/common/eth_types_rlp import rlpHash
export historical_hashes_accumulator
func validateHeader(header: Header, blockHash: Hash32): Result[void, string] =
if not (header.rlpHash() == blockHash):
err("Header hash does not match")
else:
ok()
func validateHeader(header: Header, number: uint64): Result[void, string] =
if not (header.number == number):
err("Header number does not match")
else:
ok()
func validateHeaderBytes*(
bytes: openArray[byte], id: uint64 | Hash32
): Result[Header, string] =
# Note:
# No additional quick-checks are addedhere such as timestamp vs the optional
# (later forks) added fields. E.g. Shanghai field, Cancun fields,
# zero ommersHash, etc.
# This is because the block hash comparison + canonical verification will
# catch these. For comparison by number this is will also be caught by the
# canonical verification.
let header = ?decodeRlp(bytes, Header)
?header.validateHeader(id)
ok(header)
func verifyBlockHeaderProof*(
a: FinishedHistoricalHashesAccumulator, header: Header, proof: BlockHeaderProof
): Result[void, string] =
case proof.proofType
of BlockHeaderProofType.historicalHashesAccumulatorProof:
a.verifyAccumulatorProof(header, proof.historicalHashesAccumulatorProof)
of BlockHeaderProofType.none:
if header.isPreMerge():
err("Pre merge header requires HistoricalHashesAccumulatorProof")
else:
# TODO:
# Add verification post merge based on historical_roots & historical_summaries
ok()
func validateCanonicalHeaderBytes*(
bytes: openArray[byte], id: uint64 | Hash32, a: FinishedHistoricalHashesAccumulator
): Result[Header, string] =
let headerWithProof = decodeSsz(bytes, BlockHeaderWithProof).valueOr:
return err("Failed decoding header with proof: " & error)
let header = ?validateHeaderBytes(headerWithProof.header.asSeq(), id)
?a.verifyBlockHeaderProof(header, headerWithProof.proof)
ok(header)
func validateBlockBody*(
body: PortalBlockBodyLegacy, header: Header
): Result[void, string] =
## Validate the block body against the txRoot and ommersHash from the header.
let calculatedOmmersHash = keccak256(body.uncles.asSeq())
if calculatedOmmersHash != header.ommersHash:
return err("Invalid ommers hash")
let calculatedTxsRoot = orderedTrieRoot(body.transactions.asSeq)
if calculatedTxsRoot != header.txRoot:
return err(
"Invalid transactions root: expected " & $header.txRoot & " - got " &
$calculatedTxsRoot
)
ok()
func validateBlockBody*(
body: PortalBlockBodyShanghai, header: Header
): Result[void, string] =
## Validate the block body against the txRoot, ommersHash and withdrawalsRoot
## from the header.
# Shortcut the ommersHash calculation as uncles must be an RLP encoded
# empty list
if body.uncles.asSeq() != @[byte 0xc0]:
return err("Invalid ommers hash, uncles list is not empty")
let calculatedTxsRoot = orderedTrieRoot(body.transactions.asSeq)
if calculatedTxsRoot != header.txRoot:
return err(
"Invalid transactions root: expected " & $header.txRoot & " - got " &
$calculatedTxsRoot
)
# TODO: This check is done higher up but perhaps this can become cleaner with
# some refactor.
doAssert(header.withdrawalsRoot.isSome())
let
calculatedWithdrawalsRoot = orderedTrieRoot(body.withdrawals.asSeq)
headerWithdrawalsRoot = header.withdrawalsRoot.get()
if calculatedWithdrawalsRoot != headerWithdrawalsRoot:
return err(
"Invalid withdrawals root: expected " & $headerWithdrawalsRoot & " - got " &
$calculatedWithdrawalsRoot
)
ok()
func validateBlockBodyBytes*(
bytes: openArray[byte], header: Header
): Result[BlockBody, string] =
## Fully decode the SSZ encoded Portal Block Body and validate it against the
## header.
## TODO: improve this decoding in combination with the block body validation
## calls.
let timestamp = Moment.init(header.timestamp.int64, Second)
# TODO: The additional header checks are not needed as header is implicitly
# verified by means of the accumulator? Except that we don't use this yet
# post merge, so the checks are still useful, for now.
if isShanghai(chainConfig, timestamp):
if header.withdrawalsRoot.isNone():
err("Expected withdrawalsRoot for Shanghai block")
elif header.ommersHash != EMPTY_UNCLE_HASH:
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():
err("Expected no withdrawalsRoot for pre Shanghai block")
elif header.ommersHash != EMPTY_UNCLE_HASH:
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():
err("Expected no withdrawalsRoot for pre Shanghai block")
else:
let body = ?decodeSsz(bytes, PortalBlockBodyLegacy)
?validateBlockBody(body, header)
BlockBody.fromPortalBlockBody(body)
func validateReceipts*(
receipts: PortalReceipts, receiptsRoot: Hash32
): Result[void, string] =
if orderedTrieRoot(receipts.asSeq) != receiptsRoot:
err("Unexpected receipt root")
else:
ok()
func validateReceiptsBytes*(
bytes: openArray[byte], receiptsRoot: Hash32
): Result[seq[Receipt], string] =
## Fully decode the SSZ encoded receipts and validate it against the header's
## receipts root.
let receipts = ?decodeSsz(bytes, PortalReceipts)
?validateReceipts(receipts, receiptsRoot)
seq[Receipt].fromPortalReceipts(receipts)

View File

@ -175,24 +175,6 @@ func verifyAccumulatorProof*(
else: else:
err("Cannot verify post merge header with accumulator proof") err("Cannot verify post merge header with accumulator proof")
func verifyHeader*(
a: FinishedHistoricalHashesAccumulator, header: Header, proof: BlockHeaderProof
): Result[void, string] =
case proof.proofType
of BlockHeaderProofType.historicalHashesAccumulatorProof:
a.verifyAccumulatorProof(header, proof.historicalHashesAccumulatorProof)
of BlockHeaderProofType.none:
if header.isPreMerge():
err("Pre merge header requires HistoricalHashesAccumulatorProof")
else:
# TODO:
# Currently there is no proof solution for verifying headers post-merge.
# Skipping canonical verification will allow for nodes to push block data
# that is not part of the canonical chain.
# For now we accept this flaw as the focus lies on testing data
# availability up to the head of the chain.
ok()
func buildProof*( func buildProof*(
header: Header, epochRecord: EpochRecord | EpochRecordCached header: Header, epochRecord: EpochRecord | EpochRecordCached
): Result[HistoricalHashesAccumulatorProof, string] = ): Result[HistoricalHashesAccumulatorProof, string] =

View File

@ -13,7 +13,7 @@ import
eth/common/[headers_rlp, blocks_rlp, receipts_rlp], eth/common/[headers_rlp, blocks_rlp, receipts_rlp],
json_rpc/rpcclient, json_rpc/rpcclient,
../common/common_types, ../common/common_types,
../network/history/[history_content, history_network], ../network/history/[history_content, history_type_conversions, history_validation],
./rpc_calls/[rpc_discovery_calls, rpc_portal_calls, rpc_portal_debug_calls] ./rpc_calls/[rpc_discovery_calls, rpc_portal_calls, rpc_portal_debug_calls]
export rpcclient, rpc_discovery_calls, rpc_portal_calls, rpc_portal_debug_calls, results export rpcclient, rpc_discovery_calls, rpc_portal_calls, rpc_portal_debug_calls, results
@ -110,7 +110,7 @@ proc historyGetBlockHeader*(
headerBytes = headerWithProof.header.asSeq() headerBytes = headerWithProof.header.asSeq()
if validateContent: if validateContent:
validateBlockHeaderBytes(headerBytes, blockHash).valueOrErr(ContentValidationFailed) validateHeaderBytes(headerBytes, blockHash).valueOrErr(ContentValidationFailed)
else: else:
decodeRlp(headerBytes, Header).valueOrErr(InvalidContentValue) decodeRlp(headerBytes, Header).valueOrErr(InvalidContentValue)
@ -132,7 +132,7 @@ proc historyGetBlockBody*(
ContentValidationFailed ContentValidationFailed
) )
else: else:
decodeBlockBodyBytes(content.toBytes()).valueOrErr(InvalidContentValue) fromPortalBlockBodyBytes(content.toBytes()).valueOrErr(InvalidContentValue)
proc historyGetReceipts*( proc historyGetReceipts*(
client: PortalRpcClient, blockHash: Hash32, validateContent = true client: PortalRpcClient, blockHash: Hash32, validateContent = true

View File

@ -15,8 +15,7 @@ import
eth/common/headers_rlp, eth/common/headers_rlp,
../../network_metadata, ../../network_metadata,
../../eth_data/[history_data_json_store, history_data_ssz_e2s], ../../eth_data/[history_data_json_store, history_data_ssz_e2s],
../../network/history/ ../../network/history/[history_content, history_type_conversions, history_validation],
[history_content, history_network, validation/historical_hashes_accumulator],
../../eth_data/yaml_utils, ../../eth_data/yaml_utils,
./test_history_util ./test_history_util
@ -73,7 +72,7 @@ suite "History Content Values":
check res.isOk() check res.isOk()
let header = res.get() let header = res.get()
check accumulator.verifyHeader(header, blockHeaderWithProof.proof).isOk() check accumulator.verifyBlockHeaderProof(header, blockHeaderWithProof.proof).isOk()
# Encode content # Encode content
check: check:
@ -108,7 +107,9 @@ suite "History Content Values":
check res.isOk() check res.isOk()
let header = res.get() let header = res.get()
check accumulator.verifyHeader(header, blockHeaderWithProof.proof).isOk() check accumulator
.verifyBlockHeaderProof(header, blockHeaderWithProof.proof)
.isOk()
# Encode content # Encode content
check: check:
@ -172,7 +173,7 @@ suite "History Content Values":
check contentKey.isOk() check contentKey.isOk()
# Decode (SSZ + RLP decode step) and validate block body # Decode (SSZ + RLP decode step) and validate block body
let contentValue = decodeBlockBodyBytes(contentValueEncoded) let contentValue = fromPortalBlockBodyBytes(contentValueEncoded)
check contentValue.isOk() check contentValue.isOk()
# Encode content and content key # Encode content and content key

View File

@ -17,7 +17,7 @@ import
eth/common/headers_rlp, eth/common/headers_rlp,
../../common/common_types, ../../common/common_types,
../../eth_data/history_data_json_store, ../../eth_data/history_data_json_store,
../../network/history/history_network ../../network/history/[history_type_conversions, history_validation]
const const
dataFile = "./fluffy/tests/blocks/mainnet_blocks_selected.json" dataFile = "./fluffy/tests/blocks/mainnet_blocks_selected.json"
@ -51,12 +51,12 @@ suite "History Content Values Validation":
) )
test "Valid Header": test "Valid Header":
check validateBlockHeaderBytes(blockHeaderBytes, blockHash).isOk() check validateHeaderBytes(blockHeaderBytes, blockHash).isOk()
test "Malformed Header": test "Malformed Header":
let malformedBytes = blockHeaderBytes[10 .. blockHeaderBytes.high] let malformedBytes = blockHeaderBytes[10 .. blockHeaderBytes.high]
check validateBlockHeaderBytes(malformedBytes, blockHash).isErr() check validateHeaderBytes(malformedBytes, blockHash).isErr()
test "Invalid Header - Different gasUsed": test "Invalid Header - Different gasUsed":
var modifiedHeader = blockHeader var modifiedHeader = blockHeader
@ -65,7 +65,7 @@ suite "History Content Values Validation":
let modifiedHeaderBytes = rlp.encode(modifiedHeader) let modifiedHeaderBytes = rlp.encode(modifiedHeader)
check validateBlockHeaderBytes(modifiedHeaderBytes, blockHash).isErr() check validateHeaderBytes(modifiedHeaderBytes, blockHash).isErr()
test "Valid Block Body": test "Valid Block Body":
check validateBlockBodyBytes(blockBodyBytes, blockHeader).isOk() check validateBlockBodyBytes(blockBodyBytes, blockHeader).isOk()

View File

@ -19,7 +19,7 @@ import
eth/p2p/discoveryv5/protocol as discv5_protocol, eth/p2p/discoveryv5/protocol as discv5_protocol,
../../network/wire/[portal_protocol, portal_stream, portal_protocol_config], ../../network/wire/[portal_protocol, portal_stream, portal_protocol_config],
../../network/history/ ../../network/history/
[history_network, history_content, validation/historical_hashes_accumulator], [history_network, history_content, history_type_conversions, history_validation],
../../database/content_db, ../../database/content_db,
../../rpc/[portal_rpc_client, rpc_portal_history_api], ../../rpc/[portal_rpc_client, rpc_portal_history_api],
../test_helpers ../test_helpers

View File

@ -16,7 +16,7 @@ import
eth/p2p/discoveryv5/routing_table, eth/p2p/discoveryv5/routing_table,
../../network/wire/[portal_protocol, portal_stream, portal_protocol_config], ../../network/wire/[portal_protocol, portal_stream, portal_protocol_config],
../../../nimbus/common/chain_config, ../../../nimbus/common/chain_config,
../../network/history/[history_content, history_network], ../../network/history/[history_content, history_network, history_validation],
../../network/state/[state_content, state_utils, state_network], ../../network/state/[state_content, state_utils, state_network],
../../eth_data/yaml_utils, ../../eth_data/yaml_utils,
../../database/content_db, ../../database/content_db,

View File

@ -20,7 +20,7 @@ import
../../../nimbus/beacon/web3_eth_conv, ../../../nimbus/beacon/web3_eth_conv,
../../../hive_integration/nodocker/engine/engine_client, ../../../hive_integration/nodocker/engine/engine_client,
../../rpc/portal_rpc_client, ../../rpc/portal_rpc_client,
../../network/history/[history_content, history_network], ../../network/history/[history_content, history_type_conversions, history_validation],
../../network_metadata, ../../network_metadata,
../../eth_data/[era1, history_data_ssz_e2s, history_data_seeding], ../../eth_data/[era1, history_data_ssz_e2s, history_data_seeding],
../../database/era1_db, ../../database/era1_db,
@ -239,7 +239,7 @@ proc runLatestLoop(
let hash = blockObject.hash let hash = blockObject.hash
if validate: if validate:
if validateBlockHeaderBytes(headerWithProof.header.asSeq(), hash).isErr(): if validateHeaderBytes(headerWithProof.header.asSeq(), hash).isErr():
error "Block header is invalid" error "Block header is invalid"
continue continue
if validateBlockBody(body, ethBlock.header).isErr(): if validateBlockBody(body, ethBlock.header).isErr():