Rename to EpochRecord and other accumulator spec changes (#2473)

- EpochAccumulator got renamed to EpochRecord
- MasterAccumulator is not HistoricalHashesAccumulator
- The List size for the accumulator got a different maximum which
also result in a different encoding and HTR
This commit is contained in:
Kim De Mey 2024-07-11 17:42:45 +02:00 committed by GitHub
parent a6764670f0
commit d996e60347
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
23 changed files with 214 additions and 217 deletions

View File

@ -62,9 +62,9 @@ proc getBlockTuple*(db: Era1DB, blockNumber: uint64): Result[BlockTuple, string]
proc getAccumulator*(
db: Era1DB, blockNumber: uint64
): Result[EpochAccumulatorCached, string] =
## Get the Epoch Accumulator that the block with `blockNumber` is part of.
# TODO: Probably want this `EpochAccumulatorCached` also actually cached in
): Result[EpochRecordCached, string] =
## Get the Epoch Record that the block with `blockNumber` is part of.
# TODO: Probably want this `EpochRecordCached` also actually cached in
# the Era1File or EraDB object.
let f = ?db.getEra1File(blockNumber.era)

View File

@ -95,18 +95,18 @@ the assigned `--data-dir`.
3. Build the master accumulator and the epoch accumulators:
```bash
./build/eth_data_exporter history exportAccumulatorData --writeEpochAccumulators --data-dir:"./user_data_dir/"
./build/eth_data_exporter history exportAccumulatorData --write-epoch-records --data-dir:"./user_data_dir/"
```
#### Step 2: Seed the epoch accumulators into the Portal network
Run Fluffy and trigger the propagation of data with the
`portal_history_propagateEpochAccumulators` JSON-RPC API call:
`portal_history_propagateEpochRecords` JSON-RPC API call:
```bash
./build/fluffy --rpc
# From another terminal
curl -s -X POST -H 'Content-Type: application/json' -d '{"jsonrpc":"2.0","id":"1","method":"portal_history_propagateEpochAccumulators","params":["./user_data_dir/"]}' http://localhost:8545 | jq
curl -s -X POST -H 'Content-Type: application/json' -d '{"jsonrpc":"2.0","id":"1","method":"portal_history_propagateEpochRecords","params":["./user_data_dir/"]}' http://localhost:8545 | jq
```

View File

@ -438,7 +438,7 @@ proc getAccumulatorRoot*(f: Era1File): Result[Digest, string] =
ok(Digest(data: array[32, byte].initCopyFrom(bytes)))
proc buildAccumulator*(f: Era1File): Result[EpochAccumulatorCached, string] =
proc buildAccumulator*(f: Era1File): Result[EpochRecordCached, string] =
let
startNumber = f.blockIdx.startNumber
endNumber = f.blockIdx.endNumber()
@ -453,7 +453,7 @@ proc buildAccumulator*(f: Era1File): Result[EpochAccumulatorCached, string] =
HeaderRecord(blockHash: blockHeader.blockHash(), totalDifficulty: totalDifficulty)
)
ok(EpochAccumulatorCached.init(headerRecords))
ok(EpochRecordCached.init(headerRecords))
proc verify*(f: Era1File): Result[Digest, string] =
let
@ -483,7 +483,7 @@ proc verify*(f: Era1File): Result[Digest, string] =
)
let expectedRoot = ?f.getAccumulatorRoot()
let accumulatorRoot = getEpochAccumulatorRoot(headerRecords)
let accumulatorRoot = getEpochRecordRoot(headerRecords)
if accumulatorRoot != expectedRoot:
err("Invalid accumulator root")

View File

