exit/validatorchange pool includes BLS to execution messages; REST support for new pool (#4519)

* exit/validatorchange pool includes BLS to execution messages; REST
support for new pool

* catch failed individual futures

* increase BLS changes bound and keep BLS seen consistent with subpool

* deque capacities should be powers of 2
This commit is contained in:
tersec 2023-01-19 22:00:40 +00:00 committed by GitHub
parent 6818120849
commit 819e007689
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
21 changed files with 429 additions and 162 deletions

View File

@ -177,14 +177,6 @@ OK: 5/5 Fail: 0/5 Skip: 0/5
+ Subnet query after ENR update OK
```
OK: 3/3 Fail: 0/3 Skip: 0/3
## Exit pool testing suite
```diff
+ addExitMessage/getAttesterSlashingMessage OK
+ addExitMessage/getProposerSlashingMessage OK
+ addExitMessage/getVoluntaryExitMessage OK
+ pre-pre-fork voluntary exit OK
```
OK: 4/4 Fail: 0/4 Skip: 0/4
## Fee recipient management [Beacon Node] [Preset: mainnet]
```diff
+ Configuring the fee recpient [Beacon Node] [Preset: mainnet] OK
@ -516,6 +508,16 @@ OK: 4/4 Fail: 0/4 Skip: 0/4
+ [SyncQueue] hasEndGap() test OK
```
OK: 23/23 Fail: 0/23 Skip: 0/23
## Validator change pool testing suite
```diff
+ addValidatorChangeMessage/getAttesterSlashingMessage OK
+ addValidatorChangeMessage/getBlsToExecutionChange (post-capella) OK
+ addValidatorChangeMessage/getBlsToExecutionChange (pre-capella) OK
+ addValidatorChangeMessage/getProposerSlashingMessage OK
+ addValidatorChangeMessage/getVoluntaryExitMessage OK
+ pre-pre-fork voluntary exit OK
```
OK: 6/6 Fail: 0/6 Skip: 0/6
## Validator pool
```diff
+ Activation after check OK
@ -621,4 +623,4 @@ OK: 2/2 Fail: 0/2 Skip: 0/2
OK: 9/9 Fail: 0/9 Skip: 0/9
---TOTAL---
OK: 346/351 Fail: 0/351 Skip: 5/351
OK: 348/353 Fail: 0/353 Skip: 5/353

View File

@ -67,7 +67,7 @@ type
attestationPool*: ref AttestationPool
syncCommitteeMsgPool*: ref SyncCommitteeMsgPool
lightClientPool*: ref LightClientPool
exitPool*: ref ExitPool
validatorChangePool*: ref ValidatorChangePool
eth1Monitor*: Eth1Monitor
payloadBuilderRestClient*: RestClientRef
restServer*: RestServerRef

View File

@ -29,13 +29,17 @@ const
PROPOSER_SLASHINGS_BOUND = MAX_PROPOSER_SLASHINGS * 4
VOLUNTARY_EXITS_BOUND = MAX_VOLUNTARY_EXITS * 4
# For Capella launch; scale back later
BLS_TO_EXECUTION_CHANGES_BOUND = 32768'u64
type
OnVoluntaryExitCallback =
proc(data: SignedVoluntaryExit) {.gcsafe, raises: [Defect].}
ExitPool* = object
## The exit pool tracks attester slashings, proposer slashings, and
## voluntary exits that could be added to a proposed block.
ValidatorChangePool* = object
## The validator change pool tracks attester slashings, proposer slashings,
## voluntary exits, and BLS to execution changes that could be added to a
## proposed block.
attester_slashings*: Deque[AttesterSlashing] ## \
## Not a function of chain DAG branch; just used as a FIFO queue for blocks
@ -46,6 +50,9 @@ type
voluntary_exits*: Deque[SignedVoluntaryExit] ## \
## Not a function of chain DAG branch; just used as a FIFO queue for blocks
bls_to_execution_changes*: Deque[SignedBLSToExecutionChange] ## \
## Not a function of chain DAG branch; just used as a FIFO queue for blocks
prior_seen_attester_slashed_indices*: HashSet[uint64] ## \
## Records attester-slashed indices seen.
@ -55,33 +62,47 @@ type
prior_seen_voluntary_exit_indices*: HashSet[uint64] ##\
## Records voluntary exit indices seen.
prior_seen_bls_to_execution_change_indices*: HashSet[uint64] ##\
## Records BLS to execution change indices seen.
dag*: ChainDAGRef
attestationPool*: ref AttestationPool
onVoluntaryExitReceived*: OnVoluntaryExitCallback
func init*(T: type ExitPool, dag: ChainDAGRef,
func init*(T: type ValidatorChangePool, dag: ChainDAGRef,
attestationPool: ref AttestationPool = nil,
onVoluntaryExit: OnVoluntaryExitCallback = nil): T =
## Initialize an ExitPool from the dag `headState`
## Initialize an ValidatorChangePool from the dag `headState`
T(
# Allow for filtering out some exit messages during block production
# Allow filtering some validator change messages during block production
attester_slashings:
initDeque[AttesterSlashing](initialSize = ATTESTER_SLASHINGS_BOUND.int),
proposer_slashings:
initDeque[ProposerSlashing](initialSize = PROPOSER_SLASHINGS_BOUND.int),
voluntary_exits:
initDeque[SignedVoluntaryExit](initialSize = VOLUNTARY_EXITS_BOUND.int),
bls_to_execution_changes:
# TODO scale-back to BLS_TO_EXECUTION_CHANGES_BOUND post-capella, but
# given large bound, allow to grow dynamically rather than statically
# allocate all at once
initDeque[SignedBLSToExecutionChange](initialSize = 1024),
dag: dag,
attestationPool: attestationPool,
onVoluntaryExitReceived: onVoluntaryExit
)
func addExitMessage(subpool: var auto, exitMessage, bound: auto) =
# Prefer newer to older exit messages
func addValidatorChangeMessage(
subpool: var auto, seenpool: var auto, validatorChangeMessage: auto,
bound: static[uint64]) =
# Prefer newer to older validator change messages
while subpool.lenu64 >= bound:
discard subpool.popFirst()
# TODO remove temporary workaround once capella happens
when bound == BLS_TO_EXECUTION_CHANGES_BOUND:
seenpool.excl subpool.popFirst().message.validator_index
else:
discard subpool.popFirst()
subpool.addLast(exitMessage)
subpool.addLast(validatorChangeMessage)
doAssert subpool.lenu64 <= bound
iterator getValidatorIndices(proposer_slashing: ProposerSlashing): uint64 =
@ -90,22 +111,29 @@ iterator getValidatorIndices(proposer_slashing: ProposerSlashing): uint64 =
iterator getValidatorIndices(voluntary_exit: SignedVoluntaryExit): uint64 =
yield voluntary_exit.message.validator_index
func isSeen*(pool: ExitPool, msg: AttesterSlashing): bool =
iterator getValidatorIndices(
bls_to_execution_change: SignedBLSToExecutionChange): uint64 =
yield bls_to_execution_change.message.validator_index
func isSeen*(pool: ValidatorChangePool, msg: AttesterSlashing): bool =
for idx in getValidatorIndices(msg):
# One index is enough!
if idx notin pool.prior_seen_attester_slashed_indices:
return false
true
func isSeen*(pool: ExitPool, msg: ProposerSlashing): bool =
func isSeen*(pool: ValidatorChangePool, msg: ProposerSlashing): bool =
msg.signed_header_1.message.proposer_index in
pool.prior_seen_proposer_slashed_indices
func isSeen*(pool: ExitPool, msg: SignedVoluntaryExit): bool =
msg.message.validator_index in
pool.prior_seen_voluntary_exit_indices
func isSeen*(pool: ValidatorChangePool, msg: SignedVoluntaryExit): bool =
msg.message.validator_index in pool.prior_seen_voluntary_exit_indices
func addMessage*(pool: var ExitPool, msg: AttesterSlashing) =
func isSeen*(pool: ValidatorChangePool, msg: SignedBLSToExecutionChange): bool =
msg.message.validator_index in
pool.prior_seen_bls_to_execution_change_indices
func addMessage*(pool: var ValidatorChangePool, msg: AttesterSlashing) =
for idx in getValidatorIndices(msg):
pool.prior_seen_attester_slashed_indices.incl idx
if pool.attestationPool != nil:
@ -113,17 +141,27 @@ func addMessage*(pool: var ExitPool, msg: AttesterSlashing) =
continue
pool.attestationPool.forkChoice.process_equivocation(i)
pool.attester_slashings.addExitMessage(msg, ATTESTER_SLASHINGS_BOUND)
pool.attester_slashings.addValidatorChangeMessage(
pool.prior_seen_attester_slashed_indices, msg, ATTESTER_SLASHINGS_BOUND)
func addMessage*(pool: var ExitPool, msg: ProposerSlashing) =
func addMessage*(pool: var ValidatorChangePool, msg: ProposerSlashing) =
pool.prior_seen_proposer_slashed_indices.incl(
msg.signed_header_1.message.proposer_index)
pool.proposer_slashings.addExitMessage(msg, PROPOSER_SLASHINGS_BOUND)
pool.proposer_slashings.addValidatorChangeMessage(
pool.prior_seen_proposer_slashed_indices, msg, PROPOSER_SLASHINGS_BOUND)
func addMessage*(pool: var ExitPool, msg: SignedVoluntaryExit) =
func addMessage*(pool: var ValidatorChangePool, msg: SignedVoluntaryExit) =
pool.prior_seen_voluntary_exit_indices.incl(
msg.message.validator_index)
pool.voluntary_exits.addExitMessage(msg, VOLUNTARY_EXITS_BOUND)
pool.voluntary_exits.addValidatorChangeMessage(
pool.prior_seen_voluntary_exit_indices, msg, VOLUNTARY_EXITS_BOUND)
func addMessage*(pool: var ValidatorChangePool, msg: SignedBLSToExecutionChange) =
pool.prior_seen_bls_to_execution_change_indices.incl(
msg.message.validator_index)
pool.bls_to_execution_changes.addValidatorChangeMessage(
pool.prior_seen_bls_to_execution_change_indices, msg,
BLS_TO_EXECUTION_CHANGES_BOUND)
proc validateExitMessage(
cfg: RuntimeConfig, state: ForkyBeaconState, msg: ProposerSlashing): bool =
@ -135,8 +173,13 @@ proc validateExitMessage(
cfg: RuntimeConfig, state: ForkyBeaconState, msg: SignedVoluntaryExit):
bool =
check_voluntary_exit(cfg, state, msg, {}).isOk
proc validateExitMessage(
cfg: RuntimeConfig, state: ForkyBeaconState,
msg: SignedBLSToExecutionChange): bool =
true
# TODO check_voluntary_exit(cfg, state, msg, {}).isOk
proc getExitMessagesForBlock(
proc getValidatorChangeMessagesForBlock(
subpool: var Deque, cfg: RuntimeConfig, state: ForkyBeaconState,
seen: var HashSet, output: var List) =
# Approach taken here is to simply collect messages, effectively, a circular
@ -178,18 +221,22 @@ proc getExitMessagesForBlock(
subpool.clear()
proc getBeaconBlockExits*(
pool: var ExitPool, cfg: RuntimeConfig, state: ForkyBeaconState):
BeaconBlockExits =
proc getBeaconBlockValidatorChanges*(
pool: var ValidatorChangePool, cfg: RuntimeConfig, state: ForkyBeaconState):
BeaconBlockValidatorChanges =
var
indices: HashSet[uint64]
res: BeaconBlockExits
res: BeaconBlockValidatorChanges
getExitMessagesForBlock(
getValidatorChangeMessagesForBlock(
pool.attester_slashings, cfg, state, indices, res.attester_slashings)
getExitMessagesForBlock(
getValidatorChangeMessagesForBlock(
pool.proposer_slashings, cfg, state, indices, res.proposer_slashings)
getExitMessagesForBlock(
getValidatorChangeMessagesForBlock(
pool.voluntary_exits, cfg, state, indices, res.voluntary_exits)
when typeof(state).toFork() >= BeaconStateFork.Capella:
getValidatorChangeMessagesForBlock(
pool.bls_to_execution_changes, cfg, state, indices,
res.bls_to_execution_changes)
res

View File

@ -22,7 +22,7 @@ from ../consensus_object_pools/consensus_manager import
runProposalForkchoiceUpdated, shouldSyncOptimistically, updateHead,
updateHeadWithExecution
from ../beacon_clock import GetBeaconTimeFn, toFloatSeconds
from ../consensus_object_pools/block_dag import BlockRef, root, slot
from ../consensus_object_pools/block_dag import BlockRef, root, shortLog, slot
from ../consensus_object_pools/block_pools_types import
EpochRef, VerifierError
from ../consensus_object_pools/block_quarantine import
@ -248,11 +248,9 @@ from ../consensus_object_pools/attestation_pool import
addForkChoice, selectOptimisticHead, BeaconHead
from ../consensus_object_pools/blockchain_dag import
is_optimistic, loadExecutionBlockRoot, markBlockVerified
from ../consensus_object_pools/block_dag import shortLog
from ../consensus_object_pools/spec_cache import get_attesting_indices
from ../spec/datatypes/phase0 import TrustedSignedBeaconBlock
from ../spec/datatypes/altair import SignedBeaconBlock
from ../spec/datatypes/bellatrix import SignedBeaconBlock
from eth/async_utils import awaitWithTimeout
from ../spec/datatypes/bellatrix import ExecutionPayload, SignedBeaconBlock

View File

@ -124,7 +124,7 @@ type
# Validated with no further verification required
# ----------------------------------------------------------------
exitPool: ref ExitPool
validatorChangePool: ref ValidatorChangePool
# Almost validated, pending cryptographic signature check
# ----------------------------------------------------------------
@ -151,7 +151,7 @@ proc new*(T: type Eth2Processor,
validatorMonitor: ref ValidatorMonitor,
dag: ChainDAGRef,
attestationPool: ref AttestationPool,
exitPool: ref ExitPool,
validatorChangePool: ref ValidatorChangePool,
validatorPool: ref ValidatorPool,
syncCommitteeMsgPool: ref SyncCommitteeMsgPool,
lightClientPool: ref LightClientPool,
@ -169,7 +169,7 @@ proc new*(T: type Eth2Processor,
validatorMonitor: validatorMonitor,
dag: dag,
attestationPool: attestationPool,
exitPool: exitPool,
validatorChangePool: validatorChangePool,
validatorPool: validatorPool,
syncCommitteeMsgPool: syncCommitteeMsgPool,
lightClientPool: lightClientPool,
@ -397,6 +397,23 @@ proc processSignedAggregateAndProof*(
err(v.error())
proc processBlsToExecutionChange*(
self: var Eth2Processor, src: MsgSource,
blsToExecutionChange: SignedBLSToExecutionChange): ValidationRes =
logScope:
blsToExecutionChange = shortLog(blsToExecutionChange)
debug "BLS to execution change received"
let v = self.validatorChangePool[].validateBlsToExecutionChange(
blsToExecutionChange, self.getCurrentBeaconTime().slotOrZero.epoch)
if v.isOk():
trace "BLS to execution change validated"
self.validatorChangePool[].addMessage(blsToExecutionChange)
v
proc processAttesterSlashing*(
self: var Eth2Processor, src: MsgSource,
attesterSlashing: AttesterSlashing): ValidationRes =
@ -405,12 +422,12 @@ proc processAttesterSlashing*(
debug "Attester slashing received"
let v = self.exitPool[].validateAttesterSlashing(attesterSlashing)
let v = self.validatorChangePool[].validateAttesterSlashing(attesterSlashing)
if v.isOk():
trace "Attester slashing validated"
self.exitPool[].addMessage(attesterSlashing)
self.validatorChangePool[].addMessage(attesterSlashing)
self.validatorMonitor[].registerAttesterSlashing(src, attesterSlashing)
@ -429,11 +446,11 @@ proc processProposerSlashing*(
debug "Proposer slashing received"
let v = self.exitPool[].validateProposerSlashing(proposerSlashing)
let v = self.validatorChangePool[].validateProposerSlashing(proposerSlashing)
if v.isOk():
trace "Proposer slashing validated"
self.exitPool[].addMessage(proposerSlashing)
self.validatorChangePool[].addMessage(proposerSlashing)
self.validatorMonitor[].registerProposerSlashing(src, proposerSlashing)
@ -452,11 +469,11 @@ proc processSignedVoluntaryExit*(
debug "Voluntary exit received"
let v = self.exitPool[].validateVoluntaryExit(signedVoluntaryExit)
let v = self.validatorChangePool[].validateVoluntaryExit(signedVoluntaryExit)
if v.isOk():
trace "Voluntary exit validated"
self.exitPool[].addMessage(signedVoluntaryExit)
self.validatorChangePool[].addMessage(signedVoluntaryExit)
self.validatorMonitor[].registerVoluntaryExit(
src, signedVoluntaryExit.message)

View File

@ -818,9 +818,37 @@ proc validateAggregate*(
return ok((attesting_indices, sig))
# https://github.com/ethereum/consensus-specs/blob/v1.3.0-rc.1/specs/capella/p2p-interface.md#bls_to_execution_change
proc validateBlsToExecutionChange*(
pool: ValidatorChangePool, signed_address_change: SignedBLSToExecutionChange,
wallEpoch: Epoch): Result[void, ValidationError] =
# [IGNORE] `current_epoch >= CAPELLA_FORK_EPOCH`, where `current_epoch` is
# defined by the current wall-clock time.
if not (wallEpoch >= pool.dag.cfg.CAPELLA_FORK_EPOCH):
return errIgnore("validateBlsToExecutionChange: not accepting gossip until Capella")
# [IGNORE] The `signed_bls_to_execution_change` is the first valid signed bls
# to execution change received for the validator with index
# `signed_bls_to_execution_change.message.validator_index`.
if pool.isSeen(signed_address_change):
return errIgnore("validateBlsToExecutionChange: not first signed BLS to execution change received for validator index")
# [REJECT] All of the conditions within `process_bls_to_execution_change`
# pass validation.
withState(pool.dag.headState):
when stateFork < BeaconStateFork.Capella:
return errIgnore("validateBlsToExecutionChange: can't validate against pre-Capella state")
else:
let res = check_bls_to_execution_change(
pool.dag.cfg, forkyState.data, signed_address_change)
if res.isErr:
return errReject(res.error)
ok()
# https://github.com/ethereum/consensus-specs/blob/v1.3.0-rc.1/specs/phase0/p2p-interface.md#attester_slashing
proc validateAttesterSlashing*(
pool: ExitPool, attester_slashing: AttesterSlashing):
pool: ValidatorChangePool, attester_slashing: AttesterSlashing):
Result[void, ValidationError] =
# [IGNORE] At least one index in the intersection of the attesting indices of
# each attestation has not yet been seen in any prior attester_slashing (i.e.
@ -843,7 +871,7 @@ proc validateAttesterSlashing*(
# https://github.com/ethereum/consensus-specs/blob/v1.3.0-rc.1/specs/phase0/p2p-interface.md#proposer_slashing
proc validateProposerSlashing*(
pool: ExitPool, proposer_slashing: ProposerSlashing):
pool: ValidatorChangePool, proposer_slashing: ProposerSlashing):
Result[void, ValidationError] =
# Not from spec; the rest of NBC wouldn't have correctly processed it either.
if proposer_slashing.signed_header_1.message.proposer_index > high(int).uint64:
@ -866,7 +894,7 @@ proc validateProposerSlashing*(
# https://github.com/ethereum/consensus-specs/blob/v1.3.0-rc.1/specs/phase0/p2p-interface.md#voluntary_exit
proc validateVoluntaryExit*(
pool: ExitPool, signed_voluntary_exit: SignedVoluntaryExit):
pool: ValidatorChangePool, signed_voluntary_exit: SignedVoluntaryExit):
Result[void, ValidationError] =
# [IGNORE] The voluntary exit is the first valid voluntary exit received for
# the validator with index signed_voluntary_exit.message.validator_index.

View File

@ -2623,6 +2623,13 @@ proc broadcastProposerSlashing*(
node.forkDigestAtEpoch(node.getWallEpoch))
node.broadcast(topic, slashing)
proc broadcastBlsToExecutionChange*(
node: Eth2Node, bls_to_execution_change: SignedBLSToExecutionChange):
Future[SendResult] =
let topic = getBlsToExecutionChangeTopic(
node.forkDigestAtEpoch(node.getWallEpoch))
node.broadcast(topic, bls_to_execution_change)
proc broadcastAggregateAndProof*(
node: Eth2Node, proof: SignedAggregateAndProof): Future[SendResult] =
let topic = getAggregateAndProofsTopic(

View File

@ -313,8 +313,8 @@ proc initFullNode(
SyncCommitteeMsgPool.init(rng, onSyncContribution))
lightClientPool = newClone(
LightClientPool())
exitPool = newClone(
ExitPool.init(dag, attestationPool, onVoluntaryExitAdded))
validatorChangePool = newClone(
ValidatorChangePool.init(dag, attestationPool, onVoluntaryExitAdded))
consensusManager = ConsensusManager.new(
dag, attestationPool, quarantine, node.eth1Monitor,
ActionTracker.init(rng, config.subscribeAllSubnets),
@ -335,9 +335,9 @@ proc initFullNode(
resfut
processor = Eth2Processor.new(
config.doppelgangerDetection,
blockProcessor, node.validatorMonitor, dag, attestationPool, exitPool,
node.attachedValidators, syncCommitteeMsgPool, lightClientPool,
quarantine, rng, getBeaconTime, taskpool)
blockProcessor, node.validatorMonitor, dag, attestationPool,
validatorChangePool, node.attachedValidators, syncCommitteeMsgPool,
lightClientPool, quarantine, rng, getBeaconTime, taskpool)
syncManager = newSyncManager[Peer, PeerId](
node.network.peerPool, SyncQueueKind.Forward, getLocalHeadSlot,
getLocalWallSlot, getFirstSlotAtFinalizedEpoch, getBackfillSlot,
@ -375,7 +375,7 @@ proc initFullNode(
node.attestationPool = attestationPool
node.syncCommitteeMsgPool = syncCommitteeMsgPool
node.lightClientPool = lightClientPool
node.exitPool = exitPool
node.validatorChangePool = validatorChangePool
node.processor = processor
node.blockProcessor = blockProcessor
node.consensusManager = consensusManager
@ -1463,7 +1463,7 @@ proc installMessageValidators(node: BeaconNode) =
toValidationResult(node.processor[].processSignedBeaconBlock(
MsgSource.gossip, signedBlock)))
if node.dag.cfg.EIP4844_FORK_EPOCH != FAR_FUTURE_EPOCH:
if node.dag.cfg.EIP4844_FORK_EPOCH != FAR_FUTURE_EPOCH:
node.network.addValidator(
getBeaconBlockAndBlobsSidecarTopic(forkDigests.eip4844),
proc (signedBlock: eip4844.SignedBeaconBlockAndBlobsSidecar): ValidationResult =
@ -1496,6 +1496,17 @@ proc installMessageValidators(node: BeaconNode) =
if node.dag.cfg.EIP4844_FORK_EPOCH != FAR_FUTURE_EPOCH:
installSyncCommitteeeValidators(forkDigests.eip4844)
template installBlsToExecutionChangeValidators(digest: auto) =
node.network.addValidator(
getBlsToExecutionChangeTopic(digest),
proc(msg: SignedBLSToExecutionChange): ValidationResult =
return toValidationResult(
node.processor[].processBlsToExecutionChange(MsgSource.gossip, msg)))
installBlsToExecutionChangeValidators(forkDigests.capella)
if node.dag.cfg.EIP4844_FORK_EPOCH != FAR_FUTURE_EPOCH:
installBlsToExecutionChangeValidators(forkDigests.eip4844)
node.installLightClientMessageValidators()
proc stop(node: BeaconNode) =

View File

@ -1,5 +1,5 @@
# beacon_chain
# Copyright (c) 2018-2022 Status Research & Development GmbH
# Copyright (c) 2018-2023 Status Research & Development GmbH
# Licensed and distributed under either of
# * MIT license (license terms in the root directory or at https://opensource.org/licenses/MIT).
# * Apache v2 license (license terms in the root directory or at https://www.apache.org/licenses/LICENSE-2.0).
@ -1065,7 +1065,8 @@ proc installBeaconApiHandlers*(router: var RestRouter, node: BeaconNode) =
# https://ethereum.github.io/beacon-APIs/#/Beacon/getPoolAttesterSlashings
router.api(MethodGet, "/eth/v1/beacon/pool/attester_slashings") do (
) -> RestApiResponse:
return RestApiResponse.jsonResponse(toSeq(node.exitPool.attester_slashings))
return RestApiResponse.jsonResponse(
toSeq(node.validatorChangePool.attester_slashings))
# https://ethereum.github.io/beacon-APIs/#/Beacon/submitPoolAttesterSlashings
router.api(MethodPost, "/eth/v1/beacon/pool/attester_slashings") do (
@ -1090,7 +1091,8 @@ proc installBeaconApiHandlers*(router: var RestRouter, node: BeaconNode) =
# https://ethereum.github.io/beacon-APIs/#/Beacon/getPoolProposerSlashings
router.api(MethodGet, "/eth/v1/beacon/pool/proposer_slashings") do (
) -> RestApiResponse:
return RestApiResponse.jsonResponse(toSeq(node.exitPool.proposer_slashings))
return RestApiResponse.jsonResponse(
toSeq(node.validatorChangePool.proposer_slashings))
# https://ethereum.github.io/beacon-APIs/#/Beacon/submitPoolProposerSlashings
router.api(MethodPost, "/eth/v1/beacon/pool/proposer_slashings") do (
@ -1112,6 +1114,42 @@ proc installBeaconApiHandlers*(router: var RestRouter, node: BeaconNode) =
$res.error())
return RestApiResponse.jsonMsgResponse(ProposerSlashingValidationSuccess)
# https://ethereum.github.io/beacon-APIs/?urls.primaryName=dev#/Beacon/getPoolBLSToExecutionChanges
# https://github.com/ethereum/beacon-APIs/blob/86850001845df9163da5ae9605dbf15cd318d5d0/apis/beacon/pool/bls_to_execution_changes.yaml
router.api(MethodGet, "/eth/v1/beacon/pool/bls_to_execution_changes") do (
) -> RestApiResponse:
return RestApiResponse.jsonResponse(
toSeq(node.validatorChangePool.bls_to_execution_changes))
# https://ethereum.github.io/beacon-APIs/?urls.primaryName=dev#/Beacon/submitPoolBLSToExecutionChange
# https://github.com/ethereum/beacon-APIs/blob/86850001845df9163da5ae9605dbf15cd318d5d0/apis/beacon/pool/bls_to_execution_changes.yaml
router.api(MethodPost, "/eth/v1/beacon/pool/bls_to_execution_changes") do (
contentBody: Option[ContentBody]) -> RestApiResponse:
let bls_to_execution_changes =
block:
if contentBody.isNone():
return RestApiResponse.jsonError(Http400, EmptyRequestBodyError)
let dres = decodeBody(seq[SignedBLSToExecutionChange], contentBody.get())
if dres.isErr():
return RestApiResponse.jsonError(Http400,
InvalidBlsToExecutionChangeObjectError,
$dres.error())
dres.get()
let res = await allFinished(mapIt(
bls_to_execution_changes, node.router.routeBlsToExecutionChange(it)))
for individual_res in res:
doAssert individual_res.finished()
if individual_res.failed():
return RestApiResponse.jsonError(Http400,
BlsToExecutionChangeValidationError,
$individual_res.error[].msg)
let fut_result = individual_res.read()
if fut_result.isErr():
return RestApiResponse.jsonError(Http400,
BlsToExecutionChangeValidationError,
$fut_result.error())
return RestApiResponse.jsonMsgResponse(BlsToExecutionChangeValidationSuccess)
# https://ethereum.github.io/beacon-APIs/#/Beacon/submitPoolSyncCommitteeSignatures
router.api(MethodPost, "/eth/v1/beacon/pool/sync_committees") do (
contentBody: Option[ContentBody]) -> RestApiResponse:
@ -1146,7 +1184,8 @@ proc installBeaconApiHandlers*(router: var RestRouter, node: BeaconNode) =
# https://ethereum.github.io/beacon-APIs/#/Beacon/getPoolVoluntaryExits
router.api(MethodGet, "/eth/v1/beacon/pool/voluntary_exits") do (
) -> RestApiResponse:
return RestApiResponse.jsonResponse(toSeq(node.exitPool.voluntary_exits))
return RestApiResponse.jsonResponse(
toSeq(node.validatorChangePool.voluntary_exits))
# https://ethereum.github.io/beacon-APIs/#/Beacon/submitPoolVoluntaryExit
router.api(MethodPost, "/eth/v1/beacon/pool/voluntary_exits") do (

View File

@ -1,5 +1,5 @@
# beacon_chain
# Copyright (c) 2021-2022 Status Research & Development GmbH
# Copyright (c) 2021-2023 Status Research & Development GmbH
# Licensed and distributed under either of
# * MIT license (license terms in the root directory or at https://opensource.org/licenses/MIT).
# * Apache v2 license (license terms in the root directory or at https://www.apache.org/licenses/LICENSE-2.0).
@ -36,19 +36,19 @@ const
AttestationValidationError* =
"Some errors happened while validating attestation(s)"
AttestationValidationSuccess* =
"Attestation object(s) was broadcasted"
"Attestation object(s) was broadcast"
InvalidAttesterSlashingObjectError* =
"Unable to decode attester slashing object(s)"
AttesterSlashingValidationError* =
"Invalid attester slashing, it will never pass validation so it's rejected"
AttesterSlashingValidationSuccess* =
"Attester slashing object was broadcasted"
"Attester slashing object was broadcast"
InvalidProposerSlashingObjectError* =
"Unable to decode proposer slashing object(s)"
ProposerSlashingValidationError* =
"Invalid proposer slashing, it will never pass validation so it's rejected"
ProposerSlashingValidationSuccess* =
"Proposer slashing object was broadcasted"
"Proposer slashing object was broadcast"
InvalidVoluntaryExitObjectError* =
"Unable to decode voluntary exit object(s)"
InvalidFeeRecipientRequestError* =
@ -56,14 +56,14 @@ const
VoluntaryExitValidationError* =
"Invalid voluntary exit, it will never pass validation so it's rejected"
VoluntaryExitValidationSuccess* =
"Voluntary exit object(s) was broadcasted"
"Voluntary exit object(s) was broadcast"
InvalidAggregateAndProofObjectError* =
"Unable to decode aggregate and proof object(s)"
AggregateAndProofValidationError* =
"Invalid aggregate and proof, it will never pass validation so it's " &
"rejected"
AggregateAndProofValidationSuccess* =
"Aggregate and proof object(s) was broadcasted"
"Aggregate and proof object(s) was broadcast"
BeaconCommitteeSubscriptionSuccess* =
"Beacon node processed committee subscription request(s)"
SyncCommitteeSubscriptionSuccess* =
@ -169,11 +169,11 @@ const
SyncCommitteeMessageValidationError* =
"Some errors happened while validating sync committee message(s)"
SyncCommitteeMessageValidationSuccess* =
"Sync committee message(s) was broadcasted"
"Sync committee message(s) was broadcast"
ContributionAndProofValidationError* =
"Some errors happened while validating contribution and proof(s)"
ContributionAndProofValidationSuccess* =
"Contribution and proof(s) was broadcasted"
"Contribution and proof(s) was broadcast"
ProduceContributionError* =
"Unable to produce contribution using the passed parameters"
InternalServerError* =
@ -221,3 +221,9 @@ const
"Validator inactive"
BlobsOutOfRange* =
"Requested slot is outside of blobs window"
InvalidBlsToExecutionChangeObjectError* =
"Unable to decode BLS to execution change object(s)"
BlsToExecutionChangeValidationError* =
"Invalid BLS to execution change; it won't validate, so it's rejected"
BlsToExecutionChangeValidationSuccess* =
"BLS to execution change was broadcast"

View File

@ -475,12 +475,6 @@ type
next_fork_version*: Version
next_fork_epoch*: Epoch
BeaconBlockExits* = object
# Collection of exits that are suitable for block production
proposer_slashings*: List[ProposerSlashing, Limit MAX_PROPOSER_SLASHINGS]
attester_slashings*: List[AttesterSlashing, Limit MAX_ATTESTER_SLASHINGS]
voluntary_exits*: List[SignedVoluntaryExit, Limit MAX_VOLUNTARY_EXITS]
AttnetBits* = BitArray[int ATTESTATION_SUBNET_COUNT]
type

View File

@ -39,7 +39,7 @@ const
type
SignedBLSToExecutionChangeList* =
List[SignedBLSToExecutionChange, MAX_BLS_TO_EXECUTION_CHANGES]
List[SignedBLSToExecutionChange, Limit MAX_BLS_TO_EXECUTION_CHANGES]
# https://github.com/ethereum/consensus-specs/blob/v1.3.0-rc.0/specs/capella/beacon-chain.md#withdrawal
Withdrawal* = object
@ -483,6 +483,14 @@ type
parentHash*: string
timestamp*: string
BeaconBlockValidatorChanges* = object
# Collection of exits that are suitable for block production
proposer_slashings*: List[ProposerSlashing, Limit MAX_PROPOSER_SLASHINGS]
attester_slashings*: List[AttesterSlashing, Limit MAX_ATTESTER_SLASHINGS]
voluntary_exits*: List[SignedVoluntaryExit, Limit MAX_VOLUNTARY_EXITS]
bls_to_execution_changes*:
List[SignedBLSToExecutionChange, Limit MAX_BLS_TO_EXECUTION_CHANGES]
func shortLog*(v: SomeBeaconBlock): auto =
(
slot: shortLog(v.slot),
@ -508,6 +516,19 @@ func shortLog*(v: SomeSignedBeaconBlock): auto =
signature: shortLog(v.signature)
)
func shortLog*(v: BLSToExecutionChange): auto =
(
validator_index: v.validator_index,
from_bls_pubkey: shortLog(v.from_bls_pubkey),
to_execution_address: v.to_execution_address
)
func shortLog*(v: SignedBLSToExecutionChange): auto =
(
bls_to_execution_change: shortLog(v.message),
signature: shortLog(v.signature)
)
# https://github.com/ethereum/consensus-specs/blob/v1.3.0-rc.1/specs/capella/light-client/sync-protocol.md#get_lc_execution_root
func get_lc_execution_root*(
header: LightClientHeader, cfg: RuntimeConfig): Eth2Digest =

View File

@ -344,10 +344,9 @@ template partialBeaconBlock*(
graffiti: GraffitiBytes,
attestations: seq[Attestation],
deposits: seq[Deposit],
exits: BeaconBlockExits,
validator_changes: BeaconBlockValidatorChanges,
sync_aggregate: SyncAggregate,
execution_payload: bellatrix.ExecutionPayload,
bls_to_execution_changes: SignedBLSToExecutionChangeList):
execution_payload: bellatrix.ExecutionPayload):
phase0.BeaconBlock =
phase0.BeaconBlock(
slot: state.data.slot,
@ -357,11 +356,11 @@ template partialBeaconBlock*(
randao_reveal: randao_reveal,
eth1_data: eth1data,
graffiti: graffiti,
proposer_slashings: exits.proposer_slashings,
attester_slashings: exits.attester_slashings,
proposer_slashings: validator_changes.proposer_slashings,
attester_slashings: validator_changes.attester_slashings,
attestations: List[Attestation, Limit MAX_ATTESTATIONS](attestations),
deposits: List[Deposit, Limit MAX_DEPOSITS](deposits),
voluntary_exits: exits.voluntary_exits))
voluntary_exits: validator_changes.voluntary_exits))
# https://github.com/ethereum/consensus-specs/blob/v1.3.0-rc.1/specs/altair/validator.md#preparing-a-beaconblock
template partialBeaconBlock*(
@ -373,10 +372,9 @@ template partialBeaconBlock*(
graffiti: GraffitiBytes,
attestations: seq[Attestation],
deposits: seq[Deposit],
exits: BeaconBlockExits,
validator_changes: BeaconBlockValidatorChanges,
sync_aggregate: SyncAggregate,
execution_payload: bellatrix.ExecutionPayload,
bls_to_execution_changes: SignedBLSToExecutionChangeList):
execution_payload: bellatrix.ExecutionPayload):
altair.BeaconBlock =
altair.BeaconBlock(
slot: state.data.slot,
@ -386,11 +384,11 @@ template partialBeaconBlock*(
randao_reveal: randao_reveal,
eth1_data: eth1data,
graffiti: graffiti,
proposer_slashings: exits.proposer_slashings,
attester_slashings: exits.attester_slashings,
proposer_slashings: validator_changes.proposer_slashings,
attester_slashings: validator_changes.attester_slashings,
attestations: List[Attestation, Limit MAX_ATTESTATIONS](attestations),
deposits: List[Deposit, Limit MAX_DEPOSITS](deposits),
voluntary_exits: exits.voluntary_exits,
voluntary_exits: validator_changes.voluntary_exits,
sync_aggregate: sync_aggregate))
# https://github.com/ethereum/consensus-specs/blob/v1.1.3/specs/merge/validator.md#block-proposal
@ -403,10 +401,9 @@ template partialBeaconBlock*(
graffiti: GraffitiBytes,
attestations: seq[Attestation],
deposits: seq[Deposit],
exits: BeaconBlockExits,
validator_changes: BeaconBlockValidatorChanges,
sync_aggregate: SyncAggregate,
execution_payload: bellatrix.ExecutionPayload,
bls_to_execution_changes: SignedBLSToExecutionChangeList):
execution_payload: bellatrix.ExecutionPayload):
bellatrix.BeaconBlock =
bellatrix.BeaconBlock(
slot: state.data.slot,
@ -416,11 +413,11 @@ template partialBeaconBlock*(
randao_reveal: randao_reveal,
eth1_data: eth1data,
graffiti: graffiti,
proposer_slashings: exits.proposer_slashings,
attester_slashings: exits.attester_slashings,
proposer_slashings: validator_changes.proposer_slashings,
attester_slashings: validator_changes.attester_slashings,
attestations: List[Attestation, Limit MAX_ATTESTATIONS](attestations),
deposits: List[Deposit, Limit MAX_DEPOSITS](deposits),
voluntary_exits: exits.voluntary_exits,
voluntary_exits: validator_changes.voluntary_exits,
sync_aggregate: sync_aggregate,
execution_payload: execution_payload))
@ -434,10 +431,9 @@ template partialBeaconBlock*(
graffiti: GraffitiBytes,
attestations: seq[Attestation],
deposits: seq[Deposit],
exits: BeaconBlockExits,
validator_changes: BeaconBlockValidatorChanges,
sync_aggregate: SyncAggregate,
execution_payload: capella.ExecutionPayload,
bls_to_execution_changes: SignedBLSToExecutionChangeList
):
capella.BeaconBlock =
capella.BeaconBlock(
@ -448,14 +444,14 @@ template partialBeaconBlock*(
randao_reveal: randao_reveal,
eth1_data: eth1data,
graffiti: graffiti,
proposer_slashings: exits.proposer_slashings,
attester_slashings: exits.attester_slashings,
proposer_slashings: validator_changes.proposer_slashings,
attester_slashings: validator_changes.attester_slashings,
attestations: List[Attestation, Limit MAX_ATTESTATIONS](attestations),
deposits: List[Deposit, Limit MAX_DEPOSITS](deposits),
voluntary_exits: exits.voluntary_exits,
voluntary_exits: validator_changes.voluntary_exits,
sync_aggregate: sync_aggregate,
execution_payload: execution_payload,
bls_to_execution_changes: bls_to_execution_changes
bls_to_execution_changes: validator_changes.bls_to_execution_changes
))
# https://github.com/ethereum/consensus-specs/blob/v1.1.3/specs/merge/validator.md#block-proposal
@ -468,10 +464,9 @@ template partialBeaconBlock*(
graffiti: GraffitiBytes,
attestations: seq[Attestation],
deposits: seq[Deposit],
exits: BeaconBlockExits,
validator_changes: BeaconBlockValidatorChanges,
sync_aggregate: SyncAggregate,
execution_payload: eip4844.ExecutionPayload,
bls_to_execution_changes: SignedBLSToExecutionChangeList
):
eip4844.BeaconBlock =
discard $eip4844ImplementationMissing & ": state_transition.nim: partialBeaconBlock, leaves additional fields default, okay for block_sim"
@ -483,14 +478,14 @@ template partialBeaconBlock*(
randao_reveal: randao_reveal,
eth1_data: eth1data,
graffiti: graffiti,
proposer_slashings: exits.proposer_slashings,
attester_slashings: exits.attester_slashings,
proposer_slashings: validator_changes.proposer_slashings,
attester_slashings: validator_changes.attester_slashings,
attestations: List[Attestation, Limit MAX_ATTESTATIONS](attestations),
deposits: List[Deposit, Limit MAX_DEPOSITS](deposits),
voluntary_exits: exits.voluntary_exits,
voluntary_exits: validator_changes.voluntary_exits,
sync_aggregate: sync_aggregate,
execution_payload: execution_payload,
bls_to_execution_changes: bls_to_execution_changes
bls_to_execution_changes: validator_changes.bls_to_execution_changes
))
proc makeBeaconBlock*[T: bellatrix.ExecutionPayload | capella.ExecutionPayload](
@ -502,7 +497,7 @@ proc makeBeaconBlock*[T: bellatrix.ExecutionPayload | capella.ExecutionPayload](
graffiti: GraffitiBytes,
attestations: seq[Attestation],
deposits: seq[Deposit],
exits: BeaconBlockExits,
validator_changes: BeaconBlockValidatorChanges,
sync_aggregate: SyncAggregate,
executionPayload: T,
bls_to_execution_changes: SignedBLSToExecutionChangeList,
@ -528,8 +523,8 @@ proc makeBeaconBlock*[T: bellatrix.ExecutionPayload | capella.ExecutionPayload](
ForkedBeaconBlock.init(
partialBeaconBlock(
cfg, state.`kind Data`, proposer_index, randao_reveal, eth1_data,
graffiti, attestations, deposits, exits, sync_aggregate,
executionPayload, bls_to_execution_changes))
graffiti, attestations, deposits, validator_changes, sync_aggregate,
executionPayload))
let res = process_block(
cfg, state.`kind Data`.data, blck.`kind Data`.asSigVerified(),
@ -549,15 +544,16 @@ proc makeBeaconBlock*[T: bellatrix.ExecutionPayload | capella.ExecutionPayload](
# Effectively hash_tree_root(ExecutionPayload) with the beacon block
# body, with the execution payload replaced by the execution payload
# header. htr(payload) == htr(payload header), so substitute.
discard $capellaImplementationMissing # need different htr to match capella changes
forkyState.data.latest_block_header.body_root = hash_tree_root(
[hash_tree_root(randao_reveal),
hash_tree_root(eth1_data),
hash_tree_root(graffiti),
hash_tree_root(exits.proposer_slashings),
hash_tree_root(exits.attester_slashings),
hash_tree_root(validator_changes.proposer_slashings),
hash_tree_root(validator_changes.attester_slashings),
hash_tree_root(List[Attestation, Limit MAX_ATTESTATIONS](attestations)),
hash_tree_root(List[Deposit, Limit MAX_DEPOSITS](deposits)),
hash_tree_root(exits.voluntary_exits),
hash_tree_root(validator_changes.voluntary_exits),
hash_tree_root(sync_aggregate),
execution_payload_root.get])
@ -594,9 +590,9 @@ proc makeBeaconBlock*[T](
proposer_index: ValidatorIndex, randao_reveal: ValidatorSig,
eth1_data: Eth1Data, graffiti: GraffitiBytes,
attestations: seq[Attestation], deposits: seq[Deposit],
exits: BeaconBlockExits, sync_aggregate: SyncAggregate,
exits: BeaconBlockValidatorChanges, sync_aggregate: SyncAggregate,
executionPayload: T,
bls_to_execution_changes: SignedBLSToExecutionChangeList,
bls_to_execution_changes: SignedBLSToExecutionChangeList, # TODO remove
rollback: RollbackForkedHashedProc, cache: var StateCache):
Result[ForkedBeaconBlock, cstring] =
makeBeaconBlock(
@ -612,9 +608,9 @@ proc makeBeaconBlock*[T](
proposer_index: ValidatorIndex, randao_reveal: ValidatorSig,
eth1_data: Eth1Data, graffiti: GraffitiBytes,
attestations: seq[Attestation], deposits: seq[Deposit],
exits: BeaconBlockExits, sync_aggregate: SyncAggregate,
exits: BeaconBlockValidatorChanges, sync_aggregate: SyncAggregate,
executionPayload: T,
bls_to_execution_changes: SignedBLSToExecutionChangeList,
bls_to_execution_changes: SignedBLSToExecutionChangeList, # TODO remove
rollback: RollbackForkedHashedProc,
cache: var StateCache, verificationFlags: UpdateFlags):
Result[ForkedBeaconBlock, cstring] =

View File

@ -391,8 +391,8 @@ proc process_voluntary_exit*(
ok()
# https://github.com/ethereum/consensus-specs/blob/v1.3.0-alpha.1/specs/capella/beacon-chain.md#new-process_bls_to_execution_change
proc process_bls_to_execution_change*(
cfg: RuntimeConfig, state: var (capella.BeaconState | eip4844.BeaconState),
proc check_bls_to_execution_change*(
cfg: RuntimeConfig, state: capella.BeaconState | eip4844.BeaconState,
signed_address_change: SignedBLSToExecutionChange): Result[void, cstring] =
let address_change = signed_address_change.message
@ -415,6 +415,16 @@ proc process_bls_to_execution_change*(
signed_address_change.signature):
return err("process_bls_to_execution_change: invalid signature")
ok()
proc process_bls_to_execution_change*(
cfg: RuntimeConfig, state: var (capella.BeaconState | eip4844.BeaconState),
signed_address_change: SignedBLSToExecutionChange): Result[void, cstring] =
? check_bls_to_execution_change(cfg, state, signed_address_change)
let address_change = signed_address_change.message
var withdrawal_credentials =
state.validators.item(address_change.validator_index).withdrawal_credentials
withdrawal_credentials.data[0] = ETH1_ADDRESS_WITHDRAWAL_PREFIX
withdrawal_credentials.data.fill(1, 11, 0)
withdrawal_credentials.data[12..31] =

View File

@ -1,5 +1,5 @@
# beacon_chain
# Copyright (c) 2018-2022 Status Research & Development GmbH
# Copyright (c) 2018-2023 Status Research & Development GmbH
# Licensed and distributed under either of
# * MIT license (license terms in the root directory or at https://opensource.org/licenses/MIT).
# * Apache v2 license (license terms in the root directory or at https://www.apache.org/licenses/LICENSE-2.0).
@ -452,3 +452,33 @@ proc routeProposerSlashing*(
slashing = shortLog(slashing), error = res.error()
return ok()
proc routeBlsToExecutionChange*(
router: ref MessageRouter,
bls_to_execution_change: SignedBLSToExecutionChange):
Future[SendResult] {.async.} =
block:
let res = router[].processor[].processBlsToExecutionChange(
MsgSource.api, bls_to_execution_change)
if not res.isGoodForSending:
warn "BLS to execution change request failed validation",
slashing = shortLog(bls_to_execution_change), error = res.error()
return err(res.error()[1])
if router[].getCurrentBeaconTime().slotOrZero.epoch <
router[].processor[].dag.cfg.CAPELLA_FORK_EPOCH:
# Broadcast hasn't failed, it just hasn't happened; desire seems to be to
# allow queuing up BLS to execution changes.
return ok()
let res = await router[].network.broadcastBlsToExecutionChange(
bls_to_execution_change)
if res.isOk():
notice "BLS to execution change sent",
bls_to_execution_change = shortLog(bls_to_execution_change)
else: # "no broadcast" is not a fatal error
notice "BLS to execution change not sent",
bls_to_execution_change = shortLog(bls_to_execution_change),
error = res.error()
return ok()

View File

@ -493,7 +493,8 @@ proc makeBeaconBlockForHeadAndSlot*[EP](
attestations =
node.attestationPool[].getAttestationsForBlock(state[], cache)
exits = withState(state[]):
node.exitPool[].getBeaconBlockExits(node.dag.cfg, forkyState.data)
node.validatorChangePool[].getBeaconBlockValidatorChanges(
node.dag.cfg, forkyState.data)
syncAggregate =
if slot.epoch < node.dag.cfg.ALTAIR_FORK_EPOCH:
SyncAggregate.init()
@ -517,7 +518,7 @@ proc makeBeaconBlockForHeadAndSlot*[EP](
exits,
syncAggregate,
payload,
(static(default(SignedBLSToExecutionChangeList))),
(static(default(SignedBLSToExecutionChangeList))), # TODO remove
noRollback, # Temporary state - no need for rollback
cache,
verificationFlags = {},

View File

@ -2589,6 +2589,18 @@
"body": [{"operator": "jstructcmps", "start": ["data"],"value": [{"message": {"epoch": "", "validator_index": ""}, "signature": ""}]}]
}
},
{
"topics": ["beacon", "pool_bls_to_execution_changes"],
"request": {
"url": "/eth/v1/beacon/pool/bls_to_execution_changes",
"headers": {"Accept": "application/json"}
},
"response": {
"status": {"operator": "equals", "value": "200"},
"headers": [{"key": "Content-Type", "value": "application/json", "operator": "equals"}],
"body": [{"operator": "jstructcmps", "start": ["data"],"value": [{"message": {"validator_index": "", "from_bls_pubkey": "", "to_execution_address": ""}, "signature": ""}]}]
}
},
{
"topics": ["config"],
"request": {

View File

@ -1,5 +1,5 @@
# beacon_chain
# Copyright (c) 2019-2022 Status Research & Development GmbH
# Copyright (c) 2019-2023 Status Research & Development GmbH
# Licensed and distributed under either of
# * MIT license (license terms in the root directory or at https://opensource.org/licenses/MIT).
# * Apache v2 license (license terms in the root directory or at https://www.apache.org/licenses/LICENSE-2.0).
@ -72,9 +72,6 @@ from ../beacon_chain/spec/state_transition_block import process_block
# when possible, to also use the forked version. It'll be worth keeping some
# example of the non-forked version because it enables fork bootstrapping.
const defaultSignedBLSToExecutionChangeList =
default(SignedBLSToExecutionChangeList)
proc makeBeaconBlock(
cfg: RuntimeConfig,
state: var phase0.HashedBeaconState,
@ -84,7 +81,7 @@ proc makeBeaconBlock(
graffiti: GraffitiBytes,
attestations: seq[Attestation],
deposits: seq[Deposit],
exits: BeaconBlockExits,
exits: BeaconBlockValidatorChanges,
sync_aggregate: SyncAggregate,
execution_payload: bellatrix.ExecutionPayload,
bls_to_execution_changes: SignedBLSToExecutionChangeList,
@ -104,8 +101,7 @@ proc makeBeaconBlock(
var blck = partialBeaconBlock(
cfg, state, proposer_index, randao_reveal, eth1_data, graffiti,
attestations, deposits, exits, sync_aggregate, execution_payload,
defaultSignedBLSToExecutionChangeList)
attestations, deposits, exits, sync_aggregate, execution_payload)
let res = process_block(
cfg, state.data, blck.asSigVerified(), verificationFlags, cache)
@ -128,7 +124,7 @@ proc makeBeaconBlock(
graffiti: GraffitiBytes,
attestations: seq[Attestation],
deposits: seq[Deposit],
exits: BeaconBlockExits,
exits: BeaconBlockValidatorChanges,
sync_aggregate: SyncAggregate,
execution_payload: bellatrix.ExecutionPayload,
bls_to_execution_changes: SignedBLSToExecutionChangeList,
@ -148,8 +144,7 @@ proc makeBeaconBlock(
var blck = partialBeaconBlock(
cfg, state, proposer_index, randao_reveal, eth1_data, graffiti,
attestations, deposits, exits, sync_aggregate, execution_payload,
defaultSignedBLSToExecutionChangeList)
attestations, deposits, exits, sync_aggregate, execution_payload)
# Signatures are verified elsewhere, so don't duplicate inefficiently here
let res = process_block(
@ -173,7 +168,7 @@ proc makeBeaconBlock(
graffiti: GraffitiBytes,
attestations: seq[Attestation],
deposits: seq[Deposit],
exits: BeaconBlockExits,
exits: BeaconBlockValidatorChanges,
sync_aggregate: SyncAggregate,
execution_payload: bellatrix.ExecutionPayload,
bls_to_execution_changes: SignedBLSToExecutionChangeList,
@ -193,8 +188,7 @@ proc makeBeaconBlock(
var blck = partialBeaconBlock(
cfg, state, proposer_index, randao_reveal, eth1_data, graffiti,
attestations, deposits, exits, sync_aggregate, execution_payload,
defaultSignedBLSToExecutionChangeList)
attestations, deposits, exits, sync_aggregate, execution_payload)
let res = process_block(
cfg, state.data, blck.asSigVerified(), verificationFlags, cache)
@ -217,7 +211,7 @@ proc makeBeaconBlock(
graffiti: GraffitiBytes,
attestations: seq[Attestation],
deposits: seq[Deposit],
exits: BeaconBlockExits,
exits: BeaconBlockValidatorChanges,
sync_aggregate: SyncAggregate,
execution_payload: capella.ExecutionPayload,
bls_to_execution_changes: SignedBLSToExecutionChangeList,
@ -237,8 +231,7 @@ proc makeBeaconBlock(
var blck = partialBeaconBlock(
cfg, state, proposer_index, randao_reveal, eth1_data, graffiti,
attestations, deposits, exits, sync_aggregate, execution_payload,
bls_to_execution_changes)
attestations, deposits, exits, sync_aggregate, execution_payload)
let res = process_block(
cfg, state.data, blck.asSigVerified(), verificationFlags, cache)
@ -261,7 +254,7 @@ proc makeBeaconBlock(
graffiti: GraffitiBytes,
attestations: seq[Attestation],
deposits: seq[Deposit],
exits: BeaconBlockExits,
exits: BeaconBlockValidatorChanges,
sync_aggregate: SyncAggregate,
execution_payload: eip4844.ExecutionPayload,
bls_to_execution_changes: SignedBLSToExecutionChangeList,
@ -281,8 +274,7 @@ proc makeBeaconBlock(
var blck = partialBeaconBlock(
cfg, state, proposer_index, randao_reveal, eth1_data, graffiti,
attestations, deposits, exits, sync_aggregate, execution_payload,
bls_to_execution_changes)
attestations, deposits, exits, sync_aggregate, execution_payload)
let res = process_block(
cfg, state.data, blck.asSigVerified(), verificationFlags, cache)
@ -520,7 +512,7 @@ cli do(slots = SLOTS_PER_EPOCH * 6,
default(GraffitiBytes),
attPool.getAttestationsForBlock(state, cache),
eth1ProposalData.deposits,
BeaconBlockExits(),
BeaconBlockValidatorChanges(),
sync_aggregate,
when T is eip4844.SignedBeaconBlock:
default(eip4844.ExecutionPayload)
@ -528,7 +520,7 @@ cli do(slots = SLOTS_PER_EPOCH * 6,
default(capella.ExecutionPayload)
else:
default(bellatrix.ExecutionPayload),
defaultSignedBLSToExecutionChangeList,
static(default(SignedBLSToExecutionChangeList)),
noRollback,
cache)

View File

@ -1,5 +1,5 @@
# beacon_chain
# Copyright (c) 2022 Status Research & Development GmbH
# Copyright (c) 2022-2023 Status Research & Development GmbH
# Licensed and distributed under either of
# * MIT license (license terms in the root directory or at https://opensource.org/licenses/MIT).
# * Apache v2 license (license terms in the root directory or at https://www.apache.org/licenses/LICENSE-2.0).
@ -160,7 +160,7 @@ cli do(validatorsDir: string, secretsDir: string,
GraffitiBytes.init("insecura"),
blockAggregates,
@[],
BeaconBlockExits(),
BeaconBlockValidatorChanges(),
syncAggregate,
default(bellatrix.ExecutionPayload),
(static(default(SignedBLSToExecutionChangeList))),

View File

@ -1,5 +1,5 @@
# beacon_chain
# Copyright (c) 2020-2022 Status Research & Development GmbH
# Copyright (c) 2020-2023 Status Research & Development GmbH
# Licensed and distributed under either of
# * MIT license (license terms in the root directory or at https://opensource.org/licenses/MIT).
# * Apache v2 license (license terms in the root directory or at https://www.apache.org/licenses/LICENSE-2.0).
@ -49,13 +49,14 @@ func makeSignedVoluntaryExit(
fork, genesis_validators_root, tmp,
MockPrivKeys[validator_index]).toValidatorSig)
suite "Exit pool testing suite":
suite "Validator change pool testing suite":
setup:
let
cfg = block:
var tmp = defaultRuntimeConfig
tmp.ALTAIR_FORK_EPOCH = Epoch(tmp.SHARD_COMMITTEE_PERIOD)
tmp.BELLATRIX_FORK_EPOCH = Epoch(tmp.SHARD_COMMITTEE_PERIOD) + 1
tmp.CAPELLA_FORK_EPOCH = Epoch(tmp.SHARD_COMMITTEE_PERIOD) + 2
tmp
validatorMonitor = newClone(ValidatorMonitor.init())
@ -64,9 +65,9 @@ suite "Exit pool testing suite":
validatorMonitor, {})
fork = dag.forkAtEpoch(Epoch(0))
genesis_validators_root = dag.genesis_validators_root
pool = newClone(ExitPool.init(dag))
pool = newClone(ValidatorChangePool.init(dag))
test "addExitMessage/getProposerSlashingMessage":
test "addValidatorChangeMessage/getProposerSlashingMessage":
for i in 0'u64 .. MAX_PROPOSER_SLASHINGS + 5:
for j in 0'u64 .. i:
let
@ -85,13 +86,13 @@ suite "Exit pool testing suite":
check: pool[].isSeen(msg)
withState(dag.headState):
check:
pool[].getBeaconBlockExits(
pool[].getBeaconBlockValidatorChanges(
cfg, forkyState.data).proposer_slashings.lenu64 ==
min(i + 1, MAX_PROPOSER_SLASHINGS)
pool[].getBeaconBlockExits(
pool[].getBeaconBlockValidatorChanges(
cfg, forkyState.data).proposer_slashings.len == 0
test "addExitMessage/getAttesterSlashingMessage":
test "addValidatorChangeMessage/getAttesterSlashingMessage":
for i in 0'u64 .. MAX_ATTESTER_SLASHINGS + 5:
for j in 0'u64 .. i:
let
@ -108,13 +109,13 @@ suite "Exit pool testing suite":
check: pool[].isSeen(msg)
withState(dag.headState):
check:
pool[].getBeaconBlockExits(
pool[].getBeaconBlockValidatorChanges(
cfg, forkyState.data).attester_slashings.lenu64 ==
min(i + 1, MAX_ATTESTER_SLASHINGS)
pool[].getBeaconBlockExits(
pool[].getBeaconBlockValidatorChanges(
cfg, forkyState.data).attester_slashings.len == 0
test "addExitMessage/getVoluntaryExitMessage":
test "addValidatorChangeMessage/getVoluntaryExitMessage":
# Need to advance state or it will not accept voluntary exits
var
cache: StateCache
@ -139,12 +140,67 @@ suite "Exit pool testing suite":
withState(dag.headState):
check:
pool[].getBeaconBlockExits(
pool[].getBeaconBlockValidatorChanges(
cfg, forkyState.data).voluntary_exits.lenu64 ==
min(i + 1, MAX_VOLUNTARY_EXITS)
pool[].getBeaconBlockExits(
pool[].getBeaconBlockValidatorChanges(
cfg, forkyState.data).voluntary_exits.len == 0
test "addValidatorChangeMessage/getBlsToExecutionChange (pre-capella)":
# Need to advance state or it will not accept voluntary exits
var
cache: StateCache
info: ForkedEpochInfo
process_slots(
dag.cfg, dag.headState,
Epoch(dag.cfg.SHARD_COMMITTEE_PERIOD).start_slot + 1 + SLOTS_PER_EPOCH * 1,
cache, info, {}).expect("ok")
let fork = dag.forkAtEpoch(dag.headState.get_current_epoch())
for i in 0'u64 .. MAX_BLS_TO_EXECUTION_CHANGES + 5:
for j in 0'u64 .. i:
let msg = SignedBLSToExecutionChange(
message: BLSToExecutionChange(validator_index: j))
if i == 0:
check not pool[].isSeen(msg)
pool[].addMessage(msg)
check: pool[].isSeen(msg)
withState(dag.headState):
# Too early to get BLS to execution changes for blocks
check pool[].getBeaconBlockValidatorChanges(
cfg, forkyState.data).bls_to_execution_changes.len == 0
test "addValidatorChangeMessage/getBlsToExecutionChange (post-capella)":
# Need to advance state or it will not accept voluntary exits
var
cache: StateCache
info: ForkedEpochInfo
process_slots(
dag.cfg, dag.headState,
Epoch(dag.cfg.SHARD_COMMITTEE_PERIOD).start_slot + 1 + SLOTS_PER_EPOCH * 2,
cache, info, {}).expect("ok")
let fork = dag.forkAtEpoch(dag.headState.get_current_epoch())
for i in 0'u64 .. MAX_BLS_TO_EXECUTION_CHANGES + 5:
for j in 0'u64 .. i:
let msg = SignedBLSToExecutionChange(
message: BLSToExecutionChange(validator_index: j))
if i == 0:
check not pool[].isSeen(msg)
pool[].addMessage(msg)
check: pool[].isSeen(msg)
withState(dag.headState):
check:
pool[].getBeaconBlockValidatorChanges(
cfg, forkyState.data).bls_to_execution_changes.lenu64 ==
min(i + 1, MAX_BLS_TO_EXECUTION_CHANGES)
pool[].getBeaconBlockValidatorChanges(
cfg, forkyState.data).bls_to_execution_changes.len == 0
test "pre-pre-fork voluntary exit":
var
cache: StateCache
@ -169,5 +225,5 @@ suite "Exit pool testing suite":
check:
# Message signed with a (fork-2) domain can no longer be added as that
# fork is not present in the BeaconState and thus fails transition
pool[].getBeaconBlockExits(
pool[].getBeaconBlockValidatorChanges(
cfg, forkyState.data).voluntary_exits.lenu64 == 0

View File

@ -199,7 +199,7 @@ proc addTestBlockAux[EP: bellatrix.ExecutionPayload | capella.ExecutionPayload](
graffiti,
attestations,
deposits,
BeaconBlockExits(),
BeaconBlockValidatorChanges(),
sync_aggregate,
execution_payload,
default(SignedBLSToExecutionChangeList),