`blck` --> `forkyBlck` when using `withBlck` / `withStateAndBlck` (#5451)

For symmetry with `forkyState` when using `withState`, and to avoid
problems with shadowing of `blck` when using `withBlck` in `template`,
also rename the injected `blck` to `forkyBlck`.

- https://github.com/nim-lang/Nim/issues/22698
This commit is contained in:
Etan Kissling 2023-09-21 12:49:14 +02:00 committed by GitHub
parent 351472a2d4
commit e7bc41e005
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
28 changed files with 180 additions and 166 deletions

View File

@ -46,12 +46,14 @@ proc initLightClient*(
wallSlot = node.currentSlot
withBlck(signedBlock):
when consensusFork >= ConsensusFork.Bellatrix:
if blck.message.is_execution_block:
template blckPayload(): auto = blck.message.body.execution_payload
if forkyBlck.message.is_execution_block:
template blckPayload(): auto =
forkyBlck.message.body.execution_payload
if not blckPayload.block_hash.isZero:
# engine_newPayloadV1
discard await node.elManager.newExecutionPayload(blck.message)
discard await node.elManager.newExecutionPayload(
forkyBlck.message)
# Retain optimistic head for other `forkchoiceUpdated` callers.
# May temporarily block `forkchoiceUpdatedV1` calls, e.g., Geth:
@ -60,7 +62,7 @@ proc initLightClient*(
# Once DAG sync catches up or as new optimistic heads are fetched
# the situation recovers
node.consensusManager[].setOptimisticHead(
blck.toBlockId(), blckPayload.block_hash)
forkyBlck.toBlockId(), blckPayload.block_hash)
# engine_forkchoiceUpdatedV1 or engine_forkchoiceUpdatedV2,
# depending on pre or post-Shapella
@ -73,7 +75,8 @@ proc initLightClient*(
finalizedBlockHash = beaconHead.finalizedExecutionPayloadHash,
payloadAttributes = none attributes)
case node.dag.cfg.consensusForkAtEpoch(blck.message.slot.epoch)
case node.dag.cfg.consensusForkAtEpoch(
forkyBlck.message.slot.epoch)
of ConsensusFork.Deneb:
callForkchoiceUpdated(PayloadAttributesV3)
of ConsensusFork.Capella:
@ -173,7 +176,7 @@ proc updateLightClientFromDag*(node: BeaconNode) =
const lcDataFork = lcDataForkAtConsensusFork(consensusFork)
when lcDataFork > LightClientDataFork.None:
header = ForkedLightClientHeader(kind: lcDataFork)
header.forky(lcDataFork) = blck.toLightClientHeader(lcDataFork)
header.forky(lcDataFork) = forkyBlck.toLightClientHeader(lcDataFork)
else: raiseAssert "Unreachable"
let current_sync_committee = block:
let tmpState = assignClone(node.dag.headState)

View File

@ -153,7 +153,7 @@ proc init*(T: type AttestationPool, dag: ChainDAGRef,
default(FinalityCheckpoints)
withBlck(blck):
forkChoice.process_block(
dag, epochRef, blckRef, unrealized, blck.message,
dag, epochRef, blckRef, unrealized, forkyBlck.message,
blckRef.slot.start_beacon_time)
doAssert status.isOk(), "Error in preloading the fork choice: " & $status.error

View File

@ -448,7 +448,7 @@ func init*(t: typedesc[EventBeaconBlockObject],
optimistic: Option[bool]): EventBeaconBlockObject =
withBlck(v):
EventBeaconBlockObject(
slot: blck.message.slot,
block_root: blck.root,
slot: forkyBlck.message.slot,
block_root: forkyBlck.root,
optimistic: optimistic
)

View File

@ -286,8 +286,8 @@ proc getForkedBlock*(
let fork = dag.cfg.consensusForkAtEpoch(bid.slot.epoch)
result.ok(ForkedTrustedSignedBeaconBlock(kind: fork))
withBlck(result.get()):
type T = type(blck)
blck = getBlock(dag, bid, T).valueOr:
type T = type(forkyBlck)
forkyBlck = getBlock(dag, bid, T).valueOr:
getBlock(
dag.era, getStateField(dag.headState, historical_roots).asSeq,
dag.headState.historical_summaries().asSeq,
@ -310,7 +310,7 @@ proc getBlockId*(db: BeaconChainDB, root: Eth2Digest): Opt[BlockId] =
# Shouldn't happen too often but..
let
blck = forked.get()
summary = withBlck(blck): blck.message.toBeaconBlockSummary()
summary = withBlck(blck): forkyBlck.message.toBeaconBlockSummary()
debug "Writing summary", blck = shortLog(blck)
db.putBeaconBlockSummary(root, summary)
return ok(BlockId(root: root, slot: summary.slot))
@ -1393,9 +1393,9 @@ proc computeRandaoMix(
## Compute the requested RANDAO mix for `bdata` without `state`, if possible.
withBlck(bdata):
when consensusFork >= ConsensusFork.Bellatrix:
if blck.message.is_execution_block:
var mix = eth2digest(blck.message.body.randao_reveal.toRaw())
mix.data.mxor blck.message.body.execution_payload.prev_randao.data
if forkyBlck.message.is_execution_block:
var mix = eth2digest(forkyBlck.message.body.randao_reveal.toRaw())
mix.data.mxor forkyBlck.message.body.execution_payload.prev_randao.data
return ok mix
Opt.none(Eth2Digest)
@ -1424,14 +1424,15 @@ proc computeRandaoMix*(
while bid.slot > ancestorSlot:
let bdata = ? dag.getForkedBlock(bid)
withBlck(bdata): # See `process_randao` / `process_randao_mixes_reset`
mix.data.mxor eth2digest(blck.message.body.randao_reveal.toRaw()).data
mix.data.mxor eth2digest(
forkyBlck.message.body.randao_reveal.toRaw()).data
bid = ? dag.parent(bid)
ok()
# Mix in RANDAO from `bid`
if ancestorSlot < bid.slot:
withBlck(bdata):
mix = eth2digest(blck.message.body.randao_reveal.toRaw())
mix = eth2digest(forkyBlck.message.body.randao_reveal.toRaw())
? mixToAncestor(? dag.parent(bid))
else:
mix.reset()
@ -2251,7 +2252,7 @@ proc loadExecutionBlockHash*(dag: ChainDAGRef, bid: BlockId): Eth2Digest =
withBlck(blockData):
when consensusFork >= ConsensusFork.Bellatrix:
blck.message.body.execution_payload.block_hash
forkyBlck.message.body.execution_payload.block_hash
else:
ZERO_HASH

View File

@ -218,7 +218,8 @@ proc initLightClientBootstrapForPeriod(
if not dag.lcDataStore.db.hasSyncCommittee(period):
dag.lcDataStore.db.putSyncCommittee(
period, forkyState.data.current_sync_committee)
dag.lcDataStore.db.putHeader(blck.toLightClientHeader(lcDataFork))
dag.lcDataStore.db.putHeader(
forkyBlck.toLightClientHeader(lcDataFork))
dag.lcDataStore.db.putCurrentSyncCommitteeBranch(
bid.slot, forkyState.data.build_proof(
altair.CURRENT_SYNC_COMMITTEE_INDEX).get)
@ -275,7 +276,7 @@ proc initLightClientUpdateForPeriod(
numParticipants =
withBlck(bdata):
when consensusFork >= ConsensusFork.Altair:
blck.message.body.sync_aggregate.num_active_participants
forkyBlck.message.body.sync_aggregate.num_active_participants
else: raiseAssert "Unreachable"
if numParticipants >= maxParticipants:
maxParticipants = numParticipants
@ -368,7 +369,7 @@ proc initLightClientUpdateForPeriod(
const lcDataFork = lcDataForkAtConsensusFork(consensusFork)
update = ForkedLightClientUpdate(kind: lcDataFork)
template forkyUpdate: untyped = update.forky(lcDataFork)
forkyUpdate.attested_header = blck.toLightClientHeader(lcDataFork)
forkyUpdate.attested_header = forkyBlck.toLightClientHeader(lcDataFork)
forkyUpdate.next_sync_committee = forkyState.data.next_sync_committee
forkyUpdate.next_sync_committee_branch =
forkyState.data.build_proof(altair.NEXT_SYNC_COMMITTEE_INDEX).get
@ -387,7 +388,8 @@ proc initLightClientUpdateForPeriod(
withForkyUpdate(update):
when lcDataFork > LightClientDataFork.None:
when lcDataFork >= lcDataForkAtConsensusFork(consensusFork):
forkyUpdate.finalized_header = blck.toLightClientHeader(lcDataFork)
forkyUpdate.finalized_header =
forkyBlck.toLightClientHeader(lcDataFork)
else: raiseAssert "Unreachable"
let bdata = dag.getExistingForkedBlock(signatureBid).valueOr:
dag.handleUnexpectedLightClientError(signatureBid.slot)
@ -397,7 +399,7 @@ proc initLightClientUpdateForPeriod(
withForkyUpdate(update):
when lcDataFork > LightClientDataFork.None:
forkyUpdate.sync_aggregate =
blck.asSigned().message.body.sync_aggregate
forkyBlck.asSigned().message.body.sync_aggregate
else: raiseAssert "Unreachable"
withForkyUpdate(update):
when lcDataFork > LightClientDataFork.None:
@ -478,7 +480,7 @@ template lazy_header(name: untyped): untyped {.dirty.} =
else:
withBlck(bdata.get):
when data_fork >= lcDataForkAtConsensusFork(consensusFork):
obj.name = blck.toLightClientHeader(data_fork)
obj.name = forkyBlck.toLightClientHeader(data_fork)
else: raiseAssert "Unreachable"
`name _ ptr` = addr obj.name
`name _ ok`
@ -496,7 +498,7 @@ template lazy_header(name: untyped): untyped {.dirty.} =
obj.migrateToDataFork(data_fork)
withBlck(bdata.get):
when data_fork >= lcDataForkAtConsensusFork(consensusFork):
obj.forky(data_fork).name = blck.toLightClientHeader(data_fork)
obj.forky(data_fork).name = forkyBlck.toLightClientHeader(data_fork)
else: raiseAssert "Unreachable"
`name _ ptr` = addr obj.forky(data_fork).name
`name _ ok`
@ -745,7 +747,7 @@ proc initLightClientDataCache*(dag: ChainDAGRef) =
# Create `LightClientUpdate` instances
if i < blocks.high:
dag.createLightClientUpdates(
forkyState, blck, parentBid = blocks[i + 1])
forkyState, forkyBlck, parentBid = blocks[i + 1])
else: raiseAssert "Unreachable"
let lightClientEndTick = Moment.now()
@ -882,7 +884,8 @@ proc processFinalizationForLightClient*(
withBlck(bdata):
when consensusFork >= ConsensusFork.Altair:
const lcDataFork = lcDataForkAtConsensusFork(consensusFork)
dag.lcDataStore.db.putHeader(blck.toLightClientHeader(lcDataFork))
dag.lcDataStore.db.putHeader(
forkyBlck.toLightClientHeader(lcDataFork))
else: raiseAssert "Unreachable"
dag.lcDataStore.db.putCurrentSyncCommitteeBranch(
bid.slot, dag.getLightClientData(bid).current_sync_committee_branch)
@ -1006,7 +1009,7 @@ proc getLightClientBootstrap*(
when consensusFork >= ConsensusFork.Altair:
const lcDataFork = lcDataForkAtConsensusFork(consensusFork)
let
header = blck.toLightClientHeader(lcDataFork)
header = forkyBlck.toLightClientHeader(lcDataFork)
bootstrap = dag.getLightClientBootstrap(header)
if bootstrap.kind > LightClientDataFork.None:
dag.lcDataStore.db.putHeader(header)

View File

@ -388,10 +388,10 @@ proc enqueueBlock*(
maybeFinalized = false,
validationDur = Duration()) =
withBlck(blck):
if blck.message.slot <= self.consensusManager.dag.finalizedHead.slot:
if forkyBlck.message.slot <= self.consensusManager.dag.finalizedHead.slot:
# let backfill blocks skip the queue - these are always "fast" to process
# because there are no state rewinds to deal with
let res = self.storeBackfillBlock(blck, blobs)
let res = self.storeBackfillBlock(forkyBlck, blobs)
resfut.complete(res)
return
@ -732,25 +732,25 @@ proc storeBlock(
quarantined = shortLog(quarantined.root)
withBlck(quarantined):
when typeof(blck).toFork() < ConsensusFork.Deneb:
when typeof(forkyBlck).toFork() < ConsensusFork.Deneb:
self[].enqueueBlock(
MsgSource.gossip, quarantined, Opt.none(BlobSidecars))
else:
if len(blck.message.body.blob_kzg_commitments) == 0:
if len(forkyBlck.message.body.blob_kzg_commitments) == 0:
self[].enqueueBlock(
MsgSource.gossip, quarantined, Opt.some(BlobSidecars @[]))
else:
if (let res = checkBloblessSignature(self[], blck); res.isErr):
if (let res = checkBloblessSignature(self[], forkyBlck); res.isErr):
warn "Failed to verify signature of unorphaned blobless block",
blck = shortLog(blck),
blck = shortLog(forkyBlck),
error = res.error()
continue
if self.blobQuarantine[].hasBlobs(blck):
let blobs = self.blobQuarantine[].popBlobs(blck.root)
if self.blobQuarantine[].hasBlobs(forkyBlck):
let blobs = self.blobQuarantine[].popBlobs(forkyBlck.root)
self[].enqueueBlock(MsgSource.gossip, quarantined, Opt.some(blobs))
else:
if not self.consensusManager.quarantine[].addBlobless(
dag.finalizedHead.slot, blck):
dag.finalizedHead.slot, forkyBlck):
notice "Block quarantine full (blobless)",
blockRoot = shortLog(quarantined.root),
signature = shortLog(quarantined.signature)
@ -795,7 +795,7 @@ proc processBlock(
let res = withBlck(entry.blck):
await self.storeBlock(
entry.src, wallTime, blck, entry.blobs, entry.maybeFinalized,
entry.src, wallTime, forkyBlck, entry.blobs, entry.maybeFinalized,
entry.queueTick, entry.validationDur)
if res.isErr and res.error[1] == ProcessingStatus.notCompleted:

View File

@ -344,11 +344,11 @@ proc initFullNode(
maybeFinalized: bool):
Future[Result[void, VerifierError]] =
withBlck(signedBlock):
when typeof(blck).toFork() >= ConsensusFork.Deneb:
if not blobQuarantine[].hasBlobs(blck):
when typeof(forkyBlck).toFork() >= ConsensusFork.Deneb:
if not blobQuarantine[].hasBlobs(forkyBlck):
# We don't have all the blobs for this block, so we have
# to put it in blobless quarantine.
if not quarantine[].addBlobless(dag.finalizedHead.slot, blck):
if not quarantine[].addBlobless(dag.finalizedHead.slot, forkyBlck):
Future.completed(
Result[void, VerifierError].err(VerifierError.UnviableFork),
"rmanBlockVerifier")
@ -357,7 +357,7 @@ proc initFullNode(
Result[void, VerifierError].err(VerifierError.MissingParent),
"rmanBlockVerifier")
else:
let blobs = blobQuarantine[].popBlobs(blck.root)
let blobs = blobQuarantine[].popBlobs(forkyBlck.root)
blockProcessor[].addBlock(MsgSource.gossip, signedBlock,
Opt.some(blobs),
maybeFinalized = maybeFinalized)
@ -1188,9 +1188,9 @@ proc pruneBlobs(node: BeaconNode, slot: Slot) =
for i in startIndex..<SLOTS_PER_EPOCH:
let blck = node.dag.getForkedBlock(blocks[int(i)]).valueOr: continue
withBlck(blck):
when typeof(blck).toFork() < ConsensusFork.Deneb: continue
when typeof(forkyBlck).toFork() < ConsensusFork.Deneb: continue
else:
for j in 0..len(blck.message.body.blob_kzg_commitments) - 1:
for j in 0..len(forkyBlck.message.body.blob_kzg_commitments) - 1:
if node.db.delBlobSidecar(blocks[int(i)].root, BlobIndex(j)):
count = count + 1
debug "pruned blobs", count, blobPruneEpoch

View File

@ -115,22 +115,22 @@ programMain:
# `engine_forkchoiceUpdatedV1` under any of the following conditions:
# `headBlockHash` references a block which `timestamp` is greater or
# equal to the Shanghai timestamp
if blck.message.is_execution_block:
template payload(): auto = blck.message.body.execution_payload
if forkyBlck.message.is_execution_block:
template payload(): auto = forkyBlck.message.body.execution_payload
if elManager != nil and not payload.block_hash.isZero:
discard await elManager.newExecutionPayload(blck.message)
discard await elManager.newExecutionPayload(forkyBlck.message)
discard await elManager.forkchoiceUpdated(
headBlockHash = payload.block_hash,
safeBlockHash = payload.block_hash, # stub value
finalizedBlockHash = ZERO_HASH,
payloadAttributes = none PayloadAttributesV2)
elif consensusFork >= ConsensusFork.Bellatrix:
if blck.message.is_execution_block:
template payload(): auto = blck.message.body.execution_payload
if forkyBlck.message.is_execution_block:
template payload(): auto = forkyBlck.message.body.execution_payload
if elManager != nil and not payload.block_hash.isZero:
discard await elManager.newExecutionPayload(blck.message)
discard await elManager.newExecutionPayload(forkyBlck.message)
discard await elManager.forkchoiceUpdated(
headBlockHash = payload.block_hash,
safeBlockHash = payload.block_hash, # stub value

View File

@ -799,12 +799,12 @@ proc installBeaconApiHandlers*(router: var RestRouter, node: BeaconNode) =
RestApiResponse.jsonResponseWOpt(
[
(
root: blck.root,
root: forkyBlck.root,
canonical: node.dag.isCanonical(
BlockId(root: blck.root, slot: blck.message.slot)),
BlockId(root: forkyBlck.root, slot: forkyBlck.message.slot)),
header: (
message: blck.toBeaconBlockHeader,
signature: blck.signature
message: forkyBlck.toBeaconBlockHeader,
signature: forkyBlck.signature
)
)
],
@ -826,12 +826,12 @@ proc installBeaconApiHandlers*(router: var RestRouter, node: BeaconNode) =
withBlck(bdata):
RestApiResponse.jsonResponseWOpt(
(
root: blck.root,
root: forkyBlck.root,
canonical: node.dag.isCanonical(
BlockId(root: blck.root, slot: blck.message.slot)),
BlockId(root: forkyBlck.root, slot: forkyBlck.message.slot)),
header: (
message: blck.toBeaconBlockHeader,
signature: blck.signature
message: forkyBlck.toBeaconBlockHeader,
signature: forkyBlck.signature
)
),
node.getBlockOptimistic(bdata)
@ -1024,8 +1024,8 @@ proc installBeaconApiHandlers*(router: var RestRouter, node: BeaconNode) =
return RestApiResponse.jsonError(Http400, InvalidBlockObjectError)
let res = withBlck(forked):
blck.root = hash_tree_root(blck.message)
await node.router.routeSignedBeaconBlock(blck,
forkyBlck.root = hash_tree_root(forkyBlck.message)
await node.router.routeSignedBeaconBlock(forkyBlck,
Opt.none(SignedBlobSidecars))
if res.isErr():
@ -1115,7 +1115,7 @@ proc installBeaconApiHandlers*(router: var RestRouter, node: BeaconNode) =
return
withBlck(bdata):
RestApiResponse.jsonResponseWOpt(
blck.message.body.attestations.asSeq(),
forkyBlck.message.body.attestations.asSeq(),
node.getBlockOptimistic(bdata)
)

View File

@ -414,17 +414,17 @@ proc installValidatorApiHandlers*(router: var RestRouter, node: BeaconNode) =
return
withBlck(message.blck):
let data =
when blck is deneb.BeaconBlock:
when forkyBlck is deneb.BeaconBlock:
let bundle = message.blobsBundleOpt.get()
let blockRoot = hash_tree_root(blck)
let blockRoot = hash_tree_root(forkyBlck)
var sidecars = newSeqOfCap[BlobSidecar](bundle.blobs.len)
for i in 0..<bundle.blobs.len:
let sidecar = deneb.BlobSidecar(
block_root: blockRoot,
index: BlobIndex(i),
slot: blck.slot,
block_parent_root: blck.parent_root,
proposer_index: blck.proposer_index,
slot: forkyBlck.slot,
block_parent_root: forkyBlck.parent_root,
proposer_index: forkyBlck.proposer_index,
blob: bundle.blobs[i],
kzg_commitment: bundle.kzgs[i],
kzg_proof: bundle.proofs[i]
@ -432,19 +432,21 @@ proc installValidatorApiHandlers*(router: var RestRouter, node: BeaconNode) =
sidecars.add(sidecar)
DenebBlockContents(
`block`: blck,
`block`: forkyBlck,
blob_sidecars: List[BlobSidecar,
Limit MAX_BLOBS_PER_BLOCK].init(sidecars))
elif blck is phase0.BeaconBlock or blck is altair.BeaconBlock or
blck is bellatrix.BeaconBlock or blck is capella.BeaconBlock:
blck
elif forkyBlck is phase0.BeaconBlock or
forkyBlck is altair.BeaconBlock or
forkyBlck is bellatrix.BeaconBlock or
forkyBlck is capella.BeaconBlock:
forkyBlck
else:
static: raiseAssert "produceBlockV2 received unexpected version"
if contentType == sszMediaType:
let headers = [("eth-consensus-version", message.blck.kind.toString())]
RestApiResponse.sszResponse(blck, headers)
RestApiResponse.sszResponse(forkyBlck, headers)
elif contentType == jsonMediaType:
RestApiResponse.jsonResponseWVersion(blck, message.blck.kind)
RestApiResponse.jsonResponseWVersion(forkyBlck, message.blck.kind)
else:
raiseAssert "preferredContentType() returns invalid content type"
@ -568,7 +570,7 @@ proc installValidatorApiHandlers*(router: var RestRouter, node: BeaconNode) =
if res.isErr():
return RestApiResponse.jsonError(Http400, res.error())
withBlck(res.get().blck):
return responseVersioned(blck, contextFork)
return responseVersioned(forkyBlck, contextFork)
# https://ethereum.github.io/beacon-APIs/#/Validator/produceAttestationData
router.api(MethodGet, "/eth/v1/validator/attestation_data") do (

View File

@ -278,7 +278,7 @@ proc jsonResponseBlock*(t: typedesc[RestApiResponse],
if execOpt.isSome():
writer.writeField("execution_optimistic", execOpt.get())
withBlck(data):
writer.writeField("data", blck)
writer.writeField("data", forkyBlck)
writer.endRecord()
stream.getOutput(seq[byte])
except SerializationError:
@ -1887,7 +1887,7 @@ proc readValue*(reader: var JsonReader[RestJson],
reader.raiseUnexpectedValue("Incorrect deneb block format")
value = ForkedSignedBeaconBlock.init(res.get())
withBlck(value):
blck.root = hash_tree_root(blck.message)
forkyBlck.root = hash_tree_root(forkyBlck.message)
proc writeValue*(
writer: var JsonWriter[RestJson], value: ForkedSignedBeaconBlock

View File

@ -627,7 +627,7 @@ func init*(t: typedesc[ValidatorIdent], v: ValidatorPubKey): ValidatorIdent =
func init*(t: typedesc[RestBlockInfo],
v: ForkedTrustedSignedBeaconBlock): RestBlockInfo =
withBlck(v):
RestBlockInfo(slot: blck.message.slot, blck: blck.root)
RestBlockInfo(slot: forkyBlck.message.slot, blck: forkyBlck.root)
func init*(t: typedesc[RestValidator], index: ValidatorIndex,
balance: uint64, status: string,

View File

@ -766,30 +766,30 @@ template withBlck*(
case x.kind
of ConsensusFork.Phase0:
const consensusFork {.inject, used.} = ConsensusFork.Phase0
template blck: untyped {.inject.} = x.phase0Data
template forkyBlck: untyped {.inject.} = x.phase0Data
body
of ConsensusFork.Altair:
const consensusFork {.inject, used.} = ConsensusFork.Altair
template blck: untyped {.inject.} = x.altairData
template forkyBlck: untyped {.inject.} = x.altairData
body
of ConsensusFork.Bellatrix:
const consensusFork {.inject, used.} = ConsensusFork.Bellatrix
template blck: untyped {.inject.} = x.bellatrixData
template forkyBlck: untyped {.inject.} = x.bellatrixData
body
of ConsensusFork.Capella:
const consensusFork {.inject, used.} = ConsensusFork.Capella
template blck: untyped {.inject.} = x.capellaData
template forkyBlck: untyped {.inject.} = x.capellaData
body
of ConsensusFork.Deneb:
const consensusFork {.inject, used.} = ConsensusFork.Deneb
template blck: untyped {.inject.} = x.denebData
template forkyBlck: untyped {.inject.} = x.denebData
body
func proposer_index*(x: ForkedBeaconBlock): uint64 =
withBlck(x): blck.proposer_index
withBlck(x): forkyBlck.proposer_index
func hash_tree_root*(x: ForkedBeaconBlock): Eth2Digest =
withBlck(x): hash_tree_root(blck)
withBlck(x): hash_tree_root(forkyBlck)
func hash_tree_root*(x: Web3SignerForkedBeaconBlock): Eth2Digest =
hash_tree_root(x.data)
@ -810,29 +810,29 @@ template getForkedBlockField*(
template signature*(x: ForkedSignedBeaconBlock |
ForkedMsgTrustedSignedBeaconBlock |
ForkedSignedBlindedBeaconBlock): ValidatorSig =
withBlck(x): blck.signature
withBlck(x): forkyBlck.signature
template signature*(x: ForkedTrustedSignedBeaconBlock): TrustedSig =
withBlck(x): blck.signature
withBlck(x): forkyBlck.signature
template root*(x: ForkedSignedBeaconBlock |
ForkedMsgTrustedSignedBeaconBlock |
ForkedTrustedSignedBeaconBlock): Eth2Digest =
withBlck(x): blck.root
withBlck(x): forkyBlck.root
template slot*(x: ForkedSignedBeaconBlock |
ForkedMsgTrustedSignedBeaconBlock |
ForkedTrustedSignedBeaconBlock): Slot =
withBlck(x): blck.message.slot
withBlck(x): forkyBlck.message.slot
template shortLog*(x: ForkedBeaconBlock | ForkedBlindedBeaconBlock): auto =
withBlck(x): shortLog(blck)
withBlck(x): shortLog(forkyBlck)
template shortLog*(x: ForkedSignedBeaconBlock |
ForkedMsgTrustedSignedBeaconBlock |
ForkedTrustedSignedBeaconBlock |
ForkedSignedBlindedBeaconBlock): auto =
withBlck(x): shortLog(blck)
withBlck(x): shortLog(forkyBlck)
chronicles.formatIt ForkedBeaconBlock: it.shortLog
chronicles.formatIt ForkedSignedBeaconBlock: it.shortLog
@ -849,27 +849,27 @@ template withStateAndBlck*(
of ConsensusFork.Deneb:
const consensusFork {.inject.} = ConsensusFork.Deneb
template forkyState: untyped {.inject.} = s.denebData
template blck: untyped {.inject.} = b.denebData
template forkyBlck: untyped {.inject.} = b.denebData
body
of ConsensusFork.Capella:
const consensusFork {.inject.} = ConsensusFork.Capella
template forkyState: untyped {.inject.} = s.capellaData
template blck: untyped {.inject.} = b.capellaData
template forkyBlck: untyped {.inject.} = b.capellaData
body
of ConsensusFork.Bellatrix:
const consensusFork {.inject.} = ConsensusFork.Bellatrix
template forkyState: untyped {.inject.} = s.bellatrixData
template blck: untyped {.inject.} = b.bellatrixData
template forkyBlck: untyped {.inject.} = b.bellatrixData
body
of ConsensusFork.Altair:
const consensusFork {.inject.} = ConsensusFork.Altair
template forkyState: untyped {.inject.} = s.altairData
template blck: untyped {.inject.} = b.altairData
template forkyBlck: untyped {.inject.} = b.altairData
body
of ConsensusFork.Phase0:
const consensusFork {.inject.} = ConsensusFork.Phase0
template forkyState: untyped {.inject.} = s.phase0Data
template blck: untyped {.inject.} = b.phase0Data
template forkyBlck: untyped {.inject.} = b.phase0Data
body
func toBeaconBlockHeader*(
@ -893,7 +893,7 @@ template toBeaconBlockHeader*(
blckParam: ForkedMsgTrustedSignedBeaconBlock |
ForkedTrustedSignedBeaconBlock): BeaconBlockHeader =
## Reduce a given signed beacon block to its `BeaconBlockHeader`.
withBlck(blckParam): blck.toBeaconBlockHeader()
withBlck(blckParam): forkyBlck.toBeaconBlockHeader()
func genesisFork*(cfg: RuntimeConfig): Fork =
Fork(
@ -1036,7 +1036,7 @@ func readSszForkedSignedBeaconBlock*(
kind: cfg.consensusForkAtEpoch(header.slot.epoch()))
withBlck(result):
readSszBytes(data, blck)
readSszBytes(data, forkyBlck)
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/phase0/beacon-chain.md#compute_fork_data_root
func compute_fork_data_root*(current_version: Version,
@ -1087,7 +1087,7 @@ func toBlockId*(blck: SomeForkySignedBeaconBlock): BlockId =
func toBlockId*(blck: ForkedSignedBeaconBlock |
ForkedMsgTrustedSignedBeaconBlock |
ForkedTrustedSignedBeaconBlock): BlockId =
withBlck(blck): BlockId(root: blck.root, slot: blck.message.slot)
withBlck(blck): BlockId(root: forkyBlck.root, slot: forkyBlck.message.slot)
func historical_summaries*(state: ForkedHashedBeaconState):
HashList[HistoricalSummary, Limit HISTORICAL_ROOTS_LIMIT] =

View File

@ -487,18 +487,18 @@ proc doTrustedNodeSync*(
of TrustedNodeSyncKind.TrustedBlockRoot:
# Trust-minimized sync: the server is only trusted for
# data availability, responses must be verified
dag.addBackfillBlock(blck)
dag.addBackfillBlock(forkyBlck)
of TrustedNodeSyncKind.StateId:
# The server is fully trusted to provide accurate data;
# it could have provided a malicious state
dag.addBackfillBlock(blck.asSigVerified())
dag.addBackfillBlock(forkyBlck.asSigVerified())
if res.isErr():
case res.error()
of VerifierError.Invalid,
VerifierError.MissingParent,
VerifierError.UnviableFork:
error "Got invalid block from trusted node - is it on the right network?",
blck = shortLog(blck), err = res.error()
blck = shortLog(forkyBlck), err = res.error()
quit 1
of VerifierError.Duplicate:
discard

View File

@ -114,7 +114,7 @@ proc produceBlindedBlock(
error "An unexpected error occurred while getting blinded block data",
error_name = exc.name, error_msg = exc.msg
return Opt.none(PreparedBlindedBeaconBlock)
blockRoot = withBlck(beaconBlock): hash_tree_root(blck)
blockRoot = withBlck(beaconBlock): hash_tree_root(forkyBlck)
return Opt.some(
PreparedBlindedBeaconBlock(blockRoot: blockRoot, data: beaconBlock))

View File

@ -689,7 +689,8 @@ proc getUnsignedBlindedBeaconBlock[
consensusFork == ConsensusFork.Capella)):
return err("getUnsignedBlindedBeaconBlock: mismatched block/payload types")
else:
return ok constructSignableBlindedBlock[T](blck, executionPayloadHeader)
return ok constructSignableBlindedBlock[T](
forkyBlck, executionPayloadHeader)
else:
return err("getUnsignedBlindedBeaconBlock: attempt to construct pre-Bellatrix blinded block")
@ -891,7 +892,7 @@ proc makeBlindedBeaconBlockForHeadAndSlot*[
(consensusFork == ConsensusFork.Capella and
EPH is capella.ExecutionPayloadHeader)):
return ok (constructPlainBlindedBlock[BBB, EPH](
blck, executionPayloadHeader), bidValue)
forkyBlck, executionPayloadHeader), bidValue)
else:
return err("makeBlindedBeaconBlockForHeadAndSlot: mismatched block/payload types")
else:
@ -1045,7 +1046,7 @@ proc proposeBlockAux(
withBlck(forkedBlck):
let
blockRoot = hash_tree_root(blck)
blockRoot = hash_tree_root(forkyBlck)
signingRoot = compute_block_signing_root(
fork, genesis_validators_root, slot, blockRoot)
@ -1054,7 +1055,7 @@ proc proposeBlockAux(
.registerBlock(validator_index, validator.pubkey, slot, signingRoot)
let blobSidecarsOpt =
when blck is deneb.BeaconBlock:
when forkyBlck is deneb.BeaconBlock:
var sidecars: seq[BlobSidecar]
let bundle = collectedBids.engineBlockFut.read.get().blobsBundleOpt.get
let (blobs, kzgs, proofs) = (bundle.blobs, bundle.kzgs, bundle.proofs)
@ -1063,23 +1064,25 @@ proc proposeBlockAux(
block_root: blockRoot,
index: BlobIndex(i),
slot: slot,
block_parent_root: blck.parent_root,
proposer_index: blck.proposer_index,
block_parent_root: forkyBlck.parent_root,
proposer_index: forkyBlck.proposer_index,
blob: blobs[i],
kzg_commitment: kzgs[i],
kzg_proof: proofs[i]
)
sidecars.add(sidecar)
Opt.some(sidecars)
elif blck is phase0.BeaconBlock or blck is altair.BeaconBlock or
blck is bellatrix.BeaconBlock or blck is capella.BeaconBlock:
elif forkyBlck is phase0.BeaconBlock or
forkyBlck is altair.BeaconBlock or
forkyBlck is bellatrix.BeaconBlock or
forkyBlck is capella.BeaconBlock:
Opt.none(seq[BlobSidecar])
else:
static: doAssert "Unknown BeaconBlock type"
if notSlashable.isErr:
warn "Slashing protection activated for block proposal",
blockRoot = shortLog(blockRoot), blck = shortLog(blck),
blockRoot = shortLog(blockRoot), blck = shortLog(forkyBlck),
signingRoot = shortLog(signingRoot),
validator = validator.pubkey,
slot = slot,
@ -1097,28 +1100,30 @@ proc proposeBlockAux(
return head
res.get()
signedBlock =
when blck is phase0.BeaconBlock:
when forkyBlck is phase0.BeaconBlock:
phase0.SignedBeaconBlock(
message: blck, signature: signature, root: blockRoot)
elif blck is altair.BeaconBlock:
message: forkyBlck, signature: signature, root: blockRoot)
elif forkyBlck is altair.BeaconBlock:
altair.SignedBeaconBlock(
message: blck, signature: signature, root: blockRoot)
elif blck is bellatrix.BeaconBlock:
message: forkyBlck, signature: signature, root: blockRoot)
elif forkyBlck is bellatrix.BeaconBlock:
bellatrix.SignedBeaconBlock(
message: blck, signature: signature, root: blockRoot)
elif blck is capella.BeaconBlock:
message: forkyBlck, signature: signature, root: blockRoot)
elif forkyBlck is capella.BeaconBlock:
capella.SignedBeaconBlock(
message: blck, signature: signature, root: blockRoot)
elif blck is deneb.BeaconBlock:
message: forkyBlck, signature: signature, root: blockRoot)
elif forkyBlck is deneb.BeaconBlock:
deneb.SignedBeaconBlock(
message: blck, signature: signature, root: blockRoot)
message: forkyBlck, signature: signature, root: blockRoot)
else:
static: doAssert "Unknown SignedBeaconBlock type"
signedBlobs =
when blck is phase0.BeaconBlock or blck is altair.BeaconBlock or
blck is bellatrix.BeaconBlock or blck is capella.BeaconBlock:
when forkyBlck is phase0.BeaconBlock or
forkyBlck is altair.BeaconBlock or
forkyBlck is bellatrix.BeaconBlock or
forkyBlck is capella.BeaconBlock:
Opt.none(SignedBlobSidecars)
elif blck is deneb.BeaconBlock:
elif forkyBlck is deneb.BeaconBlock:
var signed: seq[SignedBlobSidecar]
let blobSidecars = blobSidecarsOpt.get()
for i in 0..<blobs.len:
@ -1144,7 +1149,7 @@ proc proposeBlockAux(
return head # Validation errors logged in router
notice "Block proposed",
blockRoot = shortLog(blockRoot), blck = shortLog(blck),
blockRoot = shortLog(blockRoot), blck = shortLog(forkyBlck),
signature = shortLog(signature), validator = shortLog(validator)
beacon_blocks_proposed.inc()

View File

@ -134,7 +134,7 @@ proc doTransition(conf: NcliConf) =
info = ForkedEpochInfo()
let res = withTimerRet(timers[tTransition]): withBlck(blckX):
state_transition(
cfg, stateY[], blck, cache, info, flags, noRollback)
cfg, stateY[], forkyBlck, cache, info, flags, noRollback)
if res.isErr():
error "State transition failed", error = res.error()
quit 1

View File

@ -336,28 +336,28 @@ func collectFromProposerSlashings(
forkedState: ForkedHashedBeaconState,
forkedBlock: ForkedTrustedSignedBeaconBlock) =
withStateAndBlck(forkedState, forkedBlock):
for proposer_slashing in blck.message.body.proposer_slashings:
for proposer_slashing in forkyBlck.message.body.proposer_slashings:
doAssert check_proposer_slashing(
forkyState.data, proposer_slashing, {}).isOk
let slashedIndex =
proposer_slashing.signed_header_1.message.proposer_index
rewardsAndPenalties.collectFromSlashedValidator(
forkyState.data, slashedIndex.ValidatorIndex,
blck.message.proposer_index.ValidatorIndex)
forkyBlck.message.proposer_index.ValidatorIndex)
func collectFromAttesterSlashings(
rewardsAndPenalties: var seq[RewardsAndPenalties],
forkedState: ForkedHashedBeaconState,
forkedBlock: ForkedTrustedSignedBeaconBlock) =
withStateAndBlck(forkedState, forkedBlock):
for attester_slashing in blck.message.body.attester_slashings:
for attester_slashing in forkyBlck.message.body.attester_slashings:
let attester_slashing_validity = check_attester_slashing(
forkyState.data, attester_slashing, {})
doAssert attester_slashing_validity.isOk
for slashedIndex in attester_slashing_validity.value:
rewardsAndPenalties.collectFromSlashedValidator(
forkyState.data, slashedIndex,
blck.message.proposer_index.ValidatorIndex)
forkyBlck.message.proposer_index.ValidatorIndex)
func collectFromAttestations(
rewardsAndPenalties: var seq[RewardsAndPenalties],
@ -370,7 +370,7 @@ func collectFromAttestations(
let base_reward_per_increment = get_base_reward_per_increment(
get_total_active_balance(forkyState.data, cache))
doAssert base_reward_per_increment > 0
for attestation in blck.message.body.attestations:
for attestation in forkyBlck.message.body.attestations:
doAssert check_attestation(
forkyState.data, attestation, {}, cache).isOk
let proposerReward =
@ -382,8 +382,8 @@ func collectFromAttestations(
get_proposer_reward(
forkyState.data, attestation, base_reward_per_increment, cache,
epochParticipationFlags.previousEpochParticipation)
rewardsAndPenalties[blck.message.proposer_index].proposer_outcome +=
proposerReward.int64
rewardsAndPenalties[forkyBlck.message.proposer_index]
.proposer_outcome += proposerReward.int64
let inclusionDelay = forkyState.data.slot - attestation.data.slot
for index in get_attesting_indices(
forkyState.data, attestation.data, attestation.aggregation_bits,
@ -397,7 +397,7 @@ proc collectFromDeposits(
pubkeyToIndex: var PubkeyToIndexTable,
cfg: RuntimeConfig) =
withStateAndBlck(forkedState, forkedBlock):
for deposit in blck.message.body.deposits:
for deposit in forkyBlck.message.body.deposits:
let pubkey = deposit.data.pubkey
let amount = deposit.data.amount
var index = findValidatorIndex(forkyState.data, pubkey)
@ -426,7 +426,7 @@ func collectFromSyncAggregate(
indices = get_sync_committee_cache(
forkyState.data, cache).current_sync_committee
template aggregate: untyped = blck.message.body.sync_aggregate
template aggregate: untyped = forkyBlck.message.body.sync_aggregate
doAssert indices.len == SYNC_COMMITTEE_SIZE
doAssert aggregate.sync_committee_bits.len == SYNC_COMMITTEE_SIZE
@ -439,8 +439,8 @@ func collectFromSyncAggregate(
if aggregate.sync_committee_bits[i]:
rewardsAndPenalties[indices[i]].sync_committee_outcome +=
participant_reward.int64
rewardsAndPenalties[blck.message.proposer_index].proposer_outcome +=
proposer_reward.int64
rewardsAndPenalties[forkyBlck.message.proposer_index]
.proposer_outcome += proposer_reward.int64
else:
rewardsAndPenalties[indices[i]].sync_committee_outcome -=
participant_reward.int64

View File

@ -431,13 +431,13 @@ proc cmdPutBlock(conf: DbConf, cfg: RuntimeConfig) =
cfg, readAllBytes(file).tryGet())
withBlck(blck.asTrusted()):
db.putBlock(blck)
db.putBlock(forkyBlck)
if conf.setHead:
db.putHeadBlock(blck.root)
db.putHeadBlock(forkyBlck.root)
if conf.setTail:
db.putTailBlock(blck.root)
db.putTailBlock(forkyBlck.root)
if conf.setGenesis:
db.putGenesisBlock(blck.root)
db.putGenesisBlock(forkyBlck.root)
proc cmdRewindState(conf: DbConf, cfg: RuntimeConfig) =
echo "Opening database..."
@ -628,7 +628,7 @@ proc cmdImportEra(conf: DbConf, cfg: RuntimeConfig) =
continue
withBlck(blck.asTrusted()):
db.putBlock(blck)
db.putBlock(forkyBlck)
blocks += 1
elif header.typ == SnappyBeaconState:
info "Skipping beacon state (use reindexing to recreate state snapshots)"
@ -1058,13 +1058,13 @@ proc cmdValidatorDb(conf: DbConf, cfg: RuntimeConfig) =
for bi in 0 ..< blockRefs.len:
let forkedBlock = dag.getForkedBlock(blockRefs[blockRefs.len - bi - 1]).get()
withBlck(forkedBlock):
processSlots(blck.message.slot, {skipLastStateRootCalculation})
processSlots(forkyBlck.message.slot, {skipLastStateRootCalculation})
rewardsAndPenalties.collectBlockRewardsAndPenalties(
tmpState[], forkedBlock, auxiliaryState, cache, cfg)
let res = state_transition_block(
cfg, tmpState[], blck, cache, {}, noRollback)
cfg, tmpState[], forkyBlck, cache, {}, noRollback)
if res.isErr:
fatal "State transition failed (!)"
quit QuitFailure

View File

@ -48,8 +48,8 @@ proc getParentBlock(restClient: RestClientRef):
withBlck(resp):
when consensusFork >= ConsensusFork.Capella:
return Opt.some ParentHeaderInfo(
block_number: blck.message.body.execution_payload.block_number,
timestamp: blck.message.body.execution_payload.timestamp)
block_number: forkyBlck.message.body.execution_payload.block_number,
timestamp: forkyBlck.message.body.execution_payload.timestamp)
else:
discard
@ -154,7 +154,7 @@ proc setupEngineAPI*(router: var RestRouter, payloadCache:
return RestApiResponse.jsonError(Http400, "No parent head hash provided")
let execution_payload = (await getInfo(parent_hash.get)).valueOr:
return RestApiResponse.jsonError(Http400, "Error getting parent head information")
payloadCache[hash_tree_root(execution_payload)] = execution_payload
payloadCache[hash_tree_root(execution_payload)] = execution_payload
return RestApiResponse.jsonResponse(
getExecutionPayloadHeader(execution_payload))
@ -177,7 +177,7 @@ proc setupEngineAPI*(router: var RestRouter, payloadCache:
restBlock, payloadCache[execution_header_root]))
else:
return RestApiResponse.jsonError(Http400, "Unknown execution payload")
router.api(MethodGet, "/eth/v1/builder/status") do () -> RestApiResponse:
return RestApiResponse.response("", Http200, "text/plain")

View File

@ -411,7 +411,7 @@ proc doRunTest(path: string, fork: ConsensusFork) =
let status = stepOnBlock(
stores.dag, stores.fkChoice,
verifier, state[], stateCache,
blck, step.blobData, time, invalidatedRoots)
forkyBlck, step.blobData, time, invalidatedRoots)
doAssert status.isOk == step.valid
of opOnAttesterSlashing:
let indices =

View File

@ -1503,7 +1503,7 @@ template runShufflingTests(cfg: RuntimeConfig, numRandomTests: int) =
graffiti = graffiti, cfg = cfg):
let added = withBlck(forkedBlck):
const nilCallback = (consensusFork.OnBlockAddedCallback)(nil)
dag.addHeadBlock(verifier, blck, nilCallback)
dag.addHeadBlock(verifier, forkyBlck, nilCallback)
check added.isOk()
dag.updateHead(added[], quarantine[], [])

View File

@ -195,7 +195,7 @@ suite "Gossip validation - Altair":
attested = false, cfg = cfg):
let added = withBlck(blck):
const nilCallback = (consensusFork.OnBlockAddedCallback)(nil)
dag.addHeadBlock(verifier, blck, nilCallback)
dag.addHeadBlock(verifier, forkyBlck, nilCallback)
check: added.isOk()
dag.updateHead(added[], quarantine, [])

View File

@ -65,7 +65,7 @@ suite "Light client" & preset():
syncCommitteeRatio = syncCommitteeRatio, cfg = cfg):
let added = withBlck(blck):
const nilCallback = (consensusFork.OnBlockAddedCallback)(nil)
dag.addHeadBlock(verifier, blck, nilCallback)
dag.addHeadBlock(verifier, forkyBlck, nilCallback)
check: added.isOk()
dag.updateHead(added[], quarantine, [])

View File

@ -53,7 +53,7 @@ suite "Light client processor" & preset():
syncCommitteeRatio = syncCommitteeRatio, cfg = cfg):
let added = withBlck(blck):
const nilCallback = (consensusFork.OnBlockAddedCallback)(nil)
dag.addHeadBlock(verifier, blck, nilCallback)
dag.addHeadBlock(verifier, forkyBlck, nilCallback)
doAssert added.isOk()
dag.updateHead(added[], quarantine[], [])

View File

@ -804,7 +804,7 @@ block:
asyncTest "Signing BeaconBlock (getBlockSignature(bellatrix))":
let
forked = getBlock(ConsensusFork.Bellatrix)
blockRoot = withBlck(forked): hash_tree_root(blck)
blockRoot = withBlck(forked): hash_tree_root(forkyBlck)
sres1 =
await validator1.getBlockSignature(SigningFork, GenesisValidatorsRoot,
@ -839,7 +839,7 @@ block:
asyncTest "Signing BeaconBlock (getBlockSignature(capella))":
let
forked = getBlock(ConsensusFork.Capella)
blockRoot = withBlck(forked): hash_tree_root(blck)
blockRoot = withBlck(forked): hash_tree_root(forkyBlck)
sres1 =
await validator1.getBlockSignature(SigningFork, GenesisValidatorsRoot,
@ -874,7 +874,7 @@ block:
asyncTest "Signing BeaconBlock (getBlockSignature(deneb))":
let
forked = getBlock(ConsensusFork.Deneb)
blockRoot = withBlck(forked): hash_tree_root(blck)
blockRoot = withBlck(forked): hash_tree_root(forkyBlck)
sres1 =
await validator1.getBlockSignature(SigningFork, GenesisValidatorsRoot,
@ -1042,9 +1042,9 @@ block:
let
fork = ConsensusFork.Bellatrix
forked1 = getBlock(fork)
blockRoot1 = withBlck(forked1): hash_tree_root(blck)
blockRoot1 = withBlck(forked1): hash_tree_root(forkyBlck)
forked2 = getBlock(fork, SigningOtherFeeRecipient)
blockRoot2 = withBlck(forked2): hash_tree_root(blck)
blockRoot2 = withBlck(forked2): hash_tree_root(forkyBlck)
request1 = Web3SignerRequest.init(SigningFork, GenesisValidatorsRoot,
Web3SignerForkedBeaconBlock.init(forked1))
request2 = Web3SignerRequest.init(SigningFork, GenesisValidatorsRoot,
@ -1131,9 +1131,9 @@ block:
let
fork = ConsensusFork.Capella
forked1 = getBlock(fork)
blockRoot1 = withBlck(forked1): hash_tree_root(blck)
blockRoot1 = withBlck(forked1): hash_tree_root(forkyBlck)
forked2 = getBlock(fork, SigningOtherFeeRecipient)
blockRoot2 = withBlck(forked2): hash_tree_root(blck)
blockRoot2 = withBlck(forked2): hash_tree_root(forkyBlck)
request1 = Web3SignerRequest.init(SigningFork, GenesisValidatorsRoot,
Web3SignerForkedBeaconBlock.init(forked1))
request2 = Web3SignerRequest.init(SigningFork, GenesisValidatorsRoot,
@ -1220,9 +1220,9 @@ block:
let
fork = ConsensusFork.Deneb
forked1 = getBlock(fork)
blockRoot1 = withBlck(forked1): hash_tree_root(blck)
blockRoot1 = withBlck(forked1): hash_tree_root(forkyBlck)
forked2 = getBlock(fork, SigningOtherFeeRecipient)
blockRoot2 = withBlck(forked2): hash_tree_root(blck)
blockRoot2 = withBlck(forked2): hash_tree_root(forkyBlck)
request1 = Web3SignerRequest.init(SigningFork, GenesisValidatorsRoot,
Web3SignerForkedBeaconBlock.init(forked1))
request2 = Web3SignerRequest.init(SigningFork, GenesisValidatorsRoot,

View File

@ -724,7 +724,7 @@ suite "SyncManager test suite":
let sblock = await aq.popFirst()
if sblock.blck.slot == Slot(counter):
withBlck(sblock.blck):
if blck.message.proposer_index == 0xDEADBEAF'u64:
if forkyBlck.message.proposer_index == 0xDEADBEAF'u64:
sblock.fail(VerifierError.MissingParent)
else:
inc(counter)
@ -782,7 +782,7 @@ suite "SyncManager test suite":
var missingSlice = chain.getSlice(startSlot, r13)
withBlck(missingSlice[0][]):
blck.message.proposer_index = 0xDEADBEAF'u64
forkyBlck.message.proposer_index = 0xDEADBEAF'u64
var f13 = queue.push(r13, missingSlice,
Opt.none(seq[BlobSidecars]))
await allFutures(f13, f14)
@ -897,7 +897,7 @@ suite "SyncManager test suite":
let sblock = await aq.popFirst()
if sblock.blck.slot == Slot(counter):
withBlck(sblock.blck):
if blck.message.proposer_index == 0xDEADBEAF'u64:
if forkyBlck.message.proposer_index == 0xDEADBEAF'u64:
sblock.fail(VerifierError.MissingParent)
else:
lastSafeSlot = sblock.blck.slot
@ -954,7 +954,7 @@ suite "SyncManager test suite":
var missingSlice = chain.getSlice(startSlot, r13)
withBlck(missingSlice[0][]):
blck.message.proposer_index = 0xDEADBEAF'u64
forkyBlck.message.proposer_index = 0xDEADBEAF'u64
var f13 = queue.push(r13, missingSlice, Opt.none(seq[BlobSidecars]))
await allFutures(f13, f14)
check:

View File

@ -74,7 +74,7 @@ func signBlock(
fork: Fork, genesis_validators_root: Eth2Digest, forked: ForkedBeaconBlock,
privKey: ValidatorPrivKey, flags: UpdateFlags = {}): ForkedSignedBeaconBlock =
let
slot = withBlck(forked): blck.slot
slot = withBlck(forked): forkyBlck.slot
root = hash_tree_root(forked)
signature =
if skipBlsValidation notin flags: