Merge branch 'unstable' into update-libp2p-branch

This commit is contained in:
diegomrsantos 2024-04-19 00:10:55 +02:00 committed by GitHub
commit d033d03fb3
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
79 changed files with 782 additions and 387 deletions

View File

@ -14,10 +14,12 @@ import
from ./spec/datatypes/capella import
ExecutionPayloadHeader, HistoricalSummary, Withdrawal
from ./spec/datatypes/deneb import ExecutionPayloadHeader
from ./spec/datatypes/electra import ExecutionPayloadHeader
from ./spec/datatypes/electra import
ExecutionPayloadHeader, PendingConsolidation, PendingBalanceDeposit,
PendingPartialWithdrawal
type
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.6/specs/phase0/beacon-chain.md#beaconstate
# https://github.com/ethereum/consensus-specs/blob/v1.4.0/specs/phase0/beacon-chain.md#beaconstate
# Memory-representation-equivalent to a phase0 BeaconState for in-place SSZ
# reading and writing
Phase0BeaconStateNoImmutableValidators* = object
@ -398,3 +400,20 @@ type
# Deep history valid from Capella onwards
historical_summaries*:
HashList[HistoricalSummary, Limit HISTORICAL_ROOTS_LIMIT]
deposit_receipts_start_index*: uint64 # [New in Electra:EIP6110]
deposit_balance_to_consume*: Gwei # [New in Electra:EIP7251]
exit_balance_to_consume*: Gwei # [New in Electra:EIP7251]
earliest_exit_epoch*: Epoch # [New in Electra:EIP7251]
consolidation_balance_to_consume*: Gwei # [New in Electra:EIP7251]
earliest_consolidation_epoch*: Epoch # [New in Electra:EIP7251]
pending_balance_deposits*:
HashList[PendingBalanceDeposit, Limit PENDING_BALANCE_DEPOSITS_LIMIT]
## [New in Electra:EIP7251]
# [New in Electra:EIP7251]
pending_partial_withdrawals*:
HashList[PendingPartialWithdrawal, Limit PENDING_PARTIAL_WITHDRAWALS_LIMIT]
pending_consolidations*:
HashList[PendingConsolidation, Limit PENDING_CONSOLIDATIONS_LIMIT]
## [New in Electra:EIP7251]

View File

@ -27,7 +27,7 @@ type
## which blocks are valid - in particular, blocks are not valid if they
## come from the future as seen from the local clock.
##
## https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.5/specs/phase0/fork-choice.md#fork-choice
## https://github.com/ethereum/consensus-specs/blob/v1.4.0/specs/phase0/fork-choice.md#fork-choice
##
# TODO consider NTP and network-adjusted timestamps as outlined here:
# https://ethresear.ch/t/network-adjusted-timestamps/4187

View File

@ -44,7 +44,7 @@ type
EventBus* = object
headQueue*: AsyncEventQueue[HeadChangeInfoObject]
blocksQueue*: AsyncEventQueue[EventBeaconBlockObject]
attestQueue*: AsyncEventQueue[Attestation]
attestQueue*: AsyncEventQueue[phase0.Attestation]
exitQueue*: AsyncEventQueue[SignedVoluntaryExit]
blsToExecQueue*: AsyncEventQueue[SignedBLSToExecutionChange]
propSlashQueue*: AsyncEventQueue[ProposerSlashing]

View File

@ -8,7 +8,7 @@ to specs:
- aggregate attestations: https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.5/specs/phase0/p2p-interface.md#beacon_aggregate_and_proof
- unaggregated attestation: https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.5/specs/phase0/p2p-interface.md#beacon_attestation_subnet_id
- voluntary exits: https://github.com/ethereum/consensus-specs/blob/v1.4.0/specs/phase0/p2p-interface.md#voluntary_exit
- Attester slashings: https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.5/specs/phase0/p2p-interface.md#attester_slashing
- Attester slashings: https://github.com/ethereum/consensus-specs/blob/v1.4.0/specs/phase0/p2p-interface.md#attester_slashing
- Proposer slashings: https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.5/specs/phase0/p2p-interface.md#proposer_slashing
After "gossip validation" the consensus objects can be rebroadcasted as they are optimistically good, however for internal processing further verification is needed.

View File

