automated consensus spec URL updating to v1.4.0-beta.1 (#5280)

This commit is contained in:
tersec 2023-08-09 03:58:47 +00:00 committed by GitHub
parent 8c3d1cd3aa
commit 85e1976ac3
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
68 changed files with 324 additions and 324 deletions

View File

@ -16,7 +16,7 @@ from ./spec/datatypes/capella import
from ./spec/datatypes/deneb import ExecutionPayloadHeader
type
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/phase0/beacon-chain.md#beaconstate
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/phase0/beacon-chain.md#beaconstate
# Memory-representation-equivalent to a phase0 BeaconState for in-place SSZ
# reading and writing
Phase0BeaconStateNoImmutableValidators* = object
@ -69,7 +69,7 @@ type
current_justified_checkpoint*: Checkpoint
finalized_checkpoint*: Checkpoint
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/altair/beacon-chain.md#beaconstate
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/altair/beacon-chain.md#beaconstate
# Memory-representation-equivalent to an Altair BeaconState for in-place SSZ
# reading and writing
AltairBeaconStateNoImmutableValidators* = object
@ -186,7 +186,7 @@ type
# Execution
latest_execution_payload_header*: bellatrix.ExecutionPayloadHeader # [New in Bellatrix]
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/capella/beacon-chain.md#beaconstate
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/capella/beacon-chain.md#beaconstate
# with indirect changes via ExecutionPayload
# Memory-representation-equivalent to a Capella BeaconState for in-place SSZ
# reading and writing
@ -258,7 +258,7 @@ type
HashList[HistoricalSummary,
Limit HISTORICAL_ROOTS_LIMIT] # [New in Capella]
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/capella/beacon-chain.md#beaconstate
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/capella/beacon-chain.md#beaconstate
# with indirect changes via ExecutionPayloadHeader
# Memory-representation-equivalent to a Deneb BeaconState for in-place SSZ
# reading and writing

View File

@ -5,7 +5,7 @@ This folder holds the various consensus object pools needed for a blockchain cli
Object in those pools have passed the "gossip validation" filter according
to specs:
- blocks: https://github.com/ethereum/consensus-specs/blob/v1.3.0/specs/phase0/p2p-interface.md#beacon_block
- aggregate attestations: https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/phase0/p2p-interface.md#beacon_aggregate_and_proof
- aggregate attestations: https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/phase0/p2p-interface.md#beacon_aggregate_and_proof
- unaggregated attestation: https://github.com/ethereum/consensus-specs/blob/v1.3.0/specs/phase0/p2p-interface.md#beacon_attestation_subnet_id
- voluntary exits: https://github.com/ethereum/consensus-specs/blob/v1.4.0-alpha.1/specs/phase0/p2p-interface.md#voluntary_exit
- Attester slashings: https://github.com/ethereum/consensus-specs/blob/v1.3.0/specs/phase0/p2p-interface.md#attester_slashing

View File

@ -772,7 +772,7 @@ proc getBeaconHead*(
finalizedExecutionPayloadHash =
pool.dag.loadExecutionBlockHash(pool.dag.finalizedHead.blck)
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/fork_choice/safe-block.md#get_safe_execution_payload_hash
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/fork_choice/safe-block.md#get_safe_execution_payload_hash
safeBlockRoot = pool.forkChoice.get_safe_beacon_block_root()
safeBlock = pool.dag.getBlockRef(safeBlockRoot)
safeExecutionPayloadHash =

View File

@ -134,7 +134,7 @@ func link*(parent, child: BlockRef) =
func get_ancestor*(blck: BlockRef, slot: Slot,
maxDepth = 100'i64 * 365 * 24 * 60 * 60 div SECONDS_PER_SLOT.int):
BlockRef =
## https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/phase0/fork-choice.md#get_ancestor
## https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/phase0/fork-choice.md#get_ancestor
## Return the most recent block as of the time at `slot` that not more recent
## than `blck` itself
if isNil(blck): return nil

View File

@ -1931,7 +1931,7 @@ proc pruneBlocksDAG(dag: ChainDAGRef) =
prunedHeads = hlen - dag.heads.len,
dagPruneDur = Moment.now() - startTick
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/sync/optimistic.md#helpers
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/sync/optimistic.md#helpers
template is_optimistic*(dag: ChainDAGRef, bid: BlockId): bool =
let blck =
if bid.slot <= dag.finalizedHead.slot:

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.0/specs/phase0/beacon-chain.md#get_committee_count_per_slot
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/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.0/specs/phase0/beacon-chain.md#get_beacon_committee
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/phase0/beacon-chain.md#get_beacon_committee
func get_beacon_committee*(
shufflingRef: ShufflingRef, slot: Slot, committee_index: CommitteeIndex):
seq[ValidatorIndex] =

View File

@ -349,7 +349,7 @@ proc produceSyncAggregate*(
proc isEpochLeadTime*(
pool: SyncCommitteeMsgPool, epochsToSyncPeriod: uint64): bool =
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/altair/validator.md#sync-committee-subnet-stability
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/altair/validator.md#sync-committee-subnet-stability
# This ensures a uniform distribution without requiring additional state:
# (1/4) = 1/4, 4 slots out
# (3/4) * (1/3) = 1/4, 3 slots out

View File

@ -223,7 +223,7 @@ proc restValidatorExit(config: BeaconNodeConf) {.async.} =
let response = await client.getSpec()
if response.status == 200:
let spec = response.data
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/phase0/beacon-chain.md#voluntary-exits
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/phase0/beacon-chain.md#voluntary-exits
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/deneb/beacon-chain.md#modified-process_voluntary_exit
if currentEpoch >= Epoch(spec.data.DENEB_FORK_EPOCH):
Fork(

View File

@ -425,11 +425,11 @@ template toGaugeValue(x: Quantity): int64 =
# doAssert SECONDS_PER_ETH1_BLOCK * cfg.ETH1_FOLLOW_DISTANCE < GENESIS_DELAY,
# "Invalid configuration: GENESIS_DELAY is set too low"
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/phase0/validator.md#get_eth1_data
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/phase0/validator.md#get_eth1_data
func compute_time_at_slot(genesis_time: uint64, slot: Slot): uint64 =
genesis_time + slot * SECONDS_PER_SLOT
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/phase0/validator.md#get_eth1_data
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/phase0/validator.md#get_eth1_data
func voting_period_start_time(state: ForkedHashedBeaconState): uint64 =
let eth1_voting_period_start_slot =
getStateField(state, slot) - getStateField(state, slot) mod
@ -437,7 +437,7 @@ func voting_period_start_time(state: ForkedHashedBeaconState): uint64 =
compute_time_at_slot(
getStateField(state, genesis_time), eth1_voting_period_start_slot)
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/phase0/validator.md#get_eth1_data
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/phase0/validator.md#get_eth1_data
func is_candidate_block(cfg: RuntimeConfig,
blk: Eth1Block,
period_start: uint64): bool =
@ -1680,7 +1680,7 @@ template trackFinalizedState*(m: ELManager,
finalizedStateDepositIndex: uint64): bool =
trackFinalizedState(m.eth1Chain, finalizedEth1Data, finalizedStateDepositIndex)
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/phase0/validator.md#get_eth1_data
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/phase0/validator.md#get_eth1_data
proc getBlockProposalData*(chain: var Eth1Chain,
state: ForkedHashedBeaconState,
finalizedEth1Data: Eth1Data,

View File

@ -7,7 +7,7 @@
{.push raises: [].}
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/tests/core/pyspec/eth2spec/utils/merkle_minimal.py
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/tests/core/pyspec/eth2spec/utils/merkle_minimal.py
# Merkle tree helpers
# ---------------------------------------------------------------

View File

@ -107,7 +107,7 @@ proc update_justified(
self.update_justified(dag, blck, justified.epoch)
ok()
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/phase0/fork-choice.md#update_checkpoints
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/phase0/fork-choice.md#update_checkpoints
proc update_checkpoints(
self: var Checkpoints, dag: ChainDAGRef,
checkpoints: FinalityCheckpoints): FcResult[void] =
@ -124,7 +124,7 @@ proc update_checkpoints(
ok()
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/phase0/fork-choice.md#on_tick_per_slot
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/phase0/fork-choice.md#on_tick_per_slot
proc on_tick(
self: var ForkChoice, dag: ChainDAGRef, time: BeaconTime): FcResult[void] =
## Must be called at least once per slot.
@ -206,7 +206,7 @@ func contains*(self: ForkChoiceBackend, block_root: Eth2Digest): bool =
## In particular, before adding a block, its parent must be known to the fork choice
self.proto_array.indices.contains(block_root)
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/phase0/fork-choice.md#on_attestation
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/phase0/fork-choice.md#on_attestation
proc on_attestation*(
self: var ForkChoice,
dag: ChainDAGRef,
@ -235,7 +235,7 @@ proc on_attestation*(
block_root: beacon_block_root))
ok()
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/phase0/fork-choice.md#on_attester_slashing
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/phase0/fork-choice.md#on_attester_slashing
func process_equivocation*(
self: var ForkChoice,
validator_index: ValidatorIndex
@ -350,7 +350,7 @@ func find_head*(
return ok(new_head)
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/phase0/fork-choice.md#get_head
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/phase0/fork-choice.md#get_head
proc get_head*(self: var ForkChoice,
dag: ChainDAGRef,
wallTime: BeaconTime): FcResult[Eth2Digest] =
@ -364,7 +364,7 @@ proc get_head*(self: var ForkChoice,
self.checkpoints.justified.balances,
self.checkpoints.proposer_boost_root)
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/fork_choice/safe-block.md#get_safe_beacon_block_root
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/fork_choice/safe-block.md#get_safe_beacon_block_root
func get_safe_beacon_block_root*(self: ForkChoice): Eth2Digest =
# Use most recent justified block as a stopgap
self.checkpoints.justified.checkpoint.root

View File

@ -123,7 +123,7 @@ iterator realizePendingCheckpoints*(
# Reset tip tracking for new epoch
self.currentEpochTips.clear()
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/phase0/fork-choice.md#get_weight
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/phase0/fork-choice.md#get_weight
func calculateProposerBoost(validatorBalances: openArray[Gwei]): uint64 =
var total_balance: uint64
for balance in validatorBalances:
@ -190,7 +190,7 @@ func applyScoreChanges*(self: var ProtoArray,
# If we find the node matching the current proposer boost root, increase
# the delta by the new score amount.
#
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/phase0/fork-choice.md#get_weight
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/phase0/fork-choice.md#get_weight
if (not proposerBoostRoot.isZero) and proposerBoostRoot == node.bid.root:
proposerBoostScore = calculateProposerBoost(newBalances)
if nodeDelta >= 0 and

View File

@ -10,7 +10,7 @@ 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.3.0/specs/phase0/p2p-interface.md#beacon_block
- Attestations (aggregated): https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/phase0/p2p-interface.md#beacon_aggregate_and_proof
- Attestations (aggregated): https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/phase0/p2p-interface.md#beacon_aggregate_and_proof
- Attestations (unaggregated): https://github.com/ethereum/consensus-specs/blob/v1.3.0/specs/phase0/p2p-interface.md#attestation-subnets
- Voluntary exits: https://github.com/ethereum/consensus-specs/blob/v1.4.0-alpha.3/specs/phase0/p2p-interface.md#voluntary_exit
- Proposer slashings: https://github.com/ethereum/consensus-specs/blob/v1.4.0-alpha.3/specs/phase0/p2p-interface.md#proposer_slashing

View File

@ -780,7 +780,7 @@ proc processBlock(
# - MUST NOT optimistically import the block.
# - MUST NOT apply the block to the fork choice store.
# - MAY queue the block for later processing.
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/sync/optimistic.md#execution-engine-errors
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/sync/optimistic.md#execution-engine-errors
await sleepAsync(chronos.seconds(1))
self[].addBlock(
entry.src, entry.blck, entry.blobs, entry.resfut, entry.maybeFinalized,

View File

@ -657,7 +657,7 @@ proc processSignedContributionAndProof*(
err(v.error())
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/altair/light-client/sync-protocol.md#process_light_client_finality_update
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/altair/light-client/sync-protocol.md#process_light_client_finality_update
proc processLightClientFinalityUpdate*(
self: var Eth2Processor, src: MsgSource,
finality_update: ForkedLightClientFinalityUpdate
@ -673,7 +673,7 @@ proc processLightClientFinalityUpdate*(
beacon_light_client_finality_update_dropped.inc(1, [$v.error[0]])
v
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/altair/light-client/sync-protocol.md#process_light_client_optimistic_update
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/altair/light-client/sync-protocol.md#process_light_client_optimistic_update
proc processLightClientOptimisticUpdate*(
self: var Eth2Processor, src: MsgSource,
optimistic_update: ForkedLightClientOptimisticUpdate

View File

@ -95,7 +95,7 @@ func check_propagation_slot_range(
return ok(msgSlot)
if consensusFork < ConsensusFork.Deneb:
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/phase0/p2p-interface.md#configuration
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/phase0/p2p-interface.md#configuration
# The spec value of ATTESTATION_PROPAGATION_SLOT_RANGE is 32, but it can
# retransmit attestations on the cusp of being out of spec, and which by
# the time they reach their destination might be out of spec.
@ -104,13 +104,13 @@ func check_propagation_slot_range(
if msgSlot + ATTESTATION_PROPAGATION_SLOT_RANGE < pastSlot.slot:
return errIgnore("Attestation slot in the past")
else:
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/deneb/p2p-interface.md#beacon_attestation_subnet_id
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/deneb/p2p-interface.md#beacon_attestation_subnet_id
# "[IGNORE] the epoch of attestation.data.slot is either the current or
# previous epoch (with a MAXIMUM_GOSSIP_CLOCK_DISPARITY allowance) -- i.e.
# compute_epoch_at_slot(attestation.data.slot) in
# (get_previous_epoch(state), get_current_epoch(state))"
#
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/deneb/p2p-interface.md#beacon_aggregate_and_proof
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/deneb/p2p-interface.md#beacon_aggregate_and_proof
# "[IGNORE] the epoch of aggregate.data.slot is either the current or
# previous epoch (with a MAXIMUM_GOSSIP_CLOCK_DISPARITY allowance) -- i.e.
# compute_epoch_at_slot(aggregate.data.slot) in
@ -413,7 +413,7 @@ proc validateBeaconBlock*(
# proposer for the slot, signed_beacon_block.message.slot.
#
# While this condition is similar to the proposer slashing condition at
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/phase0/validator.md#proposer-slashing
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/phase0/validator.md#proposer-slashing
# it's not identical, and this check does not address slashing:
#
# (1) The beacon blocks must be conflicting, i.e. different, for the same
@ -560,8 +560,8 @@ proc validateBeaconBlock*(
ok()
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/phase0/p2p-interface.md#beacon_attestation_subnet_id
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/deneb/p2p-interface.md#beacon_aggregate_and_proof
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/phase0/p2p-interface.md#beacon_attestation_subnet_id
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/deneb/p2p-interface.md#beacon_aggregate_and_proof
proc validateAttestation*(
pool: ref AttestationPool,
batchCrypto: ref BatchCrypto,
@ -591,7 +591,7 @@ proc validateAttestation*(
# >= attestation.data.slot (a client MAY queue future attestations for
# processing at the appropriate slot).
#
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/deneb/p2p-interface.md#beacon_attestation_subnet_id
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/deneb/p2p-interface.md#beacon_attestation_subnet_id
# modifies this for Deneb and newer forks.
block:
let v = check_propagation_slot_range(
@ -731,8 +731,8 @@ proc validateAttestation*(
return ok((validator_index, sig))
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/phase0/p2p-interface.md#beacon_aggregate_and_proof
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/deneb/p2p-interface.md#beacon_aggregate_and_proof
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/phase0/p2p-interface.md#beacon_aggregate_and_proof
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/deneb/p2p-interface.md#beacon_aggregate_and_proof
proc validateAggregate*(
pool: ref AttestationPool,
batchCrypto: ref BatchCrypto,
@ -763,7 +763,7 @@ proc validateAggregate*(
# MAXIMUM_GOSSIP_CLOCK_DISPARITY allowance) -- i.e. aggregate.data.slot +
# ATTESTATION_PROPAGATION_SLOT_RANGE >= current_slot >= aggregate.data.slot
#
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/deneb/p2p-interface.md#beacon_aggregate_and_proof
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/deneb/p2p-interface.md#beacon_aggregate_and_proof
# modifies this for Deneb and newer forks.
block:
let v = check_propagation_slot_range(
@ -1067,7 +1067,7 @@ proc validateVoluntaryExit*(
ok()
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/altair/p2p-interface.md#sync_committee_subnet_id
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/altair/p2p-interface.md#sync_committee_subnet_id
proc validateSyncCommitteeMessage*(
dag: ChainDAGRef,
quarantine: ref Quarantine,
@ -1308,7 +1308,7 @@ proc validateContribution*(
return ok((blck.bid, sig, participants))
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/altair/light-client/p2p-interface.md#light_client_finality_update
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/altair/light-client/p2p-interface.md#light_client_finality_update
proc validateLightClientFinalityUpdate*(
pool: var LightClientPool, dag: ChainDAGRef,
finality_update: ForkedLightClientFinalityUpdate,
@ -1344,7 +1344,7 @@ proc validateLightClientFinalityUpdate*(
pool.latestForwardedFinalitySlot = finalized_slot
ok()
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/altair/light-client/p2p-interface.md#light_client_optimistic_update
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/altair/light-client/p2p-interface.md#light_client_optimistic_update
proc validateLightClientOptimisticUpdate*(
pool: var LightClientPool, dag: ChainDAGRef,
optimistic_update: ForkedLightClientOptimisticUpdate,

View File

@ -528,7 +528,7 @@ func toValidationError(
# previously forwarded `optimistic_update`s
errIgnore($r.error)
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/altair/light-client/sync-protocol.md#process_light_client_finality_update
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/altair/light-client/sync-protocol.md#process_light_client_finality_update
proc processLightClientFinalityUpdate*(
self: var LightClientProcessor, src: MsgSource,
finality_update: ForkedLightClientFinalityUpdate
@ -543,7 +543,7 @@ proc processLightClientFinalityUpdate*(
self.latestFinalityUpdate = finality_update.toOptimistic
v
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/altair/light-client/sync-protocol.md#process_light_client_finality_update
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/altair/light-client/sync-protocol.md#process_light_client_finality_update
proc processLightClientOptimisticUpdate*(
self: var LightClientProcessor, src: MsgSource,
optimistic_update: ForkedLightClientOptimisticUpdate

View File

@ -74,7 +74,7 @@ proc ETHConsensusConfigCreateFromYaml(
## * `NULL` - If the given `config.yaml` is malformed or incompatible.
##
## See:
## * https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/configs/README.md
## * https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/configs/README.md
let cfg = RuntimeConfig.new()
try:
cfg[] = readRuntimeConfig($configFileContent, "config.yaml")[0]
@ -138,11 +138,11 @@ proc ETHBeaconStateCreateFromSsz(
## * `NULL` - If the given `sszBytes` is malformed.
##
## See:
## * https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/phase0/beacon-chain.md#beaconstate
## * https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/altair/beacon-chain.md#beaconstate
## * https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/bellatrix/beacon-chain.md#beaconstate
## * https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/capella/beacon-chain.md#beaconstate
## * https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/configs/README.md
## * https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/phase0/beacon-chain.md#beaconstate
## * https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/altair/beacon-chain.md#beaconstate
## * https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/bellatrix/beacon-chain.md#beaconstate
## * https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/capella/beacon-chain.md#beaconstate
## * https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/configs/README.md
let
consensusFork = decodeEthConsensusVersion($consensusVersion).valueOr:
return nil
@ -191,7 +191,7 @@ proc ETHRootDestroy(root: ptr Eth2Digest) {.exported.} =
## * `root` - Merkle root.
##
## See:
## * https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/phase0/beacon-chain.md#custom-types
## * https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/phase0/beacon-chain.md#custom-types
root.destroy()
proc ETHForkDigestsCreateFromState(
@ -210,7 +210,7 @@ proc ETHForkDigestsCreateFromState(
## * Pointer to an initialized fork digests cache based on the beacon state.
##
## See:
## * https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/phase0/beacon-chain.md#compute_fork_digest
## * https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/phase0/beacon-chain.md#compute_fork_digest
let forkDigests = ForkDigests.new()
forkDigests[] = ForkDigests.init(
cfg[], getStateField(state[], genesis_validators_root))
@ -261,7 +261,7 @@ proc ETHBeaconClockGetSlot(beaconClock: ptr BeaconClock): cint {.exported.} =
## * `0` - If genesis is still pending.
##
## See:
## * https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/phase0/beacon-chain.md#custom-types
## * https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/phase0/beacon-chain.md#custom-types
beaconClock[].now().slotOrZero().cint
const lcDataFork = LightClientDataFork.high
@ -320,8 +320,8 @@ proc ETHLightClientStoreCreateFromBootstrap(
## See:
## * https://ethereum.github.io/beacon-APIs/?urls.primaryName=v2.4.1#/Beacon/getLightClientBootstrap
## * https://ethereum.github.io/beacon-APIs/?urls.primaryName=v2.4.1#/Events/eventstream
## * https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/altair/light-client/light-client.md
## * https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/phase0/weak-subjectivity.md#weak-subjectivity-period
## * https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/altair/light-client/light-client.md
## * https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/phase0/weak-subjectivity.md#weak-subjectivity-period
let
mediaType = MediaType.init($mediaType)
consensusFork = decodeEthConsensusVersion($consensusVersion).valueOr:
@ -727,7 +727,7 @@ func ETHLightClientStoreGetFinalizedHeader(
## * Latest finalized header.
##
## See:
## * https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/capella/light-client/sync-protocol.md#modified-lightclientheader
## * https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/capella/light-client/sync-protocol.md#modified-lightclientheader
addr store[].finalized_header
func ETHLightClientStoreIsNextSyncCommitteeKnown(
@ -746,8 +746,8 @@ func ETHLightClientStoreIsNextSyncCommitteeKnown(
## * Whether or not the next sync committee is currently known.
##
## See:
## * https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/altair/light-client/sync-protocol.md#is_next_sync_committee_known
## * https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/altair/light-client/light-client.md
## * https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/altair/light-client/sync-protocol.md#is_next_sync_committee_known
## * https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/altair/light-client/light-client.md
store[].is_next_sync_committee_known
func ETHLightClientStoreGetOptimisticHeader(
@ -766,7 +766,7 @@ func ETHLightClientStoreGetOptimisticHeader(
## * Latest optimistic header.
##
## See:
## * https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/capella/light-client/sync-protocol.md#modified-lightclientheader
## * https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/capella/light-client/sync-protocol.md#modified-lightclientheader
addr store[].optimistic_header
func ETHLightClientStoreGetSafetyThreshold(
@ -787,7 +787,7 @@ func ETHLightClientStoreGetSafetyThreshold(
## * Light client store safety threshold.
##
## See:
## * https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/altair/light-client/sync-protocol.md#get_safety_threshold
## * https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/altair/light-client/sync-protocol.md#get_safety_threshold
store[].get_safety_threshold.cint
proc ETHLightClientHeaderCreateCopy(
@ -833,7 +833,7 @@ proc ETHLightClientHeaderCopyBeaconRoot(
## * Pointer to a copy of the given header's beacon block root.
##
## See:
## * https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/phase0/beacon-chain.md#hash_tree_root
## * https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/phase0/beacon-chain.md#hash_tree_root
discard cfg # Future-proof against new fields, see `get_lc_execution_root`.
let root = Eth2Digest.new()
root[] = header[].beacon.hash_tree_root()
@ -855,7 +855,7 @@ func ETHLightClientHeaderGetBeacon(
## * Beacon block header.
##
## See:
## * https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/phase0/beacon-chain.md#beaconblockheader
## * https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/phase0/beacon-chain.md#beaconblockheader
addr header[].beacon
func ETHBeaconBlockHeaderGetSlot(

View File

@ -353,7 +353,7 @@ proc installMessageValidators*(
digest = forkDigests[].atConsensusFork(contextFork)
# light_client_optimistic_update
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/altair/light-client/p2p-interface.md#light_client_finality_update
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/altair/light-client/p2p-interface.md#light_client_finality_update
lightClient.network.addValidator(
getLightClientFinalityUpdateTopic(digest), proc (
msg: lcDataFork.LightClientFinalityUpdate
@ -361,7 +361,7 @@ proc installMessageValidators*(
validate(msg, contextFork, processLightClientFinalityUpdate))
# light_client_optimistic_update
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/altair/light-client/p2p-interface.md#light_client_optimistic_update
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/altair/light-client/p2p-interface.md#light_client_optimistic_update
lightClient.network.addValidator(
getLightClientOptimisticUpdateTopic(digest), proc (
msg: lcDataFork.LightClientOptimisticUpdate

View File

@ -2255,8 +2255,8 @@ proc getPersistentNetKeys*(
func gossipId(
data: openArray[byte], phase0Prefix, topic: string): seq[byte] =
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/phase0/p2p-interface.md#topics-and-messages
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/altair/p2p-interface.md#topics-and-messages
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/phase0/p2p-interface.md#topics-and-messages
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/altair/p2p-interface.md#topics-and-messages
const
MESSAGE_DOMAIN_INVALID_SNAPPY = [0x00'u8, 0x00, 0x00, 0x00]
MESSAGE_DOMAIN_VALID_SNAPPY = [0x01'u8, 0x00, 0x00, 0x00]
@ -2554,7 +2554,7 @@ proc updateStabilitySubnetMetadata*(node: Eth2Node, attnets: AttnetBits) =
node.metadata.seq_number += 1
node.metadata.attnets = attnets
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/phase0/p2p-interface.md#attestation-subnet-subscription
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/phase0/p2p-interface.md#attestation-subnet-subscription
# https://github.com/ethereum/consensus-specs/blob/v1.3.0/specs/phase0/p2p-interface.md#attestation-subnet-bitfield
let res = node.discovery.updateRecord({
enrAttestationSubnetsField: SSZ.encode(node.metadata.attnets)
@ -2567,7 +2567,7 @@ proc updateStabilitySubnetMetadata*(node: Eth2Node, attnets: AttnetBits) =
debug "Stability subnets changed; updated ENR attnets", attnets
proc updateSyncnetsMetadata*(node: Eth2Node, syncnets: SyncnetBits) =
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/altair/validator.md#sync-committee-subnet-stability
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/altair/validator.md#sync-committee-subnet-stability
if node.metadata.syncnets == syncnets:
return

View File

@ -774,7 +774,7 @@ func forkDigests(node: BeaconNode): auto =
node.dag.forkDigests.deneb]
forkDigestsArray
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/phase0/p2p-interface.md#attestation-subnet-subscription
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/phase0/p2p-interface.md#attestation-subnet-subscription
proc updateAttestationSubnetHandlers(node: BeaconNode, slot: Slot) =
if node.gossipState.card == 0:
# When disconnected, updateGossipState is responsible for all things
@ -1488,7 +1488,7 @@ proc installMessageValidators(node: BeaconNode) =
when consensusFork >= ConsensusFork.Altair:
# sync_committee_{subnet_id}
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/altair/p2p-interface.md#sync_committee_subnet_id
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/altair/p2p-interface.md#sync_committee_subnet_id
for subcommitteeIdx in SyncSubcommitteeIndex:
closureScope: # Needed for inner `proc`; don't lift it out of loop.
let idx = subcommitteeIdx
@ -1511,7 +1511,7 @@ proc installMessageValidators(node: BeaconNode) =
MsgSource.gossip, msg)))
when consensusFork >= ConsensusFork.Capella:
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/capella/p2p-interface.md#bls_to_execution_change
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/capella/p2p-interface.md#bls_to_execution_change
node.network.addAsyncValidator(
getBlsToExecutionChangeTopic(digest), proc (
msg: SignedBLSToExecutionChange
@ -1522,7 +1522,7 @@ proc installMessageValidators(node: BeaconNode) =
when consensusFork >= ConsensusFork.Deneb:
# blob_sidecar_{index}
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/deneb/p2p-interface.md#blob_sidecar_subnet_id
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/deneb/p2p-interface.md#blob_sidecar_subnet_id
for i in 0 ..< BLOB_SIDECAR_SUBNET_COUNT:
closureScope: # Needed for inner `proc`; don't lift it out of loop.
let idx = i

View File

@ -88,7 +88,7 @@ proc installConfigApiHandlers*(router: var RestRouter, node: BeaconNode) =
MAX_VOLUNTARY_EXITS:
Base10.toString(MAX_VOLUNTARY_EXITS),
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/presets/mainnet/altair.yaml
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/presets/mainnet/altair.yaml
INACTIVITY_PENALTY_QUOTIENT_ALTAIR:
Base10.toString(INACTIVITY_PENALTY_QUOTIENT_ALTAIR),
MIN_SLASHING_PENALTY_QUOTIENT_ALTAIR:
@ -104,7 +104,7 @@ proc installConfigApiHandlers*(router: var RestRouter, node: BeaconNode) =
UPDATE_TIMEOUT:
Base10.toString(UPDATE_TIMEOUT),
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/presets/mainnet/bellatrix.yaml
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/presets/mainnet/bellatrix.yaml
INACTIVITY_PENALTY_QUOTIENT_BELLATRIX:
Base10.toString(INACTIVITY_PENALTY_QUOTIENT_BELLATRIX),
MIN_SLASHING_PENALTY_QUOTIENT_BELLATRIX:
@ -120,7 +120,7 @@ proc installConfigApiHandlers*(router: var RestRouter, node: BeaconNode) =
MAX_EXTRA_DATA_BYTES:
Base10.toString(uint64(MAX_EXTRA_DATA_BYTES)),
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/presets/mainnet/capella.yaml
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/presets/mainnet/capella.yaml
MAX_BLS_TO_EXECUTION_CHANGES:
Base10.toString(uint64(MAX_BLS_TO_EXECUTION_CHANGES)),
MAX_WITHDRAWALS_PER_PAYLOAD:
@ -259,7 +259,7 @@ proc installConfigApiHandlers*(router: var RestRouter, node: BeaconNode) =
ATTESTATION_SUBNET_COUNT:
Base10.toString(ATTESTATION_SUBNET_COUNT),
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/altair/validator.md#constants
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/altair/validator.md#constants
TARGET_AGGREGATORS_PER_SYNC_SUBCOMMITTEE:
Base10.toString(uint64(TARGET_AGGREGATORS_PER_SYNC_SUBCOMMITTEE)),
SYNC_COMMITTEE_SUBNET_COUNT:

View File

@ -43,7 +43,7 @@ const
GENESIS_SLOT* = Slot(0)
GENESIS_EPOCH* = Epoch(0) # compute_epoch_at_slot(GENESIS_SLOT)
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/phase0/fork-choice.md#constant
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/phase0/fork-choice.md#constant
INTERVALS_PER_SLOT* = 3
FAR_FUTURE_BEACON_TIME* = BeaconTime(ns_since_genesis: int64.high())
@ -134,22 +134,22 @@ template `+`*(a: TimeDiff, b: Duration): TimeDiff =
const
# Offsets from the start of the slot to when the corresponding message should
# be sent
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/phase0/validator.md#attesting
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/phase0/validator.md#attesting
attestationSlotOffset* = TimeDiff(nanoseconds:
NANOSECONDS_PER_SLOT.int64 div INTERVALS_PER_SLOT)
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/phase0/validator.md#broadcast-aggregate
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/phase0/validator.md#broadcast-aggregate
aggregateSlotOffset* = TimeDiff(nanoseconds:
NANOSECONDS_PER_SLOT.int64 * 2 div INTERVALS_PER_SLOT)
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/altair/validator.md#prepare-sync-committee-message
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/altair/validator.md#prepare-sync-committee-message
syncCommitteeMessageSlotOffset* = TimeDiff(nanoseconds:
NANOSECONDS_PER_SLOT.int64 div INTERVALS_PER_SLOT)
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/altair/validator.md#broadcast-sync-committee-contribution
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/altair/validator.md#broadcast-sync-committee-contribution
syncContributionSlotOffset* = TimeDiff(nanoseconds:
NANOSECONDS_PER_SLOT.int64 * 2 div INTERVALS_PER_SLOT)
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/altair/light-client/p2p-interface.md#sync-committee
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/altair/light-client/p2p-interface.md#sync-committee
lightClientFinalityUpdateSlotOffset* = TimeDiff(nanoseconds:
NANOSECONDS_PER_SLOT.int64 div INTERVALS_PER_SLOT)
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/altair/light-client/p2p-interface.md#sync-committee
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/altair/light-client/p2p-interface.md#sync-committee
lightClientOptimisticUpdateSlotOffset* = TimeDiff(nanoseconds:
NANOSECONDS_PER_SLOT.int64 div INTERVALS_PER_SLOT)
@ -189,7 +189,7 @@ func epoch*(slot: Slot): Epoch = # aka compute_epoch_at_slot
if slot == FAR_FUTURE_SLOT: FAR_FUTURE_EPOCH
else: Epoch(slot div SLOTS_PER_EPOCH)
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/phase0/fork-choice.md#compute_slots_since_epoch_start
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/phase0/fork-choice.md#compute_slots_since_epoch_start
func since_epoch_start*(slot: Slot): uint64 = # aka compute_slots_since_epoch_start
## How many slots since the beginning of the epoch (`[0..SLOTS_PER_EPOCH-1]`)
(slot mod SLOTS_PER_EPOCH)
@ -217,7 +217,7 @@ iterator slots*(epoch: Epoch): Slot =
for slot in start_slot ..< start_slot + SLOTS_PER_EPOCH:
yield slot
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/altair/validator.md#sync-committee
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/altair/validator.md#sync-committee
template sync_committee_period*(epoch: Epoch): SyncCommitteePeriod =
if epoch == FAR_FUTURE_EPOCH: FAR_FUTURE_PERIOD
else: SyncCommitteePeriod(epoch div EPOCHS_PER_SYNC_COMMITTEE_PERIOD)

View File

@ -22,7 +22,7 @@ from ./datatypes/capella import BeaconState, ExecutionPayloadHeader, Withdrawal
export extras, forks, validator, chronicles
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/phase0/beacon-chain.md#increase_balance
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/phase0/beacon-chain.md#increase_balance
func increase_balance*(balance: var Gwei, delta: Gwei) =
balance += delta
@ -32,7 +32,7 @@ func increase_balance*(
if delta != 0: # 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.0/specs/phase0/beacon-chain.md#decrease_balance
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/phase0/beacon-chain.md#decrease_balance
func decrease_balance*(balance: var Gwei, delta: Gwei) =
balance =
if delta > balance:
@ -72,7 +72,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.0/specs/phase0/beacon-chain.md#get_validator_churn_limit
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/phase0/beacon-chain.md#get_validator_churn_limit
func get_validator_churn_limit*(
cfg: RuntimeConfig, state: ForkyBeaconState, cache: var StateCache):
uint64 =
@ -134,8 +134,8 @@ func initiate_validator_exit*(
from ./datatypes/deneb import BeaconState
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-alpha.3/specs/phase0/beacon-chain.md#slash_validator
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/altair/beacon-chain.md#modified-slash_validator
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/bellatrix/beacon-chain.md#modified-slash_validator
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/altair/beacon-chain.md#modified-slash_validator
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/bellatrix/beacon-chain.md#modified-slash_validator
func get_slashing_penalty*(state: ForkyBeaconState,
validator_effective_balance: Gwei): Gwei =
# TODO Consider whether this is better than splitting the functions apart; in
@ -150,9 +150,9 @@ func get_slashing_penalty*(state: ForkyBeaconState,
else:
{.fatal: "invalid BeaconState type".}
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/phase0/beacon-chain.md#slash_validator
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/altair/beacon-chain.md#modified-slash_validator
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/bellatrix/beacon-chain.md#modified-slash_validator
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/phase0/beacon-chain.md#slash_validator
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/altair/beacon-chain.md#modified-slash_validator
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/bellatrix/beacon-chain.md#modified-slash_validator
func get_whistleblower_reward*(validator_effective_balance: Gwei): Gwei =
validator_effective_balance div WHISTLEBLOWER_REWARD_QUOTIENT
@ -168,9 +168,9 @@ func get_proposer_reward(state: ForkyBeaconState, whistleblower_reward: Gwei): G
else:
{.fatal: "invalid BeaconState type".}
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/phase0/beacon-chain.md#slash_validator
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/altair/beacon-chain.md#modified-slash_validator
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/bellatrix/beacon-chain.md#modified-slash_validator
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/phase0/beacon-chain.md#slash_validator
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/altair/beacon-chain.md#modified-slash_validator
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/bellatrix/beacon-chain.md#modified-slash_validator
proc slash_validator*(
cfg: RuntimeConfig, state: var ForkyBeaconState,
slashed_index: ValidatorIndex, cache: var StateCache):
@ -380,7 +380,7 @@ proc is_valid_indexed_attestation*(
ok()
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/phase0/beacon-chain.md#get_attesting_indices
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/phase0/beacon-chain.md#get_attesting_indices
iterator get_attesting_indices_iter*(state: ForkyBeaconState,
data: AttestationData,
bits: CommitteeValidatorsBits,
@ -401,7 +401,7 @@ iterator get_attesting_indices_iter*(state: ForkyBeaconState,
if bits[index_in_committee]:
yield validator_index
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/phase0/beacon-chain.md#get_attesting_indices
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/phase0/beacon-chain.md#get_attesting_indices
func get_attesting_indices*(state: ForkyBeaconState,
data: AttestationData,
bits: CommitteeValidatorsBits,
@ -471,8 +471,8 @@ func check_attestation_target_epoch(
ok(data.target.epoch)
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/phase0/beacon-chain.md#attestations
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/altair/beacon-chain.md#modified-process_attestation
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/phase0/beacon-chain.md#attestations
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/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
func check_attestation_inclusion(
consensusFork: static ConsensusFork, attestation_slot: Slot,
@ -502,7 +502,7 @@ func check_attestation_index*(
Result[CommitteeIndex, cstring] =
check_attestation_index(data.index, committees_per_slot)
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/altair/beacon-chain.md#get_attestation_participation_flag_indices
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/altair/beacon-chain.md#get_attestation_participation_flag_indices
func get_attestation_participation_flag_indices(
state: altair.BeaconState | bellatrix.BeaconState | capella.BeaconState,
data: AttestationData, inclusion_delay: uint64): set[TimelyFlag] =
@ -574,7 +574,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.0/specs/phase0/beacon-chain.md#get_total_active_balance
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/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
@ -590,7 +590,7 @@ func get_total_active_balance*(state: ForkyBeaconState, cache: var StateCache):
cache.total_active_balance[epoch] = tab
return tab
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/altair/beacon-chain.md#get_base_reward_per_increment
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/altair/beacon-chain.md#get_base_reward_per_increment
func get_base_reward_per_increment_sqrt*(
total_active_balance_sqrt: uint64): Gwei =
EFFECTIVE_BALANCE_INCREMENT * BASE_REWARD_FACTOR div total_active_balance_sqrt
@ -599,7 +599,7 @@ func get_base_reward_per_increment*(
total_active_balance: Gwei): Gwei =
get_base_reward_per_increment_sqrt(integer_squareroot(total_active_balance))
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/altair/beacon-chain.md#get_base_reward
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/altair/beacon-chain.md#get_base_reward
func get_base_reward(
state: altair.BeaconState | bellatrix.BeaconState | capella.BeaconState |
deneb.BeaconState,
@ -610,7 +610,7 @@ func get_base_reward(
state.validators[index].effective_balance div EFFECTIVE_BALANCE_INCREMENT
increments * base_reward_per_increment
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/phase0/beacon-chain.md#attestations
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/phase0/beacon-chain.md#attestations
proc check_attestation*(
state: ForkyBeaconState, attestation: SomeAttestation, flags: UpdateFlags,
cache: var StateCache): Result[void, cstring] =
@ -789,7 +789,7 @@ func get_next_sync_committee_keys(
i += 1'u64
res
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/capella/beacon-chain.md#has_eth1_withdrawal_credential
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/capella/beacon-chain.md#has_eth1_withdrawal_credential
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
@ -1108,7 +1108,7 @@ proc initialize_hashed_beacon_state_from_eth1*(
execution_payload_header, flags))
result.root = hash_tree_root(result.data)
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/altair/fork.md#upgrading-the-state
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/altair/fork.md#upgrading-the-state
func translate_participation(
state: var altair.BeaconState,
pending_attestations: openArray[phase0.PendingAttestation]) =
@ -1194,7 +1194,7 @@ func upgrade_to_altair*(cfg: RuntimeConfig, pre: phase0.BeaconState):
post
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/bellatrix/fork.md#upgrading-the-state
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/bellatrix/fork.md#upgrading-the-state
func upgrade_to_bellatrix*(cfg: RuntimeConfig, pre: altair.BeaconState):
ref bellatrix.BeaconState =
let epoch = get_current_epoch(pre)
@ -1251,7 +1251,7 @@ func upgrade_to_bellatrix*(cfg: RuntimeConfig, pre: altair.BeaconState):
latest_execution_payload_header: default(bellatrix.ExecutionPayloadHeader)
)
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/capella/fork.md#upgrading-the-state
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/capella/fork.md#upgrading-the-state
func upgrade_to_capella*(cfg: RuntimeConfig, pre: bellatrix.BeaconState):
ref capella.BeaconState =
let

View File

@ -206,7 +206,7 @@ func finish*(agg: AggregateSignature): CookedSig {.inline.} =
sig.finish(agg)
CookedSig(sig)
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/phase0/beacon-chain.md#bls-signatures
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/phase0/beacon-chain.md#bls-signatures
func blsVerify*(
pubkey: CookedPubKey, message: openArray[byte],
signature: CookedSig): bool =
@ -219,7 +219,7 @@ func blsVerify*(
## to enforce correct usage.
PublicKey(pubkey).verify(message, blscurve.Signature(signature))
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/phase0/beacon-chain.md#bls-signatures
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/phase0/beacon-chain.md#bls-signatures
proc blsVerify*(
pubkey: ValidatorPubKey, message: openArray[byte],
signature: CookedSig): bool =

View File

@ -51,16 +51,16 @@ const
PARTICIPATION_FLAG_WEIGHTS*: array[TimelyFlag, uint64] =
[uint64 TIMELY_SOURCE_WEIGHT, TIMELY_TARGET_WEIGHT, TIMELY_HEAD_WEIGHT]
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/altair/validator.md#misc
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/altair/validator.md#misc
TARGET_AGGREGATORS_PER_SYNC_SUBCOMMITTEE* = 16
SYNC_COMMITTEE_SUBNET_COUNT* = 4
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/altair/light-client/sync-protocol.md#constants
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/altair/light-client/sync-protocol.md#constants
# All of these indices are rooted in `BeaconState`.
# The first member (`genesis_time`) is 32, subsequent members +1 each.
# If there are ever more than 32 members in `BeaconState`, indices change!
# `FINALIZED_ROOT_INDEX` is one layer deeper, i.e., `52 * 2 + 1`.
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/ssz/merkle-proofs.md
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/ssz/merkle-proofs.md
FINALIZED_ROOT_INDEX* = 105.GeneralizedIndex # `finalized_checkpoint` > `root`
CURRENT_SYNC_COMMITTEE_INDEX* = 54.GeneralizedIndex # `current_sync_committee`
NEXT_SYNC_COMMITTEE_INDEX* = 55.GeneralizedIndex # `next_sync_committee`
@ -88,7 +88,7 @@ type
## effectively making the cost of clearing the cache higher than the typical
## gains
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/altair/beacon-chain.md#syncaggregate
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/altair/beacon-chain.md#syncaggregate
SyncAggregate* = object
sync_committee_bits*: BitArray[SYNC_COMMITTEE_SIZE]
sync_committee_signature*: ValidatorSig
@ -102,7 +102,7 @@ type
pubkeys*: HashArray[Limit SYNC_COMMITTEE_SIZE, ValidatorPubKey]
aggregate_pubkey*: ValidatorPubKey
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/altair/validator.md#synccommitteemessage
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/altair/validator.md#synccommitteemessage
SyncCommitteeMessage* = object
slot*: Slot
## Slot to which this contribution pertains
@ -116,7 +116,7 @@ type
signature*: ValidatorSig
## Signature by the validator over the block root of `slot`
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/altair/validator.md#synccommitteecontribution
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/altair/validator.md#synccommitteecontribution
SyncCommitteeAggregationBits* =
BitArray[SYNC_SUBCOMMITTEE_SIZE]
@ -138,18 +138,18 @@ type
signature*: ValidatorSig
## Signature by the validator(s) over the block root of `slot`
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/altair/validator.md#contributionandproof
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/altair/validator.md#contributionandproof
ContributionAndProof* = object
aggregator_index*: uint64 # `ValidatorIndex` after validation
contribution*: SyncCommitteeContribution
selection_proof*: ValidatorSig
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/altair/validator.md#signedcontributionandproof
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/altair/validator.md#signedcontributionandproof
SignedContributionAndProof* = object
message*: ContributionAndProof
signature*: ValidatorSig
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/altair/validator.md#syncaggregatorselectiondata
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/altair/validator.md#syncaggregatorselectiondata
SyncAggregatorSelectionData* = object
slot*: Slot
subcommittee_index*: uint64 # `SyncSubcommitteeIndex` after validation
@ -165,12 +165,12 @@ type
NextSyncCommitteeBranch* =
array[log2trunc(NEXT_SYNC_COMMITTEE_INDEX), Eth2Digest]
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/altair/light-client/sync-protocol.md#lightclientheader
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/altair/light-client/sync-protocol.md#lightclientheader
LightClientHeader* = object
beacon*: BeaconBlockHeader
## Beacon block header
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/altair/light-client/sync-protocol.md#lightclientbootstrap
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/altair/light-client/sync-protocol.md#lightclientbootstrap
LightClientBootstrap* = object
header*: LightClientHeader
## Header matching the requested beacon block root
@ -179,7 +179,7 @@ type
## Current sync committee corresponding to `header.beacon.state_root`
current_sync_committee_branch*: CurrentSyncCommitteeBranch
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/altair/light-client/sync-protocol.md#lightclientupdate
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/altair/light-client/sync-protocol.md#lightclientupdate
LightClientUpdate* = object
attested_header*: LightClientHeader
## Header attested to by the sync committee
@ -198,7 +198,7 @@ type
signature_slot*: Slot
## Slot at which the aggregate signature was created (untrusted)
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/altair/light-client/sync-protocol.md#lightclientfinalityupdate
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/altair/light-client/sync-protocol.md#lightclientfinalityupdate
LightClientFinalityUpdate* = object
# Header attested to by the sync committee
attested_header*: LightClientHeader
@ -212,7 +212,7 @@ type
# Slot at which the aggregate signature was created (untrusted)
signature_slot*: Slot
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/altair/light-client/sync-protocol.md#lightclientoptimisticupdate
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/altair/light-client/sync-protocol.md#lightclientoptimisticupdate
LightClientOptimisticUpdate* = object
# Header attested to by the sync committee
attested_header*: LightClientHeader
@ -238,7 +238,7 @@ type
LightClientBootstrap |
SomeLightClientUpdate
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/altair/light-client/sync-protocol.md#lightclientstore
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/altair/light-client/sync-protocol.md#lightclientstore
LightClientStore* = object
finalized_header*: LightClientHeader
## Header that is finalized
@ -347,7 +347,7 @@ type
data*: BeaconState
root*: Eth2Digest # hash_tree_root(data)
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/phase0/beacon-chain.md#beaconblock
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/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
@ -404,7 +404,7 @@ type
state_root*: Eth2Digest
body*: TrustedBeaconBlockBody
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/altair/beacon-chain.md#beaconblockbody
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/altair/beacon-chain.md#beaconblockbody
BeaconBlockBody* = object
randao_reveal*: ValidatorSig
eth1_data*: Eth1Data
@ -456,7 +456,7 @@ type
SyncnetBits* = BitArray[SYNC_COMMITTEE_SUBNET_COUNT]
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/altair/p2p-interface.md#metadata
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/altair/p2p-interface.md#metadata
MetaData* = object
seq_number*: uint64
attnets*: AttnetBits
@ -659,7 +659,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.0/specs/altair/light-client/sync-protocol.md#is_valid_light_client_header
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/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

@ -82,7 +82,7 @@ const
ZERO_HASH* = Eth2Digest()
MAX_GRAFFITI_SIZE* = 32
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/phase0/p2p-interface.md#configuration
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/phase0/p2p-interface.md#configuration
MAXIMUM_GOSSIP_CLOCK_DISPARITY* = 500.millis
SLOTS_PER_ETH1_VOTING_PERIOD* =
@ -191,7 +191,7 @@ type
# SSZ / hashing purposes
JustificationBits* = distinct uint8
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/phase0/beacon-chain.md#proposerslashing
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/phase0/beacon-chain.md#proposerslashing
ProposerSlashing* = object
signed_header_1*: SignedBeaconBlockHeader
signed_header_2*: SignedBeaconBlockHeader
@ -203,7 +203,7 @@ type
signed_header_1*: TrustedSignedBeaconBlockHeader
signed_header_2*: TrustedSignedBeaconBlockHeader
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/phase0/beacon-chain.md#attesterslashing
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/phase0/beacon-chain.md#attesterslashing
AttesterSlashing* = object
attestation_1*: IndexedAttestation
attestation_2*: IndexedAttestation
@ -215,7 +215,7 @@ type
attestation_1*: TrustedIndexedAttestation
attestation_2*: TrustedIndexedAttestation
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/phase0/beacon-chain.md#indexedattestation
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/phase0/beacon-chain.md#indexedattestation
IndexedAttestation* = object
attesting_indices*: List[uint64, Limit MAX_VALIDATORS_PER_COMMITTEE]
data*: AttestationData
@ -231,7 +231,7 @@ type
CommitteeValidatorsBits* = BitList[Limit MAX_VALIDATORS_PER_COMMITTEE]
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/phase0/beacon-chain.md#attestation
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/phase0/beacon-chain.md#attestation
Attestation* = object
aggregation_bits*: CommitteeValidatorsBits
data*: AttestationData
@ -270,7 +270,7 @@ type
source*: Checkpoint
target*: Checkpoint
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/phase0/beacon-chain.md#deposit
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/phase0/beacon-chain.md#deposit
Deposit* = object
proof*: array[DEPOSIT_CONTRACT_TREE_DEPTH + 1, Eth2Digest]
## Merkle path to deposit root
@ -321,7 +321,7 @@ type
pubkey*: CookedPubKey
withdrawal_credentials*: Eth2Digest
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/phase0/beacon-chain.md#validator
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/phase0/beacon-chain.md#validator
Validator* = object
pubkey*: ValidatorPubKey
@ -352,7 +352,7 @@ type
proposer_index*: uint64 # `ValidatorIndex` after validation
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/phase0/beacon-chain.md#historicalbatch
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/phase0/beacon-chain.md#historicalbatch
HistoricalBatch* = object
block_roots* : array[SLOTS_PER_HISTORICAL_ROOT, Eth2Digest]
state_roots* : array[SLOTS_PER_HISTORICAL_ROOT, Eth2Digest]
@ -371,7 +371,7 @@ type
deposit_count*: uint64
block_hash*: Eth2Digest
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/phase0/beacon-chain.md#signedvoluntaryexit
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/phase0/beacon-chain.md#signedvoluntaryexit
SignedVoluntaryExit* = object
message*: VoluntaryExit
signature*: ValidatorSig
@ -388,7 +388,7 @@ type
state_root*: Eth2Digest
body_root*: Eth2Digest
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/phase0/beacon-chain.md#signingdata
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/phase0/beacon-chain.md#signingdata
SigningData* = object
object_root*: Eth2Digest
domain*: Eth2Domain
@ -404,13 +404,13 @@ type
message*: BeaconBlockHeader
signature*: TrustedSig
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/phase0/validator.md#aggregateandproof
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/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-beta.0/specs/phase0/validator.md#signedaggregateandproof
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/phase0/validator.md#signedaggregateandproof
SignedAggregateAndProof* = object
message*: AggregateAndProof
signature*: ValidatorSig
@ -428,7 +428,7 @@ type
sync_committees*: Table[SyncCommitteePeriod, SyncCommitteeCache]
# This matches the mutable state of the Solidity deposit contract
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/solidity_deposit_contract/deposit_contract.sol
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/solidity_deposit_contract/deposit_contract.sol
DepositContractState* = object
branch*: array[DEPOSIT_CONTRACT_TREE_DEPTH, Eth2Digest]
deposit_count*: array[32, byte] # Uint256
@ -459,7 +459,7 @@ type
withdrawable_epoch*: Epoch
## When validator can withdraw funds
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/phase0/beacon-chain.md#validator
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/phase0/beacon-chain.md#validator
ValidatorStatusCapella* = object
# This is a validator without the expensive, immutable, append-only parts
# serialized. They're represented in memory to allow in-place SSZ reading

View File

@ -34,7 +34,7 @@ const
NEWPAYLOAD_TIMEOUT* = 8.seconds
type
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/bellatrix/beacon-chain.md#custom-types
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/bellatrix/beacon-chain.md#custom-types
Transaction* = List[byte, Limit MAX_BYTES_PER_TRANSACTION]
ExecutionAddress* = object
@ -45,7 +45,7 @@ type
PayloadID* = array[8, byte]
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/bellatrix/beacon-chain.md#executionpayload
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/bellatrix/beacon-chain.md#executionpayload
ExecutionPayload* = object
# Execution block header fields
parent_hash*: Eth2Digest
@ -73,7 +73,7 @@ type
executionPayload*: ExecutionPayload
blockValue*: Wei
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/bellatrix/beacon-chain.md#executionpayloadheader
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/bellatrix/beacon-chain.md#executionpayloadheader
ExecutionPayloadHeader* = object
# Execution block header fields
parent_hash*: Eth2Digest
@ -97,13 +97,13 @@ type
ExecutePayload* = proc(
execution_payload: ExecutionPayload): bool {.gcsafe, raises: [Defect].}
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/bellatrix/fork-choice.md#powblock
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/bellatrix/fork-choice.md#powblock
PowBlock* = object
block_hash*: Eth2Digest
parent_hash*: Eth2Digest
total_difficulty*: Eth2Digest # uint256
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/bellatrix/beacon-chain.md#beaconstate
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/bellatrix/beacon-chain.md#beaconstate
BeaconState* = object
# Versioning
genesis_time*: uint64
@ -228,7 +228,7 @@ type
state_root*: Eth2Digest
body*: TrustedBeaconBlockBody
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/bellatrix/beacon-chain.md#beaconblockbody
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/bellatrix/beacon-chain.md#beaconblockbody
BeaconBlockBody* = object
randao_reveal*: ValidatorSig
eth1_data*: Eth1Data
@ -306,7 +306,7 @@ type
# Execution
execution_payload*: ExecutionPayload # [New in Bellatrix]
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/phase0/beacon-chain.md#signedbeaconblock
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/phase0/beacon-chain.md#signedbeaconblock
SignedBeaconBlock* = object
message*: BeaconBlock
signature*: ValidatorSig

View File

@ -27,43 +27,43 @@ import
export json_serialization, base
const
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/capella/light-client/sync-protocol.md#constants
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/capella/light-client/sync-protocol.md#constants
# This index is rooted in `BeaconBlockBody`.
# The first member (`randao_reveal`) is 16, subsequent members +1 each.
# If there are ever more than 16 members in `BeaconBlockBody`, indices change!
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/ssz/merkle-proofs.md
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/ssz/merkle-proofs.md
EXECUTION_PAYLOAD_INDEX* = 25.GeneralizedIndex # `execution_payload`
type
SignedBLSToExecutionChangeList* =
List[SignedBLSToExecutionChange, Limit MAX_BLS_TO_EXECUTION_CHANGES]
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/capella/beacon-chain.md#withdrawal
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/capella/beacon-chain.md#withdrawal
Withdrawal* = object
index*: WithdrawalIndex
validator_index*: uint64
address*: ExecutionAddress
amount*: Gwei
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/capella/beacon-chain.md#blstoexecutionchange
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/capella/beacon-chain.md#blstoexecutionchange
BLSToExecutionChange* = object
validator_index*: uint64
from_bls_pubkey*: ValidatorPubKey
to_execution_address*: ExecutionAddress
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/capella/beacon-chain.md#signedblstoexecutionchange
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/capella/beacon-chain.md#signedblstoexecutionchange
SignedBLSToExecutionChange* = object
message*: BLSToExecutionChange
signature*: ValidatorSig
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/capella/beacon-chain.md#historicalsummary
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/capella/beacon-chain.md#historicalsummary
HistoricalSummary* = object
# `HistoricalSummary` matches the components of the phase0
# `HistoricalBatch` making the two hash_tree_root-compatible.
block_summary_root*: Eth2Digest
state_summary_root*: Eth2Digest
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/capella/beacon-chain.md#executionpayload
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/capella/beacon-chain.md#executionpayload
ExecutionPayload* = object
# Execution block header fields
parent_hash*: Eth2Digest
@ -93,7 +93,7 @@ type
executionPayload*: ExecutionPayload
blockValue*: Wei
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/capella/beacon-chain.md#executionpayloadheader
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/capella/beacon-chain.md#executionpayloadheader
ExecutionPayloadHeader* = object
# Execution block header fields
parent_hash*: Eth2Digest
@ -122,7 +122,7 @@ type
ExecutionBranch* =
array[log2trunc(EXECUTION_PAYLOAD_INDEX), Eth2Digest]
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/capella/light-client/sync-protocol.md#modified-lightclientheader
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/capella/light-client/sync-protocol.md#modified-lightclientheader
LightClientHeader* = object
beacon*: BeaconBlockHeader
## Beacon block header
@ -131,7 +131,7 @@ type
## Execution payload header corresponding to `beacon.body_root` (from Capella onward)
execution_branch*: ExecutionBranch
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/altair/light-client/sync-protocol.md#lightclientbootstrap
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/altair/light-client/sync-protocol.md#lightclientbootstrap
LightClientBootstrap* = object
header*: LightClientHeader
## Header matching the requested beacon block root
@ -140,7 +140,7 @@ type
## Current sync committee corresponding to `header.beacon.state_root`
current_sync_committee_branch*: altair.CurrentSyncCommitteeBranch
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/altair/light-client/sync-protocol.md#lightclientupdate
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/altair/light-client/sync-protocol.md#lightclientupdate
LightClientUpdate* = object
attested_header*: LightClientHeader
## Header attested to by the sync committee
@ -159,7 +159,7 @@ type
signature_slot*: Slot
## Slot at which the aggregate signature was created (untrusted)
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/altair/light-client/sync-protocol.md#lightclientfinalityupdate
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/altair/light-client/sync-protocol.md#lightclientfinalityupdate
LightClientFinalityUpdate* = object
# Header attested to by the sync committee
attested_header*: LightClientHeader
@ -173,7 +173,7 @@ type
# Slot at which the aggregate signature was created (untrusted)
signature_slot*: Slot
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/altair/light-client/sync-protocol.md#lightclientoptimisticupdate
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/altair/light-client/sync-protocol.md#lightclientoptimisticupdate
LightClientOptimisticUpdate* = object
# Header attested to by the sync committee
attested_header*: LightClientHeader
@ -199,7 +199,7 @@ type
LightClientBootstrap |
SomeLightClientUpdate
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/altair/light-client/sync-protocol.md#lightclientstore
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/altair/light-client/sync-protocol.md#lightclientstore
LightClientStore* = object
finalized_header*: LightClientHeader
## Header that is finalized
@ -220,7 +220,7 @@ type
## (used to compute safety threshold)
current_max_active_participants*: uint64
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/capella/beacon-chain.md#beaconstate
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/capella/beacon-chain.md#beaconstate
BeaconState* = object
# Versioning
genesis_time*: uint64
@ -650,7 +650,7 @@ func shortLog*(v: SignedBLSToExecutionChange): auto =
signature: shortLog(v.signature)
)
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/capella/light-client/sync-protocol.md#get_lc_execution_root
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/capella/light-client/sync-protocol.md#get_lc_execution_root
func get_lc_execution_root*(
header: LightClientHeader, cfg: RuntimeConfig): Eth2Digest =
let epoch = header.beacon.slot.epoch
@ -660,7 +660,7 @@ func get_lc_execution_root*(
ZERO_HASH
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/capella/light-client/sync-protocol.md#modified-is_valid_light_client_header
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/capella/light-client/sync-protocol.md#modified-is_valid_light_client_header
func is_valid_light_client_header*(
header: LightClientHeader, cfg: RuntimeConfig): bool =
let epoch = header.beacon.slot.epoch
@ -677,13 +677,13 @@ func is_valid_light_client_header*(
get_subtree_index(EXECUTION_PAYLOAD_INDEX),
header.beacon.body_root)
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/capella/light-client/fork.md#upgrading-light-client-data
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/capella/light-client/fork.md#upgrading-light-client-data
func upgrade_lc_header_to_capella*(
pre: altair.LightClientHeader): LightClientHeader =
LightClientHeader(
beacon: pre.beacon)
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/capella/light-client/fork.md#upgrading-light-client-data
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/capella/light-client/fork.md#upgrading-light-client-data
func upgrade_lc_bootstrap_to_capella*(
pre: altair.LightClientBootstrap): LightClientBootstrap =
LightClientBootstrap(
@ -691,7 +691,7 @@ func upgrade_lc_bootstrap_to_capella*(
current_sync_committee: pre.current_sync_committee,
current_sync_committee_branch: pre.current_sync_committee_branch)
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/capella/light-client/fork.md#upgrading-light-client-data
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/capella/light-client/fork.md#upgrading-light-client-data
func upgrade_lc_update_to_capella*(
pre: altair.LightClientUpdate): LightClientUpdate =
LightClientUpdate(
@ -703,7 +703,7 @@ func upgrade_lc_update_to_capella*(
sync_aggregate: pre.sync_aggregate,
signature_slot: pre.signature_slot)
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/capella/light-client/fork.md#upgrading-light-client-data
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/capella/light-client/fork.md#upgrading-light-client-data
func upgrade_lc_finality_update_to_capella*(
pre: altair.LightClientFinalityUpdate): LightClientFinalityUpdate =
LightClientFinalityUpdate(
@ -713,7 +713,7 @@ func upgrade_lc_finality_update_to_capella*(
sync_aggregate: pre.sync_aggregate,
signature_slot: pre.signature_slot)
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/capella/light-client/fork.md#upgrading-light-client-data
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/capella/light-client/fork.md#upgrading-light-client-data
func upgrade_lc_optimistic_update_to_capella*(
pre: altair.LightClientOptimisticUpdate): LightClientOptimisticUpdate =
LightClientOptimisticUpdate(
@ -764,7 +764,7 @@ chronicles.formatIt LightClientUpdate: shortLog(it)
chronicles.formatIt LightClientFinalityUpdate: shortLog(it)
chronicles.formatIt LightClientOptimisticUpdate: shortLog(it)
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/capella/light-client/fork.md#upgrading-the-store
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/capella/light-client/fork.md#upgrading-the-store
func upgrade_lc_store_to_capella*(
pre: altair.LightClientStore): LightClientStore =
let best_valid_update =

View File

@ -16,10 +16,10 @@ type
DomainType* = distinct array[4, byte]
const
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/phase0/p2p-interface.md#constants
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/phase0/p2p-interface.md#constants
NODE_ID_BITS* = 256
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/phase0/p2p-interface.md#configuration
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/phase0/p2p-interface.md#configuration
EPOCHS_PER_SUBNET_SUBSCRIPTION* = 256
SUBNETS_PER_NODE* = 2'u64
ATTESTATION_SUBNET_COUNT*: uint64 = 64
@ -51,7 +51,7 @@ 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.0/specs/capella/beacon-chain.md#domain-types
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/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/v1.4.0-beta.0/specs/deneb/beacon-chain.md#domain-types
@ -60,8 +60,8 @@ const
# https://github.com/ethereum/consensus-specs/blob/v1.3.0/specs/bellatrix/beacon-chain.md#transition-settings
TERMINAL_BLOCK_HASH_ACTIVATION_EPOCH* = FAR_FUTURE_EPOCH
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/phase0/fork-choice.md#configuration
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/phase0/fork-choice.md#configuration
PROPOSER_SCORE_BOOST*: uint64 = 40
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/deneb/p2p-interface.md#configuration
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/deneb/p2p-interface.md#configuration
BLOB_SIDECAR_SUBNET_COUNT*: uint64 = 6

View File

@ -30,7 +30,7 @@ from kzg4844 import KzgCommitment, KzgProof
export json_serialization, base, kzg4844
const
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/deneb/polynomial-commitments.md#constants
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/deneb/polynomial-commitments.md#constants
BYTES_PER_FIELD_ELEMENT = 32
# https://github.com/ethereum/consensus-specs/blob/v1.3.0/specs/deneb/beacon-chain.md#blob
@ -139,7 +139,7 @@ type
ExecutePayload* = proc(
execution_payload: ExecutionPayload): bool {.gcsafe, raises: [Defect].}
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/capella/light-client/sync-protocol.md#modified-lightclientheader
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/capella/light-client/sync-protocol.md#modified-lightclientheader
LightClientHeader* = object
beacon*: BeaconBlockHeader
## Beacon block header
@ -148,7 +148,7 @@ type
## Execution payload header corresponding to `beacon.body_root` (from Capella onward)
execution_branch*: capella.ExecutionBranch
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/altair/light-client/sync-protocol.md#lightclientbootstrap
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/altair/light-client/sync-protocol.md#lightclientbootstrap
LightClientBootstrap* = object
header*: LightClientHeader
## Header matching the requested beacon block root
@ -157,7 +157,7 @@ type
## Current sync committee corresponding to `header.beacon.state_root`
current_sync_committee_branch*: altair.CurrentSyncCommitteeBranch
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/altair/light-client/sync-protocol.md#lightclientupdate
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/altair/light-client/sync-protocol.md#lightclientupdate
LightClientUpdate* = object
attested_header*: LightClientHeader
## Header attested to by the sync committee
@ -176,7 +176,7 @@ type
signature_slot*: Slot
## Slot at which the aggregate signature was created (untrusted)
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/altair/light-client/sync-protocol.md#lightclientfinalityupdate
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/altair/light-client/sync-protocol.md#lightclientfinalityupdate
LightClientFinalityUpdate* = object
# Header attested to by the sync committee
attested_header*: LightClientHeader
@ -190,7 +190,7 @@ type
# Slot at which the aggregate signature was created (untrusted)
signature_slot*: Slot
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/altair/light-client/sync-protocol.md#lightclientoptimisticupdate
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/altair/light-client/sync-protocol.md#lightclientoptimisticupdate
LightClientOptimisticUpdate* = object
# Header attested to by the sync committee
attested_header*: LightClientHeader
@ -216,7 +216,7 @@ type
LightClientBootstrap |
SomeLightClientUpdate
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/altair/light-client/sync-protocol.md#lightclientstore
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/altair/light-client/sync-protocol.md#lightclientstore
LightClientStore* = object
finalized_header*: LightClientHeader
## Header that is finalized
@ -315,7 +315,7 @@ type
data*: BeaconState
root*: Eth2Digest # hash_tree_root(data)
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/phase0/beacon-chain.md#beaconblock
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/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

View File

@ -111,7 +111,7 @@ type
data*: BeaconState
root*: Eth2Digest # hash_tree_root(data)
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/phase0/beacon-chain.md#beaconblock
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/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
@ -167,7 +167,7 @@ type
state_root*: Eth2Digest
body*: TrustedBeaconBlockBody
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/phase0/beacon-chain.md#beaconblockbody
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/phase0/beacon-chain.md#beaconblockbody
BeaconBlockBody* = object
randao_reveal*: ValidatorSig
eth1_data*: Eth1Data
@ -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.0/specs/phase0/beacon-chain.md#signedbeaconblock
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/phase0/beacon-chain.md#signedbeaconblock
SignedBeaconBlock* = object
message*: BeaconBlock
signature*: ValidatorSig

View File

@ -377,7 +377,7 @@ type
MAX_DEPOSITS*: uint64
MAX_VOLUNTARY_EXITS*: uint64
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/presets/mainnet/altair.yaml
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/presets/mainnet/altair.yaml
INACTIVITY_PENALTY_QUOTIENT_ALTAIR*: uint64
MIN_SLASHING_PENALTY_QUOTIENT_ALTAIR*: uint64
PROPORTIONAL_SLASHING_MULTIPLIER_ALTAIR*: uint64
@ -386,7 +386,7 @@ type
MIN_SYNC_COMMITTEE_PARTICIPANTS*: uint64
UPDATE_TIMEOUT*: uint64
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/presets/mainnet/bellatrix.yaml
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/presets/mainnet/bellatrix.yaml
INACTIVITY_PENALTY_QUOTIENT_BELLATRIX*: uint64
MIN_SLASHING_PENALTY_QUOTIENT_BELLATRIX*: uint64
PROPORTIONAL_SLASHING_MULTIPLIER_BELLATRIX*: uint64
@ -395,7 +395,7 @@ type
BYTES_PER_LOGS_BLOOM*: uint64
MAX_EXTRA_DATA_BYTES*: uint64
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/presets/mainnet/capella.yaml
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/presets/mainnet/capella.yaml
MAX_BLS_TO_EXECUTION_CHANGES*: uint64
MAX_WITHDRAWALS_PER_PAYLOAD*: uint64
MAX_VALIDATORS_PER_WITHDRAWALS_SWEEP*: uint64
@ -475,7 +475,7 @@ type
EPOCHS_PER_RANDOM_SUBNET_SUBSCRIPTION*: uint64
ATTESTATION_SUBNET_COUNT*: uint64
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/altair/validator.md#constants
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/altair/validator.md#constants
TARGET_AGGREGATORS_PER_SYNC_SUBCOMMITTEE*: uint64
SYNC_COMMITTEE_SUBNET_COUNT*: uint64

View File

@ -816,7 +816,7 @@ func migratingToDataFork*[
upgradedObject.migrateToDataFork(newKind)
upgradedObject
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/altair/light-client/full-node.md#block_to_light_client_header
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/altair/light-client/full-node.md#block_to_light_client_header
func toAltairLightClientHeader(
blck: # `SomeSignedBeaconBlock` doesn't work here (Nim 1.6)
phase0.SignedBeaconBlock | phase0.TrustedSignedBeaconBlock |
@ -826,7 +826,7 @@ func toAltairLightClientHeader(
altair.LightClientHeader(
beacon: blck.message.toBeaconBlockHeader())
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/capella/light-client/full-node.md#modified-block_to_light_client_header
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/capella/light-client/full-node.md#modified-block_to_light_client_header
func toCapellaLightClientHeader(
blck: # `SomeSignedBeaconBlock` doesn't work here (Nim 1.6)
phase0.SignedBeaconBlock | phase0.TrustedSignedBeaconBlock |

View File

@ -27,7 +27,7 @@ export
eth2_merkleization, forks, rlp, ssz_codec
func toEther*(gwei: Gwei): Ether =
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/phase0/weak-subjectivity.md#constants
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/phase0/weak-subjectivity.md#constants
const ETH_TO_GWEI = 1_000_000_000
(gwei div ETH_TO_GWEI).Ether
@ -48,7 +48,7 @@ func shortLog*(v: FinalityCheckpoints): auto =
chronicles.formatIt FinalityCheckpoints: it.shortLog
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/phase0/beacon-chain.md#integer_squareroot
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/phase0/beacon-chain.md#integer_squareroot
func integer_squareroot*(n: SomeInteger): SomeInteger =
## Return the largest integer ``x`` such that ``x**2 <= n``.
doAssert n >= 0'u64
@ -115,7 +115,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.0/specs/phase0/beacon-chain.md#get_randao_mix
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/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]
@ -137,7 +137,7 @@ func uint_to_bytes*(x: uint32): array[4, byte] = toBytesLE(x)
func uint_to_bytes*(x: uint16): array[2, byte] = toBytesLE(x)
func uint_to_bytes*(x: uint8): array[1, byte] = toBytesLE(x)
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/phase0/beacon-chain.md#compute_domain
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/phase0/beacon-chain.md#compute_domain
func compute_domain*(
domain_type: DomainType,
fork_version: Version,
@ -151,7 +151,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.0/specs/phase0/beacon-chain.md#get_domain
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/phase0/beacon-chain.md#get_domain
func get_domain*(
fork: Fork,
domain_type: DomainType,
@ -210,7 +210,7 @@ func has_flag*(flags: ParticipationFlags, flag_index: TimelyFlag): bool =
let flag = ParticipationFlags(1'u8 shl ord(flag_index))
(flags and flag) == flag
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/altair/light-client/sync-protocol.md#is_sync_committee_update
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/altair/light-client/sync-protocol.md#is_sync_committee_update
template is_sync_committee_update*(update: SomeForkyLightClientUpdate): bool =
when update is SomeForkyLightClientUpdateWithSyncCommittee:
update.next_sync_committee_branch !=
@ -218,25 +218,25 @@ template is_sync_committee_update*(update: SomeForkyLightClientUpdate): bool =
else:
false
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/altair/light-client/sync-protocol.md#is_finality_update
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/altair/light-client/sync-protocol.md#is_finality_update
template is_finality_update*(update: SomeForkyLightClientUpdate): bool =
when update is SomeForkyLightClientUpdateWithFinality:
update.finality_branch != default(typeof(update.finality_branch))
else:
false
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/altair/light-client/sync-protocol.md#is_next_sync_committee_known
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/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 != default(typeof(store.next_sync_committee))
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/altair/light-client/sync-protocol.md#get_safety_threshold
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/altair/light-client/sync-protocol.md#get_safety_threshold
func get_safety_threshold*(store: ForkyLightClientStore): uint64 =
max(
store.previous_max_active_participants,
store.current_max_active_participants
) div 2
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/altair/light-client/sync-protocol.md#is_better_update
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/altair/light-client/sync-protocol.md#is_better_update
type LightClientUpdateMetadata* = object
attested_slot*, finalized_slot*, signature_slot*: Slot
has_sync_committee*, has_finality*: bool
@ -327,24 +327,24 @@ template is_better_update*[
new_update: A, old_update: B): bool =
is_better_data(toMeta(new_update), toMeta(old_update))
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/altair/light-client/p2p-interface.md#getlightclientbootstrap
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/altair/light-client/p2p-interface.md#getlightclientbootstrap
func contextEpoch*(bootstrap: ForkyLightClientBootstrap): Epoch =
bootstrap.header.beacon.slot.epoch
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/altair/light-client/p2p-interface.md#lightclientupdatesbyrange
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/altair/light-client/p2p-interface.md#getlightclientfinalityupdate
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/altair/light-client/p2p-interface.md#getlightclientoptimisticupdate
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/altair/light-client/p2p-interface.md#lightclientupdatesbyrange
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/altair/light-client/p2p-interface.md#getlightclientfinalityupdate
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/altair/light-client/p2p-interface.md#getlightclientoptimisticupdate
func contextEpoch*(update: SomeForkyLightClientUpdate): Epoch =
update.attested_header.beacon.slot.epoch
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/bellatrix/beacon-chain.md#is_merge_transition_complete
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/bellatrix/beacon-chain.md#is_merge_transition_complete
func is_merge_transition_complete*(
state: bellatrix.BeaconState | capella.BeaconState | deneb.BeaconState): bool =
const defaultExecutionPayloadHeader =
default(typeof(state.latest_execution_payload_header))
state.latest_execution_payload_header != defaultExecutionPayloadHeader
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/sync/optimistic.md#helpers
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/sync/optimistic.md#helpers
func is_execution_block*(blck: SomeForkyBeaconBlock): bool =
when typeof(blck).toFork >= ConsensusFork.Bellatrix:
const defaultExecutionPayload =
@ -353,7 +353,7 @@ func is_execution_block*(blck: SomeForkyBeaconBlock): bool =
else:
false
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/bellatrix/beacon-chain.md#is_merge_transition_block
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/bellatrix/beacon-chain.md#is_merge_transition_block
func is_merge_transition_block(
state: bellatrix.BeaconState | capella.BeaconState | deneb.BeaconState,
body: bellatrix.BeaconBlockBody | bellatrix.TrustedBeaconBlockBody |

View File

@ -1366,13 +1366,13 @@ proc createWallet*(kdfKind: KdfKind,
crypto: crypto,
nextAccount: nextAccount.get(0))
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/phase0/validator.md#bls_withdrawal_prefix
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/phase0/validator.md#bls_withdrawal_prefix
func makeWithdrawalCredentials*(k: ValidatorPubKey): Eth2Digest =
var bytes = eth2digest(k.toRaw())
bytes.data[0] = BLS_WITHDRAWAL_PREFIX.uint8
bytes
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/phase0/deposit-contract.md#withdrawal-credentials
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/phase0/deposit-contract.md#withdrawal-credentials
proc makeWithdrawalCredentials*(k: CookedPubKey): Eth2Digest =
makeWithdrawalCredentials(k.toPubKey())

View File

@ -15,7 +15,7 @@ import
from ../consensus_object_pools/block_pools_types import VerifierError
export block_pools_types.VerifierError
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/altair/light-client/sync-protocol.md#initialize_light_client_store
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/altair/light-client/sync-protocol.md#initialize_light_client_store
func initialize_light_client_store*(
trusted_block_root: Eth2Digest,
bootstrap: ForkyLightClientBootstrap,
@ -42,7 +42,7 @@ func initialize_light_client_store*(
current_sync_committee: bootstrap.current_sync_committee,
optimistic_header: bootstrap.header))
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/altair/light-client/sync-protocol.md#validate_light_client_update
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/altair/light-client/sync-protocol.md#validate_light_client_update
proc validate_light_client_update*(
store: ForkyLightClientStore,
update: SomeForkyLightClientUpdate,
@ -159,7 +159,7 @@ proc validate_light_client_update*(
ok()
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/altair/light-client/sync-protocol.md#apply_light_client_update
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/altair/light-client/sync-protocol.md#apply_light_client_update
func apply_light_client_update(
store: var ForkyLightClientStore,
update: SomeForkyLightClientUpdate): bool =
@ -190,7 +190,7 @@ func apply_light_client_update(
didProgress = true
didProgress
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/altair/light-client/sync-protocol.md#process_light_client_store_force_update
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/altair/light-client/sync-protocol.md#process_light_client_store_force_update
type
ForceUpdateResult* = enum
NoUpdate,
@ -223,7 +223,7 @@ func process_light_client_store_force_update*(
store.best_valid_update.reset()
res
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/altair/light-client/sync-protocol.md#process_light_client_update
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/altair/light-client/sync-protocol.md#process_light_client_update
proc process_light_client_update*(
store: var ForkyLightClientStore,
update: SomeForkyLightClientUpdate,

View File

@ -14,8 +14,8 @@ import
export base
const
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/phase0/p2p-interface.md#topics-and-messages
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/capella/p2p-interface.md#topics-and-messages
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/phase0/p2p-interface.md#topics-and-messages
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/capella/p2p-interface.md#topics-and-messages
topicBeaconBlocksSuffix* = "beacon_block/ssz_snappy"
topicVoluntaryExitsSuffix* = "voluntary_exit/ssz_snappy"
topicProposerSlashingsSuffix* = "proposer_slashing/ssz_snappy"
@ -27,7 +27,7 @@ const
MAX_REQUEST_BLOCKS* = 1024
RESP_TIMEOUT* = 10.seconds
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/altair/light-client/p2p-interface.md#configuration
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/altair/light-client/p2p-interface.md#configuration
MAX_REQUEST_LIGHT_CLIENT_UPDATES* = 128
# https://github.com/ethereum/consensus-specs/blob/v1.3.0/specs/bellatrix/p2p-interface.md#configuration
@ -68,7 +68,7 @@ func getAttesterSlashingsTopic*(forkDigest: ForkDigest): string =
func getAggregateAndProofsTopic*(forkDigest: ForkDigest): string =
eth2Prefix(forkDigest) & topicAggregateAndProofsSuffix
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/capella/p2p-interface.md#topics-and-messages
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/capella/p2p-interface.md#topics-and-messages
func getBlsToExecutionChangeTopic*(forkDigest: ForkDigest): string =
eth2Prefix(forkDigest) & topicBlsToExecutionChangeSuffix
@ -94,13 +94,13 @@ func getAttestationTopic*(forkDigest: ForkDigest,
## For subscribing and unsubscribing to/from a subnet.
eth2Prefix(forkDigest) & "beacon_attestation_" & $(subnetId) & "/ssz_snappy"
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/altair/p2p-interface.md#topics-and-messages
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/altair/p2p-interface.md#topics-and-messages
func getSyncCommitteeTopic*(forkDigest: ForkDigest,
subcommitteeIdx: SyncSubcommitteeIndex): string =
## For subscribing and unsubscribing to/from a subnet.
eth2Prefix(forkDigest) & "sync_committee_" & $subcommitteeIdx & "/ssz_snappy"
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/altair/p2p-interface.md#topics-and-messages
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/altair/p2p-interface.md#topics-and-messages
func getSyncCommitteeContributionAndProofTopic*(forkDigest: ForkDigest): string =
## For subscribing and unsubscribing to/from a subnet.
eth2Prefix(forkDigest) & "sync_committee_contribution_and_proof/ssz_snappy"
@ -110,16 +110,16 @@ func getBlobSidecarTopic*(forkDigest: ForkDigest,
subnet_id: SubnetId): string =
eth2Prefix(forkDigest) & "blob_sidecar_" & $subnet_id & "/ssz_snappy"
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/deneb/validator.md#sidecar
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/deneb/validator.md#sidecar
func compute_subnet_for_blob_sidecar*(blob_index: BlobIndex): SubnetId =
SubnetId(blob_index mod BLOB_SIDECAR_SUBNET_COUNT)
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/altair/light-client/p2p-interface.md#light_client_finality_update
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/altair/light-client/p2p-interface.md#light_client_finality_update
func getLightClientFinalityUpdateTopic*(forkDigest: ForkDigest): string =
## For broadcasting or obtaining the latest `LightClientFinalityUpdate`.
eth2Prefix(forkDigest) & "light_client_finality_update/ssz_snappy"
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/altair/light-client/p2p-interface.md#light_client_optimistic_update
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/altair/light-client/p2p-interface.md#light_client_optimistic_update
func getLightClientOptimisticUpdateTopic*(forkDigest: ForkDigest): string =
## For broadcasting or obtaining the latest `LightClientOptimisticUpdate`.
eth2Prefix(forkDigest) & "light_client_optimistic_update/ssz_snappy"
@ -156,7 +156,7 @@ func getDiscoveryForkID*(cfg: RuntimeConfig,
next_fork_version: current_fork_version,
next_fork_epoch: FAR_FUTURE_EPOCH)
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/altair/p2p-interface.md#transitioning-the-gossip
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/altair/p2p-interface.md#transitioning-the-gossip
type GossipState* = set[ConsensusFork]
func getTargetGossipState*(
epoch, ALTAIR_FORK_EPOCH, BELLATRIX_FORK_EPOCH, CAPELLA_FORK_EPOCH,
@ -198,7 +198,7 @@ func getTargetGossipState*(
targetForks
func nearSyncCommitteePeriod*(epoch: Epoch): Opt[uint64] =
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/altair/validator.md#sync-committee-subnet-stability
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/altair/validator.md#sync-committee-subnet-stability
if epoch.is_sync_committee_period():
return Opt.some 0'u64
let epochsBefore =
@ -217,7 +217,7 @@ func getSyncSubnets*(
if not nodeHasPubkey(pubkey):
continue
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/altair/validator.md#broadcast-sync-committee-message
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/altair/validator.md#broadcast-sync-committee-message
# The first quarter of the pubkeys map to subnet 0, the second quarter to
# subnet 1, the third quarter to subnet 2 and the final quarter to subnet
# 3.

View File

@ -6,7 +6,7 @@
# at your option. This file may not be copied, modified, or distributed except according to those terms.
# Mainnet preset - Altair
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/presets/mainnet/altair.yaml
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/presets/mainnet/altair.yaml
const
# Updated penalty values
# ---------------------------------------------------------------

View File

@ -6,7 +6,7 @@
# at your option. This file may not be copied, modified, or distributed except according to those terms.
# Gnosis preset - Bellatrix
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/presets/mainnet/bellatrix.yaml
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/presets/mainnet/bellatrix.yaml
const
# Updated penalty values
# ---------------------------------------------------------------

View File

@ -6,7 +6,7 @@
# at your option. This file may not be copied, modified, or distributed except according to those terms.
# Mainnet preset - Altair
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/presets/mainnet/altair.yaml
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/presets/mainnet/altair.yaml
const
# Updated penalty values
# ---------------------------------------------------------------

View File

@ -6,7 +6,7 @@
# at your option. This file may not be copied, modified, or distributed except according to those terms.
# Mainnet preset - Bellatrix
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/presets/mainnet/bellatrix.yaml
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/presets/mainnet/bellatrix.yaml
const
# Updated penalty values
# ---------------------------------------------------------------

View File

@ -6,7 +6,7 @@
# at your option. This file may not be copied, modified, or distributed except according to those terms.
# Mainnet preset - Capella
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/presets/mainnet/capella.yaml
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/presets/mainnet/capella.yaml
const
# Max operations per block
# ---------------------------------------------------------------

View File

@ -6,7 +6,7 @@
# at your option. This file may not be copied, modified, or distributed except according to those terms.
# Mainnet preset - Deneb
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/presets/mainnet/deneb.yaml
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/presets/mainnet/deneb.yaml
const
# `uint64(4096)`
FIELD_ELEMENTS_PER_BLOB*: uint64 = 4096

View File

@ -6,7 +6,7 @@
# at your option. This file may not be copied, modified, or distributed except according to those terms.
# Minimal preset - Altair
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/presets/minimal/altair.yaml
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/presets/minimal/altair.yaml
const
# Updated penalty values
# ---------------------------------------------------------------

View File

@ -6,7 +6,7 @@
# at your option. This file may not be copied, modified, or distributed except according to those terms.
# Minimal preset - Bellatrix
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/presets/minimal/bellatrix.yaml
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/presets/minimal/bellatrix.yaml
const
# Updated penalty values
# ---------------------------------------------------------------

View File

@ -6,7 +6,7 @@
# at your option. This file may not be copied, modified, or distributed except according to those terms.
# Minimal preset - Capella
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/presets/minimal/capella.yaml
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/presets/minimal/capella.yaml
const
# Max operations per block
# ---------------------------------------------------------------

View File

@ -6,7 +6,7 @@
# at your option. This file may not be copied, modified, or distributed except according to those terms.
# Minimal preset - Deneb
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/presets/minimal/deneb.yaml
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/presets/minimal/deneb.yaml
const
# [customized]
FIELD_ELEMENTS_PER_BLOB*: uint64 = 4

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.0/specs/phase0/validator.md#aggregation-selection
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/phase0/validator.md#aggregation-selection
func get_slot_signature*(
fork: Fork, genesis_validators_root: Eth2Digest, slot: Slot,
privkey: ValidatorPrivKey): CookedSig =
@ -59,7 +59,7 @@ func compute_epoch_signing_root*(
let domain = get_domain(fork, DOMAIN_RANDAO, epoch, genesis_validators_root)
compute_signing_root(epoch, domain)
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/phase0/validator.md#randao-reveal
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/phase0/validator.md#randao-reveal
func get_epoch_signature*(
fork: Fork, genesis_validators_root: Eth2Digest, epoch: Epoch,
privkey: ValidatorPrivKey): CookedSig =
@ -146,7 +146,7 @@ func compute_aggregate_and_proof_signing_root*(
fork, DOMAIN_AGGREGATE_AND_PROOF, epoch, genesis_validators_root)
compute_signing_root(aggregate_and_proof, domain)
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/phase0/validator.md#broadcast-aggregate
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/phase0/validator.md#broadcast-aggregate
func get_aggregate_and_proof_signature*(fork: Fork, genesis_validators_root: Eth2Digest,
aggregate_and_proof: AggregateAndProof,
privkey: ValidatorPrivKey): CookedSig =
@ -264,7 +264,7 @@ proc verify_voluntary_exit_signature*(
blsVerify(pubkey, signing_root.data, signature)
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/altair/validator.md#prepare-sync-committee-message
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/altair/validator.md#prepare-sync-committee-message
func compute_sync_committee_message_signing_root*(
fork: Fork, genesis_validators_root: Eth2Digest,
slot: Slot, beacon_block_root: Eth2Digest): Eth2Digest =
@ -299,7 +299,7 @@ proc verify_sync_committee_signature*(
blsFastAggregateVerify(pubkeys, signing_root.data, signature)
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/altair/validator.md#aggregation-selection
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/altair/validator.md#aggregation-selection
func compute_sync_committee_selection_proof_signing_root*(
fork: Fork, genesis_validators_root: Eth2Digest,
slot: Slot, subcommittee_index: SyncSubcommitteeIndex): Eth2Digest =
@ -330,7 +330,7 @@ proc verify_sync_committee_selection_proof*(
blsVerify(pubkey, signing_root.data, signature)
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/altair/validator.md#signature
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/altair/validator.md#signature
func compute_contribution_and_proof_signing_root*(
fork: Fork, genesis_validators_root: Eth2Digest,
msg: ContributionAndProof): Eth2Digest =
@ -348,7 +348,7 @@ proc get_contribution_and_proof_signature*(
blsSign(privkey, signing_root.data)
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/altair/validator.md#aggregation-selection
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/altair/validator.md#aggregation-selection
func is_sync_committee_aggregator*(signature: ValidatorSig): bool =
let
signatureDigest = eth2digest(signature.blob)
@ -388,7 +388,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.0/specs/capella/beacon-chain.md#new-process_bls_to_execution_change
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/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

@ -364,7 +364,7 @@ template partialBeaconBlock*(
deposits: List[Deposit, Limit MAX_DEPOSITS](deposits),
voluntary_exits: validator_changes.voluntary_exits))
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/altair/validator.md#preparing-a-beaconblock
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/altair/validator.md#preparing-a-beaconblock
template partialBeaconBlock*(
cfg: RuntimeConfig,
state: var altair.HashedBeaconState,
@ -423,7 +423,7 @@ template partialBeaconBlock*(
sync_aggregate: sync_aggregate,
execution_payload: execution_payload.executionPayload))
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/capella/validator.md#block-proposal
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/capella/validator.md#block-proposal
template partialBeaconBlock*(
cfg: RuntimeConfig,
state: var capella.HashedBeaconState,
@ -540,7 +540,7 @@ proc makeBeaconBlock*(
forkyState.data.latest_execution_payload_header.transactions_root =
transactions_root.get
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/bellatrix/beacon-chain.md#beaconblockbody
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/bellatrix/beacon-chain.md#beaconblockbody
# Effectively hash_tree_root(ExecutionPayload) with the beacon block
# body, with the execution payload replaced by the execution payload
# header. htr(payload) == htr(payload header), so substitute.

View File

@ -9,7 +9,7 @@
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-alpha.3/specs/phase0/beacon-chain.md#block-processing
# https://github.com/ethereum/consensus-specs/blob/v1.3.0/specs/altair/beacon-chain.md#block-processing
# https://github.com/ethereum/consensus-specs/blob/v1.3.0/specs/bellatrix/beacon-chain.md#block-processing
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/capella/beacon-chain.md#block-processing
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/capella/beacon-chain.md#block-processing
# https://github.com/ethereum/consensus-specs/blob/v1.3.0/specs/deneb/beacon-chain.md#block-processing
#
# The entry point is `process_block` which is at the bottom of this file.
@ -126,7 +126,7 @@ func process_eth1_data(
state.eth1_data = body.eth1_data
ok()
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/phase0/beacon-chain.md#is_slashable_validator
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/phase0/beacon-chain.md#is_slashable_validator
func is_slashable_validator(validator: Validator, epoch: Epoch): bool =
# Check if ``validator`` is slashable.
(not validator.slashed) and
@ -182,7 +182,7 @@ proc check_proposer_slashing*(
withState(state):
check_proposer_slashing(forkyState.data, proposer_slashing, flags)
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/phase0/beacon-chain.md#proposer-slashings
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/phase0/beacon-chain.md#proposer-slashings
proc process_proposer_slashing*(
cfg: RuntimeConfig, state: var ForkyBeaconState,
proposer_slashing: SomeProposerSlashing, flags: UpdateFlags,
@ -246,7 +246,7 @@ proc check_attester_slashing*(
withState(state):
check_attester_slashing(forkyState.data, attester_slashing, flags)
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/phase0/beacon-chain.md#attester-slashings
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/phase0/beacon-chain.md#attester-slashings
proc process_attester_slashing*(
cfg: RuntimeConfig,
state: var ForkyBeaconState,
@ -330,7 +330,7 @@ proc process_deposit*(cfg: RuntimeConfig,
ok()
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/phase0/beacon-chain.md#voluntary-exits
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/phase0/beacon-chain.md#voluntary-exits
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/deneb/beacon-chain.md#modified-process_voluntary_exit
proc check_voluntary_exit*(
cfg: RuntimeConfig,
@ -388,7 +388,7 @@ proc check_voluntary_exit*(
withState(state):
check_voluntary_exit(cfg, forkyState.data, signed_voluntary_exit, flags)
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/phase0/beacon-chain.md#voluntary-exits
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/phase0/beacon-chain.md#voluntary-exits
proc process_voluntary_exit*(
cfg: RuntimeConfig,
state: var ForkyBeaconState,
@ -419,7 +419,7 @@ proc process_bls_to_execution_change*(
ok()
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-alpha.3/specs/phase0/beacon-chain.md#operations
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/capella/beacon-chain.md#modified-process_operations
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/capella/beacon-chain.md#modified-process_operations
proc process_operations(cfg: RuntimeConfig,
state: var ForkyBeaconState,
body: SomeForkyBeaconBlockBody,
@ -465,11 +465,11 @@ func get_participant_reward*(total_active_balance: Gwei): Gwei =
WEIGHT_DENOMINATOR div SLOTS_PER_EPOCH
max_participant_rewards div SYNC_COMMITTEE_SIZE
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/altair/beacon-chain.md#sync-aggregate-processing
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/altair/beacon-chain.md#sync-aggregate-processing
func get_proposer_reward*(participant_reward: Gwei): Gwei =
participant_reward * PROPOSER_WEIGHT div (WEIGHT_DENOMINATOR - PROPOSER_WEIGHT)
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/altair/beacon-chain.md#sync-aggregate-processing
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/altair/beacon-chain.md#sync-aggregate-processing
proc process_sync_aggregate*(
state: var (altair.BeaconState | bellatrix.BeaconState |
capella.BeaconState | deneb.BeaconState),
@ -845,7 +845,7 @@ proc process_block*(
ok()
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/bellatrix/beacon-chain.md#block-processing
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/bellatrix/beacon-chain.md#block-processing
# TODO workaround for https://github.com/nim-lang/Nim/issues/18095
type SomeBellatrixBlock =
bellatrix.BeaconBlock | bellatrix.SigVerifiedBeaconBlock | bellatrix.TrustedBeaconBlock

View File

@ -243,7 +243,7 @@ func is_unslashed_participating_index(
has_flag(epoch_participation[].item(validator_index), flag_index) and
not state.validators[validator_index].slashed
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/phase0/beacon-chain.md#justification-and-finalization
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/phase0/beacon-chain.md#justification-and-finalization
type FinalityState = object
slot: Slot
current_epoch_ancestor_root: Eth2Digest
@ -311,7 +311,7 @@ proc weigh_justification_and_finalization(
## state.justification_bits[1:] = state.justification_bits[:-1]
## state.justification_bits[0] = 0b0
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/phase0/beacon-chain.md#misc
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/phase0/beacon-chain.md#misc
const JUSTIFICATION_BITS_LENGTH = 4
state.justification_bits = JustificationBits(
@ -388,7 +388,7 @@ proc weigh_justification_and_finalization(
current_epoch = current_epoch,
checkpoint = shortLog(state.finalized_checkpoint)
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/phase0/beacon-chain.md#justification-and-finalization
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/phase0/beacon-chain.md#justification-and-finalization
proc process_justification_and_finalization*(
state: var phase0.BeaconState,
balances: TotalBalances, flags: UpdateFlags = {}) =
@ -424,7 +424,7 @@ proc compute_unrealized_finality*(
justified: finalityState.current_justified_checkpoint,
finalized: finalityState.finalized_checkpoint)
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/altair/beacon-chain.md#justification-and-finalization
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/altair/beacon-chain.md#justification-and-finalization
proc process_justification_and_finalization*(
state: var (altair.BeaconState | bellatrix.BeaconState |
capella.BeaconState | deneb.BeaconState),
@ -460,7 +460,7 @@ proc compute_unrealized_finality*(
justified: finalityState.current_justified_checkpoint,
finalized: finalityState.finalized_checkpoint)
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/phase0/beacon-chain.md#helpers
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/phase0/beacon-chain.md#helpers
func get_base_reward_sqrt*(state: phase0.BeaconState, index: ValidatorIndex,
total_balance_sqrt: auto): Gwei =
# Spec function recalculates total_balance every time, which creates an
@ -508,7 +508,7 @@ func get_attestation_component_delta(is_unslashed_attester: bool,
else:
RewardDelta(penalties: base_reward)
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/phase0/beacon-chain.md#components-of-attestation-deltas
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/phase0/beacon-chain.md#components-of-attestation-deltas
func get_source_delta*(validator: RewardStatus,
base_reward: uint64,
balances: TotalBalances,
@ -631,7 +631,7 @@ func get_attestation_deltas(
info.validators[proposer_index].delta.add(
proposer_delta.get()[1])
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/altair/beacon-chain.md#get_base_reward
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/altair/beacon-chain.md#get_base_reward
func get_base_reward_increment*(
state: altair.BeaconState | bellatrix.BeaconState | capella.BeaconState |
deneb.BeaconState,
@ -656,7 +656,7 @@ func get_flag_index_reward*(
else:
0.Gwei
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/altair/beacon-chain.md#get_flag_index_deltas
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/altair/beacon-chain.md#get_flag_index_deltas
func get_unslashed_participating_increment*(
info: altair.EpochInfo | bellatrix.BeaconState, flag_index: TimelyFlag): Gwei =
info.balances.previous_epoch[flag_index] div EFFECTIVE_BALANCE_INCREMENT
@ -666,7 +666,7 @@ func get_active_increments*(
info: altair.EpochInfo | bellatrix.BeaconState): Gwei =
info.balances.current_epoch div EFFECTIVE_BALANCE_INCREMENT
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/altair/beacon-chain.md#get_flag_index_deltas
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/altair/beacon-chain.md#get_flag_index_deltas
iterator get_flag_index_deltas*(
state: altair.BeaconState | bellatrix.BeaconState | capella.BeaconState |
deneb.BeaconState,
@ -889,12 +889,12 @@ func get_adjusted_total_slashing_balance*(
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-alpha.3/specs/phase0/beacon-chain.md#slashings
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-alpha.3/specs/altair/beacon-chain.md#slashings
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/bellatrix/beacon-chain.md#slashings
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/bellatrix/beacon-chain.md#slashings
func slashing_penalty_applies*(validator: Validator, epoch: Epoch): bool =
validator.slashed and
epoch + EPOCHS_PER_SLASHINGS_VECTOR div 2 == validator.withdrawable_epoch
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/phase0/beacon-chain.md#slashings
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/phase0/beacon-chain.md#slashings
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-alpha.3/specs/altair/beacon-chain.md#slashings
# https://github.com/ethereum/consensus-specs/blob/v1.3.0/specs/bellatrix/beacon-chain.md#slashings
func get_slashing_penalty*(validator: Validator,
@ -907,8 +907,8 @@ func get_slashing_penalty*(validator: Validator,
penalty_numerator div total_balance * increment
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-alpha.3/specs/phase0/beacon-chain.md#slashings
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/altair/beacon-chain.md#slashings
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/bellatrix/beacon-chain.md#slashings
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/altair/beacon-chain.md#slashings
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/bellatrix/beacon-chain.md#slashings
func process_slashings*(state: var ForkyBeaconState, total_balance: Gwei) =
let
epoch = get_current_epoch(state)
@ -952,7 +952,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.0/specs/phase0/beacon-chain.md#slashings-balances-updates
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/phase0/beacon-chain.md#slashings-balances-updates
func process_slashings_reset*(state: var ForkyBeaconState) =
let next_epoch = get_current_epoch(state) + 1
@ -985,7 +985,7 @@ func process_historical_roots_update*(state: var ForkyBeaconState) =
if next_epoch mod (SLOTS_PER_HISTORICAL_ROOT div SLOTS_PER_EPOCH) == 0:
# Equivalent to hash_tree_root(foo: HistoricalBatch), but without using
# significant additional stack or heap.
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/phase0/beacon-chain.md#historicalbatch
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/phase0/beacon-chain.md#historicalbatch
# In response to https://github.com/status-im/nimbus-eth2/issues/921
if not state.historical_roots.add state.compute_historical_root():
raiseAssert "no more room for historical roots, so long and thanks for the fish!"
@ -1057,7 +1057,7 @@ func process_inactivity_updates*(
if pre_inactivity_score != inactivity_score:
state.inactivity_scores[index] = inactivity_score
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/capella/beacon-chain.md#historical-summaries-updates
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/capella/beacon-chain.md#historical-summaries-updates
func process_historical_summaries_update*(
state: var (capella.BeaconState | deneb.BeaconState)):
Result[void, cstring] =
@ -1073,7 +1073,7 @@ func process_historical_summaries_update*(
ok()
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/phase0/beacon-chain.md#epoch-processing
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/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] =
@ -1177,7 +1177,7 @@ proc process_epoch*(
ok()
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/capella/beacon-chain.md#epoch-processing
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/capella/beacon-chain.md#epoch-processing
proc process_epoch*(
cfg: RuntimeConfig,
state: var (capella.BeaconState | deneb.BeaconState),
@ -1188,7 +1188,7 @@ proc process_epoch*(
info.init(state)
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/altair/beacon-chain.md#justification-and-finalization
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/altair/beacon-chain.md#justification-and-finalization
process_justification_and_finalization(state, info.balances, flags)
# state.slot hasn't been incremented yet.
@ -1207,10 +1207,10 @@ proc process_epoch*(
# https://github.com/ethereum/consensus-specs/blob/v1.3.0/specs/altair/beacon-chain.md#rewards-and-penalties
process_rewards_and_penalties(cfg, state, info)
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/phase0/beacon-chain.md#registry-updates
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/phase0/beacon-chain.md#registry-updates
? process_registry_updates(cfg, state, cache)
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/altair/beacon-chain.md#slashings
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/altair/beacon-chain.md#slashings
process_slashings(state, info.balances.current_epoch)
process_eth1_data_reset(state)

View File

@ -21,7 +21,7 @@ const
PIVOT_VIEW_SIZE = SEED_SIZE + ROUND_SIZE
TOTAL_SIZE = PIVOT_VIEW_SIZE + POSITION_WINDOW_SIZE
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/phase0/beacon-chain.md#compute_shuffled_index
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/phase0/beacon-chain.md#compute_shuffled_index
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-alpha.3/specs/phase0/beacon-chain.md#compute_committee
# Port of https://github.com/protolambda/zrnt/blob/master/eth2/beacon/shuffle.go
# Shuffles or unshuffles, depending on the `dir` (true for shuffling, false for unshuffling
@ -301,7 +301,7 @@ func get_beacon_committee_len*(
withState(state):
get_beacon_committee_len(forkyState.data, slot, index, cache)
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/phase0/beacon-chain.md#compute_shuffled_index
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/phase0/beacon-chain.md#compute_shuffled_index
func compute_shuffled_index*(
index: uint64, index_count: uint64, seed: Eth2Digest): uint64 =
## Return the shuffled index corresponding to ``seed`` (and ``index_count``).
@ -336,7 +336,7 @@ func compute_shuffled_index*(
cur_idx_permuted
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/phase0/beacon-chain.md#compute_proposer_index
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/phase0/beacon-chain.md#compute_proposer_index
func compute_proposer_index(state: ForkyBeaconState,
indices: seq[ValidatorIndex], seed: Eth2Digest): Opt[ValidatorIndex] =
## Return from ``indices`` a random index sampled by effective balance.
@ -363,7 +363,7 @@ func compute_proposer_index(state: ForkyBeaconState,
return Opt.some(candidate_index)
i += 1
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/phase0/beacon-chain.md#get_beacon_proposer_index
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/phase0/beacon-chain.md#get_beacon_proposer_index
func get_beacon_proposer_index*(
state: ForkyBeaconState, cache: var StateCache, slot: Slot):
Opt[ValidatorIndex] =
@ -401,7 +401,7 @@ func get_beacon_proposer_index*(
return res
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/phase0/beacon-chain.md#get_beacon_proposer_index
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/phase0/beacon-chain.md#get_beacon_proposer_index
func get_beacon_proposer_index*(state: ForkyBeaconState, cache: var StateCache):
Opt[ValidatorIndex] =
## Return the beacon proposer index at the current slot.
@ -413,7 +413,7 @@ func get_beacon_proposer_index*(state: ForkedHashedBeaconState,
withState(state):
get_beacon_proposer_index(forkyState.data, cache, slot)
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/phase0/validator.md#aggregation-selection
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/phase0/validator.md#aggregation-selection
func is_aggregator*(committee_len: uint64, slot_signature: ValidatorSig): bool =
let modulo = max(1'u64, committee_len div TARGET_AGGREGATORS_PER_COMMITTEE)
bytes_to_uint64(eth2digest(

View File

@ -10,10 +10,10 @@
import
./datatypes/base, ./beaconstate, ./forks, ./helpers
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/phase0/weak-subjectivity.md#configuration
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/phase0/weak-subjectivity.md#configuration
const SAFETY_DECAY* = 10'u64
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/phase0/weak-subjectivity.md#compute_weak_subjectivity_period
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/phase0/weak-subjectivity.md#compute_weak_subjectivity_period
func compute_weak_subjectivity_period(
cfg: RuntimeConfig, state: ForkyBeaconState): uint64 =
## Returns the weak subjectivity period for the current ``state``.
@ -49,7 +49,7 @@ func compute_weak_subjectivity_period(
ws_period
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/phase0/weak-subjectivity.md#is_within_weak_subjectivity_period
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/phase0/weak-subjectivity.md#is_within_weak_subjectivity_period
func is_within_weak_subjectivity_period*(cfg: RuntimeConfig, current_slot: Slot,
ws_state: ForkedHashedBeaconState,
ws_checkpoint: Checkpoint): bool =

View File

@ -110,7 +110,7 @@ proc isGossipSupported*(
finalizedPeriod = self.getFinalizedPeriod(),
isNextSyncCommitteeKnown = self.isNextSyncCommitteeKnown())
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/altair/light-client/p2p-interface.md#getlightclientbootstrap
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/altair/light-client/p2p-interface.md#getlightclientbootstrap
proc doRequest(
e: typedesc[Bootstrap],
peer: Peer,
@ -119,7 +119,7 @@ proc doRequest(
raises: [Defect, IOError].} =
peer.lightClientBootstrap(blockRoot)
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/altair/light-client/p2p-interface.md#lightclientupdatesbyrange
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/altair/light-client/p2p-interface.md#lightclientupdatesbyrange
type LightClientUpdatesByRangeResponse =
NetRes[List[ForkedLightClientUpdate, MAX_REQUEST_LIGHT_CLIENT_UPDATES]]
proc doRequest(
@ -138,7 +138,7 @@ proc doRequest(
raise newException(ResponseError, e.error)
return response
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/altair/light-client/p2p-interface.md#getlightclientfinalityupdate
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/altair/light-client/p2p-interface.md#getlightclientfinalityupdate
proc doRequest(
e: typedesc[FinalityUpdate],
peer: Peer
@ -146,7 +146,7 @@ proc doRequest(
raises: [Defect, IOError].} =
peer.lightClientFinalityUpdate()
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/altair/light-client/p2p-interface.md#getlightclientoptimisticupdate
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/altair/light-client/p2p-interface.md#getlightclientoptimisticupdate
proc doRequest(
e: typedesc[OptimisticUpdate],
peer: Peer
@ -335,7 +335,7 @@ template query[E](
): Future[bool] =
self.query(e, Nothing())
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/altair/light-client/light-client.md#light-client-sync-process
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/altair/light-client/light-client.md#light-client-sync-process
proc loop(self: LightClientManager) {.async.} =
var nextSyncTaskTime = self.getBeaconTime()
while true:

View File

@ -548,7 +548,7 @@ p2pProtocol BeaconSync(version = 1,
debug "BlobSidecar range request done",
peer, startSlot, count = reqCount, found
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/altair/light-client/p2p-interface.md#getlightclientbootstrap
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/altair/light-client/p2p-interface.md#getlightclientbootstrap
proc lightClientBootstrap(
peer: Peer,
blockRoot: Eth2Digest,
@ -576,7 +576,7 @@ p2pProtocol BeaconSync(version = 1,
debug "LC bootstrap request done", peer, blockRoot
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/altair/light-client/p2p-interface.md#lightclientupdatesbyrange
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/altair/light-client/p2p-interface.md#lightclientupdatesbyrange
proc lightClientUpdatesByRange(
peer: Peer,
startPeriod: SyncCommitteePeriod,
@ -621,7 +621,7 @@ p2pProtocol BeaconSync(version = 1,
debug "LC updates by range request done", peer, startPeriod, count, found
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/altair/light-client/p2p-interface.md#getlightclientfinalityupdate
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/altair/light-client/p2p-interface.md#getlightclientfinalityupdate
proc lightClientFinalityUpdate(
peer: Peer,
response: SingleChunkResponse[ForkedLightClientFinalityUpdate])
@ -648,7 +648,7 @@ p2pProtocol BeaconSync(version = 1,
debug "LC finality update request done", peer
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/altair/light-client/p2p-interface.md#getlightclientoptimisticupdate
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/altair/light-client/p2p-interface.md#getlightclientoptimisticupdate
proc lightClientOptimisticUpdate(
peer: Peer,
response: SingleChunkResponse[ForkedLightClientOptimisticUpdate])

View File

@ -181,7 +181,7 @@ proc doTrustedNodeSync*(
let stateId =
case syncTarget.kind
of TrustedNodeSyncKind.TrustedBlockRoot:
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/altair/light-client/light-client.md#light-client-sync-process
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/altair/light-client/light-client.md#light-client-sync-process
const lcDataFork = LightClientDataFork.high
var bestViableCheckpoint: Opt[tuple[slot: Slot, state_root: Eth2Digest]]
func trackBestViableCheckpoint(store: lcDataFork.LightClientStore) =

View File

@ -405,7 +405,7 @@ proc pollForSyncCommitteeDuties*(service: DutiesServiceRef) {.async.} =
let vc = service.client
let
currentSlot = vc.getCurrentSlot().get(Slot(0))
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/altair/validator.md#sync-committee
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/altair/validator.md#sync-committee
dutySlot = currentSlot + 1
dutyEpoch = dutySlot.epoch()

View File

@ -1717,7 +1717,7 @@ proc handleValidatorDuties*(node: BeaconNode, lastSlot, slot: Slot) {.async.} =
updateValidatorMetrics(node) # the important stuff is done, update the vanity numbers
# https://github.com/ethereum/consensus-specs/blob/v1.3.0/specs/phase0/validator.md#broadcast-aggregate
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/altair/validator.md#broadcast-sync-committee-contribution
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/altair/validator.md#broadcast-sync-committee-contribution
# Wait 2 / 3 of the slot time to allow messages to propagate, then collect
# the result in aggregates
static:

View File

@ -429,7 +429,7 @@ proc signData(v: AttachedValidator,
else:
v.signWithDistributedKey(request)
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/phase0/validator.md#signature
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/phase0/validator.md#signature
proc getBlockSignature*(v: AttachedValidator, fork: Fork,
genesis_validators_root: Eth2Digest, slot: Slot,
block_root: Eth2Digest,
@ -618,7 +618,7 @@ proc getAggregateAndProofSignature*(v: AttachedValidator,
fork, genesis_validators_root, aggregate_and_proof)
await v.signData(request)
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/altair/validator.md#prepare-sync-committee-message
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/altair/validator.md#prepare-sync-committee-message
proc getSyncCommitteeMessage*(v: AttachedValidator,
fork: Fork,
genesis_validators_root: Eth2Digest,
@ -649,7 +649,7 @@ proc getSyncCommitteeMessage*(v: AttachedValidator,
)
)
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/altair/validator.md#aggregation-selection
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/altair/validator.md#aggregation-selection
proc getSyncCommitteeSelectionProof*(v: AttachedValidator, fork: Fork,
genesis_validators_root: Eth2Digest,
slot: Slot,
@ -669,7 +669,7 @@ proc getSyncCommitteeSelectionProof*(v: AttachedValidator, fork: Fork,
)
await v.signData(request)
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/altair/validator.md#broadcast-sync-committee-contribution
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/altair/validator.md#broadcast-sync-committee-contribution
proc getContributionAndProofSignature*(v: AttachedValidator, fork: Fork,
genesis_validators_root: Eth2Digest,
contribution_and_proof: ContributionAndProof
@ -700,7 +700,7 @@ proc getEpochSignature*(v: AttachedValidator, fork: Fork,
fork, genesis_validators_root, epoch)
await v.signData(request)
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/phase0/validator.md#aggregation-selection
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/phase0/validator.md#aggregation-selection
proc getSlotSignature*(v: AttachedValidator, fork: Fork,
genesis_validators_root: Eth2Digest, slot: Slot
): Future[SignatureResult] {.async.} =

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.0/specs/phase0/p2p-interface.md#beacon_attestation_subnet_id
- Unaggregated: https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/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-alpha.3/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.0/specs/phase0/p2p-interface.md#beacon_aggregate_and_proof
- https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/phase0/p2p-interface.md#beacon_aggregate_and_proof
- https://github.com/ethereum/consensus-specs/blob/v1.4.0-alpha.3/specs/phase0/p2p-interface.md#attestation-subnets
Finally, valid attestations are added to the local `attestationPool`.

View File

@ -6,10 +6,10 @@ 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.0/specs/phase0/p2p-interface.md#beacon_block.
https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/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.0/specs/phase0/beacon-chain.md#block-processing
https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/phase0/beacon-chain.md#block-processing
A block needs to be verified to enter fork choice, the DAG and the BeaconChainDB
In particular in terms of costly checks validating a block only requires checking:

View File

@ -104,7 +104,7 @@ The following sections explain how to do this for certain EL clients.
## Running the light client
The light client starts syncing from a trusted block.
This trusted block should be somewhat recent ([~1-2 weeks](https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/phase0/weak-subjectivity.md)) and needs to be configured each time when starting the light client.
This trusted block should be somewhat recent ([~1-2 weeks](https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/phase0/weak-subjectivity.md)) and needs to be configured each time when starting the light client.
### 1. Obtaining a trusted block root
@ -188,7 +188,7 @@ NOT 2022-11-21 18:04:03.982+01:00 New LC optimistic block opt
```
!!! note
The [light client protocol](https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/altair/light-client/sync-protocol.md) depends on consensus layer (CL) full nodes to serve additional data.
The [light client protocol](https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/altair/light-client/sync-protocol.md) depends on consensus layer (CL) full nodes to serve additional data.
As this is a new protocol, not all implementations are supporting it yet.
Therefore, it may take several minutes to discover supporting peers, during which no log messages may be produced.

View File

@ -2,7 +2,7 @@
Nimbus is configured by default to serve data that allows light clients to stay in sync with the Ethereum network.
Light client data is imported incrementally and does not affect validator performance.
Information about the light client sync protocol can be found in the [Ethereum consensus specs](https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/altair/light-client/sync-protocol.md).
Information about the light client sync protocol can be found in the [Ethereum consensus specs](https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/altair/light-client/sync-protocol.md).
!!! note
Nimbus also implements a [standalone light client](./el-light-client.md) that may be used to sync an execution layer (EL) client.

View File

@ -125,7 +125,7 @@ If you are already using a threshold signing setup (e.g. based on Vouch and Dirk
The verifying Web3Signer is an experimental extension to the [Web3Signer protocol](https://consensys.github.io/web3signer/web3signer-eth2.html#tag/Signing/operation/ETH2_SIGN) which allows the remote signer to verify certain details of the signed blocks before creating a signature (for example, the signer may require the signed block to have a particular fee recipient value).
To enable this use case, the `BLOCK_V2` request type of the `/api/v1/eth2/sign/{identifier}` endpoint is extended with an additional array field named `proofs`. The array consists of objects with the properties `index`, `proof` and `value`, where `index` is an arbitrary [generalized index](https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/ssz/merkle-proofs.md#generalized-merkle-tree-index) of any property nested under the block body and `proof` is its corresponding Merkle proof against the block body root included in the request. The `value` property is optional and it is included only when the SSZ hash of the field included in the Merkle proof doesn't match its value.
To enable this use case, the `BLOCK_V2` request type of the `/api/v1/eth2/sign/{identifier}` endpoint is extended with an additional array field named `proofs`. The array consists of objects with the properties `index`, `proof` and `value`, where `index` is an arbitrary [generalized index](https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/ssz/merkle-proofs.md#generalized-merkle-tree-index) of any property nested under the block body and `proof` is its corresponding Merkle proof against the block body root included in the request. The `value` property is optional and it is included only when the SSZ hash of the field included in the Merkle proof doesn't match its value.
Since the generalized index of a particular field may change in a hard-fork, in the remote keystore format the proven fields are usually specified by their name:

View File

@ -22,7 +22,7 @@ import
# Test utilities
../../testutil, ../../testblockutil
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/tests/core/pyspec/eth2spec/test/helpers/sync_committee.py#L27-L44
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/tests/core/pyspec/eth2spec/test/helpers/sync_committee.py#L27-L44
proc compute_aggregate_sync_committee_signature(
cfg: RuntimeConfig,
forked: ForkedHashedBeaconState,
@ -53,7 +53,7 @@ proc compute_aggregate_sync_committee_signature(
aggregateSig.aggregate(signature)
aggregateSig.finish.toValidatorSig
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/tests/core/pyspec/eth2spec/test/helpers/light_client.py#L11-L41
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/tests/core/pyspec/eth2spec/test/helpers/light_client.py#L11-L41
proc get_sync_aggregate(
cfg: RuntimeConfig,
forked: ForkedHashedBeaconState,
@ -137,7 +137,7 @@ let full_sync_committee_bits = block:
res.bytes.fill(byte.high)
res
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/altair/light-client/sync-protocol.md#initialize_light_client_store
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/altair/light-client/sync-protocol.md#initialize_light_client_store
func initialize_light_client_store(
state: auto, storeDataFork: static LightClientDataFork): auto =
storeDataFork.LightClientStore(

View File

@ -75,7 +75,7 @@ type
rewards*: List[uint64, Limit VALIDATOR_REGISTRY_LIMIT]
penalties*: List[uint64, Limit VALIDATOR_REGISTRY_LIMIT]
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/phase0/validator.md#eth1block
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/phase0/validator.md#eth1block
Eth1Block* = object
timestamp*: uint64
deposit_root*: Eth2Digest

View File

@ -25,7 +25,7 @@ const
MockPrivKeys* = MockPrivKeysT()
MockPubKeys* = MockPubKeysT()
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/tests/core/pyspec/eth2spec/test/helpers/keys.py
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/tests/core/pyspec/eth2spec/test/helpers/keys.py
func `[]`*(_: MockPrivKeysT, index: ValidatorIndex|uint64): ValidatorPrivKey =
var bytes = (index.uint64 + 1'u64).toBytesLE() # Consistent with EF tests
static: doAssert sizeof(bytes) <= sizeof(result)
@ -266,7 +266,7 @@ func makeAttestationData*(
"Computed epoch was " & $slot.epoch &
" while the state current_epoch was " & $current_epoch
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.0/specs/phase0/validator.md#attestation-data
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.1/specs/phase0/validator.md#attestation-data
AttestationData(
slot: slot,
index: committee_index.uint64,