use ForkedTrustedSignedBeaconBlock (#2720)

* use ForkedTrustedSignedBeaconBlock

* remove --subscribe-all-subnets

* https://ethereum.github.io/eth2.0-APIs/#/Beacon/getBlock implementation was passing through forked beaconblocks
This commit is contained in:
tersec 2021-07-14 12:18:52 +00:00 committed by GitHub
parent bb0a086be3
commit e4afc36d71
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
15 changed files with 142 additions and 79 deletions

View File

@ -50,8 +50,6 @@ type
vcProcess*: Process
requestManager*: RequestManager
syncManager*: SyncManager[Peer, PeerID]
topicBeaconBlocks*: string
topicAggregateAndProofs*: string
genesisSnapshotContent*: string
attestationSubnets*: AttestationSubnets
processor*: ref Eth2Processor

View File

@ -64,7 +64,7 @@ proc init*(T: type AttestationPool, dag: ChainDAGRef, quarantine: QuarantineRef)
var epochRef = finalizedEpochRef
for i in 0..<blocks.len:
let
blck = blocks[blocks.len - i - 1]
blckRef = blocks[blocks.len - i - 1]
status =
if i < (blocks.len - ForkChoiceHorizon) and (i mod 1024 != 0):
# Fork choice needs to know about the full block tree up to the
@ -74,13 +74,14 @@ proc init*(T: type AttestationPool, dag: ChainDAGRef, quarantine: QuarantineRef)
# and then to make sure the fork choice data structure doesn't grow
# too big - getting an EpochRef can be expensive.
forkChoice.backend.process_block(
blck.root, blck.parent.root,
blckRef.root, blckRef.parent.root,
epochRef.current_justified_checkpoint.epoch,
epochRef.finalized_checkpoint.epoch)
else:
epochRef = dag.getEpochRef(blck, blck.slot.epoch)
forkChoice.process_block(
dag, epochRef, blck, dag.get(blck).data.message, blck.slot)
epochRef = dag.getEpochRef(blckRef, blckRef.slot.epoch)
withBlck(dag.get(blckRef).data):
forkChoice.process_block(
dag, epochRef, blckRef, blck.message, blckRef.slot)
doAssert status.isOk(), "Error in preloading the fork choice: " & $status.error

View File

@ -212,7 +212,7 @@ type
BlockData* = object
## Body and graph in one
data*: phase0.TrustedSignedBeaconBlock # We trust all blocks we have a ref for
data*: ForkedTrustedSignedBeaconBlock # We trust all blocks we have a ref for
refs*: BlockRef
StateData* = object

View File

@ -425,6 +425,7 @@ proc init*(T: type ChainDAGRef,
cfg: cfg,
)
doAssert cfg.GENESIS_FORK_VERSION != cfg.ALTAIR_FORK_VERSION
doAssert dag.updateFlags in [{}, {verifyFinalization}]
var cache: StateCache
@ -657,15 +658,6 @@ func getBlockBySlot*(dag: ChainDAGRef, slot: Slot): BlockRef =
## with slot number less or equal to `slot`.
dag.head.atSlot(slot).blck
proc get*(dag: ChainDAGRef, blck: BlockRef): BlockData =
## Retrieve the associated block body of a block reference
doAssert (not blck.isNil), "Trying to get nil BlockRef"
let data = dag.db.getBlock(blck.root)
doAssert data.isSome, "BlockRef without backing data, database corrupt?"
BlockData(data: data.get(), refs: blck)
proc getForkedBlock*(dag: ChainDAGRef, blck: BlockRef): ForkedTrustedSignedBeaconBlock =
# TODO implement this properly
let phase0Block = dag.db.getBlock(blck.root)
@ -680,6 +672,12 @@ proc getForkedBlock*(dag: ChainDAGRef, blck: BlockRef): ForkedTrustedSignedBeaco
raiseAssert "BlockRef without backing data, database corrupt?"
proc get*(dag: ChainDAGRef, blck: BlockRef): BlockData =
## Retrieve the associated block body of a block reference
doAssert (not blck.isNil), "Trying to get nil BlockRef"
BlockData(data: dag.getForkedBlock(blck), refs: blck)
proc get*(dag: ChainDAGRef, root: Eth2Digest): Option[BlockData] =
## Retrieve a resolved block reference and its associated body, if available
let refs = dag.getRef(root)
@ -723,9 +721,17 @@ proc applyBlock(
loadStateCache(dag, cache, state.blck, getStateField(state.data, slot).epoch)
let ok = state_transition(
dag.cfg, state.data, blck.data,
cache, rewards, flags + dag.updateFlags + {slotProcessed}, restore)
# TODO some abstractions
let ok =
case blck.data.kind:
of BeaconBlockFork.Phase0:
state_transition(
dag.cfg, state.data, blck.data.phase0Block,
cache, rewards, flags + dag.updateFlags + {slotProcessed}, restore)
of BeaconBlockFork.Altair:
state_transition(
dag.cfg, state.data, blck.data.altairBlock,
cache, rewards, flags + dag.updateFlags + {slotProcessed}, restore)
if ok:
state.blck = blck.refs

View File

@ -563,13 +563,13 @@ proc isValidBeaconBlock*(
if not slotBlockRef.isNil:
let blck = dag.get(slotBlockRef).data
if blck.message.proposer_index ==
if getForkedBlockField(blck, proposer_index) ==
signed_beacon_block.message.proposer_index and
blck.message.slot == signed_beacon_block.message.slot and
getForkedBlockField(blck, slot) == signed_beacon_block.message.slot and
blck.signature.toRaw() != signed_beacon_block.signature.toRaw():
notice "block isn't first block with valid signature received for the proposer",
blckRef = slotBlockRef,
existing_block = shortLog(blck.message)
blckRef = slotBlockRef
#existing_block = shortLog(blck.message)
return err((ValidationResult.Ignore, Invalid))
# [IGNORE] The block's parent (defined by block.parent_root) has been seen

View File

@ -313,9 +313,6 @@ proc init*(T: type BeaconNode,
network = createEth2Node(
rng, config, netKeys, cfg, dag.forkDigests,
getStateField(dag.headState.data, genesis_validators_root))
# TODO altair-transition
topicBeaconBlocks = getBeaconBlocksTopic(dag.forkDigests.phase0)
topicAggregateAndProofs = getAggregateAndProofsTopic(dag.forkDigests.phase0)
attestationPool = newClone(AttestationPool.init(dag, quarantine))
exitPool = newClone(ExitPool.init(dag, quarantine))
@ -365,8 +362,6 @@ proc init*(T: type BeaconNode,
eth1Monitor: eth1Monitor,
rpcServer: rpcServer,
restServer: restServer,
topicBeaconBlocks: topicBeaconBlocks,
topicAggregateAndProofs: topicAggregateAndProofs,
processor: processor,
blockProcessor: blockProcessor,
consensusManager: consensusManager,
@ -379,7 +374,7 @@ proc init*(T: type BeaconNode,
# TODO altair-transition
var
topics = @[
topicBeaconBlocks,
getBeaconBlocksTopic(network.forkDigests.phase0),
getAttesterSlashingsTopic(network.forkDigests.phase0),
getProposerSlashingsTopic(network.forkDigests.phase0),
getVoluntaryExitsTopic(network.forkDigests.phase0),
@ -751,7 +746,8 @@ proc addMessageHandlers(node: BeaconNode) {.raises: [Defect, CatchableError].} =
basicParams.validateParameters.tryGet()
# TODO altair-transition
node.network.subscribe(node.topicBeaconBlocks, blocksTopicParams, enableTopicMetrics = true)
node.network.subscribe(
getBeaconBlocksTopic(node.dag.forkDigests.phase0), blocksTopicParams, enableTopicMetrics = true)
node.network.subscribe(getAttesterSlashingsTopic(node.dag.forkDigests.phase0), basicParams)
node.network.subscribe(getProposerSlashingsTopic(node.dag.forkDigests.phase0), basicParams)
node.network.subscribe(getVoluntaryExitsTopic(node.dag.forkDigests.phase0), basicParams)
@ -1121,7 +1117,7 @@ proc installMessageValidators(node: BeaconNode) =
node.processor.aggregateValidator(signedAggregateAndProof))
node.network.addValidator(
node.topicBeaconBlocks,
getBeaconBlocksTopic(node.dag.forkDigests.phase0),
proc (signedBlock: SignedBeaconBlock): ValidationResult =
node.processor[].blockValidator(signedBlock))

View File

@ -385,7 +385,8 @@ proc installBeaconApiHandlers*(rpcServer: RpcServer, node: BeaconNode) {.
blockId: string) ->
tuple[canonical: bool, header: SignedBeaconBlockHeader]:
let bd = node.getBlockDataFromBlockId(blockId)
let tsbb = bd.data
# TODO check for Altair blocks and fail, because /v1/
let tsbb = bd.data.phase0Block
static: doAssert tsbb.signature is TrustedSig and
sizeof(ValidatorSig) == sizeof(tsbb.signature)
result.header.signature = cast[ValidatorSig](tsbb.signature)
@ -404,7 +405,8 @@ proc installBeaconApiHandlers*(rpcServer: RpcServer, node: BeaconNode) {.
"Beacon node is currently syncing, try again later.")
let head = node.dag.head
if head.slot >= blck.message.slot:
# TODO altair-transition
# TODO altair-transition, but not immediate testnet-priority to detect
# Altair and fail, since /v1/ doesn't support Altair
let blocksTopic = getBeaconBlocksTopic(node.dag.forkDigests.phase0)
node.network.broadcast(blocksTopic, blck)
# The block failed validation, but was successfully broadcast anyway.
@ -413,7 +415,7 @@ proc installBeaconApiHandlers*(rpcServer: RpcServer, node: BeaconNode) {.
else:
let res = await proposeSignedBlock(node, head, AttachedValidator(), blck)
if res == head:
# TODO altair-transition
# TODO altair-transition, but not immediate testnet-priority
let blocksTopic = getBeaconBlocksTopic(node.dag.forkDigests.phase0)
node.network.broadcast(blocksTopic, blck)
# The block failed validation, but was successfully broadcast anyway.
@ -426,15 +428,18 @@ proc installBeaconApiHandlers*(rpcServer: RpcServer, node: BeaconNode) {.
rpcServer.rpc("get_v1_beacon_blocks_blockId") do (
blockId: string) -> TrustedSignedBeaconBlock:
return node.getBlockDataFromBlockId(blockId).data
# TODO detect Altair and fail: /v1/ APIs don't support Altair
return node.getBlockDataFromBlockId(blockId).data.phase0Block
rpcServer.rpc("get_v1_beacon_blocks_blockId_root") do (
blockId: string) -> Eth2Digest:
return node.getBlockDataFromBlockId(blockId).data.message.state_root
# TODO detect Altair and fail: /v1/ APIs don't support Altair
return node.getBlockDataFromBlockId(blockId).data.phase0Block.message.state_root
rpcServer.rpc("get_v1_beacon_blocks_blockId_attestations") do (
blockId: string) -> seq[TrustedAttestation]:
return node.getBlockDataFromBlockId(blockId).data.message.body.attestations.asSeq
# TODO detect Altair and fail: /v1/ APIs don't support Altair
return node.getBlockDataFromBlockId(blockId).data.phase0Block.message.body.attestations.asSeq
rpcServer.rpc("get_v1_beacon_pool_attestations") do (
slot: Option[uint64], committee_index: Option[uint64]) ->

View File

@ -539,17 +539,19 @@ proc installBeaconApiHandlers*(router: var RestRouter, node: BeaconNode) =
return RestApiResponse.jsonResponse(
(
root: bdata.data.root,
# TODO Altair insofar as it should detect the error condition rather
# than crashing. This API is only specified for phase 0
root: bdata.data.phase0Block.root,
canonical: bdata.refs.isAncestorOf(node.dag.head),
header: (
message: (
slot: bdata.data.message.slot,
proposer_index: bdata.data.message.proposer_index,
parent_root: bdata.data.message.parent_root,
state_root: bdata.data.message.state_root,
body_root: bdata.data.message.body.hash_tree_root()
slot: bdata.data.phase0Block.message.slot,
proposer_index: bdata.data.phase0Block.message.proposer_index,
parent_root: bdata.data.phase0Block.message.parent_root,
state_root: bdata.data.phase0Block.message.state_root,
body_root: bdata.data.phase0Block.message.body.hash_tree_root()
),
signature: bdata.data.signature
signature: bdata.data.phase0Block.signature
)
)
)
@ -569,17 +571,19 @@ proc installBeaconApiHandlers*(router: var RestRouter, node: BeaconNode) =
return RestApiResponse.jsonResponse(
(
root: bdata.data.root,
# TODO for Altair, check that it's a phase 0 block and return error if
# not, since /v1/ APIs don't support Altair
root: bdata.data.phase0Block.root,
canonical: bdata.refs.isAncestorOf(node.dag.head),
header: (
message: (
slot: bdata.data.message.slot,
proposer_index: bdata.data.message.proposer_index,
parent_root: bdata.data.message.parent_root,
state_root: bdata.data.message.state_root,
body_root: bdata.data.message.body.hash_tree_root()
slot: bdata.data.phase0Block.message.slot,
proposer_index: bdata.data.phase0Block.message.proposer_index,
parent_root: bdata.data.phase0Block.message.parent_root,
state_root: bdata.data.phase0Block.message.state_root,
body_root: bdata.data.phase0Block.message.body.hash_tree_root()
),
signature: bdata.data.signature
signature: bdata.data.phase0Block.signature
)
)
)
@ -606,14 +610,14 @@ proc installBeaconApiHandlers*(router: var RestRouter, node: BeaconNode) =
return RestApiResponse.jsonError(Http503, BeaconNodeInSyncError)
if head.slot >= blck.message.slot:
# TODO altair-transition
# TODO altair-transition, but not for immediate testnet-priority
let blocksTopic = getBeaconBlocksTopic(node.dag.forkDigests.phase0)
node.network.broadcast(blocksTopic, blck)
return RestApiResponse.jsonError(Http202, BlockValidationError)
else:
let res = await proposeSignedBlock(node, head, AttachedValidator(), blck)
if res == head:
# TODO altair-transition
# TODO altair-transition, but not for immediate testnet-priority
let blocksTopic = getBeaconBlocksTopic(node.dag.forkDigests.phase0)
node.network.broadcast(blocksTopic, blck)
return RestApiResponse.jsonError(Http202, BlockValidationError)
@ -632,7 +636,8 @@ proc installBeaconApiHandlers*(router: var RestRouter, node: BeaconNode) =
if res.isErr():
return RestApiResponse.jsonError(Http404, BlockNotFoundError)
res.get()
return RestApiResponse.jsonResponse(bdata.data)
static: doAssert bdata.data.phase0Block is TrustedSignedBeaconBlock
return RestApiResponse.jsonResponse(bdata.data.phase0Block)
# https://ethereum.github.io/eth2.0-APIs/#/Beacon/getBlockRoot
router.api(MethodGet, "/api/eth/v1/beacon/blocks/{block_id}/root") do (
@ -646,7 +651,8 @@ proc installBeaconApiHandlers*(router: var RestRouter, node: BeaconNode) =
if res.isErr():
return RestApiResponse.jsonError(Http404, BlockNotFoundError)
res.get()
return RestApiResponse.jsonResponse((root: bdata.data.root))
# TODO check whether block is altair, and if so, return error
return RestApiResponse.jsonResponse((root: bdata.data.phase0Block.root))
# https://ethereum.github.io/eth2.0-APIs/#/Beacon/getBlockAttestations
router.api(MethodGet,
@ -661,8 +667,9 @@ proc installBeaconApiHandlers*(router: var RestRouter, node: BeaconNode) =
if res.isErr():
return RestApiResponse.jsonError(Http404, BlockNotFoundError)
res.get()
# TODO check whether block is altair, and if so, return error
return RestApiResponse.jsonResponse(
bdata.data.message.body.attestations.asSeq()
bdata.data.phase0Block.message.body.attestations.asSeq()
)
# https://ethereum.github.io/eth2.0-APIs/#/Beacon/getPoolAttestations

View File

@ -56,7 +56,8 @@ proc installValidatorApiHandlers*(rpcServer: RpcServer, node: BeaconNode) {.
if head.slot >= body.message.slot:
raise newException(CatchableError,
"Proposal is for a past slot: " & $body.message.slot)
if head == await proposeSignedBlock(node, head, AttachedValidator(), body):
if head == await proposeSignedBlock(
node, head, AttachedValidator(), body):
raise newException(CatchableError, "Could not propose block")
return true
@ -79,7 +80,8 @@ proc installValidatorApiHandlers*(rpcServer: RpcServer, node: BeaconNode) {.
rpcServer.rpc("post_v1_validator_aggregate_and_proofs") do (
payload: SignedAggregateAndProof) -> bool:
debug "post_v1_validator_aggregate_and_proofs"
node.network.broadcast(node.topicAggregateAndProofs, payload)
node.network.broadcast(
getAggregateAndProofsTopic(node.dag.forkDigests.phase0), payload)
notice "Aggregated attestation sent",
attestation = shortLog(payload.message.aggregate)

View File

@ -288,7 +288,8 @@ proc installValidatorApiHandlers*(router: var RestRouter, node: BeaconNode) =
return RestApiResponse.jsonError(Http400,
AggregateAndProofValidationError,
$res.error())
node.network.broadcast(node.topicAggregateAndProofs, item)
node.network.broadcast(
getAggregateAndProofsTopic(node.dag.forkDigests.phase0), item)
return RestApiResponse.jsonMsgResponse(AggregateAndProofValidationSuccess)

View File

@ -51,6 +51,16 @@ type
ForkDigestsRef* = ref ForkDigests
template init*(T: type ForkedSignedBeaconBlock, blck: phase0.SignedBeaconBlock): T =
T(kind: BeaconBlockFork.Phase0, phase0Block: blck)
template init*(T: type ForkedSignedBeaconBlock, blck: altair.SignedBeaconBlock): T =
T(kind: BeaconBlockFork.Altair, altairBlock: blck)
template init*(T: type ForkedTrustedSignedBeaconBlock, blck: phase0.TrustedSignedBeaconBlock): T =
T(kind: BeaconBlockFork.Phase0, phase0Block: blck)
template init*(T: type ForkedTrustedSignedBeaconBlock, blck: altair.TrustedSignedBeaconBlock): T =
T(kind: BeaconBlockFork.Altair, altairBlock: blck)
# State-related functionality based on ForkedHashedBeaconState instead of BeaconState
# Dispatch functions
@ -219,21 +229,52 @@ func init*(T: type ForkDigests,
template asSigned*(x: phase0.TrustedSignedBeaconBlock or phase0.SigVerifiedBeaconBlock):
phase0.SignedBeaconBlock =
static: # TODO See isomorphicCast
doAssert sizeof(x) == sizeof(phase0.SignedBeaconBlock)
cast[ptr phase0.SignedBeaconBlock](x.unsafeAddr)[]
isomorphicCast[phase0.SignedBeaconBlock](x)
template asSigned*(x: altair.TrustedSignedBeaconBlock or altair.SigVerifiedBeaconBlock):
altair.SignedBeaconBlock =
static: # TODO See isomorphicCast
doAssert sizeof(x) == sizeof(altair.SignedBeaconBlock)
cast[ptr altair.SignedBeaconBlock](x.unsafeAddr)[]
isomorphicCast[altair.SignedBeaconBlock](x)
template asSigned*(x: ForkedTrustedSignedBeaconBlock): ForkedSignedBeaconBlock =
static: # TODO See isomorphicCast
doAssert sizeof(x) == sizeof(ForkedSignedBeaconBlock)
isomorphicCast[ForkedSignedBeaconBlock](x)
cast[ptr ForkedSignedBeaconBlock](x.unsafeAddr)[]
template asTrusted*(x: phase0.SignedBeaconBlock or phase0.SigVerifiedBeaconBlock):
phase0.TrustedSignedBeaconBlock =
isomorphicCast[phase0.TrustedSignedBeaconBlock](x)
template asTrusted*(x: altair.SignedBeaconBlock or altair.SigVerifiedBeaconBlock):
altair.TrustedSignedBeaconBlock =
isomorphicCast[altair.TrustedSignedBeaconBlock](x)
template asTrusted*(x: ForkedSignedBeaconBlock): ForkedSignedBeaconBlock =
isomorphicCast[ForkedTrustedSignedBeaconBlock](x)
template withBlck*(x: ForkedSignedBeaconBlock | ForkedTrustedSignedBeaconBlock, body: untyped): untyped =
case x.kind
of BeaconBlockFork.Phase0:
template blck: untyped = x.phase0Block
body
of BeaconBlockFork.Altair:
template blck: untyped = x.altairBlock
body
template getForkedBlockField*(x: ForkedSignedBeaconBlock | ForkedTrustedSignedBeaconBlock, y: untyped): untyped =
withBlck(x): blck.message.y
template signature*(x: ForkedSignedBeaconBlock): ValidatorSig =
withBlck(x): blck.signature
template signature*(x: ForkedTrustedSignedBeaconBlock): TrustedSig =
withBlck(x): blck.signature
template root*(x: ForkedSignedBeaconBlock | ForkedTrustedSignedBeaconBlock): Eth2Digest =
withBlck(x): blck.root
template slot*(x: ForkedSignedBeaconBlock | ForkedTrustedSignedBeaconBlock): Slot =
getForkedBlockField(x, slot)
func shortLog*(x: ForkedSignedBeaconBlock | ForkedTrustedSignedBeaconBlock): auto =
withBlck(x): shortLog(blck.message)
chronicles.formatIt ForkedSignedBeaconBlock: it.shortLog
chronicles.formatIt ForkedTrustedSignedBeaconBlock: it.shortLog

View File

@ -225,7 +225,7 @@ proc state_transition_block_aux(
state: var SomeHashedBeaconState,
signedBlock: phase0.SignedBeaconBlock | phase0.SigVerifiedSignedBeaconBlock |
phase0.TrustedSignedBeaconBlock | altair.SignedBeaconBlock |
altair.SigVerifiedSignedBeaconBlock,
altair.SigVerifiedSignedBeaconBlock | altair.TrustedSignedBeaconBlock,
cache: var StateCache, flags: UpdateFlags): bool {.nbench.} =
# Block updates - these happen when there's a new block being suggested
# by the block proposer. Every actor in the network will update its state
@ -274,7 +274,8 @@ proc state_transition_block*(
state: var ForkedHashedBeaconState,
signedBlock: phase0.SignedBeaconBlock | phase0.SigVerifiedSignedBeaconBlock |
phase0.TrustedSignedBeaconBlock |
altair.SignedBeaconBlock | altair.SigVerifiedSignedBeaconBlock,
altair.SignedBeaconBlock | altair.SigVerifiedSignedBeaconBlock |
altair.TrustedSignedBeaconBlock,
cache: var StateCache, flags: UpdateFlags,
rollback: RollbackForkedHashedProc): bool {.nbench.} =
## `rollback` is called if the transition fails and the given state has been
@ -303,7 +304,8 @@ proc state_transition*(
cfg: RuntimeConfig,
state: var ForkedHashedBeaconState,
signedBlock: phase0.SignedBeaconBlock | phase0.SigVerifiedSignedBeaconBlock |
phase0.TrustedSignedBeaconBlock | altair.SignedBeaconBlock,
phase0.TrustedSignedBeaconBlock | altair.SignedBeaconBlock |
altair.TrustedSignedBeaconBlock,
cache: var StateCache, rewards: var RewardInfo, flags: UpdateFlags,
rollback: RollbackForkedHashedProc): bool {.nbench.} =
## Apply a block to the state, advancing the slot counter as necessary. The

View File

@ -230,7 +230,8 @@ p2pProtocol BeaconSync(version = 1,
trace "wrote response block",
slot = blocks[i].slot, roor = shortLog(blocks[i].root)
let blk = dag.get(blocks[i]).data
await response.write(blk.asSigned)
# TODO Altair
await response.write(blk.phase0Block.asSigned)
debug "Block range request done",
peer, startSlot, count, reqStep, found = count - startIndex
@ -259,7 +260,8 @@ p2pProtocol BeaconSync(version = 1,
let blockRef = dag.getRef(blockRoots[i])
if not isNil(blockRef):
let blk = dag.get(blockRef).data
await response.write(blk.asSigned)
# TODO Altair
await response.write(blk.phase0Block.asSigned)
inc found
peer.updateRequestQuota(found.float * blockResponseCost)

View File

@ -367,7 +367,8 @@ proc proposeSignedBlock*(node: BeaconNode,
if node.config.dumpEnabled:
dump(node.config.dumpDirOutgoing, newBlock)
node.network.broadcast(node.topicBeaconBlocks, newBlock)
node.network.broadcast(
getBeaconBlocksTopic(node.dag.forkDigests.phase0), newBlock)
beacon_blocks_proposed.inc()
@ -574,7 +575,8 @@ proc broadcastAggregatedAttestations(
var signedAP = SignedAggregateAndProof(
message: aggregateAndProof.get,
signature: sig)
node.network.broadcast(node.topicAggregateAndProofs, signedAP)
node.network.broadcast(
getAggregateAndProofsTopic(node.dag.forkDigests.phase0), signedAP)
notice "Aggregated attestation sent",
attestation = shortLog(signedAP.message.aggregate),
validator = shortLog(curr[0].v),

View File

@ -514,7 +514,7 @@ suite "chain DAG finalization tests" & preset():
assign(tmpStateData[], dag.headState)
dag.updateStateData(tmpStateData[], cur.atSlot(cur.slot), false, cache)
check:
dag.get(cur).data.message.state_root == getStateRoot(tmpStateData[].data)
dag.get(cur).data.phase0Block.message.state_root == getStateRoot(tmpStateData[].data)
getStateRoot(tmpStateData[].data) == hash_tree_root(tmpStateData[].data)
cur = cur.parent