@ -30,7 +30,7 @@ const
## that potentially could be added to a newly created block
type
OnAttestationCallback = proc(data: Attestation) {.gcsafe, raises: [].}
OnAttestationCallback = proc(data: phase0.Attestation) {.gcsafe, raises: [].}
Validation = object
## Validations collect a set of signatures for a distict attestation - in
@ -222,8 +222,9 @@ func oneIndex(bits: CommitteeValidatorsBits): Opt[int] =
return Opt.none(int)
res
func toAttestation(entry: AttestationEntry, validation: Validation): Attestation =
Attestation(
func toAttestation(entry: AttestationEntry, validation: Validation):
phase0.Attestation =
phase0.Attestation(
aggregation_bits: validation.aggregation_bits,
data: entry.data,
signature: validation.aggregate_signature.finish().toValidatorSig()
@ -293,7 +294,7 @@ func covers(entry: AttestationEntry, bits: CommitteeValidatorsBits): bool =
false
proc addAttestation(entry: var AttestationEntry,
attestation: Attestation,
attestation: phase0.Attestation,
signature: CookedSig): bool =
logScope:
attestation = shortLog(attestation)
@ -335,7 +336,7 @@ proc addAttestation(entry: var AttestationEntry,
true
proc addAttestation*(pool: var AttestationPool,
attestation: Attestation,
attestation: phase0.Attestation,
attesting_indices: openArray[ValidatorIndex],
signature: CookedSig,
wallTime: BeaconTime) =
@ -419,8 +420,9 @@ proc addForkChoice*(pool: var AttestationPool,
error "Couldn't add block to fork choice, bug?",
blck = shortLog(blck), err = state.error
iterator attestations*(pool: AttestationPool, slot: Opt[Slot],
committee_index: Opt[CommitteeIndex]): Attestation =
iterator attestations*(
pool: AttestationPool, slot: Opt[Slot],
committee_index: Opt[CommitteeIndex]): phase0.Attestation =
let candidateIndices =
if slot.isSome():
let candidateIdx = pool.candidateIdx(slot.get())
@ -434,7 +436,7 @@ iterator attestations*(pool: AttestationPool, slot: Opt[Slot],
for candidateIndex in candidateIndices:
for _, entry in pool.candidates[candidateIndex]:
if committee_index.isNone() or entry.data.index == committee_index.get():
var singleAttestation = Attestation(
var singleAttestation = phase0.Attestation(
aggregation_bits: CommitteeValidatorsBits.init(entry.committee_len),
data: entry.data)
@ -555,7 +557,7 @@ proc check_attestation_compatible*(
proc getAttestationsForBlock*(pool: var AttestationPool,
state: ForkyHashedBeaconState,
cache: var StateCache): seq[Attestation] =
cache: var StateCache): seq[phase0.Attestation] =
## Retrieve attestations that may be added to a new block at the slot of the
## given state
## https://github.com/ethereum/consensus-specs/blob/v1.4.0/specs/phase0/validator.md#attestations
@ -641,7 +643,7 @@ proc getAttestationsForBlock*(pool: var AttestationPool,
state.data.previous_epoch_attestations.maxLen -
state.data.previous_epoch_attestations.len()
var res: seq[Attestation]
var res: seq[phase0.Attestation]
let totalCandidates = candidates.len()
while candidates.len > 0 and res.lenu64() < MAX_ATTESTATIONS:
let entryCacheKey = block:
@ -698,7 +700,7 @@ proc getAttestationsForBlock*(pool: var AttestationPool,
proc getAttestationsForBlock*(pool: var AttestationPool,
state: ForkedHashedBeaconState,
cache: var StateCache): seq[Attestation] =
cache: var StateCache): seq[phase0.Attestation] =
withState(state):
pool.getAttestationsForBlock(forkyState, cache)
@ -717,13 +719,13 @@ func bestValidation(aggregates: openArray[Validation]): (int, int) =
bestIndex = i
(bestIndex, best)
func getAggregatedAttestation*(pool: var AttestationPool,
slot: Slot,
attestation_data_root: Eth2Digest): Opt[Attestation] =
func getAggregatedAttestation*(
pool: var AttestationPool, slot: Slot, attestation_data_root: Eth2Digest):
Opt[phase0.Attestation] =
let
candidateIdx = pool.candidateIdx(slot)
if candidateIdx.isNone:
return Opt.none(Attestation)
return Opt.none(phase0.Attestation)
pool.candidates[candidateIdx.get].withValue(attestation_data_root, entry):
entry[].updateAggregates()
@ -733,19 +735,19 @@ func getAggregatedAttestation*(pool: var AttestationPool,
# Found the right hash, no need to look further
return Opt.some(entry[].toAttestation(entry[].aggregates[bestIndex]))
Opt.none(Attestation)
Opt.none(phase0.Attestation)
func getAggregatedAttestation*(pool: var AttestationPool,
slot: Slot,
index: CommitteeIndex): Opt[Attestation] =
func getAggregatedAttestation*(
pool: var AttestationPool, slot: Slot, index: CommitteeIndex):
Opt[phase0.Attestation] =
## Select the attestation that has the most votes going for it in the given
## slot/index
## https://github.com/ethereum/consensus-specs/blob/v1.4.0/specs/phase0/validator.md#construct-aggregate
let candidateIdx = pool.candidateIdx(slot)
if candidateIdx.isNone:
return Opt.none(Attestation)
return Opt.none(phase0.Attestation)
var res: Opt[Attestation]
var res: Opt[phase0.Attestation]
for _, entry in pool.candidates[candidateIdx.get].mpairs():
doAssert entry.data.slot == slot
if index != entry.data.index:

View File

@ -25,7 +25,7 @@ logScope: topics = "spec_cache"
func count_active_validators*(shufflingRef: ShufflingRef): uint64 =
shufflingRef.shuffled_active_validator_indices.lenu64
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.6/specs/phase0/beacon-chain.md#get_committee_count_per_slot
# https://github.com/ethereum/consensus-specs/blob/v1.4.0/specs/phase0/beacon-chain.md#get_committee_count_per_slot
func get_committee_count_per_slot*(shufflingRef: ShufflingRef): uint64 =
get_committee_count_per_slot(count_active_validators(shufflingRef))
@ -51,7 +51,7 @@ iterator get_beacon_committee*(
committees_per_slot * SLOTS_PER_EPOCH
): yield (index_in_committee, idx)
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.6/specs/phase0/beacon-chain.md#get_beacon_committee
# https://github.com/ethereum/consensus-specs/blob/v1.4.0/specs/phase0/beacon-chain.md#get_beacon_committee
func get_beacon_committee*(
shufflingRef: ShufflingRef, slot: Slot, committee_index: CommitteeIndex):
seq[ValidatorIndex] =
@ -98,7 +98,7 @@ iterator get_attesting_indices*(shufflingRef: ShufflingRef,
yield validator_index
iterator get_attesting_indices*(
dag: ChainDAGRef, attestation: TrustedAttestation): ValidatorIndex =
dag: ChainDAGRef, attestation: phase0.TrustedAttestation): ValidatorIndex =
block: # `return` is not allowed in an inline iterator
let
slot =

View File

@ -498,11 +498,12 @@ func asConsensusType*(rpcExecutionPayload: ExecutionPayloadV4):
signature: ValidatorSig(blob: dr.signature.distinctBase),
index: dr.index.uint64)
template getExecutionLayerExit(ele: ExitV1): ExecutionLayerExit =
ExecutionLayerExit(
source_address: ExecutionAddress(data: ele.sourceAddress.distinctBase),
template getExecutionLayerWithdrawalRequest(elwr: ExitV1):
ExecutionLayerWithdrawalRequest =
ExecutionLayerWithdrawalRequest(
source_address: ExecutionAddress(data: elwr.sourceAddress.distinctBase),
validator_pubkey: ValidatorPubKey(
blob: ele.validatorPublicKey.distinctBase))
blob: elwr.validatorPublicKey.distinctBase))
electra.ExecutionPayload(
parent_hash: rpcExecutionPayload.parentHash.asEth2Digest,
@ -529,9 +530,9 @@ func asConsensusType*(rpcExecutionPayload: ExecutionPayloadV4):
deposit_receipts:
List[electra.DepositReceipt, MAX_DEPOSIT_RECEIPTS_PER_PAYLOAD].init(
mapIt(rpcExecutionPayload.depositReceipts, it.getDepositReceipt)),
exits:
List[electra.ExecutionLayerExit, MAX_EXECUTION_LAYER_EXITS_PER_PAYLOAD].init(
mapIt(rpcExecutionPayload.exits, it.getExecutionLayerExit)))
withdrawal_requests:
List[electra.ExecutionLayerWithdrawalRequest, MAX_WITHDRAWAL_REQUESTS_PER_PAYLOAD].init(
mapIt(rpcExecutionPayload.exits, it.getExecutionLayerWithdrawalRequest)))
func asConsensusType*(payload: engine_api.GetPayloadV4Response):
electra.ExecutionPayloadForSigning =
@ -639,10 +640,13 @@ func asEngineExecutionPayload*(executionPayload: electra.ExecutionPayload):
signature: FixedBytes[RawSigSize](dr.signature.blob),
index: dr.index.Quantity)
template getExecutionLayerExit(ele: ExecutionLayerExit): ExitV1 =
template getExecutionLayerWithdrawalRequest(
elwr: ExecutionLayerWithdrawalRequest): ExitV1 =
ExitV1(
sourceAddress: Address(ele.source_address.data),
validatorPublicKey: FixedBytes[RawPubKeySize](ele.validator_pubkey.blob))
sourceAddress: Address(elwr.source_address.data),
validatorPublicKey: FixedBytes[RawPubKeySize](elwr.validator_pubkey.blob))
debugRaiseAssert "nim-web3 needs to change exits to withdrawalRequests; maybe it already has been"
engine_api.ExecutionPayloadV4(
parentHash: executionPayload.parent_hash.asBlockHash,
@ -665,7 +669,9 @@ func asEngineExecutionPayload*(executionPayload: electra.ExecutionPayload):
excessBlobGas: Quantity(executionPayload.excess_blob_gas),
depositReceipts: mapIt(
executionPayload.deposit_receipts, it.getDepositReceipt),
exits: mapIt(executionPayload.exits, it.getExecutionLayerExit))
exits:
mapIt(executionPayload.withdrawal_requests,
it.getExecutionLayerWithdrawalRequest))
func isConnected(connection: ELConnection): bool =
connection.web3.isSome

View File

@ -113,7 +113,7 @@ proc update_justified(
self.update_justified(dag, blck, justified.epoch)
ok()
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.5/specs/phase0/fork-choice.md#update_checkpoints
# https://github.com/ethereum/consensus-specs/blob/v1.4.0/specs/phase0/fork-choice.md#update_checkpoints
proc update_checkpoints(
self: var Checkpoints, dag: ChainDAGRef,
checkpoints: FinalityCheckpoints): FcResult[void] =

View File

@ -10,8 +10,8 @@ This folder holds a collection of modules to:
Gossip validation is different from consensus verification in particular for blocks.
- Blocks: https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.5/specs/phase0/p2p-interface.md#beacon_block
- Attestations (aggregated): https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.5/specs/phase0/p2p-interface.md#beacon_aggregate_and_proof
- Attestations (unaggregated): https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.5/specs/phase0/p2p-interface.md#attestation-subnets
- Attestations (aggregated): https://github.com/ethereum/consensus-specs/blob/v1.4.0/specs/phase0/p2p-interface.md#beacon_aggregate_and_proof
- Attestations (unaggregated): https://github.com/ethereum/consensus-specs/blob/v1.4.0/specs/phase0/p2p-interface.md#attestation-subnets
- Voluntary exits: https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.4/specs/phase0/p2p-interface.md#voluntary_exit
- Proposer slashings: https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.5/specs/phase0/p2p-interface.md#proposer_slashing
- Attester slashing: https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.5/specs/phase0/p2p-interface.md#attester_slashing

View File

@ -342,7 +342,7 @@ proc clearDoppelgangerProtection*(self: var Eth2Processor) =
self.doppelgangerDetection.broadcastStartEpoch = FAR_FUTURE_EPOCH
proc checkForPotentialDoppelganger(
self: var Eth2Processor, attestation: Attestation,
self: var Eth2Processor, attestation: phase0.Attestation,
attesterIndices: openArray[ValidatorIndex]) =
# Only check for attestations after node launch. There might be one slot of
# overlap in quick intra-slot restarts so trade off a few true negatives in
@ -364,7 +364,7 @@ proc checkForPotentialDoppelganger(
proc processAttestation*(
self: ref Eth2Processor, src: MsgSource,
attestation: Attestation, subnet_id: SubnetId,
attestation: phase0.Attestation, subnet_id: SubnetId,
checkSignature: bool = true): Future[ValidationRes] {.async: (raises: [CancelledError]).} =
var wallTime = self.getCurrentBeaconTime()
let (afterGenesis, wallSlot) = wallTime.toSlot()

View File

@ -158,7 +158,8 @@ func check_beacon_and_target_block(
ok(target)
func check_aggregation_count(
attestation: Attestation, singular: bool): Result[void, ValidationError] =
attestation: phase0.Attestation, singular: bool):
Result[void, ValidationError] =
let ones = attestation.aggregation_bits.countOnes()
if singular and ones != 1:
return errReject("Attestation must have a single attestation bit set")
@ -650,7 +651,7 @@ proc validateBeaconBlock*(
proc validateAttestation*(
pool: ref AttestationPool,
batchCrypto: ref BatchCrypto,
attestation: Attestation,
attestation: phase0.Attestation,
wallTime: BeaconTime,
subnet_id: SubnetId, checkSignature: bool):
Future[Result[

View File

@ -149,10 +149,10 @@ typedef struct ETHBeaconState ETHBeaconState;
* representation - If successful.
* @return `NULL` - If the given `sszBytes` is malformed.
*
* @see https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.6/specs/phase0/beacon-chain.md#beaconstate
* @see https://github.com/ethereum/consensus-specs/blob/v1.4.0/specs/phase0/beacon-chain.md#beaconstate
* @see https://github.com/ethereum/consensus-specs/blob/v1.4.0/specs/altair/beacon-chain.md#beaconstate
* @see https://github.com/ethereum/consensus-specs/blob/v1.4.0/specs/bellatrix/beacon-chain.md#beaconstate
* @see https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.5/specs/capella/beacon-chain.md#beaconstate
* @see https://github.com/ethereum/consensus-specs/blob/v1.4.0/specs/capella/beacon-chain.md#beaconstate
* @see https://github.com/ethereum/consensus-specs/blob/v1.4.0/configs/README.md
*/
ETH_RESULT_USE_CHECK
@ -198,7 +198,7 @@ ETHRoot *ETHBeaconStateCopyGenesisValidatorsRoot(const ETHBeaconState *state);
*
* @param root Merkle root.
*
* @see https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.6/specs/phase0/beacon-chain.md#custom-types
* @see https://github.com/ethereum/consensus-specs/blob/v1.4.0/specs/phase0/beacon-chain.md#custom-types
*/
void ETHRootDestroy(ETHRoot *root);
@ -271,7 +271,7 @@ void ETHBeaconClockDestroy(ETHBeaconClock *beaconClock);
* @return Slot number for the current wall clock time - If genesis has occurred.
* @return `0` - If genesis is still pending.
*
* @see https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.6/specs/phase0/beacon-chain.md#custom-types
* @see https://github.com/ethereum/consensus-specs/blob/v1.4.0/specs/phase0/beacon-chain.md#custom-types
*/
ETH_RESULT_USE_CHECK
int ETHBeaconClockGetSlot(const ETHBeaconClock *beaconClock);
@ -597,7 +597,7 @@ const ETHLightClientHeader *ETHLightClientStoreGetFinalizedHeader(
*
* @return Whether or not the next sync committee is currently known.
*
* @see https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.5/specs/altair/light-client/sync-protocol.md#is_next_sync_committee_known
* @see https://github.com/ethereum/consensus-specs/blob/v1.4.0/specs/altair/light-client/sync-protocol.md#is_next_sync_committee_known
* @see https://github.com/ethereum/consensus-specs/blob/v1.4.0/specs/altair/light-client/light-client.md
*/
ETH_RESULT_USE_CHECK

View File

@ -142,10 +142,10 @@ proc ETHBeaconStateCreateFromSsz(
## * `NULL` - If the given `sszBytes` is malformed.
##
## See:
## * https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.6/specs/phase0/beacon-chain.md#beaconstate
## * https://github.com/ethereum/consensus-specs/blob/v1.4.0/specs/phase0/beacon-chain.md#beaconstate
## * https://github.com/ethereum/consensus-specs/blob/v1.4.0/specs/altair/beacon-chain.md#beaconstate
## * https://github.com/ethereum/consensus-specs/blob/v1.4.0/specs/bellatrix/beacon-chain.md#beaconstate
## * https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.5/specs/capella/beacon-chain.md#beaconstate
## * https://github.com/ethereum/consensus-specs/blob/v1.4.0/specs/capella/beacon-chain.md#beaconstate
## * https://github.com/ethereum/consensus-specs/blob/v1.4.0/configs/README.md
let
consensusFork = ConsensusFork.decodeString($consensusVersion).valueOr:
@ -755,7 +755,7 @@ func ETHLightClientStoreIsNextSyncCommitteeKnown(
## * Whether or not the next sync committee is currently known.
##
## See:
## * https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.5/specs/altair/light-client/sync-protocol.md#is_next_sync_committee_known
## * https://github.com/ethereum/consensus-specs/blob/v1.4.0/specs/altair/light-client/sync-protocol.md#is_next_sync_committee_known
## * https://github.com/ethereum/consensus-specs/blob/v1.4.0/specs/altair/light-client/light-client.md
store[].is_next_sync_committee_known
@ -796,7 +796,7 @@ func ETHLightClientStoreGetSafetyThreshold(
## * Light client store safety threshold.
##
## See:
## * https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.5/specs/altair/light-client/sync-protocol.md#get_safety_threshold
## * https://github.com/ethereum/consensus-specs/blob/v1.4.0/specs/altair/light-client/sync-protocol.md#get_safety_threshold
store[].get_safety_threshold.cint
proc ETHLightClientHeaderCreateCopy(

View File

@ -837,7 +837,7 @@ template gossipMaxSize(T: untyped): uint32 =
# Attestation, AttesterSlashing, and SignedAggregateAndProof, which all
# have lists bounded at MAX_VALIDATORS_PER_COMMITTEE (2048) items, thus
# having max sizes significantly smaller than GOSSIP_MAX_SIZE.
elif T is Attestation or T is AttesterSlashing or
elif T is phase0.Attestation or T is AttesterSlashing or
T is SignedAggregateAndProof or T is phase0.SignedBeaconBlock or
T is altair.SignedBeaconBlock or T is SomeForkyLightClientObject:
GOSSIP_MAX_SIZE
@ -2204,7 +2204,7 @@ proc getPersistentNetKeys*(
func gossipId(
data: openArray[byte], phase0Prefix, topic: string): seq[byte] =
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.5/specs/phase0/p2p-interface.md#topics-and-messages
# https://github.com/ethereum/consensus-specs/blob/v1.4.0/specs/phase0/p2p-interface.md#topics-and-messages
# https://github.com/ethereum/consensus-specs/blob/v1.4.0/specs/altair/p2p-interface.md#topics-and-messages
const MESSAGE_DOMAIN_VALID_SNAPPY = [0x01'u8, 0x00, 0x00, 0x00]
let messageDigest = withEth2Hash:
@ -2565,7 +2565,7 @@ proc getWallEpoch(node: Eth2Node): Epoch =
node.getBeaconTime().slotOrZero.epoch
proc broadcastAttestation*(
node: Eth2Node, subnet_id: SubnetId, attestation: Attestation):
node: Eth2Node, subnet_id: SubnetId, attestation: phase0.Attestation):
Future[SendResult] {.async: (raises: [CancelledError], raw: true).} =
# Regardless of the contents of the attestation,
# https://github.com/ethereum/consensus-specs/blob/v1.3.0/specs/altair/p2p-interface.md#transitioning-the-gossip

View File

@ -277,7 +277,7 @@ proc initFullNode(
getBeaconTime: GetBeaconTimeFn) {.async.} =
template config(): auto = node.config
proc onAttestationReceived(data: Attestation) =
proc onAttestationReceived(data: phase0.Attestation) =
node.eventBus.attestQueue.emit(data)
proc onSyncContribution(data: SignedContributionAndProof) =
node.eventBus.contribQueue.emit(data)
@ -1753,7 +1753,7 @@ proc installMessageValidators(node: BeaconNode) =
let subnet_id = it
node.network.addAsyncValidator(
getAttestationTopic(digest, subnet_id), proc (
attestation: Attestation
attestation: phase0.Attestation
): Future[ValidationResult] {.async: (raises: [CancelledError]).} =
return toValidationResult(
await node.processor.processAttestation(

View File

@ -1192,7 +1192,7 @@ proc installBeaconApiHandlers*(router: var RestRouter, node: BeaconNode) =
Opt.some(rslot.get())
else:
Opt.none(Slot)
var res: seq[Attestation]
var res: seq[phase0.Attestation]
for item in node.attestationPool[].attestations(vslot, vindex):
res.add(item)
RestApiResponse.jsonResponse(res)
@ -1204,7 +1204,7 @@ proc installBeaconApiHandlers*(router: var RestRouter, node: BeaconNode) =
block:
if contentBody.isNone():
return RestApiResponse.jsonError(Http400, EmptyRequestBodyError)
let dres = decodeBody(seq[Attestation], contentBody.get())
let dres = decodeBody(seq[phase0.Attestation], contentBody.get())
if dres.isErr():
return RestApiResponse.jsonError(Http400,
InvalidAttestationObjectError,

View File

@ -196,7 +196,7 @@ func since_epoch_start*(slot: Slot): uint64 = # aka compute_slots_since_epoch_st
template is_epoch*(slot: Slot): bool =
slot.since_epoch_start == 0
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.7/specs/phase0/beacon-chain.md#compute_start_slot_at_epoch
# https://github.com/ethereum/consensus-specs/blob/v1.4.0/specs/phase0/beacon-chain.md#compute_start_slot_at_epoch
func start_slot*(epoch: Epoch): Slot = # aka compute_start_slot_at_epoch
## Return the start slot of ``epoch``.
const maxEpoch = Epoch(FAR_FUTURE_SLOT div SLOTS_PER_EPOCH)

View File

@ -30,7 +30,7 @@ func increase_balance*(
if delta != 0.Gwei: # avoid dirtying the balance cache if not needed
increase_balance(state.balances.mitem(index), delta)
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.6/specs/phase0/beacon-chain.md#decrease_balance
# https://github.com/ethereum/consensus-specs/blob/v1.4.0/specs/phase0/beacon-chain.md#decrease_balance
func decrease_balance*(balance: var Gwei, delta: Gwei) =
balance =
if delta > balance:
@ -71,7 +71,7 @@ func compute_activation_exit_epoch*(epoch: Epoch): Epoch =
## ``epoch`` take effect.
epoch + 1 + MAX_SEED_LOOKAHEAD
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.7/specs/phase0/beacon-chain.md#get_validator_churn_limit
# https://github.com/ethereum/consensus-specs/blob/v1.4.0/specs/phase0/beacon-chain.md#get_validator_churn_limit
func get_validator_churn_limit*(
cfg: RuntimeConfig, state: ForkyBeaconState, cache: var StateCache):
uint64 =
@ -158,7 +158,7 @@ func initiate_validator_exit*(
from ./datatypes/deneb import BeaconState
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.7/specs/phase0/beacon-chain.md#slash_validator
# https://github.com/ethereum/consensus-specs/blob/v1.4.0/specs/phase0/beacon-chain.md#slash_validator
# https://github.com/ethereum/consensus-specs/blob/v1.4.0/specs/altair/beacon-chain.md#modified-slash_validator
# https://github.com/ethereum/consensus-specs/blob/v1.4.0/specs/bellatrix/beacon-chain.md#modified-slash_validator
func get_slashing_penalty*(state: ForkyBeaconState,
@ -181,7 +181,7 @@ func get_slashing_penalty*(state: ForkyBeaconState,
func get_whistleblower_reward*(validator_effective_balance: Gwei): Gwei =
validator_effective_balance div WHISTLEBLOWER_REWARD_QUOTIENT
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.7/specs/phase0/beacon-chain.md#slash_validator
# https://github.com/ethereum/consensus-specs/blob/v1.4.0/specs/phase0/beacon-chain.md#slash_validator
# https://github.com/ethereum/consensus-specs/blob/v1.4.0/specs/altair/beacon-chain.md#modified-slash_validator
# https://github.com/ethereum/consensus-specs/blob/v1.4.0/specs/bellatrix/beacon-chain.md#modified-slash_validator
func get_proposer_reward(state: ForkyBeaconState, whistleblower_reward: Gwei): Gwei =
@ -280,7 +280,7 @@ func get_initial_beacon_block*(state: bellatrix.HashedBeaconState):
bellatrix.TrustedSignedBeaconBlock(
message: message, root: hash_tree_root(message))
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.5/specs/capella/beacon-chain.md#testing
# https://github.com/ethereum/consensus-specs/blob/v1.4.0/specs/capella/beacon-chain.md#testing
func get_initial_beacon_block*(state: capella.HashedBeaconState):
capella.TrustedSignedBeaconBlock =
# The genesis block is implicitly trusted
@ -341,7 +341,7 @@ func get_block_root_at_slot*(
withState(state):
get_block_root_at_slot(forkyState.data, slot)
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.6/specs/phase0/beacon-chain.md#get_block_root
# https://github.com/ethereum/consensus-specs/blob/v1.4.0/specs/phase0/beacon-chain.md#get_block_root
func get_block_root*(state: ForkyBeaconState, epoch: Epoch): Eth2Digest =
## Return the block root at the start of a recent ``epoch``.
get_block_root_at_slot(state, epoch.start_slot())
@ -378,7 +378,7 @@ func is_eligible_for_activation*(
# Has not yet been activated
validator.activation_epoch == FAR_FUTURE_EPOCH
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.6/specs/phase0/beacon-chain.md#is_valid_indexed_attestation
# https://github.com/ethereum/consensus-specs/blob/v1.4.0/specs/phase0/beacon-chain.md#is_valid_indexed_attestation
proc is_valid_indexed_attestation*(
state: ForkyBeaconState, indexed_attestation: SomeIndexedAttestation,
flags: UpdateFlags): Result[void, cstring] =
@ -490,7 +490,7 @@ proc is_valid_indexed_attestation(
# Attestation validation
# ------------------------------------------------------------------------------------------
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.6/specs/phase0/beacon-chain.md#attestations
# https://github.com/ethereum/consensus-specs/blob/v1.4.0/specs/phase0/beacon-chain.md#attestations
# https://github.com/ethereum/consensus-specs/blob/v1.3.0/specs/phase0/p2p-interface.md#beacon_attestation_subnet_id
func check_attestation_slot_target*(data: AttestationData): Result[Slot, cstring] =
@ -509,7 +509,7 @@ func check_attestation_target_epoch(
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.6/specs/phase0/beacon-chain.md#attestations
# https://github.com/ethereum/consensus-specs/blob/v1.4.0/specs/altair/beacon-chain.md#modified-process_attestation
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/deneb/beacon-chain.md#modified-process_attestation
# https://github.com/ethereum/consensus-specs/blob/v1.4.0/specs/deneb/beacon-chain.md#modified-process_attestation
func check_attestation_inclusion(
consensusFork: static ConsensusFork, attestation_slot: Slot,
current_slot: Slot): Result[void, cstring] =
@ -611,7 +611,7 @@ func get_attestation_participation_flag_indices(
# TODO these duplicate some stuff in state_transition_epoch which uses TotalBalances
# better to centralize around that if feasible
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.6/specs/phase0/beacon-chain.md#get_total_active_balance
# https://github.com/ethereum/consensus-specs/blob/v1.4.0/specs/phase0/beacon-chain.md#get_total_active_balance
func get_total_active_balance*(state: ForkyBeaconState, cache: var StateCache): Gwei =
## Return the combined effective balance of the active validators.
## Note: ``get_total_balance`` returns ``EFFECTIVE_BALANCE_INCREMENT`` Gwei
@ -838,7 +838,7 @@ func has_eth1_withdrawal_credential*(validator: Validator): bool =
## Check if ``validator`` has an 0x01 prefixed "eth1" withdrawal credential.
validator.withdrawal_credentials.data[0] == ETH1_ADDRESS_WITHDRAWAL_PREFIX
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.5/specs/capella/beacon-chain.md#is_fully_withdrawable_validator
# https://github.com/ethereum/consensus-specs/blob/v1.4.0/specs/capella/beacon-chain.md#is_fully_withdrawable_validator
func is_fully_withdrawable_validator(
validator: Validator, balance: Gwei, epoch: Epoch): bool =
## Check if ``validator`` is fully withdrawable.
@ -856,6 +856,23 @@ func is_partially_withdrawable_validator(
has_eth1_withdrawal_credential(validator) and
has_max_effective_balance and has_excess_balance
# https://github.com/ethereum/consensus-specs/blob/82133085a1295e93394ebdf71df8f2f6e0962588/specs/electra/beacon-chain.md#new-is_compounding_withdrawal_credential
func is_compounding_withdrawal_credential(
withdrawal_credentials: Eth2Digest): bool =
withdrawal_credentials.data[0] == COMPOUNDING_WITHDRAWAL_PREFIX
# https://github.com/ethereum/consensus-specs/blob/82133085a1295e93394ebdf71df8f2f6e0962588/specs/electra/beacon-chain.md#new-has_compounding_withdrawal_credential
func has_compounding_withdrawal_credential(validator: Validator): bool =
## Check if ``validator`` has an 0x02 prefixed "compounding" withdrawal
## credential.
is_compounding_withdrawal_credential(validator.withdrawal_credentials)
# https://github.com/ethereum/consensus-specs/blob/82133085a1295e93394ebdf71df8f2f6e0962588/specs/electra/beacon-chain.md#new-has_execution_withdrawal_credential
func has_execution_withdrawal_credential(validator: Validator): bool =
## Check if ``validator`` has a 0x01 or 0x02 prefixed withdrawal credential.
has_compounding_withdrawal_credential(validator) or
has_eth1_withdrawal_credential(validator)
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.5/specs/capella/beacon-chain.md#new-get_expected_withdrawals
func get_expected_withdrawals*(
state: capella.BeaconState | deneb.BeaconState | electra.BeaconState):
@ -1016,7 +1033,7 @@ proc initialize_hashed_beacon_state_from_eth1*(
result.root = hash_tree_root(result.data)
# https://github.com/ethereum/consensus-specs/blob/v1.4.0/specs/bellatrix/beacon-chain.md#testing
# https://github.com/ethereum/consensus-specs/blob/v1.3.0/specs/capella/beacon-chain.md#testing
# https://github.com/ethereum/consensus-specs/blob/v1.4.0/specs/capella/beacon-chain.md#testing
# https://github.com/ethereum/consensus-specs/blob/v1.3.0/specs/deneb/beacon-chain.md#testing
proc initialize_beacon_state_from_eth1*(
cfg: RuntimeConfig,

View File

@ -161,7 +161,7 @@ type
NextSyncCommitteeBranch* =
array[log2trunc(NEXT_SYNC_COMMITTEE_GINDEX), Eth2Digest]
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.5/specs/altair/light-client/sync-protocol.md#lightclientheader
# https://github.com/ethereum/consensus-specs/blob/v1.4.0/specs/altair/light-client/sync-protocol.md#lightclientheader
LightClientHeader* = object
beacon*: BeaconBlockHeader
## Beacon block header
@ -665,7 +665,7 @@ chronicles.formatIt SyncCommitteeContribution: shortLog(it)
chronicles.formatIt ContributionAndProof: shortLog(it)
chronicles.formatIt SignedContributionAndProof: shortLog(it)
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.5/specs/altair/light-client/sync-protocol.md#is_valid_light_client_header
# https://github.com/ethereum/consensus-specs/blob/v1.4.0/specs/altair/light-client/sync-protocol.md#is_valid_light_client_header
func is_valid_light_client_header*(
header: LightClientHeader, cfg: RuntimeConfig): bool =
true

View File

@ -210,7 +210,7 @@ type
# SSZ / hashing purposes
JustificationBits* = distinct uint8
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.6/specs/phase0/beacon-chain.md#proposerslashing
# https://github.com/ethereum/consensus-specs/blob/v1.4.0/specs/phase0/beacon-chain.md#proposerslashing
ProposerSlashing* = object
signed_header_1*: SignedBeaconBlockHeader
signed_header_2*: SignedBeaconBlockHeader
@ -250,20 +250,6 @@ type
CommitteeValidatorsBits* = BitList[Limit MAX_VALIDATORS_PER_COMMITTEE]
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.6/specs/phase0/beacon-chain.md#attestation
Attestation* = object
aggregation_bits*: CommitteeValidatorsBits
data*: AttestationData
signature*: ValidatorSig
TrustedAttestation* = object
# The Trusted version, at the moment, implies that the cryptographic signature was checked.
# It DOES NOT imply that the state transition was verified.
# Currently the code MUST verify the state transition as soon as the signature is verified
aggregation_bits*: CommitteeValidatorsBits
data*: AttestationData
signature*: TrustedSig
ForkDigest* = distinct array[4, byte]
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.6/specs/phase0/beacon-chain.md#forkdata
@ -271,7 +257,7 @@ type
current_version*: Version
genesis_validators_root*: Eth2Digest
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.6/specs/phase0/beacon-chain.md#checkpoint
# https://github.com/ethereum/consensus-specs/blob/v1.4.0/specs/phase0/beacon-chain.md#checkpoint
Checkpoint* = object
epoch*: Epoch
root*: Eth2Digest
@ -312,13 +298,12 @@ type
signature*: ValidatorSig
## Signing over DepositMessage
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.6/specs/phase0/beacon-chain.md#voluntaryexit
# https://github.com/ethereum/consensus-specs/blob/v1.4.0/specs/phase0/beacon-chain.md#voluntaryexit
VoluntaryExit* = object
epoch*: Epoch
## Earliest epoch when voluntary exit can be processed
validator_index*: uint64 # `ValidatorIndex` after validation
SomeAttestation* = Attestation | TrustedAttestation
SomeIndexedAttestation* = IndexedAttestation | TrustedIndexedAttestation
SomeProposerSlashing* = ProposerSlashing | TrustedProposerSlashing
SomeAttesterSlashing* = AttesterSlashing | TrustedAttesterSlashing
@ -347,7 +332,7 @@ type
HashedValidatorPubKey* = object
value*: ptr HashedValidatorPubKeyItem
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.6/specs/phase0/beacon-chain.md#validator
# https://github.com/ethereum/consensus-specs/blob/v1.4.0/specs/phase0/beacon-chain.md#validator
Validator* = object
pubkeyData*{.serializedFieldName: "pubkey".}: HashedValidatorPubKey
@ -369,7 +354,7 @@ type
withdrawable_epoch*: Epoch
## When validator can withdraw funds
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.6/specs/phase0/beacon-chain.md#pendingattestation
# https://github.com/ethereum/consensus-specs/blob/v1.4.0/specs/phase0/beacon-chain.md#pendingattestation
PendingAttestation* = object
aggregation_bits*: CommitteeValidatorsBits
data*: AttestationData
@ -397,7 +382,7 @@ type
deposit_count*: uint64
block_hash*: Eth2Digest
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.6/specs/phase0/beacon-chain.md#signedvoluntaryexit
# https://github.com/ethereum/consensus-specs/blob/v1.4.0/specs/phase0/beacon-chain.md#signedvoluntaryexit
SignedVoluntaryExit* = object
message*: VoluntaryExit
signature*: ValidatorSig
@ -421,7 +406,7 @@ type
GraffitiBytes* = distinct array[MAX_GRAFFITI_SIZE, byte]
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.7/specs/phase0/beacon-chain.md#signedbeaconblockheader
# https://github.com/ethereum/consensus-specs/blob/v1.4.0/specs/phase0/beacon-chain.md#signedbeaconblockheader
SignedBeaconBlockHeader* = object
message*: BeaconBlockHeader
signature*: ValidatorSig
@ -430,17 +415,6 @@ type
message*: BeaconBlockHeader
signature*: TrustedSig
# https://github.com/ethereum/consensus-specs/blob/v1.4.0/specs/phase0/validator.md#aggregateandproof
AggregateAndProof* = object
aggregator_index*: uint64 # `ValidatorIndex` after validation
aggregate*: Attestation
selection_proof*: ValidatorSig
# https://github.com/ethereum/consensus-specs/blob/v1.4.0/specs/phase0/validator.md#signedaggregateandproof
SignedAggregateAndProof* = object
message*: AggregateAndProof
signature*: ValidatorSig
SyncCommitteeCache* = object
current_sync_committee*: array[SYNC_COMMITTEE_SIZE, ValidatorIndex]
next_sync_committee*: array[SYNC_COMMITTEE_SIZE, ValidatorIndex]
@ -470,7 +444,7 @@ type
execution_block_hash*: Eth2Digest
execution_block_height*: uint64
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.7/specs/phase0/beacon-chain.md#validator
# https://github.com/ethereum/consensus-specs/blob/v1.4.0/specs/phase0/beacon-chain.md#validator
ValidatorStatus* = object
# This is a validator without the expensive, immutable, append-only parts
# serialized. They're represented in memory to allow in-place SSZ reading
@ -842,16 +816,6 @@ func shortLog*(v: PendingAttestation): auto =
proposer_index: v.proposer_index
)
func shortLog*(v: SomeAttestation): auto =
(
aggregation_bits: v.aggregation_bits,
data: shortLog(v.data),
signature: shortLog(v.signature)
)
template asTrusted*(x: Attestation): TrustedAttestation =
isomorphicCast[TrustedAttestation](x)
func shortLog*(v: SomeIndexedAttestation): auto =
(
attestating_indices: v.attesting_indices,
@ -894,7 +858,6 @@ func shortLog*(v: SomeSignedVoluntaryExit): auto =
)
chronicles.formatIt AttestationData: it.shortLog
chronicles.formatIt Attestation: it.shortLog
chronicles.formatIt Checkpoint: it.shortLog
const
@ -926,23 +889,6 @@ func init*(T: type GraffitiBytes, input: string): GraffitiBytes
raise newException(ValueError, "The graffiti value should be 32 characters or less")
distinctBase(result)[0 ..< input.len] = toBytes(input)
func init*(
T: type Attestation,
indices_in_committee: openArray[uint64],
committee_len: int,
data: AttestationData,
signature: ValidatorSig): Result[T, cstring] =
var bits = CommitteeValidatorsBits.init(committee_len)
for index_in_committee in indices_in_committee:
if index_in_committee >= committee_len.uint64: return err("Invalid index for committee")
bits.setBit index_in_committee
ok Attestation(
aggregation_bits: bits,
data: data,
signature: signature
)
func defaultGraffitiBytes*(): GraffitiBytes =
const graffitiBytes =
toBytes("Nimbus/" & fullVersionStr)

View File

@ -305,7 +305,7 @@ type
# Execution
execution_payload*: ExecutionPayload # [New in Bellatrix]
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.7/specs/phase0/beacon-chain.md#signedbeaconblock
# https://github.com/ethereum/consensus-specs/blob/v1.4.0/specs/phase0/beacon-chain.md#signedbeaconblock
SignedBeaconBlock* = object
message*: BeaconBlock
signature*: ValidatorSig

View File

@ -52,12 +52,12 @@ type
from_bls_pubkey*: ValidatorPubKey
to_execution_address*: ExecutionAddress
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.5/specs/capella/beacon-chain.md#signedblstoexecutionchange
# https://github.com/ethereum/consensus-specs/blob/v1.4.0/specs/capella/beacon-chain.md#signedblstoexecutionchange
SignedBLSToExecutionChange* = object
message*: BLSToExecutionChange
signature*: ValidatorSig
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.5/specs/capella/beacon-chain.md#historicalsummary
# https://github.com/ethereum/consensus-specs/blob/v1.4.0/specs/capella/beacon-chain.md#historicalsummary
HistoricalSummary* = object
# `HistoricalSummary` matches the components of the phase0
# `HistoricalBatch` making the two hash_tree_root-compatible.

View File

@ -40,7 +40,7 @@ const
FAR_FUTURE_EPOCH* = Epoch(not 0'u64)
FAR_FUTURE_PERIOD* = SyncCommitteePeriod(not 0'u64)
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.7/specs/phase0/beacon-chain.md#domain-types
# https://github.com/ethereum/consensus-specs/blob/v1.4.0/specs/phase0/beacon-chain.md#domain-types
DOMAIN_BEACON_PROPOSER* = DomainType([byte 0x00, 0x00, 0x00, 0x00])
DOMAIN_BEACON_ATTESTER* = DomainType([byte 0x01, 0x00, 0x00, 0x00])
DOMAIN_RANDAO* = DomainType([byte 0x02, 0x00, 0x00, 0x00])
@ -55,9 +55,12 @@ const
DOMAIN_SYNC_COMMITTEE_SELECTION_PROOF* = DomainType([byte 0x08, 0x00, 0x00, 0x00])
DOMAIN_CONTRIBUTION_AND_PROOF* = DomainType([byte 0x09, 0x00, 0x00, 0x00])
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.5/specs/capella/beacon-chain.md#domain-types
# https://github.com/ethereum/consensus-specs/blob/v1.4.0/specs/capella/beacon-chain.md#domain-types
DOMAIN_BLS_TO_EXECUTION_CHANGE* = DomainType([byte 0x0a, 0x00, 0x00, 0x00])
# https://github.com/ethereum/consensus-specs/blob/82133085a1295e93394ebdf71df8f2f6e0962588/specs/electra/beacon-chain.md#domains
DOMAIN_CONSOLIDATION* = DomainType([byte 0x0b, 0x00, 0x00, 0x00])
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.5/specs/phase0/fork-choice.md#configuration
PROPOSER_SCORE_BOOST*: uint64 = 40
REORG_HEAD_WEIGHT_THRESHOLD*: uint64 = 20
@ -77,5 +80,8 @@ const
GOSSIP_MAX_SIZE* = 10'u64 * 1024 * 1024 # bytes
MAX_CHUNK_SIZE* = 10'u64 * 1024 * 1024 # bytes
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.4/specs/deneb/p2p-interface.md#configuration
# https://github.com/ethereum/consensus-specs/blob/v1.4.0/specs/deneb/p2p-interface.md#configuration
MAX_REQUEST_BLOCKS_DENEB*: uint64 = 128 # TODO Make use of in request code
# https://github.com/ethereum/consensus-specs/blob/82133085a1295e93394ebdf71df8f2f6e0962588/specs/electra/beacon-chain.md#withdrawal-prefixes
COMPOUNDING_WITHDRAWAL_PREFIX* = 0x02

View File

@ -53,7 +53,7 @@ type
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.5/specs/deneb/polynomial-commitments.md#custom-types
Blob* = array[BYTES_PER_FIELD_ELEMENT * FIELD_ELEMENTS_PER_BLOB, byte]
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.7/specs/deneb/p2p-interface.md#blobsidecar
# https://github.com/ethereum/consensus-specs/blob/v1.4.0/specs/deneb/p2p-interface.md#blobsidecar
BlobSidecar* = object
index*: BlobIndex
## Index of blob in block

View File

@ -30,14 +30,8 @@ from ./deneb import Blobs, BlobsBundle, KzgCommitments, KzgProofs
export json_serialization, base, kzg4844
const
# Keep these here for now, since things still in flux
# https://github.com/ethereum/consensus-specs/pull/3615
MAX_DEPOSIT_RECEIPTS_PER_PAYLOAD* = 8192
MAX_EXECUTION_LAYER_EXITS_PER_PAYLOAD* = 16 # there's a discrepancy here, _PER_PAYLOAD or not
type
# https://github.com/ethereum/consensus-specs/pull/3615
# https://github.com/ethereum/consensus-specs/blob/94a0b6c581f2809aa8aca4ef7ee6fbb63f9d74e9/specs/electra/beacon-chain.md#depositreceipt
DepositReceipt* = object
pubkey*: ValidatorPubKey
withdrawal_credentials*: Eth2Digest
@ -45,12 +39,7 @@ type
signature*: ValidatorSig
index*: uint64
# https://github.com/ethereum/consensus-specs/pull/3615
ExecutionLayerExit* = object
source_address*: ExecutionAddress
validator_pubkey*: ValidatorPubKey
# https://github.com/ethereum/consensus-specs/pull/3615
# https://github.com/ethereum/consensus-specs/blob/82133085a1295e93394ebdf71df8f2f6e0962588/specs/electra/beacon-chain.md#executionpayload
ExecutionPayload* = object
# Execution block header fields
parent_hash*: Eth2Digest
@ -76,14 +65,17 @@ type
blob_gas_used*: uint64
excess_blob_gas*: uint64
deposit_receipts*: List[DepositReceipt, MAX_DEPOSIT_RECEIPTS_PER_PAYLOAD]
exits*: List[ExecutionLayerExit, MAX_EXECUTION_LAYER_EXITS_PER_PAYLOAD]
## [New in Electra:EIP6110]
withdrawal_requests*:
List[ExecutionLayerWithdrawalRequest, MAX_WITHDRAWAL_REQUESTS_PER_PAYLOAD]
## [New in Electra:EIP6110]
ExecutionPayloadForSigning* = object
executionPayload*: ExecutionPayload
blockValue*: Wei
blobsBundle*: BlobsBundle
# https://github.com/ethereum/consensus-specs/blob/v1.4.0/specs/deneb/beacon-chain.md#executionpayloadheader
# https://github.com/ethereum/consensus-specs/blob/82133085a1295e93394ebdf71df8f2f6e0962588/specs/electra/beacon-chain.md#executionpayloadheader
ExecutionPayloadHeader* = object
# Execution block header fields
parent_hash*: Eth2Digest
@ -104,14 +96,51 @@ type
## Hash of execution block
transactions_root*: Eth2Digest
withdrawals_root*: Eth2Digest
blob_gas_used*: uint64 # [New in Deneb:EIP4844]
excess_blob_gas*: uint64 # [New in Deneb:EIP4844]
deposit_receipts_root*: Eth2Digest
exits_root*: Eth2Digest
blob_gas_used*: uint64
excess_blob_gas*: uint64
deposit_receipts_root*: Eth2Digest # [New in Electra:EIP6110]
withdrawal_requests_root*: Eth2Digest # [New in Electra:EIP7002:EIP7251]
ExecutePayload* = proc(
execution_payload: ExecutionPayload): bool {.gcsafe, raises: [].}
# https://github.com/ethereum/consensus-specs/blob/82133085a1295e93394ebdf71df8f2f6e0962588/specs/electra/beacon-chain.md#depositreceipt
PendingBalanceDeposit* = object
index*: uint64
amount*: Gwei
# https://github.com/ethereum/consensus-specs/blob/82133085a1295e93394ebdf71df8f2f6e0962588/specs/electra/beacon-chain.md#pendingpartialwithdrawal
PendingPartialWithdrawal* = object
index*: uint64
amount*: Gwei
withdrawable_epoch*: Epoch
# https://github.com/ethereum/consensus-specs/blob/82133085a1295e93394ebdf71df8f2f6e0962588/specs/electra/beacon-chain.md#executionlayerwithdrawalrequest
ExecutionLayerWithdrawalRequest* = object
source_address*: ExecutionAddress
validator_pubkey*: ValidatorPubKey
amount*: Gwei
# https://github.com/ethereum/consensus-specs/blob/82133085a1295e93394ebdf71df8f2f6e0962588/specs/electra/beacon-chain.md#consolidation
Consolidation* = object
source_index*: uint64
target_index*: uint64
epoch*: Epoch
# https://github.com/ethereum/consensus-specs/blob/82133085a1295e93394ebdf71df8f2f6e0962588/specs/electra/beacon-chain.md#signedconsolidation
SignedConsolidation* = object
message*: Consolidation
signature*: ValidatorSig
TrustedSignedConsolidation* = object
message*: Consolidation
signature*: TrustedSig
# https://github.com/ethereum/consensus-specs/blob/82133085a1295e93394ebdf71df8f2f6e0962588/specs/electra/beacon-chain.md#pendingconsolidation
PendingConsolidation* = object
source_index*: uint64
target_index*: uint64
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.5/specs/capella/light-client/sync-protocol.md#modified-lightclientheader
LightClientHeader* = object
beacon*: BeaconBlockHeader
@ -210,8 +239,7 @@ type
## (used to compute safety threshold)
current_max_active_participants*: uint64
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.5/specs/capella/beacon-chain.md#beaconstate
# changes indirectly via ExecutionPayloadHeader
# https://github.com/ethereum/consensus-specs/blob/82133085a1295e93394ebdf71df8f2f6e0962588/specs/electra/beacon-chain.md#beaconstate
BeaconState* = object
# Versioning
genesis_time*: uint64
@ -268,6 +296,7 @@ type
# Execution
latest_execution_payload_header*: ExecutionPayloadHeader
## [Modified in Electra:EIP6110:EIP7002]
# Withdrawals
next_withdrawal_index*: WithdrawalIndex
@ -277,6 +306,23 @@ type
historical_summaries*:
HashList[HistoricalSummary, Limit HISTORICAL_ROOTS_LIMIT]
deposit_receipts_start_index*: uint64 # [New in Electra:EIP6110]
deposit_balance_to_consume*: Gwei # [New in Electra:EIP7251]
exit_balance_to_consume*: Gwei # [New in Electra:EIP7251]
earliest_exit_epoch*: Epoch # [New in Electra:EIP7251]
consolidation_balance_to_consume*: Gwei # [New in Electra:EIP7251]
earliest_consolidation_epoch*: Epoch # [New in Electra:EIP7251]
pending_balance_deposits*:
HashList[PendingBalanceDeposit, Limit PENDING_BALANCE_DEPOSITS_LIMIT]
## [New in Electra:EIP7251]
# [New in Electra:EIP7251]
pending_partial_withdrawals*:
HashList[PendingPartialWithdrawal, Limit PENDING_PARTIAL_WITHDRAWALS_LIMIT]
pending_consolidations*:
HashList[PendingConsolidation, Limit PENDING_CONSOLIDATIONS_LIMIT]
## [New in Electra:EIP7251]
# TODO Careful, not nil analysis is broken / incomplete and the semantics will
# likely change in future versions of the language:
# https://github.com/nim-lang/RFCs/issues/250
@ -345,7 +391,7 @@ type
state_root*: Eth2Digest
body*: TrustedBeaconBlockBody
# https://github.com/ethereum/consensus-specs/blob/v1.4.0/specs/deneb/beacon-chain.md#beaconblockbody
# https://github.com/ethereum/consensus-specs/blob/82133085a1295e93394ebdf71df8f2f6e0962588/specs/electra/beacon-chain.md#beaconblockbody
BeaconBlockBody* = object
randao_reveal*: ValidatorSig
eth1_data*: Eth1Data
@ -356,17 +402,22 @@ type
# Operations
proposer_slashings*: List[ProposerSlashing, Limit MAX_PROPOSER_SLASHINGS]
attester_slashings*: List[AttesterSlashing, Limit MAX_ATTESTER_SLASHINGS]
attestations*: List[Attestation, Limit MAX_ATTESTATIONS]
attester_slashings*:
List[AttesterSlashing, Limit MAX_ATTESTER_SLASHINGS_ELECTRA]
## [Modified in Electra:EIP7549]
attestations*: List[Attestation, Limit MAX_ATTESTATIONS_ELECTRA]
## [Modified in Electra:EIP7549]
deposits*: List[Deposit, Limit MAX_DEPOSITS]
voluntary_exits*: List[SignedVoluntaryExit, Limit MAX_VOLUNTARY_EXITS]
sync_aggregate*: SyncAggregate
# Execution
execution_payload*: ExecutionPayload # [Modified in Deneb]
execution_payload*: ExecutionPayload # [Modified in Electra:EIP6110:EIP7002]
bls_to_execution_changes*: SignedBLSToExecutionChangeList
blob_kzg_commitments*: KzgCommitments # [New in Deneb]
blob_kzg_commitments*: KzgCommitments
consolidations*: List[SignedConsolidation, Limit MAX_CONSOLIDATIONS]
## [New in Electra:EIP7251]
SigVerifiedBeaconBlockBody* = object
## A BeaconBlock body with signatures verified
@ -390,18 +441,24 @@ type
## Arbitrary data
# Operations
proposer_slashings*: List[TrustedProposerSlashing, Limit MAX_PROPOSER_SLASHINGS]
attester_slashings*: List[TrustedAttesterSlashing, Limit MAX_ATTESTER_SLASHINGS]
attestations*: List[TrustedAttestation, Limit MAX_ATTESTATIONS]
proposer_slashings*:
List[TrustedProposerSlashing, Limit MAX_PROPOSER_SLASHINGS]
attester_slashings*:
List[TrustedAttesterSlashing, Limit MAX_ATTESTER_SLASHINGS_ELECTRA]
## [Modified in Electra:EIP7549]
attestations*: List[TrustedAttestation, Limit MAX_ATTESTATIONS_ELECTRA]
## [Modified in Electra:EIP7549]
deposits*: List[Deposit, Limit MAX_DEPOSITS]
voluntary_exits*: List[TrustedSignedVoluntaryExit, Limit MAX_VOLUNTARY_EXITS]
sync_aggregate*: TrustedSyncAggregate
# Execution
execution_payload*: ExecutionPayload # [Modified in Deneb]
execution_payload*: ExecutionPayload # [Modified in Electra:EIP6110:EIP7002]
bls_to_execution_changes*: SignedBLSToExecutionChangeList
blob_kzg_commitments*: KzgCommitments # [New in Deneb]
blob_kzg_commitments*: KzgCommitments
consolidations*: List[TrustedSignedConsolidation, Limit MAX_CONSOLIDATIONS]
## [New in Electra:EIP7251]
TrustedBeaconBlockBody* = object
## A full verified block
@ -413,18 +470,24 @@ type
## Arbitrary data
# Operations
proposer_slashings*: List[TrustedProposerSlashing, Limit MAX_PROPOSER_SLASHINGS]
attester_slashings*: List[TrustedAttesterSlashing, Limit MAX_ATTESTER_SLASHINGS]
attestations*: List[TrustedAttestation, Limit MAX_ATTESTATIONS]
proposer_slashings*:
List[TrustedProposerSlashing, Limit MAX_PROPOSER_SLASHINGS]
attester_slashings*:
List[TrustedAttesterSlashing, Limit MAX_ATTESTER_SLASHINGS_ELECTRA]
## [Modified in Electra:EIP7549]
attestations*: List[TrustedAttestation, Limit MAX_ATTESTATIONS_ELECTRA]
## [Modified in Electra:EIP7549]
deposits*: List[Deposit, Limit MAX_DEPOSITS]
voluntary_exits*: List[TrustedSignedVoluntaryExit, Limit MAX_VOLUNTARY_EXITS]
sync_aggregate*: TrustedSyncAggregate
# Execution
execution_payload*: ExecutionPayload # [Modified in Deneb]
execution_payload*: ExecutionPayload # [Modified in Electra:EIP6110:EIP7002]
bls_to_execution_changes*: SignedBLSToExecutionChangeList
blob_kzg_commitments*: KzgCommitments # [New in Deneb]
blob_kzg_commitments*: KzgCommitments
consolidations*: List[TrustedSignedConsolidation, Limit MAX_CONSOLIDATIONS]
## [New in Electra:EIP7251]
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.6/specs/phase0/beacon-chain.md#signedbeaconblock
SignedBeaconBlock* = object

View File

@ -73,7 +73,7 @@ type
current_justified_checkpoint*: Checkpoint
finalized_checkpoint*: Checkpoint
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.6/specs/phase0/beacon-chain.md#get_total_balance
# https://github.com/ethereum/consensus-specs/blob/v1.4.0/specs/phase0/beacon-chain.md#get_total_balance
TotalBalances* = object
# The total effective balance of all active validators during the _current_
# epoch.
@ -111,7 +111,7 @@ type
data*: BeaconState
root*: Eth2Digest # hash_tree_root(data)
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.6/specs/phase0/beacon-chain.md#beaconblock
# https://github.com/ethereum/consensus-specs/blob/v1.4.0/specs/phase0/beacon-chain.md#beaconblock
BeaconBlock* = object
## For each slot, a proposer is chosen from the validator pool to propose
## a new block. Once the block as been proposed, it is transmitted to
@ -219,7 +219,7 @@ type
deposits*: List[Deposit, Limit MAX_DEPOSITS]
voluntary_exits*: List[TrustedSignedVoluntaryExit, Limit MAX_VOLUNTARY_EXITS]
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.6/specs/phase0/beacon-chain.md#signedbeaconblock
# https://github.com/ethereum/consensus-specs/blob/v1.4.0/specs/phase0/beacon-chain.md#signedbeaconblock
SignedBeaconBlock* = object
message*: BeaconBlock
signature*: ValidatorSig
@ -257,6 +257,31 @@ type
root* {.dontSerialize.}: Eth2Digest # cached root of signed beacon block
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.6/specs/phase0/beacon-chain.md#attestation
Attestation* = object
aggregation_bits*: CommitteeValidatorsBits
data*: AttestationData
signature*: ValidatorSig
TrustedAttestation* = object
# The Trusted version, at the moment, implies that the cryptographic signature was checked.
# It DOES NOT imply that the state transition was verified.
# Currently the code MUST verify the state transition as soon as the signature is verified
aggregation_bits*: CommitteeValidatorsBits
data*: AttestationData
signature*: TrustedSig
# https://github.com/ethereum/consensus-specs/blob/v1.4.0/specs/phase0/validator.md#aggregateandproof
AggregateAndProof* = object
aggregator_index*: uint64 # `ValidatorIndex` after validation
aggregate*: Attestation
selection_proof*: ValidatorSig
# https://github.com/ethereum/consensus-specs/blob/v1.4.0/specs/phase0/validator.md#signedaggregateandproof
SignedAggregateAndProof* = object
message*: AggregateAndProof
signature*: ValidatorSig
SomeSignedBeaconBlock* =
SignedBeaconBlock |
SigVerifiedSignedBeaconBlock |
@ -270,6 +295,7 @@ type
BeaconBlockBody |
SigVerifiedBeaconBlockBody |
TrustedBeaconBlockBody
SomeAttestation* = Attestation | TrustedAttestation
EpochInfo* = object
## Information about the outcome of epoch processing
@ -277,6 +303,7 @@ type
balances*: TotalBalances
chronicles.formatIt BeaconBlock: it.shortLog
chronicles.formatIt Attestation: it.shortLog
func clear*(info: var EpochInfo) =
info.validators.setLen(0)
@ -314,6 +341,16 @@ func shortLog*(v: SomeSignedBeaconBlock): auto =
signature: shortLog(v.signature)
)
func shortLog*(v: SomeAttestation): auto =
(
aggregation_bits: v.aggregation_bits,
data: shortLog(v.data),
signature: shortLog(v.signature)
)
template asTrusted*(x: Attestation): TrustedAttestation =
isomorphicCast[TrustedAttestation](x)
template asSigned*(
x: SigVerifiedSignedBeaconBlock |
MsgTrustedSignedBeaconBlock |
@ -341,3 +378,20 @@ template asTrusted*(
SigVerifiedSignedBeaconBlock |
MsgTrustedSignedBeaconBlock): TrustedSignedBeaconBlock =
isomorphicCast[TrustedSignedBeaconBlock](x)
func init*(
T: type Attestation,
indices_in_committee: openArray[uint64],
committee_len: int,
data: AttestationData,
signature: ValidatorSig): Result[T, cstring] =
var bits = CommitteeValidatorsBits.init(committee_len)
for index_in_committee in indices_in_committee:
if index_in_committee >= committee_len.uint64: return err("Invalid index for committee")
bits.setBit index_in_committee
ok Attestation(
aggregation_bits: bits,
data: data,
signature: signature
)

View File

@ -9,7 +9,7 @@
# Consensus hash function / digest
#
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.6/specs/phase0/beacon-chain.md#hash
# https://github.com/ethereum/consensus-specs/blob/v1.4.0/specs/phase0/beacon-chain.md#hash
#
# In Phase 0 the beacon chain is deployed with SHA256 (SHA2-256).
# Note that is is different from Keccak256 (often mistakenly called SHA3-256)

View File

@ -45,7 +45,6 @@ createJsonFlavor RestJson
RestJson.useDefaultSerializationFor(
AggregateAndProof,
Attestation,
AttestationData,
AttesterSlashing,
BLSToExecutionChange,
@ -54,6 +53,7 @@ RestJson.useDefaultSerializationFor(
BlobSidecarInfoObject,
BlobsBundle,
Checkpoint,
Consolidation,
ContributionAndProof,
DataEnclosedObject,
DataMetaEnclosedObject,
@ -74,7 +74,7 @@ RestJson.useDefaultSerializationFor(
EmptyBody,
Eth1Data,
EventBeaconBlockObject,
ExecutionLayerExit,
ExecutionLayerWithdrawalRequest,
Fork,
GetBlockAttestationsResponse,
GetBlockHeaderResponse,
@ -115,6 +115,9 @@ RestJson.useDefaultSerializationFor(
GetGraffitiResponse,
GraffitiResponse,
PendingAttestation,
PendingBalanceDeposit,
PendingConsolidation,
PendingPartialWithdrawal,
PostKeystoresResponse,
PrepareBeaconProposer,
ProposerSlashing,
@ -170,6 +173,7 @@ RestJson.useDefaultSerializationFor(
SignedAggregateAndProof,
SignedBLSToExecutionChange,
SignedBeaconBlockHeader,
SignedConsolidation,
SignedContributionAndProof,
SignedValidatorRegistrationV1,
SignedVoluntaryExit,
@ -180,7 +184,6 @@ RestJson.useDefaultSerializationFor(
SyncCommittee,
SyncCommitteeContribution,
SyncCommitteeMessage,
TrustedAttestation,
Validator,
ValidatorRegistrationV1,
VoluntaryExit,
@ -258,10 +261,12 @@ RestJson.useDefaultSerializationFor(
electra_mev.ExecutionPayloadAndBlobsBundle,
electra_mev.SignedBlindedBeaconBlock,
electra_mev.SignedBuilderBid,
phase0.Attestation,
phase0.BeaconBlock,
phase0.BeaconBlockBody,
phase0.BeaconState,
phase0.SignedBeaconBlock,
phase0.TrustedAttestation
)
# TODO
@ -349,7 +354,7 @@ type
ForkedMaybeBlindedBeaconBlock
EncodeArrays* =
seq[Attestation] |
seq[phase0.Attestation] |
seq[PrepareBeaconProposer] |
seq[RemoteKeystoreInfo] |
seq[RestCommitteeSubscription] |
@ -1705,7 +1710,7 @@ proc readValue*(reader: var JsonReader[RestJson],
Opt[List[ProposerSlashing, Limit MAX_PROPOSER_SLASHINGS]]
attester_slashings:
Opt[List[AttesterSlashing, Limit MAX_ATTESTER_SLASHINGS]]
attestations: Opt[List[Attestation, Limit MAX_ATTESTATIONS]]
attestations: Opt[List[phase0.Attestation, Limit MAX_ATTESTATIONS]]
deposits: Opt[List[Deposit, Limit MAX_DEPOSITS]]
voluntary_exits: Opt[List[SignedVoluntaryExit, Limit MAX_VOLUNTARY_EXITS]]
sync_aggregate: Opt[SyncAggregate]
@ -1749,7 +1754,7 @@ proc readValue*(reader: var JsonReader[RestJson],
reader.raiseUnexpectedField("Multiple `attestations` fields found",
"RestPublishedBeaconBlockBody")
attestations = Opt.some(
reader.readValue(List[Attestation, Limit MAX_ATTESTATIONS]))
reader.readValue(List[phase0.Attestation, Limit MAX_ATTESTATIONS]))
of "deposits":
if deposits.isSome():
reader.raiseUnexpectedField("Multiple `deposits` fields found",
@ -1937,8 +1942,8 @@ proc readValue*(reader: var JsonReader[RestJson],
eth1_data: eth1_data.get(),
graffiti: graffiti.get(),
proposer_slashings: proposer_slashings.get(),
attester_slashings: attester_slashings.get(),
attestations: attestations.get(),
#attester_slashings: attester_slashings.get(),
#attestations: attestations.get(),
deposits: deposits.get(),
voluntary_exits: voluntary_exits.get(),
sync_aggregate: sync_aggregate.get(),
@ -1957,7 +1962,7 @@ proc readValue*(reader: var JsonReader[RestJson],
value.electraBody.execution_payload.excess_blob_gas,
ep_src.excess_blob_gas.get())
debugRaiseAssert "electra support missing"
debugRaiseAssert "electra support missing, including attslashing/atts"
## RestPublishedBeaconBlock
proc readValue*(reader: var JsonReader[RestJson],
@ -3403,10 +3408,7 @@ proc writeValue*(writer: var JsonWriter[RestJson],
writer.beginRecord()
withForkyMaybeBlindedBlck(value):
writer.writeField("version", consensusFork.toString())
when isBlinded:
writer.writeField("execution_payload_blinded", "true")
else:
writer.writeField("execution_payload_blinded", "false")
writer.writeField("execution_payload_blinded", isBlinded)
if value.executionValue.isSome():
writer.writeField("execution_payload_value",
$(value.executionValue.get()))

View File

@ -321,7 +321,8 @@ proc getPoolAttestations*(
meth: MethodGet.}
## https://ethereum.github.io/beacon-APIs/#/Beacon/getPoolAttestations
proc submitPoolAttestations*(body: seq[Attestation]): RestPlainResponse {.
proc submitPoolAttestations*(body: seq[phase0.Attestation]):
RestPlainResponse {.
rest, endpoint: "/eth/v1/beacon/pool/attestations",
meth: MethodPost.}
## https://ethereum.github.io/beacon-APIs/#/Beacon/submitPoolAttestations

View File

@ -289,7 +289,7 @@ type
RestWithdrawalPrefix* = distinct array[1, byte]
# https://github.com/ethereum/consensus-specs/blob/v1.3.0/specs/capella/beacon-chain.md#executionpayload
# https://github.com/ethereum/consensus-specs/blob/v1.4.0/specs/capella/beacon-chain.md#executionpayload
RestExecutionPayload* = object
# Execution block header fields
parent_hash*: Eth2Digest
@ -535,9 +535,9 @@ type
# Types based on the OAPI yaml file - used in responses to requests
GetBeaconHeadResponse* = DataEnclosedObject[Slot]
GetAggregatedAttestationResponse* = DataEnclosedObject[Attestation]
GetAggregatedAttestationResponse* = DataEnclosedObject[phase0.Attestation]
GetAttesterDutiesResponse* = DataRootEnclosedObject[seq[RestAttesterDuty]]
GetBlockAttestationsResponse* = DataEnclosedObject[seq[Attestation]]
GetBlockAttestationsResponse* = DataEnclosedObject[seq[phase0.Attestation]]
GetBlockHeaderResponse* = DataOptimisticAndFinalizedObject[RestBlockHeaderInfo]
GetBlockHeadersResponse* = DataEnclosedObject[seq[RestBlockHeaderInfo]]
GetBlockRootResponse* = DataOptimisticObject[RestRoot]
@ -553,7 +553,7 @@ type
GetPeerCountResponse* = DataMetaEnclosedObject[RestPeerCount]
GetPeerResponse* = DataMetaEnclosedObject[RestNodePeer]
GetPeersResponse* = DataMetaEnclosedObject[seq[RestNodePeer]]
GetPoolAttestationsResponse* = DataEnclosedObject[seq[Attestation]]
GetPoolAttestationsResponse* = DataEnclosedObject[seq[phase0.Attestation]]
GetPoolAttesterSlashingsResponse* = DataEnclosedObject[seq[AttesterSlashing]]
GetPoolProposerSlashingsResponse* = DataEnclosedObject[seq[ProposerSlashing]]
GetPoolVoluntaryExitsResponse* = DataEnclosedObject[seq[SignedVoluntaryExit]]

View File

@ -98,10 +98,10 @@ func init*(T: type HashedValidatorPubKey, key: ValidatorPubKey): HashedValidator
# The interface of HashSet is such that we must construct a full
# instance to check if it's in the set - then we can return that
# instace and discard the one we just created temporarily
keys[tmp]
addr keys[tmp][]
except KeyError:
raiseAssert "just checked"
else:
tmp
addr tmp[]
HashedValidatorPubKey(value: addr cached[])
HashedValidatorPubKey(value: cached) # https://github.com/nim-lang/Nim/issues/23505

View File

@ -1389,7 +1389,7 @@ func readSszForkedSignedBeaconBlock*(
withBlck(result):
readSszBytes(data, forkyBlck)
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.6/specs/phase0/beacon-chain.md#compute_fork_data_root
# https://github.com/ethereum/consensus-specs/blob/v1.4.0/specs/phase0/beacon-chain.md#compute_fork_data_root
func compute_fork_data_root*(current_version: Version,
genesis_validators_root: Eth2Digest): Eth2Digest =
## Return the 32-byte fork data root for the ``current_version`` and

View File

@ -107,12 +107,12 @@ func get_active_validator_indices_len*(
withState(state):
get_active_validator_indices_len(forkyState.data, epoch)
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.6/specs/phase0/beacon-chain.md#get_current_epoch
# https://github.com/ethereum/consensus-specs/blob/v1.4.0/specs/phase0/beacon-chain.md#get_current_epoch
func get_current_epoch*(state: ForkyBeaconState): Epoch =
## Return the current epoch.
state.slot.epoch
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.6/specs/phase0/beacon-chain.md#get_current_epoch
# https://github.com/ethereum/consensus-specs/blob/v1.4.0/specs/phase0/beacon-chain.md#get_current_epoch
func get_current_epoch*(state: ForkedHashedBeaconState): Epoch =
## Return the current epoch.
withState(state): get_current_epoch(forkyState.data)
@ -123,7 +123,7 @@ func get_previous_epoch*(
## Return the previous epoch (unless the current epoch is ``GENESIS_EPOCH``).
get_previous_epoch(get_current_epoch(state))
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.6/specs/phase0/beacon-chain.md#get_randao_mix
# https://github.com/ethereum/consensus-specs/blob/v1.4.0/specs/phase0/beacon-chain.md#get_randao_mix
func get_randao_mix*(state: ForkyBeaconState, epoch: Epoch): Eth2Digest =
## Return the randao mix at a recent ``epoch``.
state.randao_mixes[epoch mod EPOCHS_PER_HISTORICAL_VECTOR]
@ -159,7 +159,7 @@ func compute_domain*(
result[0..3] = domain_type.data
result[4..31] = fork_data_root.data.toOpenArray(0, 27)
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.6/specs/phase0/beacon-chain.md#get_domain
# https://github.com/ethereum/consensus-specs/blob/v1.4.0/specs/phase0/beacon-chain.md#get_domain
func get_domain*(
fork: Fork,
domain_type: DomainType,
@ -271,7 +271,7 @@ template is_finality_update*(update: SomeForkyLightClientUpdate): bool =
else:
false
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.5/specs/altair/light-client/sync-protocol.md#is_next_sync_committee_known
# https://github.com/ethereum/consensus-specs/blob/v1.4.0/specs/altair/light-client/sync-protocol.md#is_next_sync_committee_known
template is_next_sync_committee_known*(store: ForkyLightClientStore): bool =
store.next_sync_committee !=
static(default(typeof(store.next_sync_committee)))

View File

@ -8,6 +8,7 @@
{.push raises: [].}
import ".."/datatypes/altair
from ".."/datatypes/phase0 import Attestation
from ".."/datatypes/bellatrix import ExecutionPayloadHeader
from ".."/eth2_merkleization import hash_tree_root

View File

@ -8,6 +8,7 @@
{.push raises: [].}
import ".."/datatypes/[altair, capella]
from ".."/datatypes/phase0 import Attestation
from stew/byteutils import to0xHex
from ../eth2_merkleization import fromSszBytes, hash_tree_root, toSszType

View File

@ -10,6 +10,7 @@
import ".."/datatypes/[altair, deneb]
from stew/byteutils import to0xHex
from ".."/datatypes/phase0 import Attestation
from ../datatypes/bellatrix import ExecutionAddress
from ".."/datatypes/capella import SignedBLSToExecutionChange
from ".."/eth2_merkleization import hash_tree_root

View File

@ -10,6 +10,7 @@
import ".."/datatypes/[altair, electra]
from stew/byteutils import to0xHex
from ".."/datatypes/phase0 import Attestation
from ../datatypes/bellatrix import ExecutionAddress
from ".."/datatypes/capella import SignedBLSToExecutionChange
from ".."/datatypes/deneb import BlobsBundle, KzgCommitments
@ -32,8 +33,9 @@ type
eth1_data*: Eth1Data
graffiti*: GraffitiBytes
proposer_slashings*: List[ProposerSlashing, Limit MAX_PROPOSER_SLASHINGS]
attester_slashings*: List[AttesterSlashing, Limit MAX_ATTESTER_SLASHINGS]
attestations*: List[Attestation, Limit MAX_ATTESTATIONS]
attester_slashings*:
List[AttesterSlashing, Limit MAX_ATTESTER_SLASHINGS_ELECTRA]
attestations*: List[phase0.Attestation, Limit MAX_ATTESTATIONS_ELECTRA]
deposits*: List[Deposit, Limit MAX_DEPOSITS]
voluntary_exits*: List[SignedVoluntaryExit, Limit MAX_VOLUNTARY_EXITS]
sync_aggregate*: SyncAggregate
@ -42,6 +44,7 @@ type
List[SignedBLSToExecutionChange,
Limit MAX_BLS_TO_EXECUTION_CHANGES]
blob_kzg_commitments*: KzgCommitments # [New in Deneb]
consolidations*: List[SignedConsolidation, Limit MAX_CONSOLIDATIONS]
# https://github.com/ethereum/builder-specs/blob/v0.4.0/specs/bellatrix/builder.md#blindedbeaconblock
BlindedBeaconBlock* = object
@ -141,8 +144,9 @@ func toSignedBlindedBeaconBlock*(blck: electra.SignedBeaconBlock):
hash_tree_root(blck.message.body.execution_payload.withdrawals),
deposit_receipts_root: hash_tree_root(
blck.message.body.execution_payload.deposit_receipts),
exits_root:
hash_tree_root(blck.message.body.execution_payload.exits)),
withdrawal_requests_root:
hash_tree_root(
blck.message.body.execution_payload.withdrawal_requests)),
bls_to_execution_changes: blck.message.body.bls_to_execution_changes,
blob_kzg_commitments: blck.message.body.blob_kzg_commitments)),
signature: blck.signature)

View File

@ -779,7 +779,7 @@ proc readRuntimeConfig*(
"MAX_REQUEST_BLOB_SIDECARS"
checkCompatibility BLOB_SIDECAR_SUBNET_COUNT
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.5/specs/phase0/fork-choice.md#configuration
# https://github.com/ethereum/consensus-specs/blob/v1.4.0/specs/phase0/fork-choice.md#configuration
# Isn't being used as a preset in the usual way: at any time, there's one correct value
checkCompatibility PROPOSER_SCORE_BOOST
checkCompatibility REORG_HEAD_WEIGHT_THRESHOLD

View File

@ -1,15 +1,17 @@
# beacon_chain
# Copyright (c) 2023 Status Research & Development GmbH
# Copyright (c) 2023-2024 Status Research & Development GmbH
# Licensed and distributed under either of
# * MIT license (license terms in the root directory or at https://opensource.org/licenses/MIT).
# * Apache v2 license (license terms in the root directory or at https://www.apache.org/licenses/LICENSE-2.0).
# at your option. This file may not be copied, modified, or distributed except according to those terms.
{.push raises: [].}
import
./gnosis/[
phase0_preset, altair_preset, bellatrix_preset, capella_preset,
deneb_preset]
deneb_preset, electra_preset]
export
phase0_preset, altair_preset, bellatrix_preset, capella_preset,
deneb_preset
deneb_preset, electra_preset

View File

@ -0,0 +1,55 @@
# beacon_chain
# Copyright (c) 2024 Status Research & Development GmbH
# Licensed and distributed under either of
# * MIT license (license terms in the root directory or at https://opensource.org/licenses/MIT).
# * Apache v2 license (license terms in the root directory or at https://www.apache.org/licenses/LICENSE-2.0).
# at your option. This file may not be copied, modified, or distributed except according to those terms.
{.push raises: [].}
# Gnosis preset - Electra (Gnosis version not avilable yet; EF mainnet for now)
# https://github.com/ethereum/consensus-specs/blob/82133085a1295e93394ebdf71df8f2f6e0962588/presets/mainnet/electra.yaml
const
# Gwei values
# ---------------------------------------------------------------
# 2**5 * 10**9 (= 32,000,000,000) Gwei
MIN_ACTIVATION_BALANCE* = 32000000000
# 2**11 * 10**9 (= 2,048,000,000,000) Gwei
MAX_EFFECTIVE_BALANCE_ELECTRA* = 2048000000000
# State list lengths
# ---------------------------------------------------------------
# `uint64(2**27)` (= 134,217,728)
PENDING_BALANCE_DEPOSITS_LIMIT*: uint64 = 134217728
# `uint64(2**27)` (= 134,217,728)
PENDING_PARTIAL_WITHDRAWALS_LIMIT*: uint64 = 134217728
# `uint64(2**18)` (= 262,144)
PENDING_CONSOLIDATIONS_LIMIT*: uint64 = 262144
# Reward and penalty quotients
# ---------------------------------------------------------------
# `uint64(2**12)` (= 4,096)
MIN_SLASHING_PENALTY_QUOTIENT_ELECTRA*: uint64 = 4096
# `uint64(2**12)` (= 4,096)
WHISTLEBLOWER_REWARD_QUOTIENT_ELECTRA*: uint64 = 4096
# # Max operations per block
# ---------------------------------------------------------------
# `uint64(2**0)` (= 1)
MAX_ATTESTER_SLASHINGS_ELECTRA*: uint64 = 1
# `uint64(2**3)` (= 8)
MAX_ATTESTATIONS_ELECTRA*: uint64 = 8
# `uint64(2**0)` (= 1)
MAX_CONSOLIDATIONS*: uint64 = 1
# Execution
# ---------------------------------------------------------------
# 2**13 (= 8192) receipts
MAX_DEPOSIT_RECEIPTS_PER_PAYLOAD* = 8192
# 2**4 (= 16) withdrawal requests
MAX_WITHDRAWAL_REQUESTS_PER_PAYLOAD* = 16
# Withdrawals processing
# ---------------------------------------------------------------
# 2**3 ( = 8) pending withdrawals
MAX_PENDING_PARTIALS_PER_WITHDRAWALS_SWEEP* = 8

View File

@ -5,11 +5,13 @@
# * Apache v2 license (license terms in the root directory or at https://www.apache.org/licenses/LICENSE-2.0).
# at your option. This file may not be copied, modified, or distributed except according to those terms.
{.push raises: [].}
import
./mainnet/[
phase0_preset, altair_preset, bellatrix_preset, capella_preset,
deneb_preset]
deneb_preset, electra_preset]
export
phase0_preset, altair_preset, bellatrix_preset, capella_preset,
deneb_preset
deneb_preset, electra_preset

View File

@ -0,0 +1,55 @@
# beacon_chain
# Copyright (c) 2024 Status Research & Development GmbH
# Licensed and distributed under either of
# * MIT license (license terms in the root directory or at https://opensource.org/licenses/MIT).
# * Apache v2 license (license terms in the root directory or at https://www.apache.org/licenses/LICENSE-2.0).
# at your option. This file may not be copied, modified, or distributed except according to those terms.
{.push raises: [].}
# Electra preset - Electra
# https://github.com/ethereum/consensus-specs/blob/82133085a1295e93394ebdf71df8f2f6e0962588/presets/mainnet/electra.yaml
const
# Gwei values
# ---------------------------------------------------------------
# 2**5 * 10**9 (= 32,000,000,000) Gwei
MIN_ACTIVATION_BALANCE* = 32000000000
# 2**11 * 10**9 (= 2,048,000,000,000) Gwei
MAX_EFFECTIVE_BALANCE_ELECTRA* = 2048000000000
# State list lengths
# ---------------------------------------------------------------
# `uint64(2**27)` (= 134,217,728)
PENDING_BALANCE_DEPOSITS_LIMIT*: uint64 = 134217728
# `uint64(2**27)` (= 134,217,728)
PENDING_PARTIAL_WITHDRAWALS_LIMIT*: uint64 = 134217728
# `uint64(2**18)` (= 262,144)
PENDING_CONSOLIDATIONS_LIMIT*: uint64 = 262144
# Reward and penalty quotients
# ---------------------------------------------------------------
# `uint64(2**12)` (= 4,096)
MIN_SLASHING_PENALTY_QUOTIENT_ELECTRA*: uint64 = 4096
# `uint64(2**12)` (= 4,096)
WHISTLEBLOWER_REWARD_QUOTIENT_ELECTRA*: uint64 = 4096
# # Max operations per block
# ---------------------------------------------------------------
# `uint64(2**0)` (= 1)
MAX_ATTESTER_SLASHINGS_ELECTRA*: uint64 = 1
# `uint64(2**3)` (= 8)
MAX_ATTESTATIONS_ELECTRA*: uint64 = 8
# `uint64(2**0)` (= 1)
MAX_CONSOLIDATIONS*: uint64 = 1
# Execution
# ---------------------------------------------------------------
# 2**13 (= 8192) receipts
MAX_DEPOSIT_RECEIPTS_PER_PAYLOAD* = 8192
# 2**4 (= 16) withdrawal requests
MAX_WITHDRAWAL_REQUESTS_PER_PAYLOAD* = 16
# Withdrawals processing
# ---------------------------------------------------------------
# 2**3 ( = 8) pending withdrawals
MAX_PENDING_PARTIALS_PER_WITHDRAWALS_SWEEP* = 8

View File

@ -5,11 +5,13 @@
# * Apache v2 license (license terms in the root directory or at https://www.apache.org/licenses/LICENSE-2.0).
# at your option. This file may not be copied, modified, or distributed except according to those terms.
{.push raises: [].}
import
./minimal/[
phase0_preset, altair_preset, bellatrix_preset, capella_preset,
deneb_preset]
deneb_preset, electra_preset]
export
phase0_preset, altair_preset, bellatrix_preset, capella_preset,
deneb_preset
deneb_preset, electra_preset

View File

@ -0,0 +1,55 @@
# beacon_chain
# Copyright (c) 2024 Status Research & Development GmbH
# Licensed and distributed under either of
# * MIT license (license terms in the root directory or at https://opensource.org/licenses/MIT).
# * Apache v2 license (license terms in the root directory or at https://www.apache.org/licenses/LICENSE-2.0).
# at your option. This file may not be copied, modified, or distributed except according to those terms.
{.push raises: [].}
# Minimal preset - Electra
# https://github.com/ethereum/consensus-specs/blob/82133085a1295e93394ebdf71df8f2f6e0962588/presets/minimal/electra.yaml
const
# Gwei values
# ---------------------------------------------------------------
# 2**5 * 10**9 (= 32,000,000,000) Gwei
MIN_ACTIVATION_BALANCE* = 32000000000
# 2**11 * 10**9 (= 2,048,000,000,000) Gwei
MAX_EFFECTIVE_BALANCE_ELECTRA* = 2048000000000
# State list lengths
# ---------------------------------------------------------------
# `uint64(2**27)` (= 134,217,728)
PENDING_BALANCE_DEPOSITS_LIMIT*: uint64 = 134217728
# customized] `uint64(2**6)` (= 64)
PENDING_PARTIAL_WITHDRAWALS_LIMIT*: uint64 = 64
# [customized] `uint64(2**6)` (= 64)
PENDING_CONSOLIDATIONS_LIMIT*: uint64 = 64
# Reward and penalty quotients
# ---------------------------------------------------------------
# `uint64(2**12)` (= 4,096)
MIN_SLASHING_PENALTY_QUOTIENT_ELECTRA*: uint64 = 4096
# `uint64(2**12)` (= 4,096)
WHISTLEBLOWER_REWARD_QUOTIENT_ELECTRA*: uint64 = 4096
# # Max operations per block
# ---------------------------------------------------------------
# `uint64(2**0)` (= 1)
MAX_ATTESTER_SLASHINGS_ELECTRA*: uint64 = 1
# `uint64(2**3)` (= 8)
MAX_ATTESTATIONS_ELECTRA*: uint64 = 8
# `uint64(2**0)` (= 1)
MAX_CONSOLIDATIONS*: uint64 = 1
# Execution
# ---------------------------------------------------------------
# [customized]
MAX_DEPOSIT_RECEIPTS_PER_PAYLOAD* = 4
# [customized] 2**1 (= 2) withdrawal requests
MAX_WITHDRAWAL_REQUESTS_PER_PAYLOAD* = 2
# Withdrawals processing
# ---------------------------------------------------------------
# 2**0 ( = 1) pending withdrawals
MAX_PENDING_PARTIALS_PER_WITHDRAWALS_SWEEP* = 1

View File

@ -44,7 +44,7 @@ func compute_slot_signing_root*(
fork, DOMAIN_SELECTION_PROOF, epoch, genesis_validators_root)
compute_signing_root(slot, domain)
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.3/specs/phase0/validator.md#aggregation-selection
# https://github.com/ethereum/consensus-specs/blob/v1.4.0/specs/phase0/validator.md#aggregation-selection
func get_slot_signature*(
fork: Fork, genesis_validators_root: Eth2Digest, slot: Slot,
privkey: ValidatorPrivKey): CookedSig =
@ -171,7 +171,7 @@ func compute_deposit_signing_root(
domain = compute_domain(DOMAIN_DEPOSIT, version)
compute_signing_root(deposit_message, domain)
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.6/specs/phase0/beacon-chain.md#deposits
# https://github.com/ethereum/consensus-specs/blob/v1.4.0/specs/phase0/beacon-chain.md#deposits
func get_deposit_signature*(preset: RuntimeConfig,
deposit: DepositData,
privkey: ValidatorPrivKey): CookedSig =
@ -392,7 +392,7 @@ proc verify_builder_signature*(
let signing_root = compute_builder_signing_root(fork, msg)
blsVerify(pubkey, signing_root.data, signature)
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.5/specs/capella/beacon-chain.md#new-process_bls_to_execution_change
# https://github.com/ethereum/consensus-specs/blob/v1.4.0/specs/capella/beacon-chain.md#new-process_bls_to_execution_change
func compute_bls_to_execution_change_signing_root*(
genesisFork: Fork, genesis_validators_root: Eth2Digest,
msg: BLSToExecutionChange): Eth2Digest =

View File

@ -70,7 +70,7 @@ proc verify_block_signature(
ok()
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.6/specs/phase0/beacon-chain.md#beacon-chain-state-transition-function
# https://github.com/ethereum/consensus-specs/blob/v1.4.0/specs/phase0/beacon-chain.md#beacon-chain-state-transition-function
func verifyStateRoot(
state: ForkyBeaconState,
blck: ForkyBeaconBlock | ForkySigVerifiedBeaconBlock):
@ -348,12 +348,14 @@ proc state_transition*(
func partialBeaconBlock*(
cfg: RuntimeConfig,
state: var ForkyHashedBeaconState,
state: var (phase0.HashedBeaconState | altair.HashedBeaconState |
bellatrix.HashedBeaconState | capella.HashedBeaconState |
deneb.HashedBeaconState),
proposer_index: ValidatorIndex,
randao_reveal: ValidatorSig,
eth1_data: Eth1Data,
graffiti: GraffitiBytes,
attestations: seq[Attestation],
attestations: seq[phase0.Attestation],
deposits: seq[Deposit],
validator_changes: BeaconBlockValidatorChanges,
sync_aggregate: SyncAggregate,
@ -372,7 +374,8 @@ func partialBeaconBlock*(
graffiti: graffiti,
proposer_slashings: validator_changes.proposer_slashings,
attester_slashings: validator_changes.attester_slashings,
attestations: List[Attestation, Limit MAX_ATTESTATIONS](attestations),
attestations:
List[phase0.Attestation, Limit MAX_ATTESTATIONS](attestations),
deposits: List[Deposit, Limit MAX_DEPOSITS](deposits),
voluntary_exits: validator_changes.voluntary_exits))
@ -380,7 +383,7 @@ func partialBeaconBlock*(
when consensusFork >= ConsensusFork.Altair:
res.body.sync_aggregate = sync_aggregate
# https://github.com/ethereum/consensus-specs/blob/v1.3.0/specs/bellatrix/validator.md#block-proposal
# https://github.com/ethereum/consensus-specs/blob/v1.4.0/specs/bellatrix/validator.md#block-proposal
when consensusFork >= ConsensusFork.Bellatrix:
res.body.execution_payload = execution_payload.executionPayload
@ -393,7 +396,57 @@ func partialBeaconBlock*(
when consensusFork >= ConsensusFork.Deneb:
res.body.blob_kzg_commitments = execution_payload.blobsBundle.commitments
debugRaiseAssert "check for new fields or conditions to ensure in electra"
res
func partialBeaconBlock*(
cfg: RuntimeConfig,
state: var electra.HashedBeaconState,
proposer_index: ValidatorIndex,
randao_reveal: ValidatorSig,
eth1_data: Eth1Data,
graffiti: GraffitiBytes,
attestations: seq[phase0.Attestation],
deposits: seq[Deposit],
validator_changes: BeaconBlockValidatorChanges,
sync_aggregate: SyncAggregate,
execution_payload: ForkyExecutionPayloadForSigning
): auto =
const consensusFork = typeof(state).kind
# https://github.com/ethereum/consensus-specs/blob/v1.3.0/specs/phase0/validator.md#preparing-for-a-beaconblock
var res = consensusFork.BeaconBlock(
slot: state.data.slot,
proposer_index: proposer_index.uint64,
parent_root: state.latest_block_root,
body: consensusFork.BeaconBlockBody(
randao_reveal: randao_reveal,
eth1_data: eth1_data,
graffiti: graffiti,
proposer_slashings: validator_changes.proposer_slashings,
#attester_slashings: validator_changes.attester_slashings,
attestations:
List[phase0.Attestation, Limit MAX_ATTESTATIONS_ELECTRA](attestations),
deposits: List[Deposit, Limit MAX_DEPOSITS](deposits),
voluntary_exits: validator_changes.voluntary_exits))
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/altair/validator.md#preparing-a-beaconblock
when consensusFork >= ConsensusFork.Altair:
res.body.sync_aggregate = sync_aggregate
# https://github.com/ethereum/consensus-specs/blob/v1.4.0/specs/bellatrix/validator.md#block-proposal
when consensusFork >= ConsensusFork.Bellatrix:
res.body.execution_payload = execution_payload.executionPayload
# https://github.com/ethereum/consensus-specs/blob/v1.4.0/specs/capella/validator.md#block-proposal
when consensusFork >= ConsensusFork.Capella:
res.body.bls_to_execution_changes =
validator_changes.bls_to_execution_changes
# https://github.com/ethereum/consensus-specs/blob/v1.3.0/specs/deneb/validator.md#constructing-the-beaconblockbody
when consensusFork >= ConsensusFork.Deneb:
res.body.blob_kzg_commitments = execution_payload.blobsBundle.commitments
debugRaiseAssert "either consolidate this within separate function or recombine, re when consensusFork >= foo and atts/attslashings; here to allow noninterference with pre-pectra"
res
@ -404,7 +457,7 @@ proc makeBeaconBlockWithRewards*(
randao_reveal: ValidatorSig,
eth1_data: Eth1Data,
graffiti: GraffitiBytes,
attestations: seq[Attestation],
attestations: seq[phase0.Attestation],
deposits: seq[Deposit],
validator_changes: BeaconBlockValidatorChanges,
sync_aggregate: SyncAggregate,
@ -462,7 +515,9 @@ proc makeBeaconBlockWithRewards*(
hash_tree_root(graffiti),
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[phase0.Attestation, Limit MAX_ATTESTATIONS](
attestations)),
hash_tree_root(List[Deposit, Limit MAX_DEPOSITS](deposits)),
hash_tree_root(validator_changes.voluntary_exits),
hash_tree_root(sync_aggregate),
@ -508,7 +563,7 @@ proc makeBeaconBlock*(
cfg: RuntimeConfig, state: var ForkedHashedBeaconState,
proposer_index: ValidatorIndex, randao_reveal: ValidatorSig,
eth1_data: Eth1Data, graffiti: GraffitiBytes,
attestations: seq[Attestation], deposits: seq[Deposit],
attestations: seq[phase0.Attestation], deposits: seq[Deposit],
validator_changes: BeaconBlockValidatorChanges,
sync_aggregate: SyncAggregate,
executionPayload: ForkyExecutionPayloadForSigning,
@ -530,7 +585,7 @@ proc makeBeaconBlock*(
cfg: RuntimeConfig, state: var ForkedHashedBeaconState,
proposer_index: ValidatorIndex, randao_reveal: ValidatorSig,
eth1_data: Eth1Data, graffiti: GraffitiBytes,
attestations: seq[Attestation], deposits: seq[Deposit],
attestations: seq[phase0.Attestation], deposits: seq[Deposit],
validator_changes: BeaconBlockValidatorChanges,
sync_aggregate: SyncAggregate,
executionPayload: ForkyExecutionPayloadForSigning,
@ -548,7 +603,7 @@ proc makeBeaconBlock*(
cfg: RuntimeConfig, state: var ForkedHashedBeaconState,
proposer_index: ValidatorIndex, randao_reveal: ValidatorSig,
eth1_data: Eth1Data, graffiti: GraffitiBytes,
attestations: seq[Attestation], deposits: seq[Deposit],
attestations: seq[phase0.Attestation], deposits: seq[Deposit],
validator_changes: BeaconBlockValidatorChanges,
sync_aggregate: SyncAggregate,
executionPayload: ForkyExecutionPayloadForSigning,

View File

@ -360,8 +360,8 @@ proc process_deposit*(
ok()
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.6/specs/phase0/beacon-chain.md#voluntary-exits
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/deneb/beacon-chain.md#modified-process_voluntary_exit
# https://github.com/ethereum/consensus-specs/blob/v1.4.0/specs/phase0/beacon-chain.md#voluntary-exits
# https://github.com/ethereum/consensus-specs/blob/v1.4.0/specs/deneb/beacon-chain.md#modified-process_voluntary_exit
proc check_voluntary_exit*(
cfg: RuntimeConfig,
state: ForkyBeaconState,
@ -444,6 +444,44 @@ proc process_bls_to_execution_change*(
ok()
# https://github.com/ethereum/consensus-specs/blob/94a0b6c581f2809aa8aca4ef7ee6fbb63f9d74e9/specs/electra/beacon-chain.md#new-process_execution_layer_exit
func process_execution_layer_withdrawal_request(
cfg: RuntimeConfig, state: var electra.BeaconState,
execution_layer_withdrawal_request: ExecutionLayerWithdrawalRequest,
exit_queue_info: ExitQueueInfo, cache: var StateCache):
Result[ExitQueueInfo, cstring] =
# Verify pubkey exists
let
pubkey_to_exit = execution_layer_withdrawal_request.validator_pubkey
validator_index = findValidatorIndex(state, pubkey_to_exit).valueOr:
return err("process_execution_layer_withdrawal_request: unknown index for validator pubkey")
validator = state.validators.item(validator_index)
# Verify withdrawal credentials
let
is_execution_address = validator.has_eth1_withdrawal_credential
is_correct_source_address =
validator.withdrawal_credentials.data.toOpenArray(12, 31) ==
execution_layer_withdrawal_request.source_address.data
if not (is_execution_address and is_correct_source_address):
return err("process_execution_layer_withdrawal_request: not both execution address and correct source address")
# Verify the validator is active
if not is_active_validator(validator, get_current_epoch(state)):
return err("process_execution_layer_withdrawal_request: not active validator")
# Verify exit has not been initiated
if validator.exit_epoch != FAR_FUTURE_EPOCH:
return err("process_execution_layer_withdrawal_request: validator exit already initiated")
# Verify the validator has been active long enough
if get_current_epoch(state) < validator.activation_epoch + cfg.SHARD_COMMITTEE_PERIOD:
return err("process_execution_layer_withdrawal_request: validator not active long enough")
# Initiate exit
ok(? initiate_validator_exit(
cfg, state, validator_index, exit_queue_info, cache))
type
# https://ethereum.github.io/beacon-APIs/?urls.primaryName=v2.5.0#/Rewards/getBlockRewards
BlockRewards* = object
@ -454,6 +492,7 @@ type
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.6/specs/phase0/beacon-chain.md#operations
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.5/specs/capella/beacon-chain.md#modified-process_operations
# https://github.com/ethereum/consensus-specs/blob/94a0b6c581f2809aa8aca4ef7ee6fbb63f9d74e9/specs/electra/beacon-chain.md#modified-process_operations
proc process_operations(cfg: RuntimeConfig,
state: var ForkyBeaconState,
body: SomeForkyBeaconBlockBody,
@ -501,6 +540,10 @@ proc process_operations(cfg: RuntimeConfig,
for op in body.voluntary_exits:
exit_queue_info = ? process_voluntary_exit(
cfg, state, op, flags, exit_queue_info, cache)
when typeof(body).kind >= ConsensusFork.Electra:
for op in body.execution_payload.withdrawal_requests:
exit_queue_info = ? process_execution_layer_withdrawal_request(
cfg, state, op, exit_queue_info, cache)
when typeof(body).kind >= ConsensusFork.Capella:
for op in body.bls_to_execution_changes:
? process_bls_to_execution_change(cfg, state, op)

View File

@ -40,7 +40,7 @@ export extras, phase0, altair
logScope: topics = "consens"
# Accessors that implement the max condition in `get_total_balance`:
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.6/specs/phase0/beacon-chain.md#get_total_balance
# https://github.com/ethereum/consensus-specs/blob/v1.4.0/specs/phase0/beacon-chain.md#get_total_balance
template current_epoch*(v: TotalBalances): Gwei =
max(EFFECTIVE_BALANCE_INCREMENT.Gwei, v.current_epoch_raw)
template previous_epoch*(v: TotalBalances): Gwei =
@ -806,7 +806,7 @@ iterator get_flag_and_inactivity_deltas*(
active_increments, penalty_denominator, epoch_participation,
participating_increments, info, vidx)
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-alpha.3/specs/phase0/beacon-chain.md#rewards-and-penalties-1
# https://github.com/ethereum/consensus-specs/blob/v1.4.0/specs/phase0/beacon-chain.md#rewards-and-penalties-1
func process_rewards_and_penalties*(
state: var phase0.BeaconState, info: var phase0.EpochInfo) =
# No rewards are applied at the end of `GENESIS_EPOCH` because rewards are
@ -1012,7 +1012,7 @@ template effective_balance_might_update*(
balance + DOWNWARD_THRESHOLD < effective_balance or
effective_balance + UPWARD_THRESHOLD < balance
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.7/specs/phase0/beacon-chain.md#effective-balances-updates
# https://github.com/ethereum/consensus-specs/blob/v1.4.0/specs/phase0/beacon-chain.md#effective-balances-updates
func process_effective_balance_updates*(state: var ForkyBeaconState) =
# Update effective balances with hysteresis
for vidx in state.validators.vindices:
@ -1028,7 +1028,7 @@ func process_effective_balance_updates*(state: var ForkyBeaconState) =
if new_effective_balance != effective_balance:
state.validators.mitem(vidx).effective_balance = new_effective_balance
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.6/specs/phase0/beacon-chain.md#slashings-balances-updates
# https://github.com/ethereum/consensus-specs/blob/v1.4.0/specs/phase0/beacon-chain.md#slashings-balances-updates
func process_slashings_reset*(state: var ForkyBeaconState) =
let next_epoch = get_current_epoch(state) + 1
@ -1166,7 +1166,7 @@ func process_historical_summaries_update*(
ok()
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.6/specs/phase0/beacon-chain.md#epoch-processing
# https://github.com/ethereum/consensus-specs/blob/v1.4.0/specs/phase0/beacon-chain.md#epoch-processing
proc process_epoch*(
cfg: RuntimeConfig, state: var phase0.BeaconState, flags: UpdateFlags,
cache: var StateCache, info: var phase0.EpochInfo): Result[void, cstring] =
@ -1254,7 +1254,7 @@ proc process_epoch*(
# https://github.com/ethereum/consensus-specs/blob/v1.4.0/specs/altair/beacon-chain.md#rewards-and-penalties
process_rewards_and_penalties(cfg, state, info) # [Modified in Altair]
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.6/specs/phase0/beacon-chain.md#registry-updates
# https://github.com/ethereum/consensus-specs/blob/v1.4.0/specs/phase0/beacon-chain.md#registry-updates
? process_registry_updates(cfg, state, cache)
# https://github.com/ethereum/consensus-specs/blob/v1.4.0/specs/altair/beacon-chain.md#slashings

View File

@ -162,7 +162,7 @@ func count_active_validators*(state: ForkyBeaconState,
cache: var StateCache): uint64 =
cache.get_shuffled_active_validator_indices(state, epoch).lenu64
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.7/specs/phase0/beacon-chain.md#get_committee_count_per_slot
# https://github.com/ethereum/consensus-specs/blob/v1.4.0/specs/phase0/beacon-chain.md#get_committee_count_per_slot
func get_committee_count_per_slot*(num_active_validators: uint64): uint64 =
clamp(
num_active_validators div SLOTS_PER_EPOCH div TARGET_COMMITTEE_SIZE,

View File

@ -90,7 +90,7 @@ p2pProtocol LightClientSync(version = 1,
debug "LC bootstrap request done", peer, blockRoot
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.5/specs/altair/light-client/p2p-interface.md#lightclientupdatesbyrange
# https://github.com/ethereum/consensus-specs/blob/v1.4.0/specs/altair/light-client/p2p-interface.md#lightclientupdatesbyrange
proc lightClientUpdatesByRange(
peer: Peer,
startPeriod: SyncCommitteePeriod,

View File

@ -1489,7 +1489,7 @@ proc produceAttestationData*(
proc submitPoolAttestations*(
vc: ValidatorClientRef,
data: seq[Attestation],
data: seq[phase0.Attestation],
strategy: ApiStrategyKind
): Future[bool] {.async.} =
const
@ -1635,7 +1635,7 @@ proc getAggregatedAttestation*(
slot: Slot,
root: Eth2Digest,
strategy: ApiStrategyKind
): Future[Attestation] {.async.} =
): Future[phase0.Attestation] {.async.} =
const
RequestName = "getAggregatedAttestation"

View File

@ -161,7 +161,7 @@ proc getSyncCommitteeMessageDataScore*(
vc.rootsSeen, vc.beaconClock.now().slotOrZero(), cdata)
proc processVotes(bits: var CommitteeBitsArray,
attestation: Attestation): int =
attestation: phase0.Attestation): int =
doAssert(len(attestation.aggregation_bits) <= len(bits))
var res = 0
for index in 0 ..< len(attestation.aggregation_bits):
@ -171,7 +171,7 @@ proc processVotes(bits: var CommitteeBitsArray,
bits[index] = true
res
proc getUniqueVotes*(attestations: openArray[Attestation]): int =
proc getUniqueVotes*(attestations: openArray[phase0.Attestation]): int =
var
res = 0
attested: Table[Slot, CommitteeTable]

View File

@ -1530,7 +1530,7 @@ proc signAndSendAggregate(
shufflingRef, slot, committee_index, selectionProof):
return
# https://github.com/ethereum/consensus-specs/blob/v1.3.0/specs/phase0/validator.md#construct-aggregate
# https://github.com/ethereum/consensus-specs/blob/v1.4.0/specs/phase0/validator.md#construct-aggregate
# https://github.com/ethereum/consensus-specs/blob/v1.4.0/specs/phase0/validator.md#aggregateandproof
var
msg = SignedAggregateAndProof(

View File

@ -191,7 +191,7 @@ proc routeSignedBeaconBlock*(
ok(blockRef)
proc routeAttestation*(
router: ref MessageRouter, attestation: Attestation,
router: ref MessageRouter, attestation: phase0.Attestation,
subnet_id: SubnetId, checkSignature: bool):
Future[SendResult] {.async: (raises: [CancelledError]).} =
## Process and broadcast attestation - processing will register the it with
@ -223,7 +223,7 @@ proc routeAttestation*(
return ok()
proc routeAttestation*(
router: ref MessageRouter, attestation: Attestation):
router: ref MessageRouter, attestation: phase0.Attestation):
Future[SendResult] {.async: (raises: [CancelledError]).} =
# Compute subnet, then route attestation
let

View File

@ -39,7 +39,7 @@ macro copyFields*(
# unblinded objects, and can't simply be copied.
"transactions_root", "execution_payload",
"execution_payload_header", "body", "withdrawals_root",
"deposit_receipts_root", "exits_root"]:
"deposit_receipts_root", "withdrawal_requests_root"]:
# TODO use stew/assign2
result.add newAssignment(
newDotExpr(dst, ident(name)), newDotExpr(src, ident(name)))

View File

@ -30,8 +30,9 @@ type
data*: AttestationData
proc toAttestation*(
registered: RegisteredAttestation, signature: ValidatorSig): Attestation =
Attestation.init(
registered: RegisteredAttestation, signature: ValidatorSig):
phase0.Attestation =
phase0.Attestation.init(
[registered.index_in_committee], registered.committee_len,
registered.data, signature).expect("valid data")

View File

@ -660,7 +660,7 @@ proc registerAttestation*(
self: var ValidatorMonitor,
src: MsgSource,
seen_timestamp: BeaconTime,
attestation: Attestation,
attestation: phase0.Attestation,
idx: ValidatorIndex) =
let
slot = attestation.data.slot

2
ci/Jenkinsfile vendored
View File

@ -165,5 +165,5 @@ def getAgentLabel() {
}
def nimCommitForJob() {
return JOB_NAME.contains('-nimv2/') ? 'upstream/version-2-0' : ''
return JOB_NAME.contains('nimv2') ? 'upstream/version-2-0' : ''
}

View File

@ -7,7 +7,7 @@ This is a WIP document to explain the attestation flows.
It is important to distinguish attestation `validation` from attestation `verification`.
- Attestation `validation` is defined in the P2P specs. Validated attestations can be forwarded on GossipSub.
- Aggregated: https://github.com/ethereum/consensus-specs/blob/v1.4.0-alpha.1/specs/phase0/p2p-interface.md#beacon_aggregate_and_proof
- Unaggregated: https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.5/specs/phase0/p2p-interface.md#beacon_attestation_subnet_id
- Unaggregated: https://github.com/ethereum/consensus-specs/blob/v1.4.0/specs/phase0/p2p-interface.md#beacon_attestation_subnet_id
- Attestation `verification` is defined in the consensus specs. Verified attestations can affect fork choice and may be included in a block.
- https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.6/specs/phase0/beacon-chain.md#attestations
@ -51,7 +51,7 @@ These GossipSub topics are used to listen for attestations:
- Unaggregated: `/eth2/{$forkDigest}/beacon_attestation_{subnetIndex}/ssz_snappy`
The attestations are then validated by `validateAttestation()` or `validateAggregate()` in either `attestationValidator()` or `aggregateValidator()` according to the P2P specs.
- https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.5/specs/phase0/p2p-interface.md#beacon_aggregate_and_proof
- https://github.com/ethereum/consensus-specs/blob/v1.4.0/specs/phase0/p2p-interface.md#beacon_aggregate_and_proof
- https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.5/specs/phase0/p2p-interface.md#attestation-subnets
Finally, valid attestations are added to the local `attestationPool`.

View File

@ -6,7 +6,7 @@ This is a WIP document to explain the beacon block flows.
Important distinction:
- We distinguish block `validation` which is defined in the P2P specs:
https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.5/specs/phase0/p2p-interface.md#beacon_block.
https://github.com/ethereum/consensus-specs/blob/v1.4.0/specs/phase0/p2p-interface.md#beacon_block.
A validated block can be forwarded on gossipsub.
- and we distinguish `verification` which is defined in consensus specs:
https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.6/specs/phase0/beacon-chain.md#block-processing
@ -115,7 +115,7 @@ Logs:
### Gossip flow out
- After validation in `blockValidator()` in the Eth2Processor by `validateBeaconBlock()` according to spec https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.5/specs/phase0/p2p-interface.md#beacon_block
- After validation in `blockValidator()` in the Eth2Processor by `validateBeaconBlock()` according to spec https://github.com/ethereum/consensus-specs/blob/v1.4.0/specs/phase0/p2p-interface.md#beacon_block
- Important: P2P validation is not full verification (state transition and internal cryptographic signatures were not checked)
- We jump into libp2p/protocols/pubsub/pubsub.nim in the method `validate(PubSub, message)`
- which was called by `rpcHandler(GossipSub, PubSubPeer, RPCMsg)`

View File

@ -233,7 +233,7 @@ proc doSSZ(conf: NcliConf) =
case kind
of "attester_slashing": printit(AttesterSlashing)
of "attestation": printit(Attestation)
of "attestation": printit(phase0.Attestation)
of "phase0_signed_block": printit(phase0.SignedBeaconBlock)
of "altair_signed_block": printit(altair.SignedBeaconBlock)
of "bellatrix_signed_block": printit(bellatrix.SignedBeaconBlock)

View File

@ -21,7 +21,7 @@ import
type
AttestationInput = object
state: phase0.BeaconState
attestation: Attestation
attestation: phase0.Attestation
AttesterSlashingInput = object
state: phase0.BeaconState
attesterSlashing: AttesterSlashing

View File

@ -63,7 +63,7 @@ proc makeSimulationBlock(
randao_reveal: ValidatorSig,
eth1_data: Eth1Data,
graffiti: GraffitiBytes,
attestations: seq[Attestation],
attestations: seq[phase0.Attestation],
deposits: seq[Deposit],
exits: BeaconBlockValidatorChanges,
sync_aggregate: SyncAggregate,
@ -106,7 +106,7 @@ proc makeSimulationBlock(
randao_reveal: ValidatorSig,
eth1_data: Eth1Data,
graffiti: GraffitiBytes,
attestations: seq[Attestation],
attestations: seq[phase0.Attestation],
deposits: seq[Deposit],
exits: BeaconBlockValidatorChanges,
sync_aggregate: SyncAggregate,
@ -149,7 +149,7 @@ proc makeSimulationBlock(
randao_reveal: ValidatorSig,
eth1_data: Eth1Data,
graffiti: GraffitiBytes,
attestations: seq[Attestation],
attestations: seq[phase0.Attestation],
deposits: seq[Deposit],
exits: BeaconBlockValidatorChanges,
sync_aggregate: SyncAggregate,
@ -259,7 +259,7 @@ cli do(slots = SLOTS_PER_EPOCH * 7,
get_attestation_signature(
fork, genesis_validators_root, data,
MockPrivKeys[validator_index])
attestation = Attestation.init(
attestation = phase0.Attestation.init(
[uint64 index_in_committee], committee.len, data,
sig.toValidatorSig()).expect("valid data")

View File

@ -164,7 +164,7 @@ cli do(validatorsDir: string, secretsDir: string,
blockRoot = withState(state[]): forkyState.latest_block_root
cache: StateCache
info: ForkedEpochInfo
aggregates: seq[Attestation]
aggregates: seq[phase0.Attestation]
syncAggregate = SyncAggregate.init()
let
@ -341,7 +341,7 @@ cli do(validatorsDir: string, secretsDir: string,
forkyState.data, slot, committee_index, cache)
var
attestation = Attestation(
attestation = phase0.Attestation(
data: makeAttestationData(
forkyState.data, slot, committee_index, blockRoot),
aggregation_bits: CommitteeValidatorsBits.init(committee.len))
@ -395,4 +395,4 @@ cli do(validatorsDir: string, secretsDir: string,
syncAggregate.sync_committee_bits.setBit(i)
if inited:
syncAggregate.sync_committee_signature = finish(agg).toValidatorSig()
syncAggregate.sync_committee_signature = finish(agg).toValidatorSig()

View File

@ -21,7 +21,7 @@ source "${SCRIPTS_DIR}/bash_utils.sh"
download_geth_stable() {
if [[ ! -e "${STABLE_GETH_BINARY}" ]]; then
GETH_VERSION="1.13.14-2bd6bd01" # https://geth.ethereum.org/downloads
GETH_VERSION="1.13.15-c5ba367e" # https://geth.ethereum.org/downloads
GETH_URL="https://gethstore.blob.core.windows.net/builds/"
case "${OS}-${ARCH}" in

View File

@ -70,7 +70,7 @@ proc runTest[T, U](
suite baseDescription & "Attestation " & preset():
proc applyAttestation(
preState: var altair.BeaconState, attestation: Attestation):
preState: var altair.BeaconState, attestation: phase0.Attestation):
Result[void, cstring] =
var cache: StateCache
let
@ -85,7 +85,7 @@ suite baseDescription & "Attestation " & preset():
ok()
for path in walkTests(OpAttestationsDir):
runTest[Attestation, typeof applyAttestation](
runTest[phase0.Attestation, typeof applyAttestation](
OpAttestationsDir, suiteName, "Attestation", "attestation",
applyAttestation, path)

View File

@ -68,7 +68,7 @@ proc runTest[T, U](
suite baseDescription & "Attestation " & preset():
proc applyAttestation(
preState: var phase0.BeaconState, attestation: Attestation):
preState: var phase0.BeaconState, attestation: phase0.Attestation):
Result[void, cstring] =
var cache: StateCache
doAssert (? process_attestation(
@ -76,7 +76,7 @@ suite baseDescription & "Attestation " & preset():
ok()
for path in walkTests(OpAttestationsDir):
runTest[Attestation, typeof applyAttestation](
runTest[phase0.Attestation, typeof applyAttestation](
OpAttestationsDir, suiteName, "Attestation", "attestation",
applyAttestation, path)

View File

@ -55,7 +55,7 @@ type
of opOnTick:
tick: int
of opOnAttestation:
att: Attestation
att: phase0.Attestation
of opOnBlock:
blck: ForkedSignedBeaconBlock
blobData: Opt[BlobData]
@ -115,7 +115,7 @@ proc loadOps(
let filename = step["attestation"].getStr()
let att = parseTest(
path/filename & ".ssz_snappy",
SSZ, Attestation
SSZ, phase0.Attestation
)
result.add Operation(kind: opOnAttestation,
att: att)

View File

@ -43,7 +43,7 @@ proc runBlobToKzgCommitmentTest(suiteName, suitePath, path: string) =
output = data["output"]
blob = fromHex[131072](data["input"]["blob"].getStr)
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.6/tests/formats/kzg/blob_to_kzg_commitment.md#condition
# https://github.com/ethereum/consensus-specs/blob/v1.4.0/tests/formats/kzg/blob_to_kzg_commitment.md#condition
# If the blob is invalid (e.g. incorrect length or one of the 32-byte
# blocks does not represent a BLS field element) it should error, i.e. the
# output should be `null`.
@ -117,7 +117,7 @@ proc runVerifyBlobKzgProofBatchTest(suiteName, suitePath, path: string) =
commitments = data["input"]["commitments"].mapIt(fromHex[48](it.getStr))
proofs = data["input"]["proofs"].mapIt(fromHex[48](it.getStr))
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.6/tests/formats/kzg/verify_blob_kzg_proof_batch.md#condition
# https://github.com/ethereum/consensus-specs/blob/v1.4.0/tests/formats/kzg/verify_blob_kzg_proof_batch.md#condition
# "If any of the commitments or proofs are invalid (e.g. not on the curve or
# not in the G1 subgroup of the BLS curve) or any blob is invalid (e.g.
# incorrect length or one of the 32-byte blocks does not represent a BLS
@ -143,7 +143,7 @@ proc runComputeKzgProofTest(suiteName, suitePath, path: string) =
blob = fromHex[131072](data["input"]["blob"].getStr)
z = fromHex[32](data["input"]["z"].getStr)
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.6/tests/formats/kzg/compute_kzg_proof.md#condition
# https://github.com/ethereum/consensus-specs/blob/v1.4.0/tests/formats/kzg/compute_kzg_proof.md#condition
# "If the blob is invalid (e.g. incorrect length or one of the 32-byte
# blocks does not represent a BLS field element) or z is not a valid BLS
# field element, it should error, i.e. the output should be null."
@ -227,4 +227,4 @@ suite suiteName:
for kind, path in walkDir(testsDir, relative = true, checkDir = true):
runComputeBlobKzgProofTest(suiteName, testsDir, testsDir / path)
doAssert Kzg.freeTrustedSetup().isOk
doAssert Kzg.freeTrustedSetup().isOk

View File

@ -28,7 +28,7 @@ import
from std/sequtils import toSeq
from ./testbcutil import addHeadBlock
func combine(tgt: var Attestation, src: Attestation) =
func combine(tgt: var phase0.Attestation, src: phase0.Attestation) =
## Combine the signature and participation bitfield, with the assumption that
## the same data is being signed - if the signatures overlap, they are not
## combined.
@ -47,7 +47,7 @@ func combine(tgt: var Attestation, src: Attestation) =
agg.aggregate(src.signature.load.get())
tgt.signature = agg.finish().toValidatorSig()
func loadSig(a: Attestation): CookedSig =
func loadSig(a: phase0.Attestation): CookedSig =
a.signature.load.get()
proc pruneAtFinalization(dag: ChainDAGRef, attPool: AttestationPool) =
@ -666,7 +666,7 @@ suite "Attestation pool processing" & preset():
# -------------------------------------------------------------
# Pass an epoch
var attestations: seq[Attestation]
var attestations: seq[phase0.Attestation]
for epoch in 0 ..< 5:
let start_slot = start_slot(Epoch epoch)
@ -703,7 +703,7 @@ suite "Attestation pool processing" & preset():
for v in 0 ..< committee.len * 2 div 3 + 1:
aggregation_bits[v] = true
attestations.add Attestation(
attestations.add phase0.Attestation(
aggregation_bits: aggregation_bits,
data: makeAttestationData(state[], getStateField(state[], slot),
committee_index, blockRef.get().root)
@ -729,4 +729,4 @@ suite "Attestation pool processing" & preset():
epochRef, blckRef, unrealized, signedBlock.message,
blckRef.slot.start_beacon_time)
doAssert: b10Add_clone.error == VerifierError.Duplicate
doAssert: b10Add_clone.error == VerifierError.Duplicate

View File

@ -1641,7 +1641,7 @@ suite "Eth1 monitor":
signature: ValidatorSig(blob: hexToByteArray[96]("0x943c3c3818d5aa98fbf8344ef4cdc9c13cdabfdeb7762efc8bb32d2ea32d3bbb4ef069a254f5f35325f48609fad7bbafb6389e204767a9b3bbe46a04f8baa850bfd4d3747aaf2816c7e18fc2ebe4fa41088d195d09c761819c7a2e57a3451148")),
index: 900883336538271514'u64),
]),
exits: List[ExecutionLayerExit, MAX_EXECUTION_LAYER_EXITS_PER_PAYLOAD].init(@[
withdrawal_requests: List[ExecutionLayerWithdrawalRequest, MAX_WITHDRAWAL_REQUESTS_PER_PAYLOAD].init(@[
]),
),
(electra.ExecutionPayload)(
@ -1699,14 +1699,14 @@ suite "Eth1 monitor":
signature: ValidatorSig(blob: hexToByteArray[96]("0x0d543b0e9b934586fb877615c8d2551e11998f020bce6b96901fb8045ef42eb41f6039e813136043fe5c63d91a11e1e15e5c4063d1775f95ae1715cb87b21b7690b44ec38efd1a825e1e3ac68d21940f772b3309edb3ddebb24204e06d4924c2")),
index: 12423850076890731216'u64),
]),
exits: List[ExecutionLayerExit, MAX_EXECUTION_LAYER_EXITS_PER_PAYLOAD].init(@[
ExecutionLayerExit(
withdrawal_requests: List[ExecutionLayerWithdrawalRequest, MAX_WITHDRAWAL_REQUESTS_PER_PAYLOAD].init(@[
ExecutionLayerWithdrawalRequest(
source_address: ExecutionAddress.fromHex("0x39554fbddf13facd81344d536c08ed5769304749"),
validator_pubkey: ValidatorPubKey(blob: hexToByteArray[48]("0xc4f5b2c07cc2f6758dd8eaef217247f767bcd88a8f5c93b030023d420568f47735d113df344627759f4ea1b56c53136f"))),
ExecutionLayerExit(
ExecutionLayerWithdrawalRequest(
source_address: ExecutionAddress.fromHex("0x243c496e83f955ef23dc3d121b3cbe5f56305d73"),
validator_pubkey: ValidatorPubKey(blob: hexToByteArray[48]("0xe9a3d62cdf9acae4966e5682958d0cc9223065b4d68ed3b12a024a56744ab9656736326061f9fb41a8f15564cb4d241f"))),
ExecutionLayerExit(
ExecutionLayerWithdrawalRequest(
source_address: ExecutionAddress.fromHex("0x462f156d8d950c7ffd40d7ba149bcc34093bbdb7"),
validator_pubkey: ValidatorPubKey(blob: hexToByteArray[48]("0xd6d7f2281c2b9c98a8a5dc0b7f41783eb91b838973207239852e817ed412e164e330003ac9ab0e96bc65886e15b5cbe9"))),
]),
@ -1739,14 +1739,14 @@ suite "Eth1 monitor":
signature: ValidatorSig(blob: hexToByteArray[96]("0x143a1a4dcac6db342901feb541dc0c95830a4ca1aca9c3fcb55e2dcb9a5b31e2bd9214b1a3a12e17e140d37ba7ebfd11d6d8a38eea5d0755402dd400386aaefcc70d12fb1409f92797923bf964bea3f916b562f3ff2b522c48b748c8e8c632d4")),
index: 15872726372973140071'u64),
]),
exits: List[ExecutionLayerExit, MAX_EXECUTION_LAYER_EXITS_PER_PAYLOAD].init(@[
ExecutionLayerExit(
withdrawal_requests: List[ExecutionLayerWithdrawalRequest, MAX_WITHDRAWAL_REQUESTS_PER_PAYLOAD].init(@[
ExecutionLayerWithdrawalRequest(
source_address: ExecutionAddress.fromHex("0x3a8a707225d47dbddb01c1ca39181af823d57d97"),
validator_pubkey: ValidatorPubKey(blob: hexToByteArray[48]("0x9cf008ca8159512ffffa1fe56de68bb9e44f9c4bb3c2c4924f5d7bf1bb810cc807b155f11ddd55a4972346f8e75f06ab"))),
ExecutionLayerExit(
ExecutionLayerWithdrawalRequest(
source_address: ExecutionAddress.fromHex("0x7c55f3e4f648bcfb47db2122233b25881785709b"),
validator_pubkey: ValidatorPubKey(blob: hexToByteArray[48]("0xb9e559b137b8ab79ddfbc6ea2fb44d96d1925c2b7b6e4c0e1b69f66d82b656065af06bd62e8fe9210276a116ad78c382"))),
ExecutionLayerExit(
ExecutionLayerWithdrawalRequest(
source_address: ExecutionAddress.fromHex("0xcf25ed583b463f3a57acd97c398e27877b9bf6a6"),
validator_pubkey: ValidatorPubKey(blob: hexToByteArray[48]("0xa14ac0d85ae38dd91a9f7da12b6c7cb4e879f78effc5ac0da8f9ee56059460f31152009fc1b88d0e0a0bf576950f45e0"))),
]),
@ -1801,8 +1801,8 @@ suite "Eth1 monitor":
signature: ValidatorSig(blob: hexToByteArray[96]("0x94cb986143fdae936c832d08977520c41a6a9b1569b012a8486678571ea5ce2e913f55c9d5631a8f02d1b75aca414969c56f22cf0b6e5193f7ac3568b09c9ae955581c69095908ccab9c5ff5c47b2edef262f2843ccc7cbc69eb35b14c66886c")),
index: 11423537419700559218'u64),
]),
exits: List[ExecutionLayerExit, MAX_EXECUTION_LAYER_EXITS_PER_PAYLOAD].init(@[
ExecutionLayerExit(
withdrawal_requests: List[ExecutionLayerWithdrawalRequest, MAX_WITHDRAWAL_REQUESTS_PER_PAYLOAD].init(@[
ExecutionLayerWithdrawalRequest(
source_address: ExecutionAddress.fromHex("0x44754b90f7b23eee1dddafa745ac723dcc147404"),
validator_pubkey: ValidatorPubKey(blob: hexToByteArray[48]("0x8d13edc45159cdcf6ed780fc7b93e74434fa392b0842dfa92458cc59515aaac127317df24def9701eb6d5ea060eaffea"))),
]),
@ -1841,7 +1841,7 @@ suite "Eth1 monitor":
signature: ValidatorSig(blob: hexToByteArray[96]("0x9e2864164d275e436ed45120245d2063dbedc87d555cceabe8c18622fe462411ecbe7fa4a262989a45795efea09d21f8e4254cedd5c787bf80211be0a3c6ffc1bcc5f364387f32f746647e0194a599653f3af5f6e1151244df02bb7b3f7270cc")),
index: 1665528005288012054'u64),
]),
exits: List[ExecutionLayerExit, MAX_EXECUTION_LAYER_EXITS_PER_PAYLOAD].init(@[
withdrawal_requests: List[ExecutionLayerWithdrawalRequest, MAX_WITHDRAWAL_REQUESTS_PER_PAYLOAD].init(@[
]),
),
(electra.ExecutionPayload)(
@ -1890,7 +1890,7 @@ suite "Eth1 monitor":
signature: ValidatorSig(blob: hexToByteArray[96]("0xca03e6b82cd3df289ed574c2deca216a089928bc944abd3efd26ee3124c45b22e7f541c02cc95f71ea8b52f0fed044ca14863b651c07b6e52abbce8afb500556a32e33a5f57a33ca6103237aa1c5bc409f4a2745b9828d6eff5360a2ba63d162")),
index: 18335046601207607970'u64),
]),
exits: List[ExecutionLayerExit, MAX_EXECUTION_LAYER_EXITS_PER_PAYLOAD].init(@[
withdrawal_requests: List[ExecutionLayerWithdrawalRequest, MAX_WITHDRAWAL_REQUESTS_PER_PAYLOAD].init(@[
]),
),
(electra.ExecutionPayload)(
@ -1947,8 +1947,8 @@ suite "Eth1 monitor":
signature: ValidatorSig(blob: hexToByteArray[96]("0x42a5b14b6d5018eedf1dc9bb07cd64ae2d25f583ad805d58d89b4c8381db8740fe188a70f1a1d2eb0e486807cefff900f93ebed94fbe2539edddf06f91bf347281f9dcc891db49d6107c2f88d678d32e5e9849a2be7b082919edb769b7c70abf")),
index: 16997402741851403011'u64),
]),
exits: List[ExecutionLayerExit, MAX_EXECUTION_LAYER_EXITS_PER_PAYLOAD].init(@[
ExecutionLayerExit(
withdrawal_requests: List[ExecutionLayerWithdrawalRequest, MAX_WITHDRAWAL_REQUESTS_PER_PAYLOAD].init(@[
ExecutionLayerWithdrawalRequest(
source_address: ExecutionAddress.fromHex("0x4bd763bcdfcf9fd2ce667c75408bc1157fa9730a"),
validator_pubkey: ValidatorPubKey(blob: hexToByteArray[48]("0xdf62e8946d1457a50ce017fae0c36e5dc5177e642c18b74dd6df192620f8a32bef5f02453f0835583f6082f213df7245"))),
]),
@ -2002,11 +2002,11 @@ suite "Eth1 monitor":
signature: ValidatorSig(blob: hexToByteArray[96]("0xbd837ceae239191f7e958fabc91efc7b3830da9814f4d888ec278ed0fbf870e811db948bf81377fd53339db9095f3c71b36de09b6f5b38a18caba6d3e8f337bbcb107380ee3d50058e3d266653860b1c6a9309eb60f142948f53041a07109f4d")),
index: 2237248193846176262'u64),
]),
exits: List[ExecutionLayerExit, MAX_EXECUTION_LAYER_EXITS_PER_PAYLOAD].init(@[
ExecutionLayerExit(
withdrawal_requests: List[ExecutionLayerWithdrawalRequest, MAX_WITHDRAWAL_REQUESTS_PER_PAYLOAD].init(@[
ExecutionLayerWithdrawalRequest(
source_address: ExecutionAddress.fromHex("0x614d16bedf5dfe9d06171e3ef50671e66fadfce4"),
validator_pubkey: ValidatorPubKey(blob: hexToByteArray[48]("0x9f92a4aa0e58f82ff2ec0bbe4aca6d338fd08ffff3213f64bef81148f7dbb163eb25add8ccc540ec0dd1bf9d237e26f9"))),
ExecutionLayerExit(
ExecutionLayerWithdrawalRequest(
source_address: ExecutionAddress.fromHex("0x4cff44c8f0353fa6dee31f6c87e4b8c3bcaf1c38"),
validator_pubkey: ValidatorPubKey(blob: hexToByteArray[48]("0x3166f8e41daae4a0af1549a00b95ad9280d73e91a882d49c827bc078c88300264e7171cbbf50e3598da77bcdb175a203"))),
]),
@ -2048,20 +2048,20 @@ suite "Eth1 monitor":
signature: ValidatorSig(blob: hexToByteArray[96]("0x7a4df2b27bded4e1cc2e20120e70f576e9991369d77dfad54186d3067416bfe1f7cb7a1021a9c0722370680367fe4c12e571902c2f4ce4c2754a4738c10ead67b1d9a1a82b2ecd4ce3b6567c87e0066c979664bf79025851cd9583c5ed2f7c2f")),
index: 4361690020859323832'u64),
]),
exits: List[ExecutionLayerExit, MAX_EXECUTION_LAYER_EXITS_PER_PAYLOAD].init(@[
ExecutionLayerExit(
withdrawal_requests: List[ExecutionLayerWithdrawalRequest, MAX_WITHDRAWAL_REQUESTS_PER_PAYLOAD].init(@[
ExecutionLayerWithdrawalRequest(
source_address: ExecutionAddress.fromHex("0x9c2b1570328c29ef47c715cd021aead97695741e"),
validator_pubkey: ValidatorPubKey(blob: hexToByteArray[48]("0x6d3da7dd6f61e0818830bf11df8c91af8be664041d8832ca48b0c90566963acaa54695da7fb9ae2904d1aa0d7de5dcbd"))),
ExecutionLayerExit(
ExecutionLayerWithdrawalRequest(
source_address: ExecutionAddress.fromHex("0xf3fff390ae583278167deb91dba09b4ba089acaf"),
validator_pubkey: ValidatorPubKey(blob: hexToByteArray[48]("0xaeaef2b0928efd727bab75c3859118681492d7aaa0ceb7cb0897e21d4689ce7a6a9306850b2dbd801cb3ec165bb97d68"))),
ExecutionLayerExit(
ExecutionLayerWithdrawalRequest(
source_address: ExecutionAddress.fromHex("0x9a89ea1df940046760d3a84e134ea525a05a91fd"),
validator_pubkey: ValidatorPubKey(blob: hexToByteArray[48]("0x7afe11eec6aa2da5eb2bb7f9c6f2329ef9b9c17cd2f2ea35fee5e4169bc4e26c73c30cbbde16cbe4ae2351266454c31f"))),
ExecutionLayerExit(
ExecutionLayerWithdrawalRequest(
source_address: ExecutionAddress.fromHex("0xf77580ffa7329925db0934de5f3667b1a32effd1"),
validator_pubkey: ValidatorPubKey(blob: hexToByteArray[48]("0x3f5026c08a653bb8cc9f46a5cb1c35200c43efb6c44f729b48d12400828f5029fdc88f4672f1f9393d7d764ba3599bf1"))),
ExecutionLayerExit(
ExecutionLayerWithdrawalRequest(
source_address: ExecutionAddress.fromHex("0xc61710d4969b77326cfe3ee23b65023c23e8789e"),
validator_pubkey: ValidatorPubKey(blob: hexToByteArray[48]("0xb2952e0f7d6581c3032f95f4908bf76f6df8d7e866b7b67996254597ef73ce9a15dac375b78a3456d4f7f156af2b5ed5"))),
]),
@ -2086,11 +2086,11 @@ suite "Eth1 monitor":
excess_blob_gas: 1,
deposit_receipts: List[DepositReceipt, MAX_DEPOSIT_RECEIPTS_PER_PAYLOAD].init(@[
]),
exits: List[ExecutionLayerExit, MAX_EXECUTION_LAYER_EXITS_PER_PAYLOAD].init(@[
ExecutionLayerExit(
withdrawal_requests: List[ExecutionLayerWithdrawalRequest, MAX_WITHDRAWAL_REQUESTS_PER_PAYLOAD].init(@[
ExecutionLayerWithdrawalRequest(
source_address: ExecutionAddress.fromHex("0xe4bed2d5de111ca1d0a77bf6006c09ced6c6cc89"),
validator_pubkey: ValidatorPubKey(blob: hexToByteArray[48]("0x814d7cd0b6d428414fa787584e1eb52a5f215b8f0e7792499365f465ac43f5696e8d18ab579568c348f6dde75c189301"))),
ExecutionLayerExit(
ExecutionLayerWithdrawalRequest(
source_address: ExecutionAddress.fromHex("0x95137ca91b36a9a753441d911bdf91677931615c"),
validator_pubkey: ValidatorPubKey(blob: hexToByteArray[48]("0x0e223fadbfa2985e293f13e083bbe22a9a208d0e9f37fd99d24be92b3e329d77f1d40d61b891e2bdfed12ca746eeec50"))),
]),
@ -2138,8 +2138,8 @@ suite "Eth1 monitor":
signature: ValidatorSig(blob: hexToByteArray[96]("0xc614759dcdc309a46d9f24ae6b2840625bc5ddecd802c2907f9141d9091966e3367d78b3963717877a6110d741f40b45486acd32ac0e7bf1b4c36e681411570a7d1156dda127c1c5e5c6011ff857222ea51086016c01346e6cd2c8764bc7e7f4")),
index: 9892892756897161299'u64),
]),
exits: List[ExecutionLayerExit, MAX_EXECUTION_LAYER_EXITS_PER_PAYLOAD].init(@[
ExecutionLayerExit(
withdrawal_requests: List[ExecutionLayerWithdrawalRequest, MAX_WITHDRAWAL_REQUESTS_PER_PAYLOAD].init(@[
ExecutionLayerWithdrawalRequest(
source_address: ExecutionAddress.fromHex("0x9892501906b7abf06fdb6893b8e1767884bc17f5"),
validator_pubkey: ValidatorPubKey(blob: hexToByteArray[48]("0x30099e0ee2adf0d51a0a96d10fd2fd5cf6f17cdb4b4ea88b5a0e205bd10d40319595e0403891aaa1bac82b980ef76f23"))),
]),
@ -2179,7 +2179,7 @@ suite "Eth1 monitor":
signature: ValidatorSig(blob: hexToByteArray[96]("0x967057d2edd3b53ae9fb665fe668ab2319403b4f8e4620064b11f0933f9def18d952cae5f50395ffd1a8e8554604d95371b7643386df808a18c913e186a7a915e5a5c65908dd6668f2c0d02e404eb88d3499c096967e93b791d814429caae9a2")),
index: 7603599240231509693'u64),
]),
exits: List[ExecutionLayerExit, MAX_EXECUTION_LAYER_EXITS_PER_PAYLOAD].init(@[
withdrawal_requests: List[ExecutionLayerWithdrawalRequest, MAX_WITHDRAWAL_REQUESTS_PER_PAYLOAD].init(@[
]),
),
(electra.ExecutionPayload)(
@ -2210,8 +2210,8 @@ suite "Eth1 monitor":
signature: ValidatorSig(blob: hexToByteArray[96]("0xd6a92f4de599923ba4955c360b2cd54bd544e2b75947127fefa9ec08f5e53cf02bf398b63a0420226dd356fc5d50683eaead8a5aa8a6d4fdbe62296506c813e5e02a2513b6457c1ca408e1189fba32e80d74c48e389f62c7b0b0ff3c1881ec55")),
index: 14462442824619447645'u64),
]),
exits: List[ExecutionLayerExit, MAX_EXECUTION_LAYER_EXITS_PER_PAYLOAD].init(@[
ExecutionLayerExit(
withdrawal_requests: List[ExecutionLayerWithdrawalRequest, MAX_WITHDRAWAL_REQUESTS_PER_PAYLOAD].init(@[
ExecutionLayerWithdrawalRequest(
source_address: ExecutionAddress.fromHex("0xf55f4b626328f2b7a725d8a3f8485072eebf7f6e"),
validator_pubkey: ValidatorPubKey(blob: hexToByteArray[48]("0x3eb1812d045ff1d2f7d96f919c41230db2993ed8194de6ba564fad54047e3b45fb925e5216cc47f69e184a4e2c45ce39"))),
]),
@ -2258,20 +2258,20 @@ suite "Eth1 monitor":
signature: ValidatorSig(blob: hexToByteArray[96]("0x2f470357ded569d4fe968b5da6619cbeb414271e71ec7abc8e0e6c7c962b1932934bef085f682bc6af358670bdaf80572dd4ee3fdf80711e60205868aad5859971a858f30eaeee2883bad62b5c4e6ada3ea38ae1ab516f294a16b18c099fa760")),
index: 3956355178667798015'u64),
]),
exits: List[ExecutionLayerExit, MAX_EXECUTION_LAYER_EXITS_PER_PAYLOAD].init(@[
ExecutionLayerExit(
withdrawal_requests: List[ExecutionLayerWithdrawalRequest, MAX_WITHDRAWAL_REQUESTS_PER_PAYLOAD].init(@[
ExecutionLayerWithdrawalRequest(
source_address: ExecutionAddress.fromHex("0x98410af351e5be94f9d37f7cc9f97a85e9bd0dad"),
validator_pubkey: ValidatorPubKey(blob: hexToByteArray[48]("0xd96132438444f4582e21aaa4950d907a84d56f5edaf5d4262439210d6b6aae00ef67d15caa1e95040484b977ba677f31"))),
ExecutionLayerExit(
ExecutionLayerWithdrawalRequest(
source_address: ExecutionAddress.fromHex("0xe640e25259ffe5aa8b481e98684b41a14f3d2192"),
validator_pubkey: ValidatorPubKey(blob: hexToByteArray[48]("0xfb8bad5edefcf4a76157dd4df48c345b10966ebe21c5265519a3d166ee6f43b92bc67707a7bcc478c05cb5d5aaa5e217"))),
ExecutionLayerExit(
ExecutionLayerWithdrawalRequest(
source_address: ExecutionAddress.fromHex("0x6544a67710ed5b8466aea7bb74de9e275c7a7338"),
validator_pubkey: ValidatorPubKey(blob: hexToByteArray[48]("0xae821baad66de4d33dc8a8ea9088ab97cce0be2f1be1243c3c640377fd57f3f7389087ace339f953d52372d198300f8c"))),
ExecutionLayerExit(
ExecutionLayerWithdrawalRequest(
source_address: ExecutionAddress.fromHex("0x48f380f0b267ceec6fbe39f80b7108991acf97b5"),
validator_pubkey: ValidatorPubKey(blob: hexToByteArray[48]("0x2c9a9040e72e095e347f8ba9ad33947a1ec5ffddaa2e86a112fd73c30af209625f1bf19eb7b4fcee28409707679781d1"))),
ExecutionLayerExit(
ExecutionLayerWithdrawalRequest(
source_address: ExecutionAddress.fromHex("0x04fb4574aa211ef818aa9c13135f20f4694b8ce3"),
validator_pubkey: ValidatorPubKey(blob: hexToByteArray[48]("0x8d381b3ee22253692bd5861ede4c0d62cb2f6c90df6afd180831ec183ac3f8bcccbbfb5fa1f3ee38d5c3871ca8e28ba3"))),
]),
@ -2300,17 +2300,17 @@ suite "Eth1 monitor":
excess_blob_gas: 1233408100755176706'u64,
deposit_receipts: List[DepositReceipt, MAX_DEPOSIT_RECEIPTS_PER_PAYLOAD].init(@[
]),
exits: List[ExecutionLayerExit, MAX_EXECUTION_LAYER_EXITS_PER_PAYLOAD].init(@[
ExecutionLayerExit(
withdrawal_requests: List[ExecutionLayerWithdrawalRequest, MAX_WITHDRAWAL_REQUESTS_PER_PAYLOAD].init(@[
ExecutionLayerWithdrawalRequest(
source_address: ExecutionAddress.fromHex("0x16c6ba72f97bd60af3008e747aa0045eace969dd"),
validator_pubkey: ValidatorPubKey(blob: hexToByteArray[48]("0x79b68340894f69a82de6d6ac26b6cffd1f84be9008f7cec5a8f740c5dcd73103e50366cb45ec0c2a0984b37597011784"))),
ExecutionLayerExit(
ExecutionLayerWithdrawalRequest(
source_address: ExecutionAddress.fromHex("0xf950853d752ff1e8dfd3ffb9bdb504e851361060"),
validator_pubkey: ValidatorPubKey(blob: hexToByteArray[48]("0x9924a9bf1759d436f9dcc185cdb646d06af53ddf9e86351b69bda506eaaf4b47739a0737ebfcb7d734d33237eb77983c"))),
ExecutionLayerExit(
ExecutionLayerWithdrawalRequest(
source_address: ExecutionAddress.fromHex("0x7ef709dcc026c545a1707a4161948637f4c1afce"),
validator_pubkey: ValidatorPubKey(blob: hexToByteArray[48]("0xfe1b2e2cd818d436f9cfd7ad7e9efb8e8940bff9ac2c5094793d26f9a50f76436e25b40d375d7b9d461ac7fac81887d3"))),
ExecutionLayerExit(
ExecutionLayerWithdrawalRequest(
source_address: ExecutionAddress.fromHex("0x1e6d99ec506e2b79322f77283f3e18dfc0561346"),
validator_pubkey: ValidatorPubKey(blob: hexToByteArray[48]("0x25931e58a52265a5a90f7004706cd736fdb762d50aff67039d5e0242039dfc49fd6670e6f4cf62639d7debe3efe5298b"))),
]),
@ -2349,20 +2349,20 @@ suite "Eth1 monitor":
signature: ValidatorSig(blob: hexToByteArray[96]("0x0b7a4a77b5554a3be5f9338c31158e9f0b0b5fc95e9ef176ca38183ceb3aaf214711af03ecf194091cbc99a11aa7a376d721b3c1e27e71447828326ee811a07f4680c5a73fb52106bfe9b66eadd40cf80f027f0db90e41c77c78552edaccf295")),
index: 659556622372086172'u64),
]),
exits: List[ExecutionLayerExit, MAX_EXECUTION_LAYER_EXITS_PER_PAYLOAD].init(@[
ExecutionLayerExit(
withdrawal_requests: List[ExecutionLayerWithdrawalRequest, MAX_WITHDRAWAL_REQUESTS_PER_PAYLOAD].init(@[
ExecutionLayerWithdrawalRequest(
source_address: ExecutionAddress.fromHex("0xa80127ae927ef2fc72e527bee414d2a899e1050f"),
validator_pubkey: ValidatorPubKey(blob: hexToByteArray[48]("0x463d04b11a5f2b3a5ff5d93f7c20acb46b06d8a434d9dcbbcde024be06f50b6542ebca1a759d8cf8381e7142bce4bd1c"))),
ExecutionLayerExit(
ExecutionLayerWithdrawalRequest(
source_address: ExecutionAddress.fromHex("0x91e2ec291b66f267104a11157c46ef32fd40c22f"),
validator_pubkey: ValidatorPubKey(blob: hexToByteArray[48]("0xc05bcf497d5e305552b041c7a239536c938fff8bc755fadd28fd907f070f7f4a5553660a3351739a0b1bec2e6ec3d2aa"))),
ExecutionLayerExit(
ExecutionLayerWithdrawalRequest(
source_address: ExecutionAddress.fromHex("0x1281069954affabc619e8092861136ada40cb869"),
validator_pubkey: ValidatorPubKey(blob: hexToByteArray[48]("0x781f234560ec5d2197a33908a66fcb156330141f51212a51a0f0117417b5370f3fd0266c9dd1bf2c66d47eaf98375327"))),
ExecutionLayerExit(
ExecutionLayerWithdrawalRequest(
source_address: ExecutionAddress.fromHex("0x56855acbe00c442f0d20d489deb80fc02b31a173"),
validator_pubkey: ValidatorPubKey(blob: hexToByteArray[48]("0x805d72db2998bbfaf07d13f5328db300ea7a2fa156d049bf072590d61dca40ae142de4a204e36768f6e546af62d7e1fb"))),
ExecutionLayerExit(
ExecutionLayerWithdrawalRequest(
source_address: ExecutionAddress.fromHex("0x722d597ea5a6f82a0f9b06bd8af0449d18f78795"),
validator_pubkey: ValidatorPubKey(blob: hexToByteArray[48]("0x1de64f16597d52214d1a5987abc026398d310712ad0db48d48e747e7783204579a886bbd9a58a47704d9874a83726a50"))),
]),
@ -2390,11 +2390,11 @@ suite "Eth1 monitor":
excess_blob_gas: 10785611890433610477'u64,
deposit_receipts: List[DepositReceipt, MAX_DEPOSIT_RECEIPTS_PER_PAYLOAD].init(@[
]),
exits: List[ExecutionLayerExit, MAX_EXECUTION_LAYER_EXITS_PER_PAYLOAD].init(@[
ExecutionLayerExit(
withdrawal_requests: List[ExecutionLayerWithdrawalRequest, MAX_WITHDRAWAL_REQUESTS_PER_PAYLOAD].init(@[
ExecutionLayerWithdrawalRequest(
source_address: ExecutionAddress.fromHex("0x027404a69d1a1a8b931d0deb6ef4c90cc23fe74e"),
validator_pubkey: ValidatorPubKey(blob: hexToByteArray[48]("0x144cd543ddf6cc88499595246d2373629467e69048b4c638824b8c4d82296fb635028f495c7516174670ed1c5b320462"))),
ExecutionLayerExit(
ExecutionLayerWithdrawalRequest(
source_address: ExecutionAddress.fromHex("0x748168ee6835196ae76808fe3232a422b40e42a7"),
validator_pubkey: ValidatorPubKey(blob: hexToByteArray[48]("0x5e024d736b5c4d340929745f59b7d681eeb151107f895a87d534491b5af13fbf7bed890a2f41dc8debacf2f65fce2c20"))),
]),
@ -2440,11 +2440,11 @@ suite "Eth1 monitor":
signature: ValidatorSig(blob: hexToByteArray[96]("0x655e809ad38376a8d7fdd895a30d8a1ac52861864f67e1ce885cc40cbdf3ff27a8a6f8cb1b33f74254c5bfef90de22f6b1c724e888d284438995fab628ecdc5278319435192ed259b56ab6d2f18ad3ba53aa534e85fa802e15c1a1ec9fe3b7e1")),
index: 15032238460111462081'u64),
]),
exits: List[ExecutionLayerExit, MAX_EXECUTION_LAYER_EXITS_PER_PAYLOAD].init(@[
ExecutionLayerExit(
withdrawal_requests: List[ExecutionLayerWithdrawalRequest, MAX_WITHDRAWAL_REQUESTS_PER_PAYLOAD].init(@[
ExecutionLayerWithdrawalRequest(
source_address: ExecutionAddress.fromHex("0xc8bcdf0144cd4eb45e62b4fa76b7d5963fa912ec"),
validator_pubkey: ValidatorPubKey(blob: hexToByteArray[48]("0x4569a134a3f6e0ac638b19e8d88c9010f7281449f78adcbad225d11d2358790b2454504ac56209ac54cf66d5df779bce"))),
ExecutionLayerExit(
ExecutionLayerWithdrawalRequest(
source_address: ExecutionAddress.fromHex("0x308d3b908ce2fb2ebd207120422994608d8c3354"),
validator_pubkey: ValidatorPubKey(blob: hexToByteArray[48]("0x3deec67ff0f69aeeeddf322043b694ed4ec79aa2cd2414797bb95da5691b2b9731d3fe3d3627684d022241f80504f3ad"))),
]),
@ -2498,20 +2498,20 @@ suite "Eth1 monitor":
signature: ValidatorSig(blob: hexToByteArray[96]("0x97ea0a8e3f3e73fb11ded1814f4232e8bfb1e7b71bce608f3f181e5609bdaab3ffde52b1ff98d94c3d02ffefa6b3716cd83deda00888224f24716619f685c940da205910227b976bedf7f0cfc16262e2ec48dd837509326c97e329fe666846ab")),
index: 8630770799181013738'u64),
]),
exits: List[ExecutionLayerExit, MAX_EXECUTION_LAYER_EXITS_PER_PAYLOAD].init(@[
ExecutionLayerExit(
withdrawal_requests: List[ExecutionLayerWithdrawalRequest, MAX_WITHDRAWAL_REQUESTS_PER_PAYLOAD].init(@[
ExecutionLayerWithdrawalRequest(
source_address: ExecutionAddress.fromHex("0x4e4c648248758aaba856a20f8496700f036a9177"),
validator_pubkey: ValidatorPubKey(blob: hexToByteArray[48]("0x96902ac59a4940715d171f1d6ec3e03b0c1557fc0100abb930b6626917b9792aabd48ec1bc1e37737c582fe11c966658"))),
ExecutionLayerExit(
ExecutionLayerWithdrawalRequest(
source_address: ExecutionAddress.fromHex("0xa99cc4727a81a0abfb662fe28748133420938dae"),
validator_pubkey: ValidatorPubKey(blob: hexToByteArray[48]("0x10f3cea6d52558a866988352bef57525f708aecb5fb392af8453e306cf7c5da68aea8a544d71db63dc1057317b00feb7"))),
ExecutionLayerExit(
ExecutionLayerWithdrawalRequest(
source_address: ExecutionAddress.fromHex("0xe13404d88c1418f69c92ed12d256382a462ecf4e"),
validator_pubkey: ValidatorPubKey(blob: hexToByteArray[48]("0xf8f8cffff4aa7bec351b9e084274b6e47c536671bd559c7fbf110985e684a58c0384ffc314c23c4441c0f17ce33bd767"))),
ExecutionLayerExit(
ExecutionLayerWithdrawalRequest(
source_address: ExecutionAddress.fromHex("0x6f9427252a6fa414a6501e0761cf92f0839f3bbe"),
validator_pubkey: ValidatorPubKey(blob: hexToByteArray[48]("0x4ca4f660800c2cfa68827299ddcbfddcf2cb01c51dcaf5af1abc5e8f05164846ca26f1c8c884a3e674a22dbfc0d9fa7b"))),
ExecutionLayerExit(
ExecutionLayerWithdrawalRequest(
source_address: ExecutionAddress.fromHex("0x14bce680ec1a632aac5f77cb4d5eca52f74bd1e6"),
validator_pubkey: ValidatorPubKey(blob: hexToByteArray[48]("0xb4f363283d5276f12a6c2c98c58484c6a6e8e3c7f5b3adfc044d2de76365bef427f8b9ac1e321baa7a611447010f9e8d"))),
]),
@ -2544,11 +2544,11 @@ suite "Eth1 monitor":
signature: ValidatorSig(blob: hexToByteArray[96]("0xac560bee8d8dd4dad94f2bd5b480e7799f7a8445adf3e0070747f8b5724d442453fbba2f332cc69af3a450dce80249b6b7afe19340f4fc5dc54a5c0e56cd4c484c94c61480bc56c75eef44e55c1288bd58739b8354caa93da5d2502bb38546df")),
index: 7086745948630243467'u64),
]),
exits: List[ExecutionLayerExit, MAX_EXECUTION_LAYER_EXITS_PER_PAYLOAD].init(@[
ExecutionLayerExit(
withdrawal_requests: List[ExecutionLayerWithdrawalRequest, MAX_WITHDRAWAL_REQUESTS_PER_PAYLOAD].init(@[
ExecutionLayerWithdrawalRequest(
source_address: ExecutionAddress.fromHex("0x91810ed86a3244c89274f94fd510532cf12d7074"),
validator_pubkey: ValidatorPubKey(blob: hexToByteArray[48]("0xbb480d96367f62ab5790cbfdeeac6344e21774681edd0afe64c50b48f4d07795e584468821788948c7d8c151733ad01f"))),
ExecutionLayerExit(
ExecutionLayerWithdrawalRequest(
source_address: ExecutionAddress.fromHex("0xe16b15a5256815cf6d338498a5cb0e8ec0d5bfec"),
validator_pubkey: ValidatorPubKey(blob: hexToByteArray[48]("0x79b49178606e2a5cda067c04b982d445df7b41d09d4361e5498b7a454d0e8a37a6975da56c3bd20694a3fcb467f7ff59"))),
]),
@ -2573,17 +2573,17 @@ suite "Eth1 monitor":
excess_blob_gas: 1,
deposit_receipts: List[DepositReceipt, MAX_DEPOSIT_RECEIPTS_PER_PAYLOAD].init(@[
]),
exits: List[ExecutionLayerExit, MAX_EXECUTION_LAYER_EXITS_PER_PAYLOAD].init(@[
ExecutionLayerExit(
withdrawal_requests: List[ExecutionLayerWithdrawalRequest, MAX_WITHDRAWAL_REQUESTS_PER_PAYLOAD].init(@[
ExecutionLayerWithdrawalRequest(
source_address: ExecutionAddress.fromHex("0x08396e3d726ff055f903e2b4e7b743fd8c128f4b"),
validator_pubkey: ValidatorPubKey(blob: hexToByteArray[48]("0x01c1c045960d8121bc8ab57c4728dfb3c07289818df71893c002352eca51c54f03db8840f608607bea01bd7b0f02284d"))),
ExecutionLayerExit(
ExecutionLayerWithdrawalRequest(
source_address: ExecutionAddress.fromHex("0xc7fefcefc468685bc9b8cdd3c4e1ae643952b254"),
validator_pubkey: ValidatorPubKey(blob: hexToByteArray[48]("0x816cae90cab4ca290dfaf9f32b7ad508bd82095ec815cd55b9399eee91208d30f79e548951bfdddc60b7e7560f2b9e1b"))),
ExecutionLayerExit(
ExecutionLayerWithdrawalRequest(
source_address: ExecutionAddress.fromHex("0x7eef42203641e2f5c21779289b6c48d24d578887"),
validator_pubkey: ValidatorPubKey(blob: hexToByteArray[48]("0x738dfea8a133b5fd384bd6242fa58f1119bcfed0cfca93899c95f1670d1460b905134cc91eabb429d2147b5f147d5d1f"))),
ExecutionLayerExit(
ExecutionLayerWithdrawalRequest(
source_address: ExecutionAddress.fromHex("0x032d5223828ee1c8943fdacfbcd25ce4bb2eacfd"),
validator_pubkey: ValidatorPubKey(blob: hexToByteArray[48]("0xf42315c025ae7ef0e8a04175441e9617b0e315a9e7c8fc5f0a0bba4efc9775fea3a8af9b40c4aa37633718ccb5b3260d"))),
]),
@ -2622,14 +2622,14 @@ suite "Eth1 monitor":
signature: ValidatorSig(blob: hexToByteArray[96]("0x121632563dca7d7a560e5b243d7f27dc7dc72319f1486f67cb41751c5f5a42bd9f8efdd14e3f811e03c84e3ba36295a0cb2313bb9792cfc7d80a1669f0adc30934440adbd665ef96b3c30a2762cbaf932e6eb1b4a1c93063ec7f0b6f6aa2a9db")),
index: 10368232928814555152'u64),
]),
exits: List[ExecutionLayerExit, MAX_EXECUTION_LAYER_EXITS_PER_PAYLOAD].init(@[
ExecutionLayerExit(
withdrawal_requests: List[ExecutionLayerWithdrawalRequest, MAX_WITHDRAWAL_REQUESTS_PER_PAYLOAD].init(@[
ExecutionLayerWithdrawalRequest(
source_address: ExecutionAddress.fromHex("0x7bee235a632b5f79831f376843209740d409b9f8"),
validator_pubkey: ValidatorPubKey(blob: hexToByteArray[48]("0x8f40af9186eb70dea2f3105785a930511368e60d2235055c34a0be1a591c5b580eed67542c89a0f8a024c4a6bd1f9bb7"))),
ExecutionLayerExit(
ExecutionLayerWithdrawalRequest(
source_address: ExecutionAddress.fromHex("0x72fdf4c5a62970c6d6c9ee395eec4dfd6fcca4de"),
validator_pubkey: ValidatorPubKey(blob: hexToByteArray[48]("0x540a810f4e9ad62bca1d677e9135d519100012f6f12a8f5105623762ba5de3782cb3baaf63c4a32cf03a036127d6d009"))),
ExecutionLayerExit(
ExecutionLayerWithdrawalRequest(
source_address: ExecutionAddress.fromHex("0xd3a0f8518063d55c61423dce1bfcd2abd9a27a62"),
validator_pubkey: ValidatorPubKey(blob: hexToByteArray[48]("0x82bec5cd588df021e98087c703b995075ee1cfde2257eebed5e27f53a3a16903479fa2e6864ab3c3c397cd25b6ba3d4f"))),
]),
@ -2669,8 +2669,8 @@ suite "Eth1 monitor":
signature: ValidatorSig(blob: hexToByteArray[96]("0x8e274ccbdef898449a07a296386e5983ec423f7ddee02bb9d480ec99dca4f5074b8f6cf469758a45586f031e2ae0a5448aa133531cddf88e9bd2b9fae191fdc817c1989124f1866753fbc833f79fb78f89677df12bc6d288693e5362f2a972bd")),
index: 15922103202526011942'u64),
]),
exits: List[ExecutionLayerExit, MAX_EXECUTION_LAYER_EXITS_PER_PAYLOAD].init(@[
ExecutionLayerExit(
withdrawal_requests: List[ExecutionLayerWithdrawalRequest, MAX_WITHDRAWAL_REQUESTS_PER_PAYLOAD].init(@[
ExecutionLayerWithdrawalRequest(
source_address: ExecutionAddress.fromHex("0xe368e59ddc49ffac6818f01b4be692a517b6838e"),
validator_pubkey: ValidatorPubKey(blob: hexToByteArray[48]("0x9c7a489a7498cada308db339f80aafeeff5e38ef7dc5803344a725b3b7f23d6d6162a33798a69660417b8fffb51c3d50")))
])
@ -2730,7 +2730,7 @@ suite "Eth1 monitor":
signature: ValidatorSig(blob: hexToByteArray[96]("0x232d34989ba30727e4ae0aa874a4bfc3934d61d0295d8f1c5f8416523f5cd05a3181a03543ff7318c4f4b9207d006267dde451177612bd888f69b43ebea83a4289cd6615526160d7ecf2a09842d4c2e90ae9f207a440a348ed8ef31e0cf1fe8b")),
index: 4403524705240661292'u64),
]),
exits: List[ExecutionLayerExit, MAX_EXECUTION_LAYER_EXITS_PER_PAYLOAD].init(@[
withdrawal_requests: List[ExecutionLayerWithdrawalRequest, MAX_WITHDRAWAL_REQUESTS_PER_PAYLOAD].init(@[
])
)]

View File

@ -61,7 +61,8 @@ suite "Message signatures":
test "Aggregate and proof signatures":
let aggregate_and_proof = AggregateAndProof(
aggregate: Attestation(aggregation_bits: CommitteeValidatorsBits.init(8)))
aggregate: phase0.Attestation(
aggregation_bits: CommitteeValidatorsBits.init(8)))
check:
# Matching public/private keys and genesis validator roots
@ -271,4 +272,4 @@ suite "Message signatures":
fork0, genesis_validators_root0, slot, subcommittee_index,
load(pubkey0).get, get_sync_committee_selection_proof(
fork0, genesis_validators_root1, slot,
subcommittee_index, privkey0).toValidatorSig)
subcommittee_index, privkey0).toValidatorSig)

View File

@ -56,7 +56,7 @@ template bellatrix_steps() =
check: b.message.body.attester_slashings.add(default(AttesterSlashing))
do_check
check: b.message.body.attestations.add(
Attestation(aggregation_bits: CommitteeValidatorsBits.init(1)))
phase0.Attestation(aggregation_bits: CommitteeValidatorsBits.init(1)))
do_check
check: b.message.body.deposits.add(default(Deposit))
do_check

View File

@ -465,8 +465,8 @@ proc init(t: typedesc[ProduceAttestationDataResponse],
target: Checkpoint(epoch: Epoch(ad.target))
))
proc init(t: typedesc[Attestation], bits: string,
slot: Slot = GENESIS_SLOT, index: uint64 = 0'u64): Attestation =
proc init(t: typedesc[phase0.Attestation], bits: string,
slot: Slot = GENESIS_SLOT, index: uint64 = 0'u64): phase0.Attestation =
let
jdata = "{\"data\":\"" & bits & "\"}"
bits =
@ -767,8 +767,8 @@ suite "Validator Client test suite":
test "getUniqueVotes() test vectors":
for vector in AttestationBitsVectors:
let
a1 = Attestation.init(vector[0][0][0], vector[0][0][1], vector[0][0][2])
a2 = Attestation.init(vector[0][1][0], vector[0][1][1], vector[0][1][2])
a1 = phase0.Attestation.init(vector[0][0][0], vector[0][0][1], vector[0][0][2])
a2 = phase0.Attestation.init(vector[0][1][0], vector[0][1][1], vector[0][1][2])
check getUniqueVotes([a1, a2]) == vector[1]
asyncTest "firstSuccessParallel() API timeout test":
@ -905,4 +905,4 @@ suite "Validator Client test suite":
res.isOk()
len(res.get().data) == 1
res.get().data[0].index == 100000
res.get().data[0].is_live == true
res.get().data[0].is_live == true

View File

@ -157,7 +157,7 @@ proc addTestBlock*(
state: var ForkedHashedBeaconState,
cache: var StateCache,
eth1_data: Eth1Data = Eth1Data(),
attestations: seq[Attestation] = newSeq[Attestation](),
attestations: seq[phase0.Attestation] = newSeq[phase0.Attestation](),
deposits: seq[Deposit] = newSeq[Deposit](),
sync_aggregate: SyncAggregate = SyncAggregate.init(),
graffiti: GraffitiBytes = default(GraffitiBytes),
@ -242,7 +242,7 @@ proc makeTestBlock*(
state: ForkedHashedBeaconState,
cache: var StateCache,
eth1_data = Eth1Data(),
attestations = newSeq[Attestation](),
attestations = newSeq[phase0.Attestation](),
deposits = newSeq[Deposit](),
sync_aggregate = SyncAggregate.init(),
graffiti = default(GraffitiBytes),
@ -323,7 +323,7 @@ func makeAttestation(
state: ForkedHashedBeaconState, beacon_block_root: Eth2Digest,
committee: seq[ValidatorIndex], slot: Slot, committee_index: CommitteeIndex,
validator_index: ValidatorIndex, cache: var StateCache,
flags: UpdateFlags = {}): Attestation =
flags: UpdateFlags = {}): phase0.Attestation =
let
index_in_committee = committee.find(validator_index)
data = makeAttestationData(state, slot, committee_index, beacon_block_root)
@ -341,7 +341,7 @@ func makeAttestation(
getStateField(state, genesis_validators_root),
data, committee, aggregation_bits)
Attestation(
phase0.Attestation(
data: data,
aggregation_bits: aggregation_bits,
signature: sig
@ -364,7 +364,7 @@ func find_beacon_committee(
func makeAttestation*(
state: ForkedHashedBeaconState, beacon_block_root: Eth2Digest,
validator_index: ValidatorIndex, cache: var StateCache): Attestation =
validator_index: ValidatorIndex, cache: var StateCache): phase0.Attestation =
let (committee, slot, index) =
find_beacon_committee(state, validator_index, cache)
makeAttestation(state, beacon_block_root, committee, slot, index,
@ -373,7 +373,7 @@ func makeAttestation*(
func makeFullAttestations*(
state: ForkedHashedBeaconState, beacon_block_root: Eth2Digest, slot: Slot,
cache: var StateCache,
flags: UpdateFlags = {}): seq[Attestation] =
flags: UpdateFlags = {}): seq[phase0.Attestation] =
# Create attestations in which the full committee participates for each shard
# that should be attested to during a particular slot
let committees_per_slot = get_committee_count_per_slot(
@ -384,7 +384,7 @@ func makeFullAttestations*(
data = makeAttestationData(state, slot, committee_index, beacon_block_root)
doAssert committee.len() >= 1
var attestation = Attestation(
var attestation = phase0.Attestation(
aggregation_bits: CommitteeValidatorsBits.init(committee.len),
data: data)
for i in 0..<committee.len:
@ -556,4 +556,4 @@ iterator makeTestBlocks*(
deposits = deposits,
sync_aggregate = sync_aggregate,
graffiti = graffiti,
cfg = cfg)
cfg = cfg)

2
vendor/nim-bearssl vendored

@ -1 +1 @@
Subproject commit 86f212c6a5d76b52e20fad2e318cc5436d04fc26
Subproject commit d81b37dc2011bf3a2bd93500489877c2ce8e6ac3

2
vendor/nim-chronos vendored

@ -1 +1 @@
Subproject commit ef1b077adfdc803fcce880e81a5740b964bac0bc
Subproject commit 0d050d582306e8c521c3a4a6f6dcb3c83c93a90f

@ -1 +1 @@
Subproject commit f26a9909cebf29cc5e61dd795427eda60c6e431e
Subproject commit 11b9d952a80ec87e2443405a6a5382f9daac51f8

2
vendor/nimcrypto vendored

@ -1 +1 @@
Subproject commit 0c6ddab03a99805239b7875f71d2ca95fbed6f85
Subproject commit 485f7b3cfa83c1beecc0e31be0e964d697aa74d7