@ -35,36 +35,35 @@ proc historyStore*(
ok()
proc propagateEpochAccumulator*(
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 epochAccumulatorRes = readEpochAccumulator(file)
if epochAccumulatorRes.isErr():
return err(epochAccumulatorRes.error)
let epochRecordRes = readEpochRecord(file)
if epochRecordRes.isErr():
return err(epochRecordRes.error)
else:
let
accumulator = epochAccumulatorRes.get()
rootHash = accumulator.hash_tree_root()
epochRecord = epochRecordRes.get()
rootHash = epochRecord.hash_tree_root()
key = ContentKey(
contentType: epochAccumulator,
epochAccumulatorKey: EpochAccumulatorKey(epochHash: rootHash),
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.
encodedAccumulator = SSZ.encode(accumulator)
info "Gossiping epoch accumulator", rootHash, contentKey = encKey
encodedEpochRecord = SSZ.encode(epochRecord)
info "Gossiping epoch record", rootHash, contentKey = encKey
p.storeContent(encKey, history_content.toContentId(encKey), encodedAccumulator)
p.storeContent(encKey, history_content.toContentId(encKey), encodedEpochRecord)
discard await p.neighborhoodGossip(
Opt.none(NodeId), ContentKeysList(@[encKey]), @[encodedAccumulator]
Opt.none(NodeId), ContentKeysList(@[encKey]), @[encodedEpochRecord]
)
return ok()
proc propagateEpochAccumulators*(
proc propagateEpochRecords*(
p: PortalProtocol, path: string
): Future[Result[void, string]] {.async.} =
## Propagate all epoch accumulators created when building the accumulator
@ -73,11 +72,11 @@ proc propagateEpochAccumulators*(
for i in 0 ..< preMergeEpochs:
let file =
try:
path / &"mainnet-epoch-accumulator-{i.uint64:05}.ssz"
path / &"mainnet-epoch-record-{i.uint64:05}.ssz"
except ValueError as e:
raiseAssert e.msg
let res = await p.propagateEpochAccumulator(file)
let res = await p.propagateEpochRecord(file)
if res.isErr():
return err(res.error)
@ -164,7 +163,7 @@ proc historyPropagateBlock*(
return err(blockDataTable.error)
proc historyPropagateHeadersWithProof*(
p: PortalProtocol, epochHeadersFile: string, epochAccumulatorFile: string
p: PortalProtocol, epochHeadersFile: string, epochRecordFile: string
): Future[Result[void, string]] {.async.} =
let res = readBlockHeaders(epochHeadersFile)
if res.isErr():
@ -172,14 +171,14 @@ proc historyPropagateHeadersWithProof*(
let blockHeaders = res.get()
let epochAccumulatorRes = readEpochAccumulatorCached(epochAccumulatorFile)
if epochAccumulatorRes.isErr():
let epochRecordRes = readEpochRecordCached(epochRecordFile)
if epochRecordRes.isErr():
return err(res.error)
let epochAccumulator = epochAccumulatorRes.get()
let epochRecord = epochRecordRes.get()
for header in blockHeaders:
if header.isPreMerge():
let headerWithProof = buildHeaderWithProof(header, epochAccumulator)
let headerWithProof = buildHeaderWithProof(header, epochRecord)
if headerWithProof.isErr:
return err(headerWithProof.error)
@ -210,14 +209,14 @@ proc historyPropagateHeadersWithProof*(
dataDir / &"mainnet-headers-epoch-{i.uint64:05}.e2s"
except ValueError as e:
raiseAssert e.msg
epochAccumulatorFile =
epochRecordFile =
try:
dataDir / &"mainnet-epoch-accumulator-{i.uint64:05}.ssz"
dataDir / &"mainnet-epoch-record-{i.uint64:05}.ssz"
except ValueError as e:
raiseAssert e.msg
let res =
await p.historyPropagateHeadersWithProof(epochHeadersfile, epochAccumulatorFile)
await p.historyPropagateHeadersWithProof(epochHeadersfile, epochRecordFile)
if res.isOk():
info "Finished gossiping 1 epoch of headers with proof", i
else:
@ -268,7 +267,7 @@ proc historyPropagateHeaders*(
# have great support for usage in iterators.
iterator headersWithProof*(
f: Era1File, epochAccumulator: EpochAccumulatorCached
f: Era1File, epochRecord: EpochRecordCached
): (ByteList, seq[byte]) =
for blockHeader in f.era1BlockHeaders:
doAssert blockHeader.isPreMerge()
@ -279,7 +278,7 @@ iterator headersWithProof*(
blockHeaderKey: BlockKey(blockHash: blockHeader.blockHash()),
).encode()
headerWithProof = buildHeaderWithProof(blockHeader, epochAccumulator).valueOr:
headerWithProof = buildHeaderWithProof(blockHeader, epochRecord).valueOr:
raiseAssert "Failed to build header with proof: " & $blockHeader.number
contentValue = SSZ.encode(headerWithProof)
@ -315,10 +314,7 @@ iterator blockContent*(f: Era1File): (ByteList, seq[byte]) =
##
proc historyGossipHeadersWithProof*(
p: PortalProtocol,
era1File: string,
epochAccumulatorFile: Opt[string],
verifyEra = false,
p: PortalProtocol, era1File: string, epochRecordFile: Opt[string], verifyEra = false
): Future[Result[void, string]] {.async.} =
let f = ?Era1File.open(era1File)
@ -328,13 +324,13 @@ proc historyGossipHeadersWithProof*(
# Note: building the accumulator takes about 150ms vs 10ms for reading it,
# so it is probably not really worth using the read version considering the
# UX hassle it adds to provide the accumulator ssz files.
let epochAccumulator =
if epochAccumulatorFile.isNone:
let epochRecord =
if epochRecordFile.isNone:
?f.buildAccumulator()
else:
?readEpochAccumulatorCached(epochAccumulatorFile.get())
?readEpochRecordCached(epochRecordFile.get())
for (contentKey, contentValue) in f.headersWithProof(epochAccumulator):
for (contentKey, contentValue) in f.headersWithProof(epochRecord):
let peers = await p.neighborhoodGossip(
Opt.none(NodeId), ContentKeysList(@[contentKey]), @[contentValue]
)

View File

@ -27,19 +27,19 @@ proc readAccumulator*(file: string): Result[FinishedAccumulator, string] =
except SerializationError as e:
err("Failed decoding accumulator: " & e.msg)
proc readEpochAccumulator*(file: string): Result[EpochAccumulator, string] =
proc readEpochRecord*(file: string): Result[EpochRecord, string] =
let encodedAccumulator = ?readAllFile(file).mapErr(toString)
try:
ok(SSZ.decode(encodedAccumulator, EpochAccumulator))
ok(SSZ.decode(encodedAccumulator, EpochRecord))
except SerializationError as e:
err("Decoding epoch accumulator failed: " & e.msg)
proc readEpochAccumulatorCached*(file: string): Result[EpochAccumulatorCached, string] =
proc readEpochRecordCached*(file: string): Result[EpochRecordCached, string] =
let encodedAccumulator = ?readAllFile(file).mapErr(toString)
try:
ok(SSZ.decode(encodedAccumulator, EpochAccumulatorCached))
ok(SSZ.decode(encodedAccumulator, EpochRecordCached))
except SerializationError as e:
err("Decoding epoch accumulator failed: " & e.msg)

View File

@ -22,14 +22,17 @@ export ssz_serialization, merkleization, proofs, eth_types_rlp
# But with the adjustment to finish the accumulator at merge point.
const
epochSize* = 8192 # blocks
EPOCH_SIZE* = 8192 # block roots per epoch record
MAX_HISTORICAL_EPOCHS = 2048'u64 # Should be sufficient for all networks as for
# mainnet this is not even reached: ceil(mergeBlockNumber / EPOCH_SIZE) = 1897
# Allow this to be adjusted at compile time for testing. If more constants
# need to be adjusted we can add some presets file.
mergeBlockNumber* {.intdefine.}: uint64 = 15537394
# Note: This is like a ceil(mergeBlockNumber / epochSize)
# Could use ceilDiv(mergeBlockNumber, epochSize) in future versions
preMergeEpochs* = (mergeBlockNumber + epochSize - 1) div epochSize
# Note: This is like a ceil(mergeBlockNumber / EPOCH_SIZE)
# Could use ceilDiv(mergeBlockNumber, EPOCH_SIZE) in future versions
preMergeEpochs* = (mergeBlockNumber + EPOCH_SIZE - 1) div EPOCH_SIZE
# TODO:
# Currently disabled, because issue when testing with other
@ -39,7 +42,6 @@ const
# of `mergeBlockNumber`, but:
# - Still need to store the actual `mergeBlockNumber` and run-time somewhere
# as it allows for each pre vs post merge block header checking.
# - Can't limit `historicalEpochs` SSZ list at `preMergeEpochs` value.
# - Should probably be stated in the portal network specs.
# TERMINAL_TOTAL_DIFFICULTY = u256"58750000000000000000000"
@ -48,22 +50,25 @@ type
blockHash*: BlockHash
totalDifficulty*: UInt256
EpochAccumulator* = List[HeaderRecord, epochSize]
EpochRecord* = List[HeaderRecord, EPOCH_SIZE]
# In the core code of Fluffy the `EpochAccumulator` type is solely used, as
# In the core code of Fluffy the `EpochRecord` type is solely used, as
# `hash_tree_root` is done either once or never on this object after
# serialization.
# However for the generation of the proofs for all the headers in an epoch, it
# needs to be run many times and the cached version of the SSZ list is
# obviously much faster, so this second type is added for this usage.
EpochAccumulatorCached* = HashList[HeaderRecord, epochSize]
EpochRecordCached* = HashList[HeaderRecord, EPOCH_SIZE]
# HistoricalHashesAccumulator
Accumulator* = object
historicalEpochs*: List[Bytes32, int(preMergeEpochs)]
currentEpoch*: EpochAccumulator
historicalEpochs*: List[Bytes32, int(MAX_HISTORICAL_EPOCHS)]
currentEpoch*: EpochRecord
# HistoricalHashesAccumulator in its final state
FinishedAccumulator* = object
historicalEpochs*: List[Bytes32, int(preMergeEpochs)]
historicalEpochs*: List[Bytes32, int(MAX_HISTORICAL_EPOCHS)]
currentEpoch*: EpochRecord
BlockEpochData* = object
epochHash*: Bytes32
@ -71,14 +76,14 @@ type
func init*(T: type Accumulator): T =
Accumulator(
historicalEpochs: List[Bytes32, int(preMergeEpochs)].init(@[]),
currentEpoch: EpochAccumulator.init(@[]),
historicalEpochs: List[Bytes32, int(MAX_HISTORICAL_EPOCHS)].init(@[]),
currentEpoch: EpochRecord.init(@[]),
)
func getEpochAccumulatorRoot*(headerRecords: openArray[HeaderRecord]): Digest =
let epochAccumulator = EpochAccumulator.init(@headerRecords)
func getEpochRecordRoot*(headerRecords: openArray[HeaderRecord]): Digest =
let epochRecord = EpochRecord.init(@headerRecords)
hash_tree_root(epochAccumulator)
hash_tree_root(epochRecord)
func updateAccumulator*(a: var Accumulator, header: BlockHeader) =
doAssert(
@ -95,11 +100,11 @@ func updateAccumulator*(a: var Accumulator, header: BlockHeader) =
# finish an epoch. However, if we were to move this after adding the
# `HeaderRecord`, there would be no way to get the current total difficulty,
# unless another field is introduced in the `Accumulator` object.
if a.currentEpoch.len() == epochSize:
if a.currentEpoch.len() == EPOCH_SIZE:
let epochHash = hash_tree_root(a.currentEpoch)
doAssert(a.historicalEpochs.add(epochHash.data))
a.currentEpoch = EpochAccumulator.init(@[])
a.currentEpoch = EpochRecord.init(@[])
let headerRecord = HeaderRecord(
blockHash: header.blockHash(),
@ -122,7 +127,7 @@ func finishAccumulator*(a: var Accumulator): FinishedAccumulator =
## against the Accumulator and the header proofs.
func getEpochIndex*(blockNumber: uint64): uint64 =
blockNumber div epochSize
blockNumber div EPOCH_SIZE
func getEpochIndex*(header: BlockHeader): uint64 =
## Get the index for the historical epochs
@ -130,7 +135,7 @@ func getEpochIndex*(header: BlockHeader): uint64 =
func getHeaderRecordIndex*(blockNumber: uint64, epochIndex: uint64): uint64 =
## Get the relative header index for the epoch accumulator
uint64(blockNumber - epochIndex * epochSize)
uint64(blockNumber - epochIndex * EPOCH_SIZE)
func getHeaderRecordIndex*(header: BlockHeader, epochIndex: uint64): uint64 =
## Get the relative header index for the epoch accumulator
@ -147,15 +152,15 @@ func verifyProof(
): bool =
let
epochIndex = getEpochIndex(header)
epochAccumulatorHash = Digest(data: a.historicalEpochs[epochIndex])
epochRecordHash = Digest(data: a.historicalEpochs[epochIndex])
leave = hash_tree_root(header.blockHash())
headerRecordIndex = getHeaderRecordIndex(header, epochIndex)
# TODO: Implement more generalized `get_generalized_index`
gIndex = GeneralizedIndex(epochSize * 2 * 2 + (headerRecordIndex * 2))
gIndex = GeneralizedIndex(EPOCH_SIZE * 2 * 2 + (headerRecordIndex * 2))
verify_merkle_multiproof(@[leave], proof, @[gIndex], epochAccumulatorHash)
verify_merkle_multiproof(@[leave], proof, @[gIndex], epochRecordHash)
func verifyAccumulatorProof*(
a: FinishedAccumulator, header: BlockHeader, proof: AccumulatorProof
@ -189,7 +194,7 @@ func verifyHeader*(
ok()
func buildProof*(
header: BlockHeader, epochAccumulator: EpochAccumulator | EpochAccumulatorCached
header: BlockHeader, epochRecord: EpochRecord | EpochRecordCached
): Result[AccumulatorProof, string] =
doAssert(header.isPreMerge(), "Must be pre merge header")
@ -198,17 +203,17 @@ func buildProof*(
headerRecordIndex = getHeaderRecordIndex(header, epochIndex)
# TODO: Implement more generalized `get_generalized_index`
gIndex = GeneralizedIndex(epochSize * 2 * 2 + (headerRecordIndex * 2))
gIndex = GeneralizedIndex(EPOCH_SIZE * 2 * 2 + (headerRecordIndex * 2))
var proof: AccumulatorProof
?epochAccumulator.build_proof(gIndex, proof)
?epochRecord.build_proof(gIndex, proof)
ok(proof)
func buildHeaderWithProof*(
header: BlockHeader, epochAccumulator: EpochAccumulator | EpochAccumulatorCached
header: BlockHeader, epochRecord: EpochRecord | EpochRecordCached
): Result[BlockHeaderWithProof, string] =
let proof = ?buildProof(header, epochAccumulator)
let proof = ?buildProof(header, epochRecord)
ok(
BlockHeaderWithProof(

View File

@ -31,12 +31,12 @@ type
blockHeader = 0x00
blockBody = 0x01
receipts = 0x02
epochAccumulator = 0x03
epochRecord = 0x03
BlockKey* = object
blockHash*: BlockHash
EpochAccumulatorKey* = object
EpochRecordKey* = object
epochHash*: Digest
ContentKey* = object
@ -47,8 +47,8 @@ type
blockBodyKey*: BlockKey
of receipts:
receiptsKey*: BlockKey
of epochAccumulator:
epochAccumulatorKey*: EpochAccumulatorKey
of epochRecord:
epochRecordKey*: EpochRecordKey
func init*(T: type ContentKey, contentType: ContentType, hash: BlockHash | Digest): T =
case contentType
@ -58,10 +58,9 @@ func init*(T: type ContentKey, contentType: ContentType, hash: BlockHash | Diges
ContentKey(contentType: contentType, blockBodyKey: BlockKey(blockHash: hash))
of receipts:
ContentKey(contentType: contentType, receiptsKey: BlockKey(blockHash: hash))
of epochAccumulator:
of epochRecord:
ContentKey(
contentType: contentType,
epochAccumulatorKey: EpochAccumulatorKey(epochHash: hash),
contentType: contentType, epochRecordKey: EpochRecordKey(epochHash: hash)
)
func encode*(contentKey: ContentKey): ByteList =
@ -97,8 +96,8 @@ func `$`*(x: ContentKey): string =
res.add($x.blockBodyKey)
of receipts:
res.add($x.receiptsKey)
of epochAccumulator:
let key = x.epochAccumulatorKey
of epochRecord:
let key = x.epochRecordKey
res.add("epochHash: " & $key.epochHash)
res.add(")")

View File

@ -394,7 +394,7 @@ proc get(db: ContentDB, T: type seq[Receipt], contentId: ContentId): Opt[T] =
else:
Opt.none(T)
proc get(db: ContentDB, T: type EpochAccumulator, contentId: ContentId): Opt[T] =
proc get(db: ContentDB, T: type EpochRecord, contentId: ContentId): Opt[T] =
db.getSszDecoded(contentId, T)
proc getContentFromDb(n: HistoryNetwork, T: type, contentId: ContentId): Opt[T] =
@ -567,18 +567,18 @@ proc getReceipts*(
return Opt.some(receipts)
proc getEpochAccumulator(
proc getEpochRecord(
n: HistoryNetwork, epochHash: Digest
): Future[Opt[EpochAccumulator]] {.async: (raises: [CancelledError]).} =
): Future[Opt[EpochRecord]] {.async: (raises: [CancelledError]).} =
let
contentKey = ContentKey.init(epochAccumulator, epochHash).encode()
contentKey = ContentKey.init(epochRecord, epochHash).encode()
contentId = contentKey.toContentId()
logScope:
epochHash
contentKey
let accumulatorFromDb = n.getContentFromDb(EpochAccumulator, contentId)
let accumulatorFromDb = n.getContentFromDb(EpochRecord, contentId)
if accumulatorFromDb.isSome():
info "Fetched epoch accumulator from database"
return accumulatorFromDb
@ -587,15 +587,15 @@ proc getEpochAccumulator(
let
accumulatorContent = (await n.portalProtocol.contentLookup(contentKey, contentId)).valueOr:
warn "Failed fetching epoch accumulator from the network"
return Opt.none(EpochAccumulator)
return Opt.none(EpochRecord)
epochAccumulator =
epochRecord =
try:
SSZ.decode(accumulatorContent.content, EpochAccumulator)
SSZ.decode(accumulatorContent.content, EpochRecord)
except SerializationError:
continue
let hash = hash_tree_root(epochAccumulator)
let hash = hash_tree_root(epochRecord)
if hash == epochHash:
info "Fetched epoch accumulator from the network"
n.portalProtocol.storeContent(contentKey, contentId, accumulatorContent.content)
@ -604,11 +604,11 @@ proc getEpochAccumulator(
accumulatorContent.content,
)
return Opt.some(epochAccumulator)
return Opt.some(epochRecord)
else:
warn "Validation of epoch accumulator failed", resultedEpochHash = hash
return Opt.none(EpochAccumulator)
return Opt.none(EpochRecord)
proc getBlockHashByNumber*(
n: HistoryNetwork, bn: UInt256
@ -617,7 +617,7 @@ proc getBlockHashByNumber*(
epochData = n.accumulator.getBlockEpochDataForBlockNumber(bn).valueOr:
return err(error)
digest = Digest(data: epochData.epochHash)
epoch = (await n.getEpochAccumulator(digest)).valueOr:
epoch = (await n.getEpochRecord(digest)).valueOr:
return err("Cannot retrieve epoch accumulator for given block number")
ok(epoch[epochData.blockRelativeIndex].blockHash)
@ -677,22 +677,22 @@ proc validateContent(
return false
else:
return true
of epochAccumulator:
of epochRecord:
# Check first if epochHash is part of master accumulator
let epochHash = key.epochAccumulatorKey.epochHash
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 epochAccumulator =
let epochRecord =
try:
SSZ.decode(content, EpochAccumulator)
SSZ.decode(content, EpochRecord)
except SerializationError:
warn "Failed decoding epoch accumulator"
return false
# Next check the hash tree root, as this is probably more expensive
let hash = hash_tree_root(epochAccumulator)
let hash = hash_tree_root(epochRecord)
if hash != epochHash:
warn "Epoch accumulator has invalid root hash"
return false

View File

@ -47,13 +47,14 @@ const
angelfoodBootstrapNodes* =
loadCompileTimeBootstrapNodes(portalConfigDir / "bootstrap_nodes_angelfood.txt")
finishedAccumulatorSSZ* = slurp(portalConfigDir / "finished_accumulator.ssz")
historicalHashesAccumulatorSSZ* =
slurp(portalConfigDir / "historical_hashes_accumulator.ssz")
historicalRootsSSZ* = slurp(portalConfigDir / "historical_roots.ssz")
func loadAccumulator*(): FinishedAccumulator =
try:
SSZ.decode(finishedAccumulatorSSZ, FinishedAccumulator)
SSZ.decode(historicalHashesAccumulatorSSZ, FinishedAccumulator)
except SerializationError as err:
raiseAssert "Invalid baked-in accumulator: " & err.msg

View File

@ -25,8 +25,8 @@ createRpcSigsFromNim(RpcClient):
proc portal_history_propagate(dataFile: string): bool
proc portal_history_propagateHeaders(dataFile: string): bool
proc portal_history_propagateBlock(dataFile: string, blockHash: string): bool
proc portal_history_propagateEpochAccumulator(dataFile: string): bool
proc portal_history_propagateEpochAccumulators(path: string): bool
proc portal_history_propagateEpochRecord(dataFile: string): bool
proc portal_history_propagateEpochRecords(path: string): bool
proc portal_history_storeContentInNodeRange(
dbPath: string, max: uint32, starting: uint32
): bool

View File

@ -23,9 +23,9 @@ proc installPortalDebugApiHandlers*(
) =
## Portal debug API calls related to storage and seeding from Era1 files.
rpcServer.rpc("portal_" & network & "GossipHeaders") do(
era1File: string, epochAccumulatorFile: Opt[string]
era1File: string, epochRecordFile: Opt[string]
) -> bool:
let res = await p.historyGossipHeadersWithProof(era1File, epochAccumulatorFile)
let res = await p.historyGossipHeadersWithProof(era1File, epochRecordFile)
if res.isOk():
return true
else:
@ -62,10 +62,10 @@ proc installPortalDebugApiHandlers*(
raise newException(ValueError, $res.error)
rpcServer.rpc("portal_" & network & "_propagateHeaders") do(
epochHeadersFile: string, epochAccumulatorFile: string
epochHeadersFile: string, epochRecordFile: string
) -> bool:
let res =
await p.historyPropagateHeadersWithProof(epochHeadersFile, epochAccumulatorFile)
await p.historyPropagateHeadersWithProof(epochHeadersFile, epochRecordFile)
if res.isOk():
return true
else:
@ -80,19 +80,17 @@ proc installPortalDebugApiHandlers*(
else:
raise newException(ValueError, $res.error)
rpcServer.rpc("portal_" & network & "_propagateEpochAccumulator") do(
rpcServer.rpc("portal_" & network & "_propagateEpochRecord") do(
dataFile: string
) -> bool:
let res = await p.propagateEpochAccumulator(dataFile)
let res = await p.propagateEpochRecord(dataFile)
if res.isOk():
return true
else:
raise newException(ValueError, $res.error)
rpcServer.rpc("portal_" & network & "_propagateEpochAccumulators") do(
path: string
) -> bool:
let res = await p.propagateEpochAccumulators(path)
rpcServer.rpc("portal_" & network & "_propagateEpochRecords") do(path: string) -> bool:
let res = await p.propagateEpochRecords(path)
if res.isOk():
return true
else:

View File

@ -248,14 +248,15 @@ procSuite "Portal testnet tests":
"./vendor/portal-spec-tests/tests/mainnet/history/headers/1000001-1000010.e2s"
accumulatorFile =
"./vendor/portal-spec-tests/tests/mainnet/history/accumulator/epoch-accumulator-00122.ssz"
# TODO: rename
blockDataFile = "./fluffy/tests/blocks/mainnet_blocks_1000001_1000010.json"
let
blockHeaders = readBlockHeaders(headerFile).valueOr:
raiseAssert "Invalid header file: " & headerFile
epochAccumulator = readEpochAccumulatorCached(accumulatorFile).valueOr:
epochRecord = readEpochRecordCached(accumulatorFile).valueOr:
raiseAssert "Invalid epoch accumulator file: " & accumulatorFile
blockHeadersWithProof = buildHeadersWithProof(blockHeaders, epochAccumulator).valueOr:
blockHeadersWithProof = buildHeadersWithProof(blockHeaders, epochRecord).valueOr:
raiseAssert "Could not build headers with proof"
blockData = readJsonType(blockDataFile, BlockDataTable).valueOr:
raiseAssert "Invalid block data file" & blockDataFile

View File

@ -28,15 +28,16 @@ suite "History Content Encodings":
"./vendor/portal-spec-tests/tests/mainnet/history/headers/1000001-1000010.e2s"
accumulatorFile =
"./vendor/portal-spec-tests/tests/mainnet/history/accumulator/epoch-accumulator-00122.ssz"
# TODO: rename
headersWithProofFile =
"./vendor/portal-spec-tests/tests/mainnet/history/headers_with_proof/1000001-1000010.json"
let
blockHeaders = readBlockHeaders(headerFile).valueOr:
raiseAssert "Invalid header file: " & headerFile
epochAccumulator = readEpochAccumulatorCached(accumulatorFile).valueOr:
epochRecord = readEpochRecordCached(accumulatorFile).valueOr:
raiseAssert "Invalid epoch accumulator file: " & accumulatorFile
blockHeadersWithProof = buildHeadersWithProof(blockHeaders, epochAccumulator).valueOr:
blockHeadersWithProof = buildHeadersWithProof(blockHeaders, epochRecord).valueOr:
raiseAssert "Could not build headers with proof"
accumulator = loadAccumulator()

View File

@ -136,8 +136,7 @@ suite "History ContentKey Encodings":
"9fb2175e76c6989e0fdac3ee10c40d2a81eb176af32e1c16193e3904fe56896e"
let contentKey = ContentKey(
contentType: epochAccumulator,
epochAccumulatorKey: EpochAccumulatorKey(epochHash: epochHash),
contentType: epochRecord, epochRecordKey: EpochRecordKey(epochHash: epochHash)
)
let encoded = encode(contentKey)
@ -148,7 +147,7 @@ suite "History ContentKey Encodings":
let contentKeyDecoded = decoded.get()
check:
contentKeyDecoded.contentType == contentKey.contentType
contentKeyDecoded.epochAccumulatorKey == contentKey.epochAccumulatorKey
contentKeyDecoded.epochRecordKey == contentKey.epochRecordKey
toContentId(contentKey) == parse(contentId, StUint[256], 10)
# In stint this does BE hex string

View File

@ -26,13 +26,13 @@ suite "Header Accumulator":
# Note: This test assumes at least 5 epochs
headersToTest = [
0,
epochSize - 1,
epochSize,
epochSize * 2 - 1,
epochSize * 2,
epochSize * 3 - 1,
epochSize * 3,
epochSize * 3 + 1,
EPOCH_SIZE - 1,
EPOCH_SIZE,
EPOCH_SIZE * 2 - 1,
EPOCH_SIZE * 2,
EPOCH_SIZE * 3 - 1,
EPOCH_SIZE * 3,
EPOCH_SIZE * 3 + 1,
int(amount) - 1,
]
@ -45,12 +45,12 @@ suite "Header Accumulator":
let accumulatorRes = buildAccumulatorData(headers)
check accumulatorRes.isOk()
let (accumulator, epochAccumulators) = accumulatorRes.get()
let (accumulator, epochRecords) = accumulatorRes.get()
block: # Test valid headers
for i in headersToTest:
let header = headers[i]
let proof = buildProof(header, epochAccumulators)
let proof = buildProof(header, epochRecords)
check:
proof.isOk()
verifyAccumulatorProof(accumulator, header, proof.get()).isOk()
@ -63,7 +63,7 @@ suite "Header Accumulator":
# Test altered block headers by altering the difficulty
for i in headersToTest:
let proof = buildProof(headers[i], epochAccumulators)
let proof = buildProof(headers[i], epochRecords)
check:
proof.isOk()
# Alter the block header so the proof no longer matches
@ -89,7 +89,7 @@ suite "Header Accumulator":
check accumulatorRes.isErr()
test "Header BlockNumber to EpochAccumulator Root":
test "Header BlockNumber to EpochRecord Root":
# Note: This test assumes at least 3 epochs
const amount = mergeBlockNumber
@ -108,7 +108,7 @@ suite "Header Accumulator":
# Valid response for block numbers in epoch 0
block:
for i in 0 ..< epochSize:
for i in 0 ..< EPOCH_SIZE:
let res = accumulator.getBlockEpochDataForBlockNumber(u256(i))
check:
res.isOk()
@ -116,7 +116,7 @@ suite "Header Accumulator":
# Valid response for block numbers in epoch 1
block:
for i in epochSize ..< (2 * epochSize):
for i in EPOCH_SIZE ..< (2 * EPOCH_SIZE):
let res = accumulator.getBlockEpochDataForBlockNumber(u256(i))
check:
res.isOk()
@ -124,7 +124,7 @@ suite "Header Accumulator":
# Valid response for block numbers in the incomplete (= last) epoch
block:
const startIndex = mergeBlockNumber - (mergeBlockNumber mod epochSize)
const startIndex = mergeBlockNumber - (mergeBlockNumber mod EPOCH_SIZE)
for i in startIndex ..< mergeBlockNumber:
let res = accumulator.getBlockEpochDataForBlockNumber(u256(i))
check:

View File

@ -64,50 +64,50 @@ func buildAccumulator*(headers: seq[BlockHeader]): Result[FinishedAccumulator, s
func buildAccumulatorData*(
headers: seq[BlockHeader]
): Result[(FinishedAccumulator, seq[EpochAccumulator]), string] =
): Result[(FinishedAccumulator, seq[EpochRecord]), string] =
var accumulator: Accumulator
var epochAccumulators: seq[EpochAccumulator]
var epochRecords: seq[EpochRecord]
for header in headers:
updateAccumulator(accumulator, header)
if accumulator.currentEpoch.len() == epochSize:
epochAccumulators.add(accumulator.currentEpoch)
if accumulator.currentEpoch.len() == EPOCH_SIZE:
epochRecords.add(accumulator.currentEpoch)
if header.number == mergeBlockNumber - 1:
epochAccumulators.add(accumulator.currentEpoch)
epochRecords.add(accumulator.currentEpoch)
return ok((finishAccumulator(accumulator), epochAccumulators))
return ok((finishAccumulator(accumulator), epochRecords))
err("Not enough headers provided to finish the accumulator")
func buildProof*(
header: BlockHeader, epochAccumulators: seq[EpochAccumulator]
header: BlockHeader, epochRecords: seq[EpochRecord]
): Result[AccumulatorProof, string] =
let epochIndex = getEpochIndex(header)
doAssert(epochIndex < uint64(epochAccumulators.len()))
let epochAccumulator = epochAccumulators[epochIndex]
doAssert(epochIndex < uint64(epochRecords.len()))
let epochRecord = epochRecords[epochIndex]
buildProof(header, epochAccumulator)
buildProof(header, epochRecord)
func buildHeaderWithProof*(
header: BlockHeader, epochAccumulators: seq[EpochAccumulator]
header: BlockHeader, epochRecords: seq[EpochRecord]
): Result[BlockHeaderWithProof, string] =
## Construct the accumulator proof for a specific header.
## Returns the block header with the proof
if header.isPreMerge():
let epochIndex = getEpochIndex(header)
doAssert(epochIndex < uint64(epochAccumulators.len()))
let epochAccumulator = epochAccumulators[epochIndex]
doAssert(epochIndex < uint64(epochRecords.len()))
let epochRecord = epochRecords[epochIndex]
buildHeaderWithProof(header, epochAccumulator)
buildHeaderWithProof(header, epochRecord)
else:
err("Cannot build accumulator proof for post merge header")
func buildHeadersWithProof*(
headers: seq[BlockHeader], epochAccumulators: seq[EpochAccumulator]
headers: seq[BlockHeader], epochRecords: seq[EpochRecord]
): Result[seq[BlockHeaderWithProof], string] =
var headersWithProof: seq[BlockHeaderWithProof]
for header in headers:
headersWithProof.add(?buildHeaderWithProof(header, epochAccumulators))
headersWithProof.add(?buildHeaderWithProof(header, epochRecords))
ok(headersWithProof)

View File

@ -85,13 +85,13 @@ procSuite "History Content Network":
headersToTest = [
0,
epochSize - 1,
epochSize,
epochSize * 2 - 1,
epochSize * 2,
epochSize * 3 - 1,
epochSize * 3,
epochSize * 3 + 1,
EPOCH_SIZE - 1,
EPOCH_SIZE,
EPOCH_SIZE * 2 - 1,
EPOCH_SIZE * 2,
EPOCH_SIZE * 3 - 1,
EPOCH_SIZE * 3,
EPOCH_SIZE * 3 + 1,
int(lastBlockNumber),
]
@ -100,7 +100,7 @@ procSuite "History Content Network":
check accumulatorRes.isOk()
let
(masterAccumulator, epochAccumulators) = accumulatorRes.get()
(masterAccumulator, epochRecords) = accumulatorRes.get()
historyNode1 = newHistoryNode(rng, 20302, masterAccumulator)
historyNode2 = newHistoryNode(rng, 20303, masterAccumulator)
@ -108,7 +108,7 @@ procSuite "History Content Network":
for i in headersToTest:
selectedHeaders.add(headers[i])
let headersWithProof = buildHeadersWithProof(selectedHeaders, epochAccumulators)
let headersWithProof = buildHeadersWithProof(selectedHeaders, epochRecords)
check headersWithProof.isOk()
@ -127,17 +127,17 @@ procSuite "History Content Network":
# Need to store the epoch accumulators to be able to do the block to hash
# mapping
for epochAccumulator in epochAccumulators:
for epochRecord in epochRecords:
let
rootHash = epochAccumulator.hash_tree_root()
rootHash = epochRecord.hash_tree_root()
contentKey = ContentKey(
contentType: ContentType.epochAccumulator,
epochAccumulatorKey: EpochAccumulatorKey(epochHash: rootHash),
contentType: ContentType.epochRecord,
epochRecordKey: EpochRecordKey(epochHash: rootHash),
)
encKey = encode(contentKey)
contentId = toContentId(contentKey)
historyNode2.portalProtocol().storeContent(
encKey, contentId, SSZ.encode(epochAccumulator)
encKey, contentId, SSZ.encode(epochRecord)
)
check:
@ -172,7 +172,7 @@ procSuite "History Content Network":
check accumulatorRes.isOk()
let
(masterAccumulator, epochAccumulators) = accumulatorRes.get()
(masterAccumulator, epochRecords) = accumulatorRes.get()
historyNode1 = newHistoryNode(rng, 20302, masterAccumulator)
historyNode2 = newHistoryNode(rng, 20303, masterAccumulator)
@ -191,7 +191,7 @@ procSuite "History Content Network":
getMaxOfferedContentKeys(uint32(len(PortalProtocolId)), maxContentKeySize)
let headersWithProof =
buildHeadersWithProof(headers[0 .. maxOfferedHistoryContent], epochAccumulators)
buildHeadersWithProof(headers[0 .. maxOfferedHistoryContent], epochRecords)
check headersWithProof.isOk()
# This is one header more than maxOfferedHistoryContent
@ -251,14 +251,14 @@ procSuite "History Content Network":
const
lastBlockNumber = int(mergeBlockNumber - 1)
headersToTest =
[0, 1, epochSize div 2, epochSize - 1, lastBlockNumber - 1, lastBlockNumber]
[0, 1, EPOCH_SIZE div 2, EPOCH_SIZE - 1, lastBlockNumber - 1, lastBlockNumber]
let headers = createEmptyHeaders(0, lastBlockNumber)
let accumulatorRes = buildAccumulatorData(headers)
check accumulatorRes.isOk()
let
(masterAccumulator, epochAccumulators) = accumulatorRes.get()
(masterAccumulator, epochRecords) = accumulatorRes.get()
historyNode1 = newHistoryNode(rng, 20302, masterAccumulator)
historyNode2 = newHistoryNode(rng, 20303, masterAccumulator)
@ -277,7 +277,7 @@ procSuite "History Content Network":
for i in headersToTest:
selectedHeaders.add(headers[i])
let headersWithProof = buildHeadersWithProof(selectedHeaders, epochAccumulators)
let headersWithProof = buildHeadersWithProof(selectedHeaders, epochRecords)
check headersWithProof.isOk()
let contentKVs = headersToContentKV(headersWithProof.get())

View File

@ -13,13 +13,13 @@ import
export results, accumulator, history_content
proc buildHeadersWithProof*(
blockHeaders: seq[BlockHeader], epochAccumulator: EpochAccumulatorCached
blockHeaders: seq[BlockHeader], epochRecord: EpochRecordCached
): Result[seq[(seq[byte], seq[byte])], string] =
var blockHeadersWithProof: seq[(seq[byte], seq[byte])]
for header in blockHeaders:
if header.isPreMerge():
let
content = ?buildHeaderWithProof(header, epochAccumulator)
content = ?buildHeaderWithProof(header, epochRecord)
contentKey = ContentKey(
contentType: blockHeader,
blockHeaderKey: BlockKey(blockHash: header.blockHash()),

View File

@ -6,7 +6,7 @@
# 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 `EpochAccumulator`s of the history network.
# Currently only supports checking `EpochRecord`s of the history network.
{.push raises: [].}
@ -42,24 +42,23 @@ proc checkAccumulators(client: RpcClient) {.async.} =
for i, hash in accumulator.historicalEpochs:
let root = Digest(data: hash)
let contentKey = ContentKey.init(epochAccumulator, root)
let contentKey = ContentKey.init(epochRecord, root)
try:
let contentInfo = await client.portal_historyRecursiveFindContent(
contentKey.encode.asSeq().toHex()
)
let res = decodeSsz(hexToSeqByte(contentInfo.content), EpochAccumulator)
let res = decodeSsz(hexToSeqByte(contentInfo.content), EpochRecord)
if res.isErr():
echo "[Invalid] EpochAccumulator number " & $i & ": " & $root & " error: " &
res.error
echo "[Invalid] EpochRecord number " & $i & ": " & $root & " error: " & res.error
else:
let epochAccumulator = res.get()
let resultingRoot = hash_tree_root(epochAccumulator)
let epochRecord = res.get()
let resultingRoot = hash_tree_root(epochRecord)
if resultingRoot == root:
echo "[Available] EpochAccumulator number " & $i & ": " & $root
echo "[Available] EpochRecord number " & $i & ": " & $root
else:
echo "[Invalid] EpochAccumulator number " & $i & ": " & $root &
echo "[Invalid] EpochRecord number " & $i & ": " & $root &
" error: Invalid root"
except RpcPostError as e:
# RpcPostError when for example timing out on the request. Could retry
@ -69,8 +68,7 @@ proc checkAccumulators(client: RpcClient) {.async.} =
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] EpochAccumulator number " & $i & ": " & $root & " error: " &
e.msg
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

View File

@ -227,7 +227,7 @@ proc cmdExportEra1(config: ExporterConf) =
group.update(e2, blockNumber, blck.header, blck.body, blck.receipts, ttd).get()
accumulatorRoot = getEpochAccumulatorRoot(headerRecords)
accumulatorRoot = getEpochRecordRoot(headerRecords)
group.finish(e2, accumulatorRoot, endNumber).get()
completed = true
@ -313,9 +313,9 @@ when isMainModule:
# Downloading headers from JSON RPC endpoint
info "Requesting epoch headers", epoch
var headers: seq[BlockHeader]
for j in 0 ..< epochSize.uint64:
for j in 0 ..< EPOCH_SIZE.uint64:
debug "Requesting block", number = j
let header = client.downloadHeader(epoch * epochSize + j)
let header = client.downloadHeader(epoch * EPOCH_SIZE + j)
headers.add(header)
let fh = ?openFile(file, {OpenFlags.Write, OpenFlags.Create}).mapErr(toString)
@ -393,7 +393,7 @@ when isMainModule:
quit 1
proc buildAccumulator(
dataDir: string, writeEpochAccumulators = false
dataDir: string, writeEpochRecords = false
): Result[FinishedAccumulator, string] =
var accumulator: Accumulator
for i in 0 ..< preMergeEpochs:
@ -421,11 +421,11 @@ when isMainModule:
return err("Invalid block header in " & file & ": " & e.msg)
# Quick sanity check
if blockHeader.number != i * epochSize + count:
if blockHeader.number != i * EPOCH_SIZE + count:
fatal "Incorrect block headers in file",
file = file,
blockNumber = blockHeader.number,
expectedBlockNumber = i * epochSize + count
expectedBlockNumber = i * EPOCH_SIZE + count
quit 1
updateAccumulator(accumulator, blockHeader)
@ -433,22 +433,21 @@ when isMainModule:
# Note: writing away of epoch accumulators occurs 1 iteration before
# updating the epoch accumulator, as the latter happens when passed
# a header for the next epoch (or on finishing the epoch).
if writeEpochAccumulators:
if accumulator.currentEpoch.len() == epochSize or
if writeEpochRecords:
if accumulator.currentEpoch.len() == EPOCH_SIZE or
blockHeader.number == mergeBlockNumber - 1:
let file =
try:
dataDir / &"mainnet-epoch-accumulator-{i.uint64:05}.ssz"
dataDir / &"mainnet-epoch-record-{i.uint64:05}.ssz"
except ValueError as e:
raiseAssert e.msg
let res = io2.writeFile(file, SSZ.encode(accumulator.currentEpoch))
if res.isErr():
error "Failed writing epoch accumulator to file",
file, error = res.error
error "Failed writing epoch record to file", file, error = res.error
else:
notice "Succesfully wrote epoch accumulator to file", file
notice "Succesfully wrote epoch record to file", file
if count == epochSize - 1:
if count == EPOCH_SIZE - 1:
info "Updated an epoch", epoch = i
count.inc()
@ -462,7 +461,7 @@ when isMainModule:
err("Not enough headers provided to finish the accumulator")
let accumulatorRes = buildAccumulator(dataDir, config.writeEpochAccumulators)
let accumulatorRes = buildAccumulator(dataDir, config.writeEpochRecords)
if accumulatorRes.isErr():
fatal "Could not build accumulator", error = accumulatorRes.error
quit 1
@ -474,7 +473,8 @@ when isMainModule:
file = accumulatorFile, error = res.error
quit 1
else:
notice "Succesfully wrote master accumulator to file", file = accumulatorFile
notice "Succesfully wrote Historical Hashes Accumulator to file",
file = accumulatorFile
of HistoryCmd.printAccumulatorData:
let file = dataDir / config.accumulatorFileNamePrint
@ -564,8 +564,7 @@ when isMainModule:
let
epochIndex = getEpochIndex(blockNumber)
epochHeadersFile = dataDir / &"mainnet-headers-epoch-{epochIndex:05}.e2s"
epochAccumulatorFile =
dataDir / &"mainnet-epoch-accumulator-{epochIndex:05}.ssz"
epochRecordFile = dataDir / &"mainnet-epoch-record-{epochIndex:05}.ssz"
let res = readBlockHeaders(epochHeadersFile)
if res.isErr():
@ -574,17 +573,17 @@ when isMainModule:
let blockHeaders = res.get()
let epochAccumulatorRes = readEpochAccumulatorCached(epochAccumulatorFile)
if epochAccumulatorRes.isErr():
error "Could not read epoch accumulator file", error = res.error
let epochRecordRes = readEpochRecordCached(epochRecordFile)
if epochRecordRes.isErr():
error "Could not read epoch record file", error = res.error
quit 1
let epochAccumulator = epochAccumulatorRes.get()
let epochRecord = epochRecordRes.get()
let headerIndex = getHeaderRecordIndex(blockNumber, epochIndex)
let header = blockHeaders[headerIndex]
if header.isPreMerge():
let headerWithProof = buildHeaderWithProof(header, epochAccumulator)
let headerWithProof = buildHeaderWithProof(header, epochRecord)
if headerWithProof.isErr:
error "Error building proof", error = headerWithProof.error
quit 1

View File

@ -32,7 +32,7 @@ type
const
defaultDataDirDesc* = defaultDataDir()
defaultBlockFileName* = "eth-block-data"
defaultAccumulatorFileName* = "mainnet-master-accumulator.ssz"
defaultAccumulatorFileName* = "mainnet-historical-hashes-accumulator.ssz"
defaultWeb3Url* = Web3Url(kind: HttpUrl, url: "http://127.0.0.1:8545")
type
@ -144,10 +144,10 @@ type
defaultValueDesc: $defaultAccumulatorFileName,
name: "accumulator-file-name"
.}: string
writeEpochAccumulators* {.
desc: "Write also the SSZ encoded epoch accumulators to specific files",
writeEpochRecords* {.
desc: "Write also the SSZ encoded epoch records to specific files",
defaultValue: false,
name: "write-epoch-accumulators"
name: "write-epoch-records"
.}: bool
of printAccumulatorData:
accumulatorFileNamePrint* {.

View File

@ -295,7 +295,7 @@ proc runLatestLoop(
proc gossipHeadersWithProof(
portalClient: RpcClient,
era1File: string,
epochAccumulatorFile: Opt[string] = Opt.none(string),
epochRecordFile: Opt[string] = Opt.none(string),
verifyEra = false,
): Future[Result[void, string]] {.async: (raises: []).} =
let f = ?Era1File.open(era1File)
@ -306,13 +306,13 @@ proc gossipHeadersWithProof(
# Note: building the accumulator takes about 150ms vs 10ms for reading it,
# so it is probably not really worth using the read version considering the
# UX hassle it adds to provide the accumulator ssz files.
let epochAccumulator =
if epochAccumulatorFile.isNone:
let epochRecord =
if epochRecordFile.isNone:
?f.buildAccumulator()
else:
?readEpochAccumulatorCached(epochAccumulatorFile.get())
?readEpochRecordCached(epochRecordFile.get())
for (contentKey, contentValue) in f.headersWithProof(epochAccumulator):
for (contentKey, contentValue) in f.headersWithProof(epochRecord):
let peers =
try:
await portalClient.portal_historyGossip(
@ -517,9 +517,9 @@ proc runBackfillLoopAuditMode(
# Gossip missing content
if not headerSuccess:
let
epochAccumulator = db.getAccumulator(blockNumber).valueOr:
epochRecord = db.getAccumulator(blockNumber).valueOr:
raiseAssert "Failed to get accumulator from EraDB: " & error
headerWithProof = buildHeaderWithProof(header, epochAccumulator).valueOr:
headerWithProof = buildHeaderWithProof(header, epochRecord).valueOr:
raiseAssert "Failed to build header with proof: " & error
(await portalClient.gossipBlockHeader(blockHash, headerWithProof)).isOkOr:

@ -1 +1 @@
Subproject commit c201ce4b3f1856222c31e2541a391cce70e69135
Subproject commit 35e7fabcbe2dcf392fbe8184f4187d356b6ab6e9