update to latest light client libp2p protocol (#3623)

Incorporates the latest changes to the light client sync protocol based
on Devconnect AMS feedback. Note that this breaks compatibility with the
previous prototype, due to changes to data structures and endpoints.
See https://github.com/ethereum/consensus-specs/pull/2802
This commit is contained in:
Etan Kissling 2022-05-23 14:02:54 +02:00 committed by GitHub
parent 82677c66ce
commit c808f17a37
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
34 changed files with 1676 additions and 1485 deletions

View File

@ -265,11 +265,11 @@ OK: 3/3 Fail: 0/3 Skip: 0/3
## Light client processor [Preset: mainnet]
```diff
+ Duplicate bootstrap [Preset: mainnet] OK
+ Forced update [Preset: mainnet] OK
+ Invalid bootstrap [Preset: mainnet] OK
+ Missing bootstrap (finality update) [Preset: mainnet] OK
+ Missing bootstrap (optimistic update) [Preset: mainnet] OK
+ Missing bootstrap (update) [Preset: mainnet] OK
+ Standard sync [Preset: mainnet] OK
+ Sync [Preset: mainnet] OK
```
OK: 6/6 Fail: 0/6 Skip: 0/6
## ListKeys requests [Preset: mainnet]

View File

@ -804,6 +804,11 @@ OK: 35/35 Fail: 0/35 Skip: 0/35
All tests Skip
```
OK: 0/1 Fail: 0/1 Skip: 1/1
## EF - Altair - Sync protocol - Update ranking [Preset: mainnet]
```diff
All tests Skip
```
OK: 0/1 Fail: 0/1 Skip: 1/1
## EF - Altair - Unittests - Sync protocol [Preset: mainnet]
```diff
+ process_light_client_update_finality_updated OK
@ -1215,4 +1220,4 @@ OK: 44/44 Fail: 0/44 Skip: 0/44
OK: 27/27 Fail: 0/27 Skip: 0/27
---TOTAL---
OK: 1035/1037 Fail: 0/1037 Skip: 2/1037
OK: 1035/1038 Fail: 0/1038 Skip: 3/1038

View File

@ -845,6 +845,11 @@ OK: 35/35 Fail: 0/35 Skip: 0/35
All tests Skip
```
OK: 0/1 Fail: 0/1 Skip: 1/1
## EF - Altair - Sync protocol - Update ranking [Preset: minimal]
```diff
All tests Skip
```
OK: 0/1 Fail: 0/1 Skip: 1/1
## EF - Altair - Unittests - Sync protocol [Preset: minimal]
```diff
+ process_light_client_update_finality_updated OK
@ -1292,4 +1297,4 @@ OK: 48/48 Fail: 0/48 Skip: 0/48
OK: 30/30 Fail: 0/30 Skip: 0/30
---TOTAL---
OK: 1085/1106 Fail: 0/1106 Skip: 21/1106
OK: 1085/1107 Fail: 0/1107 Skip: 22/1107

View File

@ -50,6 +50,7 @@ type
quarantine*: ref Quarantine
attestationPool*: ref AttestationPool
syncCommitteeMsgPool*: ref SyncCommitteeMsgPool
lightClientPool*: ref LightClientPool
exitPool*: ref ExitPool
eth1Monitor*: Eth1Monitor
rpcServer*: RpcServer

View File

@ -216,8 +216,10 @@ type
## On beacon chain reorganization
onFinHappened*: OnFinalizedCallback
## On finalization callback
onOptimisticLightClientUpdate*: OnOptimisticLightClientUpdateCallback
## On `OptimisticLightClientUpdate` updated callback
onLightClientFinalityUpdate*: OnLightClientFinalityUpdateCallback
## On new `LightClientFinalityUpdate` callback
onLightClientOptimisticUpdate*: OnLightClientOptimisticUpdateCallback
## On new `LightClientOptimisticUpdate` callback
headSyncCommittees*: SyncCommitteeCache
## A cache of the sync committees, as they appear in the head state -

View File

@ -18,8 +18,10 @@ import
./block_dag
type
OnOptimisticLightClientUpdateCallback* =
proc(data: OptimisticLightClientUpdate) {.gcsafe, raises: [Defect].}
OnLightClientFinalityUpdateCallback* =
proc(data: altair.LightClientFinalityUpdate) {.gcsafe, raises: [Defect].}
OnLightClientOptimisticUpdateCallback* =
proc(data: altair.LightClientOptimisticUpdate) {.gcsafe, raises: [Defect].}
ImportLightClientData* {.pure.} = enum
## Controls which classes of light client data are imported.
@ -30,18 +32,17 @@ type
Full = "full"
## Import light client data for entire weak subjectivity period.
OnDemand = "on-demand"
## No precompute of historic data. Is slow and may miss validator duties.
## Don't precompute historic data. Slow, may miss validator duties.
CachedLightClientData* = object
## Cached data from historical non-finalized states to improve speed when
## creating future `LightClientUpdate` and `LightClientBootstrap` instances.
current_sync_committee_branch*:
array[log2trunc(altair.CURRENT_SYNC_COMMITTEE_INDEX), Eth2Digest]
next_sync_committee_branch*:
array[log2trunc(altair.NEXT_SYNC_COMMITTEE_INDEX), Eth2Digest]
finalized_bid*: BlockId
finalized_slot*: Slot
finality_branch*:
array[log2trunc(altair.FINALIZED_ROOT_INDEX), Eth2Digest]
@ -55,40 +56,26 @@ type
data*: Table[BlockId, CachedLightClientData]
## Cached data for creating future `LightClientUpdate` instances.
## Key is the block ID of which the post state was used to get the data.
## Data is stored for the most recent 4 finalized checkpoints, as well as
## for all non-finalized blocks.
## Data stored for the finalized head block and all non-finalized blocks.
bootstrap*: Table[Slot, CachedLightClientBootstrap]
## Cached data for creating future `LightClientBootstrap` instances.
## Key is the block slot of which the post state was used to get the data.
## Data is stored for finalized epoch boundary blocks.
## Data stored for all finalized epoch boundary blocks.
latestCheckpoints*: array[4, Checkpoint]
## Keeps track of the latest four `finalized_checkpoint` references
## leading to `finalizedHead`. Used to prune `data`.
## Non-finalized states may only refer to these checkpoints.
lastCheckpointIndex*: int
## Last index that was modified in `latestCheckpoints`.
bestUpdates*: Table[SyncCommitteePeriod, altair.LightClientUpdate]
best*: Table[SyncCommitteePeriod, altair.LightClientUpdate]
## Stores the `LightClientUpdate` with the most `sync_committee_bits` per
## `SyncCommitteePeriod`. Updates with a finality proof have precedence.
## `SyncCommitteePeriod`. Sync committee finality gives precedence.
pendingBestUpdates*:
pendingBest*:
Table[(SyncCommitteePeriod, Eth2Digest), altair.LightClientUpdate]
## Same as `bestUpdates`, but for `SyncCommitteePeriod` with
## `next_sync_committee` that are not finalized. Key is `(period,
## Same as `best`, but for `SyncCommitteePeriod` with not yet finalized
## `next_sync_committee`. Key is `(attested_period,
## hash_tree_root(current_sync_committee | next_sync_committee)`.
latestUpdate*: altair.LightClientUpdate
## Tracks the `LightClientUpdate` for the latest slot. This may be older
## than head for empty slots or if not signed by sync committee.
optimisticUpdate*: OptimisticLightClientUpdate
## Tracks the `OptimisticLightClientUpdate` for the latest slot. This may
## be older than head for empty slots or if not signed by sync committee.
latest*: altair.LightClientFinalityUpdate
## Tracks light client data for the latest slot that was signed by
## at least `MIN_SYNC_COMMITTEE_PARTICIPANTS`. May be older than head.
importTailSlot*: Slot
## The earliest slot for which light client data is collected.
## Only relevant for `ImportLightClientData.OnlyNew`.
## The earliest slot for which light client data is imported.

View File

@ -556,10 +556,10 @@ proc updateBeaconMetrics(
import blockchain_dag_light_client
export
blockchain_dag_light_client.getBestLightClientUpdateForPeriod,
blockchain_dag_light_client.getLatestLightClientUpdate,
blockchain_dag_light_client.getOptimisticLightClientUpdate,
blockchain_dag_light_client.getLightClientBootstrap
blockchain_dag_light_client.getLightClientBootstrap,
blockchain_dag_light_client.getLightClientUpdateForPeriod,
blockchain_dag_light_client.getLightClientFinalityUpdate,
blockchain_dag_light_client.getLightClientOptimisticUpdate
proc getViableHead(cfg: RuntimeConfig, db: BeaconChainDB): Opt[BlockId] =
# When the database has been written with a pre-fork version of the
@ -659,9 +659,9 @@ proc init*(T: type ChainDAGRef, cfg: RuntimeConfig, db: BeaconChainDB,
validatorMonitor: ref ValidatorMonitor, updateFlags: UpdateFlags,
eraPath = ".",
onBlockCb: OnBlockCallback = nil, onHeadCb: OnHeadCallback = nil,
onReorgCb: OnReorgCallback = nil,
onFinCb: OnFinalizedCallback = nil,
onOptimisticLCUpdateCb: OnOptimisticLightClientUpdateCallback = nil,
onReorgCb: OnReorgCallback = nil, onFinCb: OnFinalizedCallback = nil,
onLCFinalityUpdateCb: OnLightClientFinalityUpdateCallback = nil,
onLCOptimisticUpdateCb: OnLightClientOptimisticUpdateCallback = nil,
serveLightClientData = false,
importLightClientData = ImportLightClientData.None): ChainDAGRef =
# TODO move fork version sanity checking elsewhere?
@ -712,7 +712,8 @@ proc init*(T: type ChainDAGRef, cfg: RuntimeConfig, db: BeaconChainDB,
onHeadChanged: onHeadCb,
onReorgHappened: onReorgCb,
onFinHappened: onFinCb,
onOptimisticLightClientUpdate: onOptimisticLCUpdateCb
onLightClientFinalityUpdate: onLCFinalityUpdateCb,
onLightClientOptimisticUpdate: onLCOptimisticUpdateCb
)
loadTick = Moment.now()
@ -1636,6 +1637,7 @@ proc updateHead*(
justified = shortLog(getStateField(
dag.headState, current_justified_checkpoint)),
finalized = shortLog(getStateField(dag.headState, finalized_checkpoint))
let oldFinalizedHead = dag.finalizedHead
block:
# Update `dag.finalizedBlocks` with all newly finalized blocks (those
@ -1663,7 +1665,7 @@ proc updateHead*(
dag.pruneBlocksDAG()
# Update light client data
dag.processFinalizationForLightClient()
dag.processFinalizationForLightClient(oldFinalizedHead)
# Send notification about new finalization point via callback.
if not(isNil(dag.onFinHappened)):

View File

@ -0,0 +1,37 @@
# beacon_chain
# Copyright (c) 2022 Status Research & Development GmbH
# Licensed and distributed under either of
# * MIT license (license terms in the root directory or at https://opensource.org/licenses/MIT).
# * Apache v2 license (license terms in the root directory or at https://www.apache.org/licenses/LICENSE-2.0).
# at your option. This file may not be copied, modified, or distributed except according to those terms.
{.push raises: [Defect].}
# This implements the pre-release proposal of the libp2p based light client sync
# protocol. See https://github.com/ethereum/consensus-specs/pull/2802
import
# Status libraries
chronos,
# Beacon chain internals
../spec/datatypes/base
type
LightClientPool* = object
latestForwardedFinalitySlot*: Slot
## Latest finality update that was forwarded on libp2p gossip.
## Tracks `finality_update.finalized_header.slot`.
latestForwardedOptimisticSlot*: Slot
## Latest optimistic update that was forwarded on libp2p gossip.
## Tracks `optimistic_update.attested_header.slot`.
latestBroadcastedSlot*: Slot
## Latest slot for which updates were broadcasted on libp2p gossip.
## Tracks `update.signature_slot`.
broadcastGossipFut*: Future[void]
## Task to broadcast libp2p gossip. Started when a sync committee message
## is sent. Tracked separately from `handleValidatorDuties` to catch the
## case where `node.attachedValidators[].count == 0` at function start,
## and then a sync committee message gets sent from a remote VC via REST.

View File

@ -19,15 +19,15 @@ import
../spec/datatypes/[altair, phase0],
../consensus_object_pools/[
block_clearance, block_quarantine, blockchain_dag, exit_pool, attestation_pool,
sync_committee_msg_pool],
light_client_pool, sync_committee_msg_pool],
../validators/validator_pool,
../beacon_clock,
"."/[gossip_validation, block_processor, batch_validation]
export
results, bearssl, taskpools, block_clearance, blockchain_dag, exit_pool, attestation_pool,
sync_committee_msg_pool, validator_pool, beacon_clock, gossip_validation,
block_processor, batch_validation, block_quarantine
light_client_pool, sync_committee_msg_pool, validator_pool, beacon_clock,
gossip_validation, block_processor, batch_validation, block_quarantine
# Metrics for tracking attestation and beacon block loss
declareCounter beacon_attestations_received,
@ -62,10 +62,14 @@ declareCounter beacon_sync_committee_contributions_received,
"Number of valid sync committee contributions processed by this node"
declareCounter beacon_sync_committee_contributions_dropped,
"Number of invalid sync committee contributions dropped by this node", labels = ["reason"]
declareCounter beacon_optimistic_light_client_updates_received,
"Number of valid optimistic light client updates processed by this node"
declareCounter beacon_optimistic_light_client_updates_dropped,
"Number of invalid optimistic light client updates dropped by this node", labels = ["reason"]
declareCounter beacon_light_client_finality_updates_received,
"Number of valid LC finality updates processed by this node"
declareCounter beacon_light_client_finality_updates_dropped,
"Number of invalid LC finality updates dropped by this node", labels = ["reason"]
declareCounter beacon_light_client_optimistic_updates_received,
"Number of valid LC optimistic updates processed by this node"
declareCounter beacon_light_client_optimistic_updates_dropped,
"Number of invalid LC optimistic updates dropped by this node", labels = ["reason"]
const delayBuckets = [2.0, 4.0, 6.0, 8.0, 10.0, 12.0, 14.0, Inf]
@ -113,6 +117,7 @@ type
attestationPool*: ref AttestationPool
validatorPool: ref ValidatorPool
syncCommitteeMsgPool: ref SyncCommitteeMsgPool
lightClientPool: ref LightClientPool
doppelgangerDetection*: DoppelgangerProtection
@ -152,6 +157,7 @@ proc new*(T: type Eth2Processor,
exitPool: ref ExitPool,
validatorPool: ref ValidatorPool,
syncCommitteeMsgPool: ref SyncCommitteeMsgPool,
lightClientPool: ref LightClientPool,
quarantine: ref Quarantine,
rng: ref BrHmacDrbgContext,
getBeaconTime: GetBeaconTimeFn,
@ -169,6 +175,7 @@ proc new*(T: type Eth2Processor,
exitPool: exitPool,
validatorPool: validatorPool,
syncCommitteeMsgPool: syncCommitteeMsgPool,
lightClientPool: lightClientPool,
quarantine: quarantine,
getCurrentBeaconTime: getBeaconTime,
batchCrypto: BatchCrypto.new(
@ -538,23 +545,50 @@ proc contributionValidator*(
err(v.error())
# https://github.com/ethereum/consensus-specs/blob/vFuture/specs/altair/sync-protocol.md#optimistic_light_client_update
proc optimisticLightClientUpdateValidator*(
# https://github.com/ethereum/consensus-specs/blob/vFuture/specs/altair/sync-protocol.md#light_client_finality_update
proc lightClientFinalityUpdateValidator*(
self: var Eth2Processor, src: MsgSource,
optimistic_update: OptimisticLightClientUpdate
finality_update: altair.LightClientFinalityUpdate
): Result[void, ValidationError] =
logScope:
finality_update
debug "LC finality update received"
let
wallTime = self.getCurrentBeaconTime()
v = validateLightClientFinalityUpdate(
self.lightClientPool[], self.dag, finality_update, wallTime)
if v.isOk():
trace "LC finality update validated"
beacon_light_client_finality_updates_received.inc()
else:
debug "Dropping LC finality update", error = v.error
beacon_light_client_finality_updates_dropped.inc(1, [$v.error[0]])
v
# https://github.com/ethereum/consensus-specs/blob/vFuture/specs/altair/sync-protocol.md#light_client_optimistic_update
proc lightClientOptimisticUpdateValidator*(
self: var Eth2Processor, src: MsgSource,
optimistic_update: altair.LightClientOptimisticUpdate
): Result[void, ValidationError] =
logScope:
optimistic_update
debug "Optimistic light client update received"
debug "LC optimistic update received"
let v = self.dag.validateOptimisticLightClientUpdate(optimistic_update)
let
wallTime = self.getCurrentBeaconTime()
v = validateLightClientOptimisticUpdate(
self.lightClientPool[], self.dag, optimistic_update, wallTime)
if v.isOk():
trace "Optimistic light client update validated"
trace "LC optimistic update validated"
beacon_optimistic_light_client_updates_received.inc()
beacon_light_client_optimistic_updates_received.inc()
else:
debug "Dropping optimistic light client update", error = v.error
beacon_optimistic_light_client_updates_dropped.inc(1, [$v.error[0]])
debug "Dropping LC optimistic update", error = v.error
beacon_light_client_optimistic_updates_dropped.inc(1, [$v.error[0]])
v

View File

@ -21,7 +21,7 @@ import
beaconstate, state_transition_block, forks, helpers, network, signatures],
../consensus_object_pools/[
attestation_pool, blockchain_dag, block_quarantine, exit_pool, spec_cache,
sync_committee_msg_pool],
light_client_pool, sync_committee_msg_pool],
".."/[beacon_clock],
./batch_validation
@ -1032,19 +1032,58 @@ proc validateContribution*(
return ok((sig, participants))
# https://github.com/ethereum/consensus-specs/blob/vFuture/specs/altair/sync-protocol.md#optimistic_light_client_update
proc validateOptimisticLightClientUpdate*(
dag: ChainDAGRef, optimistic_update: OptimisticLightClientUpdate):
Result[void, ValidationError] =
template local_update(): auto = dag.lightClientCache.optimisticUpdate
# https://github.com/ethereum/consensus-specs/blob/vFuture/specs/altair/sync-protocol.md#light_client_finality_update
proc validateLightClientFinalityUpdate*(
pool: var LightClientPool, dag: ChainDAGRef,
finality_update: altair.LightClientFinalityUpdate,
wallTime: BeaconTime): Result[void, ValidationError] =
let finalized_slot = finality_update.finalized_header.slot
if finalized_slot <= pool.latestForwardedFinalitySlot:
# [IGNORE] No other `finality_update` with a lower or equal
# `finalized_header.slot` was already forwarded on the network.
return errIgnore("LightClientFinalityUpdate: slot already forwarded")
if optimistic_update != local_update:
# [IGNORE] The optimistic update is not attesting to the latest block's
# parent block.
if optimistic_update.attested_header != local_update.attested_header:
return errIgnore("OptimisticLightClientUpdate: not attesting to latest")
let
signature_slot = finality_update.signature_slot
currentTime = wallTime + MAXIMUM_GOSSIP_CLOCK_DISPARITY
forwardTime = signature_slot.light_client_finality_update_time
if currentTime < forwardTime:
# [IGNORE] The `finality_update` is received after the block at
# `signature_slot` was given enough time to propagate through the network.
return errIgnore("LightClientFinalityUpdate: received too early")
# [REJECT] The optimistic update does not match the expected value.
return errReject("OptimisticLightClientUpdate: not matching expected value")
if finality_update != dag.lightClientCache.latest:
# [IGNORE] The received `finality_update` matches the locally computed one
# exactly.
return errIgnore("LightClientFinalityUpdate: not matching local")
pool.latestForwardedFinalitySlot = finalized_slot
ok()
# https://github.com/ethereum/consensus-specs/blob/vFuture/specs/altair/sync-protocol.md#light_client_optimistic_update
proc validateLightClientOptimisticUpdate*(
pool: var LightClientPool, dag: ChainDAGRef,
optimistic_update: altair.LightClientOptimisticUpdate,
wallTime: BeaconTime): Result[void, ValidationError] =
let attested_slot = optimistic_update.attested_header.slot
if attested_slot <= pool.latestForwardedOptimisticSlot:
# [IGNORE] No other `optimistic_update` with a lower or equal
# `attested_header.slot` was already forwarded on the network.
return errIgnore("LightClientOptimisticUpdate: slot already forwarded")
let
signature_slot = optimistic_update.signature_slot
currentTime = wallTime + MAXIMUM_GOSSIP_CLOCK_DISPARITY
forwardTime = signature_slot.light_client_optimistic_update_time
if currentTime < forwardTime:
# [IGNORE] The `optimistic_update` is received after the block at
# `signature_slot` was given enough time to propagate through the network.
return errIgnore("LightClientOptimisticUpdate: received too early")
if not optimistic_update.matches(dag.lightClientCache.latest):
# [IGNORE] The received `optimistic_update` matches the locally computed one
# exactly.
return errIgnore("LightClientOptimisticUpdate: not matching local")
pool.latestForwardedOptimisticSlot = attested_slot
ok()

View File

@ -33,17 +33,21 @@ type
## This manages the processing of received light client objects
##
## from:
## - Gossip (`OptimisticLightClientUpdate`)
## - SyncManager (`BestLightClientUpdatesByRange`)
## - LightClientManager (`GetLatestLightClientUpdate`,
## `GetOptimisticLightClientUpdate`, `GetLightClientBootstrap`)
## - Gossip:
## - `LightClientFinalityUpdate`
## - `LightClientOptimisticUpdate`
## - `LightClientManager`:
## - `GetLightClientBootstrap`
## - `LightClientUpdatesByRange`
## - `GetLightClientFinalityUpdate`
## - `GetLightClientOptimisticUpdate`
##
## are then verified and added to:
## - `LightClientStore`
##
## The processor will also attempt to force-update the light client state
## if no update seems to be available on the network, that is both signed by
## a supermajority of sync committee members and also has a finality proof.
## a supermajority of sync committee members and also improves finality.
## This logic is triggered if there is no progress for an extended period
## of time, and there are repeated messages indicating that this is the best
## available data on the network during that time period.
@ -131,17 +135,17 @@ proc tryForceUpdate(
store = self.store
if store[].isSome:
case store[].get.process_slot_for_light_client_store(wallSlot)
case store[].get.try_light_client_store_force_update(wallSlot)
of NoUpdate:
discard
of UpdatedWithoutSupermajority:
warn "Light client force-update without supermajority",
localHeadSlot = store[].get.optimistic_header.slot,
finalized = store[].get.finalized_header
of UpdatedWithoutFinalityProof:
warn "Light client force-update without finality proof",
localHeadSlot = store[].get.optimistic_header.slot,
finalized = store[].get.finalized_header
of DidUpdateWithoutSupermajority:
warn "Light client force-updated without supermajority",
finalizedSlot = store[].get.finalized_header.slot,
optimisticSlot = store[].get.optimistic_header.slot
of DidUpdateWithoutFinality:
warn "Light client force-updated without finality proof",
finalizedSlot = store[].get.finalized_header.slot,
optimisticSlot = store[].get.optimistic_header.slot
proc storeObject*(
self: var LightClientProcessor,
@ -167,18 +171,11 @@ proc storeObject*(
else:
store[] = some(initRes.get)
ok()
elif obj is altair.LightClientUpdate:
elif obj is SomeLightClientUpdate:
if store[].isNone:
err(BlockError.MissingParent)
else:
store[].get.process_light_client_update(
obj, wallSlot, self.cfg, self.genesis_validators_root,
allowForceUpdate = false)
elif obj is altair.OptimisticLightClientUpdate:
if store[].isNone:
err(BlockError.MissingParent)
else:
store[].get.process_optimistic_light_client_update(
obj, wallSlot, self.cfg, self.genesis_validators_root)
self.dumpObject(obj, res)
@ -186,27 +183,15 @@ proc storeObject*(
if res.isErr:
when obj is altair.LightClientUpdate:
if store[].isSome and store[].get.best_valid_update.isSome:
# `best_valid_update` gets set when no supermajority / finality proof
# is available. In that case, we will wait for a better update.
# If none is made available within reasonable time, the light client
# is force-updated using the best known data to ensure sync progress.
# `best_valid_update` gets set when no supermajority / improved finality
# is available. In that case, we will wait for a better update that once
# again fulfills those conditions. If none is received within reasonable
# time, the light client store is force-updated to `best_valid_update`.
case res.error
of BlockError.Duplicate:
if wallTime >= self.lastDuplicateTick + duplicateRateLimit:
if self.numDuplicatesSinceProgress < minForceUpdateDuplicates:
let
finalized_period =
store[].get.finalized_header.slot.sync_committee_period
update_period =
obj.get_active_header().slot.sync_committee_period
is_next_sync_committee_known =
not store[].get.next_sync_committee.isZeroMemory
update_can_advance_period =
if is_next_sync_committee_known:
update_period == finalized_period + 1
else:
update_period == finalized_period
if update_can_advance_period:
if obj.matches(store[].get.best_valid_update.get):
self.lastDuplicateTick = wallTime
inc self.numDuplicatesSinceProgress
if self.numDuplicatesSinceProgress >= minForceUpdateDuplicates and
@ -234,10 +219,14 @@ proc storeObject*(
let objSlot =
when obj is altair.LightClientBootstrap:
obj.header.slot
elif obj is SomeLightClientUpdateWithFinality:
obj.finalized_header.slot
else:
obj.attested_header.slot
debug "Light client object processed", kind = typeof(obj).name,
localHeadSlot = store[].get.optimistic_header.slot,
debug "Light client object processed",
finalizedSlot = store[].get.finalized_header.slot,
optimisticSlot = store[].get.optimistic_header.slot,
kind = typeof(obj).name,
objectSlot = objSlot,
storeObjectDur
@ -261,10 +250,14 @@ proc addObject*(
# Only one object is validated at any time -
# Light client objects are always "fast" to process
# Producers:
# - Gossip (`OptimisticLightClientUpdate`)
# - SyncManager (`BestLightClientUpdatesByRange`)
# - LightClientManager (`GetLatestLightClientUpdate`,
# `GetOptimisticLightClientUpdate`, `GetLightClientBootstrap`)
# - Gossip:
# - `LightClientFinalityUpdate`
# - `LightClientOptimisticUpdate`
# - `LightClientManager`:
# - `GetLightClientBootstrap`
# - `LightClientUpdatesByRange`
# - `GetLightClientFinalityUpdate`
# - `GetLightClientOptimisticUpdate`
let
wallTime = self.getBeaconTime()

View File

@ -619,8 +619,10 @@ proc sendResponseChunkBytes(
inc response.writtenChunks
response.stream.writeChunk(some Success, payload, contextBytes)
proc sendResponseChunk*(response: UntypedResponse, val: auto): Future[void] =
sendResponseChunkBytes(response, SSZ.encode(val))
proc sendResponseChunk*(
response: UntypedResponse, val: auto,
contextBytes: openArray[byte] = []): Future[void] =
sendResponseChunkBytes(response, SSZ.encode(val), contextBytes)
template sendUserHandlerResultAsChunkImpl*(stream: Connection,
handlerResultFut: Future): untyped =
@ -862,19 +864,23 @@ proc init*[MsgType](T: type SingleChunkResponse[MsgType],
peer: Peer, conn: Connection): T =
T(UntypedResponse(peer: peer, stream: conn))
template write*[M](r: MultipleChunksResponse[M], val: M): untyped =
template write*[M](
r: MultipleChunksResponse[M], val: M,
contextBytes: openArray[byte] = []): untyped =
mixin sendResponseChunk
sendResponseChunk(UntypedResponse(r), val)
sendResponseChunk(UntypedResponse(r), val, contextBytes)
template writeBytesSZ*[M](
r: MultipleChunksResponse[M], uncompressedLen: uint64,
bytes: openArray[byte], contextBytes: openArray[byte]): untyped =
sendResponseChunkBytesSZ(UntypedResponse(r), uncompressedLen, bytes, contextBytes)
template send*[M](r: SingleChunkResponse[M], val: M): untyped =
template send*[M](
r: SingleChunkResponse[M], val: M,
contextBytes: openArray[byte] = []): untyped =
mixin sendResponseChunk
doAssert UntypedResponse(r).writtenChunks == 0
sendResponseChunk(UntypedResponse(r), val)
sendResponseChunk(UntypedResponse(r), val, contextBytes)
proc performProtocolHandshakes*(peer: Peer, incoming: bool) {.async.} =
# Loop down serially because it's easier to reason about the connection state
@ -2551,16 +2557,14 @@ proc broadcastSignedContributionAndProof*(
node.forkDigestAtEpoch(node.getWallEpoch))
node.broadcast(topic, msg)
proc broadcastOptimisticLightClientUpdate*(
node: Eth2Node, msg: OptimisticLightClientUpdate) =
let
forkDigest =
if msg.fork_version == node.cfg.SHARDING_FORK_VERSION:
node.forkDigests.sharding
elif msg.fork_version == node.cfg.BELLATRIX_FORK_VERSION:
node.forkDigests.bellatrix
else:
doAssert msg.fork_version == node.cfg.ALTAIR_FORK_VERSION
node.forkDigests.altair
topic = getOptimisticLightClientUpdateTopic(forkDigest)
proc broadcastLightClientFinalityUpdate*(
node: Eth2Node, msg: altair.LightClientFinalityUpdate) =
let topic = getLightClientFinalityUpdateTopic(
node.forkDigestAtEpoch(msg.attested_header.slot.epoch))
node.broadcast(topic, msg)
proc broadcastLightClientOptimisticUpdate*(
node: Eth2Node, msg: altair.LightClientOptimisticUpdate) =
let topic = getLightClientOptimisticUpdateTopic(
node.forkDigestAtEpoch(msg.attested_header.slot.epoch))
node.broadcast(topic, msg)

View File

@ -158,20 +158,26 @@ proc loadChainDag(
eventBus.emit("head-change", data)
proc onChainReorg(data: ReorgInfoObject) =
eventBus.emit("chain-reorg", data)
proc onOptimisticLightClientUpdate(data: OptimisticLightClientUpdate) =
proc onLightClientFinalityUpdate(data: altair.LightClientFinalityUpdate) =
discard
proc onLightClientOptimisticUpdate(data: altair.LightClientOptimisticUpdate) =
discard
let
chainDagFlags =
if config.verifyFinalization: {verifyFinalization}
else: {}
onOptimisticLightClientUpdateCb =
if config.serveLightClientData.get: onOptimisticLightClientUpdate
onLightClientFinalityUpdateCb =
if config.serveLightClientData.get: onLightClientFinalityUpdate
else: nil
onLightClientOptimisticUpdateCb =
if config.serveLightClientData.get: onLightClientOptimisticUpdate
else: nil
dag = ChainDAGRef.init(
cfg, db, validatorMonitor, chainDagFlags, config.eraDir,
onBlockAdded, onHeadChanged, onChainReorg,
onOptimisticLCUpdateCb = onOptimisticLightClientUpdateCb,
onLCFinalityUpdateCb = onLightClientFinalityUpdateCb,
onLCOptimisticUpdateCb = onLightClientOptimisticUpdateCb,
serveLightClientData = config.serveLightClientData.get,
importLightClientData = config.importLightClientData.get)
databaseGenesisValidatorsRoot =
@ -252,6 +258,8 @@ proc initFullNode(
AttestationPool.init(dag, quarantine, onAttestationReceived))
syncCommitteeMsgPool = newClone(
SyncCommitteeMsgPool.init(rng, onSyncContribution))
lightClientPool = newClone(
LightClientPool())
exitPool = newClone(
ExitPool.init(dag, onVoluntaryExitAdded))
consensusManager = ConsensusManager.new(
@ -271,8 +279,8 @@ proc initFullNode(
processor = Eth2Processor.new(
config.doppelgangerDetection,
blockProcessor, node.validatorMonitor, dag, attestationPool, exitPool,
node.attachedValidators, syncCommitteeMsgPool, quarantine, rng,
getBeaconTime, taskpool)
node.attachedValidators, syncCommitteeMsgPool, lightClientPool,
quarantine, rng, getBeaconTime, taskpool)
syncManager = newSyncManager[Peer, PeerId](
node.network.peerPool, SyncQueueKind.Forward, getLocalHeadSlot,
getLocalWallSlot, getFirstSlotAtFinalizedEpoch, getBackfillSlot,
@ -288,6 +296,7 @@ proc initFullNode(
node.quarantine = quarantine
node.attestationPool = attestationPool
node.syncCommitteeMsgPool = syncCommitteeMsgPool
node.lightClientPool = lightClientPool
node.exitPool = exitPool
node.processor = processor
node.blockProcessor = blockProcessor
@ -866,7 +875,9 @@ proc addAltairMessageHandlers(node: BeaconNode, forkDigest: ForkDigest, slot: Sl
if node.config.serveLightClientData.get:
node.network.subscribe(
getOptimisticLightClientUpdateTopic(forkDigest), basicParams)
getLightClientFinalityUpdateTopic(forkDigest), basicParams)
node.network.subscribe(
getLightClientOptimisticUpdateTopic(forkDigest), basicParams)
proc removeAltairMessageHandlers(node: BeaconNode, forkDigest: ForkDigest) =
node.removePhase0MessageHandlers(forkDigest)
@ -880,7 +891,10 @@ proc removeAltairMessageHandlers(node: BeaconNode, forkDigest: ForkDigest) =
getSyncCommitteeContributionAndProofTopic(forkDigest))
if node.config.serveLightClientData.get:
node.network.unsubscribe(getOptimisticLightClientUpdateTopic(forkDigest))
node.network.unsubscribe(
getLightClientFinalityUpdateTopic(forkDigest))
node.network.unsubscribe(
getLightClientOptimisticUpdateTopic(forkDigest))
proc trackCurrentSyncCommitteeTopics(node: BeaconNode, slot: Slot) =
# Unlike trackNextSyncCommitteeTopics, just snap to the currently correct
@ -1395,20 +1409,29 @@ proc installMessageValidators(node: BeaconNode) =
installSyncCommitteeeValidators(forkDigests.altair)
installSyncCommitteeeValidators(forkDigests.bellatrix)
template installOptimisticLightClientUpdateValidator(digest: auto) =
template installLightClientDataValidators(digest: auto) =
node.network.addValidator(
getOptimisticLightClientUpdateTopic(digest),
proc(msg: OptimisticLightClientUpdate): ValidationResult =
getLightClientFinalityUpdateTopic(digest),
proc(msg: altair.LightClientFinalityUpdate): ValidationResult =
if node.config.serveLightClientData.get:
toValidationResult(
node.processor[].optimisticLightClientUpdateValidator(
node.processor[].lightClientFinalityUpdateValidator(
MsgSource.gossip, msg))
else:
debug "Ignoring optimistic light client update: Feature disabled"
ValidationResult.Ignore)
installOptimisticLightClientUpdateValidator(forkDigests.altair)
installOptimisticLightClientUpdateValidator(forkDigests.bellatrix)
node.network.addValidator(
getLightClientOptimisticUpdateTopic(digest),
proc(msg: altair.LightClientOptimisticUpdate): ValidationResult =
if node.config.serveLightClientData.get:
toValidationResult(
node.processor[].lightClientOptimisticUpdateValidator(
MsgSource.gossip, msg))
else:
ValidationResult.Ignore)
installLightClientDataValidators(forkDigests.altair)
installLightClientDataValidators(forkDigests.bellatrix)
proc stop(node: BeaconNode) =
bnStatus = BeaconNodeStatus.Stopping

View File

@ -150,8 +150,11 @@ const
# https://github.com/ethereum/consensus-specs/blob/v1.1.10/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/vFuture/specs/altair/sync-protocol.md#block-proposal
optimisticLightClientUpdateSlotOffset* = TimeDiff(nanoseconds:
# https://github.com/ethereum/consensus-specs/blob/vFuture/specs/altair/sync-protocol.md#light_client_finality_update
lightClientFinalityUpdateSlotOffset* = TimeDiff(nanoseconds:
NANOSECONDS_PER_SLOT.int64 div INTERVALS_PER_SLOT)
# https://github.com/ethereum/consensus-specs/blob/vFuture/specs/altair/sync-protocol.md#light_client_optimistic_update
lightClientOptimisticUpdateSlotOffset* = TimeDiff(nanoseconds:
NANOSECONDS_PER_SLOT.int64 div INTERVALS_PER_SLOT)
func toFloatSeconds*(t: TimeDiff): float =
@ -174,8 +177,10 @@ func sync_committee_message_deadline*(s: Slot): BeaconTime =
s.start_beacon_time + syncCommitteeMessageSlotOffset
func sync_contribution_deadline*(s: Slot): BeaconTime =
s.start_beacon_time + syncContributionSlotOffset
func optimistic_light_client_update_time*(s: Slot): BeaconTime =
s.start_beacon_time + optimisticLightClientUpdateSlotOffset
func light_client_finality_update_time*(s: Slot): BeaconTime =
s.start_beacon_time + lightClientFinalityUpdateSlotOffset
func light_client_optimistic_update_time*(s: Slot): BeaconTime =
s.start_beacon_time + lightClientOptimisticUpdateSlotOffset
func slotOrZero*(time: BeaconTime): Slot =
let exSlot = time.toSlot

View File

@ -160,53 +160,74 @@ type
# https://github.com/ethereum/consensus-specs/blob/vFuture/specs/altair/sync-protocol.md#lightclientbootstrap
LightClientBootstrap* = object
header*: BeaconBlockHeader ##\
## The requested beacon block header
# The requested beacon block header
header*: BeaconBlockHeader
# Current sync committee corresponding to the requested header
# Current sync committee corresponding to `header`
current_sync_committee*: SyncCommittee
current_sync_committee_branch*:
array[log2trunc(CURRENT_SYNC_COMMITTEE_INDEX), Eth2Digest]
# https://github.com/ethereum/consensus-specs/blob/vFuture/specs/altair/sync-protocol.md#lightclientupdate
LightClientUpdate* = object
attested_header*: BeaconBlockHeader ##\
## The beacon block header that is attested to by the sync committee
# The beacon block header that is attested to by the sync committee
attested_header*: BeaconBlockHeader
# Next sync committee corresponding to the active header,
# if signature is from current sync committee
# Next sync committee corresponding to `attested_header`
next_sync_committee*: SyncCommittee
next_sync_committee_branch*:
array[log2trunc(NEXT_SYNC_COMMITTEE_INDEX), Eth2Digest]
# The finalized beacon block header attested to by Merkle branch
finalized_header*: BeaconBlockHeader
finality_branch*: array[log2trunc(FINALIZED_ROOT_INDEX), Eth2Digest]
finality_branch*:
array[log2trunc(FINALIZED_ROOT_INDEX), Eth2Digest]
# Sync committee aggregate signature
sync_aggregate*: SyncAggregate
# Slot at which the aggregate signature was created (untrusted)
signature_slot*: Slot
fork_version*: Version ##\
## Fork version for the aggregate signature
# https://github.com/ethereum/consensus-specs/blob/vFuture/specs/altair/sync-protocol.md#lightclientfinalityupdate
LightClientFinalityUpdate* = object
# The beacon block header that is attested to by the sync committee
attested_header*: BeaconBlockHeader
# https://github.com/ethereum/consensus-specs/blob/vFuture/specs/altair/sync-protocol.md#optimisticlightclientupdate
OptimisticLightClientUpdate* = object
attested_header*: BeaconBlockHeader ##\
## The beacon block header that is attested to by the sync committee
# The finalized beacon block header attested to by Merkle branch
finalized_header*: BeaconBlockHeader
finality_branch*:
array[log2trunc(FINALIZED_ROOT_INDEX), Eth2Digest]
sync_aggregate*: SyncAggregate ##\
## Sync committee aggregate signature
# Sync committee aggregate signature
sync_aggregate*: SyncAggregate
# Slot at which the aggregate signature was created (untrusted)
signature_slot*: Slot
fork_version*: Version ##\
## Fork version for the aggregate signature
# https://github.com/ethereum/consensus-specs/blob/vFuture/specs/altair/sync-protocol.md#lightclientoptimisticupdate
LightClientOptimisticUpdate* = object
# The beacon block header that is attested to by the sync committee
attested_header*: BeaconBlockHeader
is_signed_by_next_sync_committee*: bool ##\
## Whether the signature was produced by `attested_header`'s next sync committee
# Sync committee aggregate signature
sync_aggregate*: SyncAggregate
# Slot at which the aggregate signature was created (untrusted)
signature_slot*: Slot
SomeLightClientUpdateWithSyncCommittee* =
LightClientUpdate
SomeLightClientUpdateWithFinality* =
LightClientUpdate |
LightClientFinalityUpdate
SomeLightClientUpdate* =
LightClientUpdate |
LightClientFinalityUpdate |
LightClientOptimisticUpdate
SomeLightClientObject* =
LightClientBootstrap |
LightClientUpdate |
OptimisticLightClientUpdate
SomeLightClientUpdate
# https://github.com/ethereum/consensus-specs/blob/v1.1.10/specs/altair/sync-protocol.md#lightclientstore
LightClientStore* = object
@ -612,37 +633,104 @@ chronicles.formatIt SyncCommitteeContribution: shortLog(it)
chronicles.formatIt ContributionAndProof: shortLog(it)
chronicles.formatIt SignedContributionAndProof: shortLog(it)
template hash*(x: LightClientUpdate): Hash =
hash(x.header)
func shortLog*(v: LightClientBootstrap): auto =
(
header: shortLog(v.header)
)
func shortLog*(v: LightClientUpdate): auto =
# `next_sync_committee` is set when the current sync committee is signing.
# When the next sync committee is signing instead, this field is kept empty,
# as it cannot be verified without already knowing the next sync committee.
# https://github.com/ethereum/consensus-specs/blob/vFuture/specs/altair/sync-protocol.md#lightclientupdate
let is_signed_by_next_sync_committee = v.next_sync_committee.isZeroMemory
(
attested: shortLog(v.attested_header),
has_next_sync_committee: not v.next_sync_committee.isZeroMemory,
finalized: shortLog(v.finalized_header),
num_active_participants: countOnes(v.sync_aggregate.sync_committee_bits),
signature_slot: v.signature_slot
)
func shortLog*(v: LightClientFinalityUpdate): auto =
(
attested: shortLog(v.attested_header),
finalized: shortLog(v.finalized_header),
num_active_participants: countOnes(v.sync_aggregate.sync_committee_bits),
is_signed_by_next: is_signed_by_next_sync_committee
signature_slot: v.signature_slot
)
func shortLog*(v: OptimisticLightClientUpdate): auto =
func shortLog*(v: LightClientOptimisticUpdate): auto =
(
attested_header: shortLog(v.attested_header),
attested: shortLog(v.attested_header),
num_active_participants: countOnes(v.sync_aggregate.sync_committee_bits),
is_signed_by_next: v.is_signed_by_next_sync_committee
signature_slot: v.signature_slot,
)
chronicles.formatIt LightClientBootstrap: shortLog(it)
chronicles.formatIt LightClientUpdate: shortLog(it)
chronicles.formatIt OptimisticLightClientUpdate: shortLog(it)
chronicles.formatIt LightClientFinalityUpdate: shortLog(it)
chronicles.formatIt LightClientOptimisticUpdate: shortLog(it)
template toFull*(
update: SomeLightClientUpdate): LightClientUpdate =
when update is LightClientUpdate:
update
elif update is SomeLightClientUpdateWithFinality:
LightClientUpdate(
attested_header: update.attested_header,
finalized_header: update.finalized_header,
finality_branch: update.finality_branch,
sync_aggregate: update.sync_aggregate,
signature_slot: update.signature_slot)
else:
LightClientUpdate(
attested_header: update.attested_header,
sync_aggregate: update.sync_aggregate,
signature_slot: update.signature_slot)
template toFinality*(
update: SomeLightClientUpdate): LightClientFinalityUpdate =
when update is LightClientFinalityUpdate:
update
elif update is SomeLightClientUpdateWithFinality:
LightClientFinalityUpdate(
attested_header: update.attested_header,
finalized_header: update.finalized_header,
finality_branch: update.finality_branch,
sync_aggregate: update.sync_aggregate,
signature_slot: update.signature_slot)
else:
LightClientFinalityUpdate(
attested_header: update.attested_header,
sync_aggregate: update.sync_aggregate,
signature_slot: update.signature_slot)
template toOptimistic*(
update: SomeLightClientUpdate): LightClientOptimisticUpdate =
when update is LightClientOptimisticUpdate:
update
else:
LightClientOptimisticUpdate(
attested_header: update.attested_header,
sync_aggregate: update.sync_aggregate,
signature_slot: update.signature_slot)
func matches*[A, B: SomeLightClientUpdate](a: A, b: B): bool =
if a.attested_header != b.attested_header:
return false
when a is SomeLightClientUpdateWithSyncCommittee and
b is SomeLightClientUpdateWithSyncCommittee:
if a.next_sync_committee != b.next_sync_committee:
return false
if a.next_sync_committee_branch != b.next_sync_committee_branch:
return false
when a is SomeLightClientUpdateWithFinality and
b is SomeLightClientUpdateWithFinality:
if a.finalized_header != b.finalized_header:
return false
if a.finality_branch != b.finality_branch:
return false
if a.sync_aggregate != b.sync_aggregate:
return false
if a.signature_slot != b.signature_slot:
return false
true
func clear*(info: var EpochInfo) =
info.validators.setLen(0)

View File

@ -330,15 +330,26 @@ func stateForkAtEpoch*(cfg: RuntimeConfig, epoch: Epoch): BeaconStateFork =
doAssert BeaconStateFork.Altair > BeaconStateFork.Phase0
doAssert GENESIS_EPOCH == 0
if epoch >= cfg.BELLATRIX_FORK_EPOCH: BeaconStateFork.Bellatrix
elif epoch >= cfg.ALTAIR_FORK_EPOCH: BeaconStateFork.Altair
else: BeaconStateFork.Phase0
if epoch >= cfg.BELLATRIX_FORK_EPOCH: BeaconStateFork.Bellatrix
elif epoch >= cfg.ALTAIR_FORK_EPOCH: BeaconStateFork.Altair
else: BeaconStateFork.Phase0
func blockForkAtEpoch*(cfg: RuntimeConfig, epoch: Epoch): BeaconBlockFork =
## Return the current fork for the given epoch.
if epoch >= cfg.BELLATRIX_FORK_EPOCH: BeaconBlockFork.Bellatrix
elif epoch >= cfg.ALTAIR_FORK_EPOCH: BeaconBlockFork.Altair
else: BeaconBlockFork.Phase0
if epoch >= cfg.BELLATRIX_FORK_EPOCH: BeaconBlockFork.Bellatrix
elif epoch >= cfg.ALTAIR_FORK_EPOCH: BeaconBlockFork.Altair
else: BeaconBlockFork.Phase0
func stateForkForDigest*(
forkDigests: ForkDigests, forkDigest: ForkDigest): Opt[BeaconStateFork] =
if forkDigest == forkDigests.bellatrix:
ok BeaconStateFork.Bellatrix
elif forkDigest == forkDigests.altair:
ok BeaconStateFork.Altair
elif forkDigest == forkDigests.phase0:
ok BeaconStateFork.Phase0
else:
err()
template asSigned*(x: ForkedTrustedSignedBeaconBlock): ForkedSignedBeaconBlock =
isomorphicCast[ForkedSignedBeaconBlock](x)

View File

@ -9,11 +9,15 @@
{.push raises: [Defect].}
# References to `vFuture` refer to the pre-release proposal of the libp2p based
# light client sync protocol. Conflicting release versions are not in use.
# https://github.com/ethereum/consensus-specs/pull/2802
import
# Standard lib
std/[algorithm, math, sequtils, sets, tables],
# Status libraries
stew/[bitops2, byteutils, endians2],
stew/[bitops2, byteutils, endians2, objects],
chronicles,
# Internal
./datatypes/[phase0, altair, bellatrix],
@ -475,11 +479,119 @@ func has_flag*(flags: ParticipationFlags, flag_index: int): bool =
let flag = ParticipationFlags(1'u8 shl flag_index)
(flags and flag) == flag
# https://github.com/ethereum/consensus-specs/blob/vFuture/specs/altair/sync-protocol.md#is_sync_committee_update
template is_sync_committee_update*(update: SomeLightClientUpdate): bool =
when update is SomeLightClientUpdateWithSyncCommittee:
not isZeroMemory(update.next_sync_committee_branch)
else:
false
# https://github.com/ethereum/consensus-specs/blob/v1.1.10/specs/altair/sync-protocol.md#get_active_header
template is_finality_update*(update: SomeLightClientUpdate): bool =
when update is SomeLightClientUpdateWithFinality:
not isZeroMemory(update.finality_branch)
else:
false
# https://github.com/ethereum/consensus-specs/blob/v1.1.10/specs/altair/sync-protocol.md#get_subtree_index
func get_subtree_index*(idx: GeneralizedIndex): uint64 =
doAssert idx > 0
uint64(idx mod (type(idx)(1) shl log2trunc(idx)))
# https://github.com/ethereum/consensus-specs/blob/vFuture/specs/altair/sync-protocol.md#is_next_sync_committee_known
template is_next_sync_committee_known*(store: LightClientStore): bool =
not isZeroMemory(store.next_sync_committee)
# https://github.com/ethereum/consensus-specs/blob/v1.1.10/specs/altair/sync-protocol.md#get_safety_threshold
func get_safety_threshold*(store: LightClientStore): uint64 =
max(
store.previous_max_active_participants,
store.current_max_active_participants
) div 2
# https://github.com/ethereum/consensus-specs/blob/vFuture/specs/altair/sync-protocol.md#is_better_update
type LightClientUpdateMetadata* = object
attested_slot*, finalized_slot*, signature_slot*: Slot
has_sync_committee*, has_finality*: bool
num_active_participants*: uint64
func toMeta*(update: SomeLightClientUpdate): LightClientUpdateMetadata =
var meta {.noinit.}: LightClientUpdateMetadata
meta.attested_slot =
update.attested_header.slot
meta.finalized_slot =
when update is SomeLightClientUpdateWithFinality:
update.finalized_header.slot
else:
GENESIS_SLOT
meta.signature_slot =
update.signature_slot
meta.has_sync_committee =
when update is SomeLightClientUpdateWithSyncCommittee:
not update.next_sync_committee_branch.isZeroMemory
else:
false
meta.has_finality =
when update is SomeLightClientUpdateWithFinality:
not update.finality_branch.isZeroMemory
else:
false
meta.num_active_participants =
countOnes(update.sync_aggregate.sync_committee_bits).uint64
meta
func is_better_data*(new_meta, old_meta: LightClientUpdateMetadata): bool =
# Compare supermajority (> 2/3) sync committee participation
const max_active_participants = SYNC_COMMITTEE_SIZE.uint64
let
new_has_supermajority =
new_meta.num_active_participants * 3 >= max_active_participants * 2
old_has_supermajority =
old_meta.num_active_participants * 3 >= max_active_participants * 2
if new_has_supermajority != old_has_supermajority:
return new_has_supermajority > old_has_supermajority
if not new_has_supermajority:
if new_meta.num_active_participants != old_meta.num_active_participants:
return new_meta.num_active_participants > old_meta.num_active_participants
# Compare presence of relevant sync committee
let
new_has_relevant_sync_committee = new_meta.has_sync_committee and
new_meta.attested_slot.sync_committee_period ==
new_meta.signature_slot.sync_committee_period
old_has_relevant_sync_committee = old_meta.has_sync_committee and
old_meta.attested_slot.sync_committee_period ==
old_meta.signature_slot.sync_committee_period
if new_has_relevant_sync_committee != old_has_relevant_sync_committee:
return new_has_relevant_sync_committee > old_has_relevant_sync_committee
# Compare indication of any finality
if new_meta.has_finality != old_meta.has_finality:
return new_meta.has_finality > old_meta.has_finality
# Compare sync committee finality
if new_meta.has_finality:
let
new_has_sync_committee_finality =
new_meta.finalized_slot.sync_committee_period ==
new_meta.attested_slot.sync_committee_period
old_has_sync_committee_finality =
old_meta.finalized_slot.sync_committee_period ==
old_meta.attested_slot.sync_committee_period
if new_has_sync_committee_finality != old_has_sync_committee_finality:
return new_has_sync_committee_finality > old_has_sync_committee_finality
# Tiebreaker 1: Sync committee participation beyond supermajority
if new_meta.num_active_participants != old_meta.num_active_participants:
return new_meta.num_active_participants > old_meta.num_active_participants
# Tiebreaker 2: Prefer older data (fewer changes to best data)
new_meta.attested_slot < old_meta.attested_slot
template is_better_update*[A, B: SomeLightClientUpdate](
new_update: A, old_update: B): bool =
is_better_data(toMeta(new_update), toMeta(old_update))
# https://github.com/ethereum/consensus-specs/blob/v1.1.10/specs/bellatrix/beacon-chain.md#is_merge_transition_complete
func is_merge_transition_complete*(state: bellatrix.BeaconState): bool =
const defaultExecutionPayloadHeader = default(ExecutionPayloadHeader)

View File

@ -17,53 +17,7 @@ import
helpers
from ../consensus_object_pools/block_pools_types import BlockError
func period_contains_fork_version(
cfg: RuntimeConfig,
period: SyncCommitteePeriod,
fork_version: Version): bool =
## Determine whether a given `fork_version` is used during a given `period`.
let
periodStartEpoch = period.start_epoch
periodEndEpoch = periodStartEpoch + EPOCHS_PER_SYNC_COMMITTEE_PERIOD - 1
return
if fork_version == cfg.SHARDING_FORK_VERSION:
periodEndEpoch >= cfg.SHARDING_FORK_EPOCH
elif fork_version == cfg.BELLATRIX_FORK_VERSION:
periodStartEpoch < cfg.SHARDING_FORK_EPOCH and
cfg.SHARDING_FORK_EPOCH != cfg.BELLATRIX_FORK_EPOCH and
periodEndEpoch >= cfg.BELLATRIX_FORK_EPOCH
elif fork_version == cfg.ALTAIR_FORK_VERSION:
periodStartEpoch < cfg.BELLATRIX_FORK_EPOCH and
cfg.BELLATRIX_FORK_EPOCH != cfg.ALTAIR_FORK_EPOCH and
periodEndEpoch >= cfg.ALTAIR_FORK_EPOCH
elif fork_version == cfg.GENESIS_FORK_VERSION:
# Light client sync protocol requires Altair
false
else:
# Unviable fork
false
# https://github.com/ethereum/consensus-specs/blob/v1.1.10/specs/altair/sync-protocol.md#get_active_header
func is_finality_update*(update: altair.LightClientUpdate): bool =
not update.finalized_header.isZeroMemory
# https://github.com/ethereum/consensus-specs/blob/v1.1.10/specs/altair/sync-protocol.md#get_active_header
func get_active_header*(update: altair.LightClientUpdate): BeaconBlockHeader =
# The "active header" is the header that the update is trying to convince
# us to accept. If a finalized header is present, it's the finalized
# header, otherwise it's the attested header
if update.is_finality_update:
update.finalized_header
else:
update.attested_header
# https://github.com/ethereum/consensus-specs/blob/v1.1.10/specs/altair/sync-protocol.md#get_safety_threshold
func get_safety_threshold(store: LightClientStore): uint64 =
max(
store.previous_max_active_participants,
store.current_max_active_participants
) div 2
export block_pools_types.BlockError
# https://github.com/ethereum/consensus-specs/blob/vFuture/specs/altair/sync-protocol.md#initialize_light_client_store
func initialize_light_client_store*(
@ -89,7 +43,7 @@ func initialize_light_client_store*(
# https://github.com/ethereum/consensus-specs/blob/vFuture/specs/altair/sync-protocol.md#validate_light_client_update
proc validate_light_client_update*(
store: LightClientStore,
update: altair.LightClientUpdate,
update: SomeLightClientUpdate,
current_slot: Slot,
cfg: RuntimeConfig,
genesis_validators_root: Eth2Digest): Result[void, BlockError] =
@ -100,88 +54,81 @@ proc validate_light_client_update*(
if num_active_participants < MIN_SYNC_COMMITTEE_PARTICIPANTS:
return err(BlockError.Invalid)
# Determine update header
template attested_header(): auto = update.attested_header
if current_slot < attested_header.slot:
return err(BlockError.UnviableFork)
let active_header = get_active_header(update)
if attested_header.slot < active_header.slot:
# Verify update does not skip a sync committee period
when update is SomeLightClientUpdateWithFinality:
if update.attested_header.slot < update.finalized_header.slot:
return err(BlockError.Invalid)
if update.signature_slot <= update.attested_header.slot:
return err(BlockError.Invalid)
if current_slot < update.signature_slot:
return err(BlockError.UnviableFork)
let
store_period = store.finalized_header.slot.sync_committee_period
signature_period = update.signature_slot.sync_committee_period
is_next_sync_committee_known = store.is_next_sync_committee_known
if is_next_sync_committee_known:
if signature_period notin [store_period, store_period + 1]:
return err(BlockError.MissingParent)
else:
if signature_period != store_period:
return err(BlockError.MissingParent)
# Verify update is relevant
let is_next_sync_committee_known = not store.next_sync_committee.isZeroMemory
if is_next_sync_committee_known:
if active_header.slot < store.finalized_header.slot:
return err(BlockError.Duplicate)
if active_header.slot == store.finalized_header.slot:
if attested_header.slot <= store.optimistic_header.slot:
let attested_period = update.attested_header.slot.sync_committee_period
when update is SomeLightClientUpdateWithSyncCommittee:
let is_sync_committee_update = update.is_sync_committee_update
if update.attested_header.slot <= store.finalized_header.slot:
when update is SomeLightClientUpdateWithSyncCommittee:
if is_next_sync_committee_known:
return err(BlockError.Duplicate)
if attested_period != store_period or not is_sync_committee_update:
return err(BlockError.Duplicate)
else:
return err(BlockError.Duplicate)
# Verify update does not skip a sync committee period
let
finalized_period = store.finalized_header.slot.sync_committee_period
update_period = active_header.slot.sync_committee_period
if update_period < finalized_period:
return err(BlockError.Duplicate)
if update_period > finalized_period + 1:
return err(BlockError.MissingParent)
let
is_signed_by_next_sync_committee =
update.next_sync_committee.isZeroMemory
signature_period =
if is_signed_by_next_sync_committee:
update_period + 1
# Verify that the `finalized_header`, if present, actually is the
# finalized header saved in the state of the `attested_header`
when update is SomeLightClientUpdateWithFinality:
if not update.is_finality_update:
if not update.finalized_header.isZeroMemory:
return err(BlockError.Invalid)
else:
var finalized_root {.noinit.}: Eth2Digest
if update.finalized_header.slot != GENESIS_SLOT:
finalized_root = hash_tree_root(update.finalized_header)
elif update.finalized_header.isZeroMemory:
finalized_root.reset()
else:
update_period
current_period = current_slot.sync_committee_period
if current_period < signature_period:
return err(BlockError.UnviableFork)
if is_next_sync_committee_known:
if signature_period notin [finalized_period, finalized_period + 1]:
return err(BlockError.MissingParent)
else:
if signature_period != finalized_period:
return err(BlockError.MissingParent)
# Verify fork version is acceptable
let fork_version = update.fork_version
if not cfg.period_contains_fork_version(signature_period, fork_version):
return err(BlockError.UnviableFork)
# Verify that the `finalized_header`, if present, actually is the finalized
# header saved in the state of the `attested_header`
if not update.is_finality_update:
if not update.finality_branch.isZeroMemory:
return err(BlockError.Invalid)
else:
if not is_valid_merkle_branch(
hash_tree_root(update.finalized_header),
update.finality_branch,
log2trunc(altair.FINALIZED_ROOT_INDEX),
get_subtree_index(altair.FINALIZED_ROOT_INDEX),
update.attested_header.state_root):
return err(BlockError.Invalid)
return err(BlockError.Invalid)
if not is_valid_merkle_branch(
finalized_root,
update.finality_branch,
log2trunc(altair.FINALIZED_ROOT_INDEX),
get_subtree_index(altair.FINALIZED_ROOT_INDEX),
update.attested_header.state_root):
return err(BlockError.Invalid)
# Verify that the `next_sync_committee`, if present, actually is the
# next sync committee saved in the state of the `active_header`
if is_signed_by_next_sync_committee:
if not update.next_sync_committee_branch.isZeroMemory:
return err(BlockError.Invalid)
else:
if update_period == finalized_period and is_next_sync_committee_known:
if update.next_sync_committee != store.next_sync_committee:
return err(BlockError.UnviableFork)
if not is_valid_merkle_branch(
hash_tree_root(update.next_sync_committee),
update.next_sync_committee_branch,
log2trunc(altair.NEXT_SYNC_COMMITTEE_INDEX),
get_subtree_index(altair.NEXT_SYNC_COMMITTEE_INDEX),
active_header.state_root):
return err(BlockError.Invalid)
# next sync committee saved in the state of the `attested_header`
when update is SomeLightClientUpdateWithSyncCommittee:
if not is_sync_committee_update:
if not update.next_sync_committee.isZeroMemory:
return err(BlockError.Invalid)
else:
if attested_period == store_period and is_next_sync_committee_known:
if update.next_sync_committee != store.next_sync_committee:
return err(BlockError.UnviableFork)
if not is_valid_merkle_branch(
hash_tree_root(update.next_sync_committee),
update.next_sync_committee_branch,
log2trunc(altair.NEXT_SYNC_COMMITTEE_INDEX),
get_subtree_index(altair.NEXT_SYNC_COMMITTEE_INDEX),
update.attested_header.state_root):
return err(BlockError.Invalid)
# Verify sync committee aggregate signature
let sync_committee =
if signature_period == finalized_period:
if signature_period == store_period:
unsafeAddr store.current_sync_committee
else:
unsafeAddr store.next_sync_committee
@ -191,240 +138,125 @@ proc validate_light_client_update*(
if bit:
participant_pubkeys.add(sync_committee.pubkeys[idx])
let
fork_version = cfg.forkVersionAtEpoch(update.signature_slot.epoch)
domain = compute_domain(
DOMAIN_SYNC_COMMITTEE, fork_version, genesis_validators_root)
signing_root = compute_signing_root(attested_header, domain)
signing_root = compute_signing_root(update.attested_header, domain)
if not blsFastAggregateVerify(
participant_pubkeys, signing_root.data,
sync_aggregate.sync_committee_signature):
return err(BlockError.Invalid)
ok()
# https://github.com/ethereum/consensus-specs/blob/vFuture/specs/altair/sync-protocol.md#validate_optimistic_light_client_update
proc validate_optimistic_light_client_update*(
store: LightClientStore,
optimistic_update: OptimisticLightClientUpdate,
current_slot: Slot,
cfg: RuntimeConfig,
genesis_validators_root: Eth2Digest): Result[void, BlockError] =
# Verify sync committee has sufficient participants
template sync_aggregate(): auto = optimistic_update.sync_aggregate
template sync_committee_bits(): auto = sync_aggregate.sync_committee_bits
let num_active_participants = countOnes(sync_committee_bits).uint64
if num_active_participants < MIN_SYNC_COMMITTEE_PARTICIPANTS:
return err(BlockError.Invalid)
# Determine update header
template attested_header(): auto = optimistic_update.attested_header
if current_slot < attested_header.slot:
return err(BlockError.Invalid)
template active_header(): auto = attested_header
# Verify update is relevant
if attested_header.slot <= store.optimistic_header.slot:
return err(BlockError.Duplicate)
# Verify update does not skip a sync committee period
let
finalized_period = store.finalized_header.slot.sync_committee_period
update_period = active_header.slot.sync_committee_period
if update_period < finalized_period:
return err(BlockError.Duplicate)
if update_period > finalized_period + 1:
return err(BlockError.MissingParent)
let
is_signed_by_next_sync_committee =
optimistic_update.is_signed_by_next_sync_committee
signature_period =
if is_signed_by_next_sync_committee:
update_period + 1
else:
update_period
current_period = current_slot.sync_committee_period
if current_period < signature_period:
return err(BlockError.Invalid)
let is_next_sync_committee_known = not store.next_sync_committee.isZeroMemory
if is_next_sync_committee_known:
if signature_period notin [finalized_period, finalized_period + 1]:
return err(BlockError.MissingParent)
else:
if signature_period != finalized_period:
return err(BlockError.MissingParent)
# Verify fork version is acceptable
let fork_version = optimistic_update.fork_version
if not cfg.period_contains_fork_version(signature_period, fork_version):
return err(BlockError.UnviableFork)
# Verify sync committee aggregate signature
let sync_committee =
if signature_period == finalized_period:
unsafeAddr store.current_sync_committee
else:
unsafeAddr store.next_sync_committee
var participant_pubkeys =
newSeqOfCap[ValidatorPubKey](num_active_participants)
for idx, bit in sync_aggregate.sync_committee_bits:
if bit:
participant_pubkeys.add(sync_committee.pubkeys[idx])
let
domain = compute_domain(
DOMAIN_SYNC_COMMITTEE, fork_version, genesis_validators_root)
signing_root = compute_signing_root(attested_header, domain)
if not blsFastAggregateVerify(
participant_pubkeys, signing_root.data,
sync_aggregate.sync_committee_signature):
return err(BlockError.Invalid)
ok()
# https://github.com/ethereum/consensus-specs/blob/vFuture/specs/altair/sync-protocol.md#apply_light_client_update
func apply_light_client_update(
store: var LightClientStore,
update: altair.LightClientUpdate): bool =
update: SomeLightClientUpdate): bool =
var didProgress = false
let
active_header = get_active_header(update)
finalized_period = store.finalized_header.slot.sync_committee_period
update_period = active_header.slot.sync_committee_period
if store.next_sync_committee.isZeroMemory:
assert update_period == finalized_period
store.next_sync_committee = update.next_sync_committee
didProgress = true
elif update_period == finalized_period + 1:
store_period = store.finalized_header.slot.sync_committee_period
finalized_period = update.finalized_header.slot.sync_committee_period
if not store.is_next_sync_committee_known:
assert finalized_period == store_period
when update is SomeLightClientUpdateWithSyncCommittee:
store.next_sync_committee = update.next_sync_committee
if store.is_next_sync_committee_known:
didProgress = true
elif finalized_period == store_period + 1:
store.current_sync_committee = store.next_sync_committee
when update is SomeLightClientUpdateWithSyncCommittee:
store.next_sync_committee = update.next_sync_committee
else:
store.next_sync_committee.reset()
store.previous_max_active_participants =
store.current_max_active_participants
store.current_max_active_participants = 0
store.current_sync_committee = store.next_sync_committee
store.next_sync_committee = update.next_sync_committee
assert not store.next_sync_committee.isZeroMemory
didProgress = true
if active_header.slot > store.finalized_header.slot:
store.finalized_header = active_header
if update.finalized_header.slot > store.finalized_header.slot:
store.finalized_header = update.finalized_header
if store.finalized_header.slot > store.optimistic_header.slot:
store.optimistic_header = store.finalized_header
didProgress = true
didProgress
# https://github.com/ethereum/consensus-specs/blob/vFuture/specs/altair/sync-protocol.md#apply_optimistic_light_client_header
func apply_optimistic_light_client_header(
store: var LightClientStore,
attested_header: BeaconBlockHeader,
num_active_participants: uint64): bool =
var didProgress = false
if store.current_max_active_participants < num_active_participants:
store.current_max_active_participants = num_active_participants
if num_active_participants > get_safety_threshold(store) and
attested_header.slot > store.optimistic_header.slot:
store.optimistic_header = attested_header
didProgress = true
didProgress
# https://github.com/ethereum/consensus-specs/blob/v1.1.10/specs/altair/sync-protocol.md#process_slot_for_light_client_store
# https://github.com/ethereum/consensus-specs/blob/vFuture/specs/altair/sync-protocol.md#try_light_client_store_force_update
type
ProcessSlotForLightClientStoreResult* = enum
ForceUpdateResult* = enum
NoUpdate,
UpdatedWithoutSupermajority,
UpdatedWithoutFinalityProof
DidUpdateWithoutSupermajority,
DidUpdateWithoutFinality
func process_slot_for_light_client_store*(
func try_light_client_store_force_update*(
store: var LightClientStore,
current_slot: Slot): ProcessSlotForLightClientStoreResult {.discardable.} =
current_slot: Slot): ForceUpdateResult {.discardable.} =
var res = NoUpdate
if store.best_valid_update.isSome and
current_slot > store.finalized_header.slot + UPDATE_TIMEOUT:
template sync_aggregate(): auto = store.best_valid_update.get.sync_aggregate
template sync_committee_bits(): auto = sync_aggregate.sync_committee_bits
let num_active_participants = countOnes(sync_committee_bits).uint64
if apply_light_client_update(store, store.best_valid_update.get):
# Forced best update when the update timeout has elapsed
template best(): auto = store.best_valid_update.get
if best.finalized_header.slot <= store.finalized_header.slot:
best.finalized_header = best.attested_header
if apply_light_client_update(store, best):
template sync_aggregate(): auto = best.sync_aggregate
template sync_committee_bits(): auto = sync_aggregate.sync_committee_bits
let num_active_participants = countOnes(sync_committee_bits).uint64
if num_active_participants * 3 < static(sync_committee_bits.len * 2):
res = UpdatedWithoutSupermajority
res = DidUpdateWithoutSupermajority
else:
res = UpdatedWithoutFinalityProof
store.best_valid_update = none(altair.LightClientUpdate)
res = DidUpdateWithoutFinality
store.best_valid_update.reset()
res
# https://github.com/ethereum/consensus-specs/blob/vFuture/specs/altair/sync-protocol.md#process_light_client_update
proc process_light_client_update*(
store: var LightClientStore,
update: altair.LightClientUpdate,
update: SomeLightClientUpdate,
current_slot: Slot,
cfg: RuntimeConfig,
genesis_validators_root: Eth2Digest,
allowForceUpdate = true): Result[void, BlockError] =
genesis_validators_root: Eth2Digest): Result[void, BlockError] =
? validate_light_client_update(
store, update, current_slot, cfg, genesis_validators_root)
var didProgress = false
# Update the best update in case we have to force-update to it
# if the timeout elapses
if store.best_valid_update.isNone or
is_better_update(update, store.best_valid_update.get):
store.best_valid_update = some(update.toFull)
didProgress = true
# Track the maximum number of active participants in the committee signatures
template sync_aggregate(): auto = update.sync_aggregate
template sync_committee_bits(): auto = sync_aggregate.sync_committee_bits
let num_active_participants = countOnes(sync_committee_bits).uint64
if num_active_participants > store.current_max_active_participants:
store.current_max_active_participants = num_active_participants
# Update the optimistic header
if apply_optimistic_light_client_header(
store, update.attested_header, num_active_participants):
didProgress = true
# Update the best update in case we have to force-update to it
# if the timeout elapses
let best_active_participants =
if store.best_valid_update.isNone:
0.uint64
else:
template best_sync_aggregate(): auto =
store.best_valid_update.get.sync_aggregate
countOnes(best_sync_aggregate.sync_committee_bits).uint64
if num_active_participants > best_active_participants:
store.best_valid_update = some(update)
if num_active_participants > get_safety_threshold(store) and
update.attested_header.slot > store.optimistic_header.slot:
store.optimistic_header = update.attested_header
didProgress = true
# Update finalized header
if num_active_participants * 3 >= static(sync_committee_bits.len * 2) and
update.is_finality_update:
# Normal update through 2/3 threshold
if apply_light_client_update(store, update):
didProgress = true
store.best_valid_update = none(altair.LightClientUpdate)
else:
if allowForceUpdate:
# Force-update to best update if the timeout elapsed
case process_slot_for_light_client_store(store, current_slot)
of UpdatedWithoutSupermajority, UpdatedWithoutFinalityProof:
didProgress = true
of NoUpdate: discard
when update is SomeLightClientUpdateWithFinality:
if num_active_participants * 3 >= static(sync_committee_bits.len * 2):
var improvesFinality =
update.finalized_header.slot > store.finalized_header.slot
when update is SomeLightClientUpdateWithSyncCommittee:
if not improvesFinality and not store.is_next_sync_committee_known:
improvesFinality =
update.is_sync_committee_update and update.is_finality_update and
update.finalized_header.slot.sync_committee_period ==
update.attested_header.slot.sync_committee_period
if improvesFinality:
# Normal update through 2/3 threshold
if apply_light_client_update(store, update):
didProgress = true
store.best_valid_update.reset()
if not didProgress:
err(BlockError.Duplicate)
else:
ok()
# https://github.com/ethereum/consensus-specs/blob/vFuture/specs/altair/sync-protocol.md#process_light_client_update
proc process_optimistic_light_client_update*(
store: var LightClientStore,
optimistic_update: OptimisticLightClientUpdate,
current_slot: Slot,
cfg: RuntimeConfig,
genesis_validators_root: Eth2Digest): Result[void, BlockError] =
? validate_optimistic_light_client_update(
store, optimistic_update, current_slot, cfg, genesis_validators_root)
var didProgress = false
template sync_aggregate(): auto = optimistic_update.sync_aggregate
template sync_committee_bits(): auto = sync_aggregate.sync_committee_bits
let num_active_participants = countOnes(sync_committee_bits).uint64
# Update the optimistic header
if apply_optimistic_light_client_header(
store, optimistic_update.attested_header, num_active_participants):
didProgress = true
if not didProgress:
err(BlockError.Duplicate)
else:
ok()
return err(BlockError.Duplicate)
ok()

View File

@ -98,10 +98,15 @@ func getSyncCommitteeContributionAndProofTopic*(forkDigest: ForkDigest): string
## For subscribing and unsubscribing to/from a subnet.
eth2Prefix(forkDigest) & "sync_committee_contribution_and_proof/ssz_snappy"
# https://github.com/ethereum/consensus-specs/blob/vFuture/specs/altair/sync-protocol.md#optimistic_light_client_update
func getOptimisticLightClientUpdateTopic*(forkDigest: ForkDigest): string =
## For broadcasting or obtaining the latest `OptimisticLightClientUpdate`.
eth2Prefix(forkDigest) & "optimistic_light_client_update_v0/ssz_snappy"
# https://github.com/ethereum/consensus-specs/blob/vFuture/specs/altair/sync-protocol.md#light_client_finality_update
func getLightClientFinalityUpdateTopic*(forkDigest: ForkDigest): string =
## For broadcasting or obtaining the latest `LightClientFinalityUpdate`.
eth2Prefix(forkDigest) & "light_client_finality_update_v0/ssz_snappy"
# https://github.com/ethereum/consensus-specs/blob/vFuture/specs/altair/sync-protocol.md#light_client_optimistic_update
func getLightClientOptimisticUpdateTopic*(forkDigest: ForkDigest): string =
## For broadcasting or obtaining the latest `LightClientOptimisticUpdate`.
eth2Prefix(forkDigest) & "light_client_optimistic_update_v0/ssz_snappy"
func getENRForkID*(cfg: RuntimeConfig,
epoch: Epoch,

View File

@ -10,11 +10,10 @@
import
std/[os, strformat],
chronicles,
./spec/[beaconstate, eth2_ssz_serialization, eth2_merkleization, forks],
./spec/[
beaconstate, eth2_ssz_serialization, eth2_merkleization, forks, helpers],
./spec/datatypes/[phase0, altair]
from spec/light_client_sync import is_finality_update
export
beaconstate, eth2_ssz_serialization, eth2_merkleization, forks
@ -59,27 +58,35 @@ proc dump*(dir: string, v: altair.LightClientBootstrap) =
SSZ.saveFile(
dir / &"{prefix}-{slot}-{blck}-{root}.ssz", v)
proc dump*(dir: string, v: altair.LightClientUpdate) =
proc dump*(dir: string, v: SomeLightClientUpdate) =
logErrors:
let
prefix = "update"
prefix =
when v is altair.LightClientUpdate:
"update"
elif v is altair.LightClientFinalityUpdate:
"finality-update"
elif v is altair.LightClientOptimisticUpdate:
"optimistic-update"
attestedSlot = v.attested_header.slot
attestedBlck = shortLog(v.attested_header.hash_tree_root())
suffix =
if v.is_finality_update:
"f"
syncCommitteeSuffix =
when v is SomeLightClientUpdateWithSyncCommittee:
if v.is_sync_committee_update:
"s"
else:
"x"
else:
"o"
""
finalitySuffix =
when v is SomeLightClientUpdateWithFinality:
if v.is_finality_update:
"f"
else:
"x"
else:
""
suffix = syncCommitteeSuffix & finalitySuffix
root = shortLog(v.hash_tree_root())
SSZ.saveFile(
dir / &"{prefix}-{attestedSlot}-{attestedBlck}-{suffix}-{root}.ssz", v)
proc dump*(dir: string, v: OptimisticLightClientUpdate) =
logErrors:
let
prefix = "optimistic-update"
attestedSlot = v.attested_header.slot
attestedBlck = shortLog(v.attested_header.hash_tree_root())
root = shortLog(v.hash_tree_root())
SSZ.saveFile(
dir / &"{prefix}-{attestedSlot}-{attestedBlck}-{root}.ssz", v)

View File

@ -26,16 +26,19 @@ logScope:
const
MAX_REQUEST_BLOCKS = 1024
# https://github.com/ethereum/consensus-specs/blob/vFuture/specs/altair/sync-protocol.md#configuration
MAX_REQUEST_LIGHT_CLIENT_UPDATES = 128
blockByRootLookupCost = allowedOpsPerSecondCost(50)
blockResponseCost = allowedOpsPerSecondCost(100)
blockByRangeLookupCost = allowedOpsPerSecondCost(20)
lightClientUpdateResponseCost = allowedOpsPerSecondCost(100)
lightClientUpdateByRangeLookupCost = allowedOpsPerSecondCost(20)
# https://github.com/ethereum/consensus-specs/blob/vFuture/specs/altair/sync-protocol.md#configuration
MAX_REQUEST_LIGHT_CLIENT_UPDATES = 128
lightClientEmptyResponseCost = allowedOpsPerSecondCost(50)
lightClientBootstrapLookupCost = allowedOpsPerSecondCost(5)
lightClientBootstrapResponseCost = allowedOpsPerSecondCost(100)
lightClientUpdateResponseCost = allowedOpsPerSecondCost(100)
lightClientUpdateByRangeLookupCost = allowedOpsPerSecondCost(20)
lightClientFinalityUpdateResponseCost = allowedOpsPerSecondCost(100)
lightClientOptimisticUpdateResponseCost = allowedOpsPerSecondCost(100)
type
StatusMsg* = object
@ -108,6 +111,29 @@ proc readChunkPayload*(
else:
return neterr InvalidContextBytes
proc readChunkPayload*(
conn: Connection, peer: Peer, maxChunkSize: uint32,
MsgType: type SomeLightClientObject):
Future[NetRes[MsgType]] {.async.} =
var contextBytes: ForkDigest
try:
await conn.readExactly(addr contextBytes, sizeof contextBytes)
except CatchableError:
return neterr UnexpectedEOF
let stateFork =
peer.network.forkDigests[].stateForkForDigest(contextBytes).valueOr:
return neterr InvalidContextBytes
let res =
if stateFork >= BeaconStateFork.Altair:
await eth2_network.readChunkPayload(conn, peer, maxChunkSize, MsgType)
else:
doAssert stateFork == BeaconStateFork.Phase0
return neterr InvalidContextBytes
if res.isErr:
return err(res.error)
return ok res.get
func shortLog*(s: StatusMsg): auto =
(
forkDigest: s.forkDigest,
@ -476,91 +502,6 @@ p2pProtocol BeaconSync(version = 1,
debug "Block root request done",
peer, roots = blockRoots.len, count, found
# https://github.com/ethereum/consensus-specs/blob/vFuture/specs/altair/sync-protocol.md#bestlightclientupdatesbyrange
proc bestLightClientUpdatesByRange(
peer: Peer,
startPeriod: SyncCommitteePeriod,
reqCount: uint64,
response: MultipleChunksResponse[altair.LightClientUpdate])
{.async, libp2pProtocol("best_light_client_updates_by_range", 0,
isLightClientRequest = true).} =
trace "Received LC updates by range request", peer, startPeriod, reqCount
if reqCount == 0'u64:
raise newException(InvalidInputsError, "Empty range requested")
let dag = peer.networkState.dag
doAssert dag.serveLightClientData
let
headPeriod = dag.head.slot.sync_committee_period
# Limit number of updates in response
maxSupportedCount =
if startPeriod > headPeriod:
0'u64
else:
min(headPeriod + 1 - startPeriod, MAX_REQUEST_LIGHT_CLIENT_UPDATES)
count = min(reqCount, maxSupportedCount)
onePastPeriod = startPeriod + count
peer.updateRequestQuota(count.float * lightClientUpdateByRangeLookupCost)
peer.awaitNonNegativeRequestQuota()
var found = 0
for period in startPeriod..<onePastPeriod:
let update = dag.getBestLightClientUpdateForPeriod(period)
if update.isSome:
await response.write(update.get)
inc found
peer.updateRequestQuota(found.float * lightClientUpdateResponseCost)
debug "LC updates by range request done", peer, startPeriod, count, found
if found == 0 and count > 0:
raise newException(ResourceUnavailableError,
"No light client update available")
# https://github.com/ethereum/consensus-specs/blob/vFuture/specs/altair/sync-protocol.md#getlatestlightclientupdate
proc latestLightClientUpdate(
peer: Peer,
response: SingleChunkResponse[altair.LightClientUpdate])
{.async, libp2pProtocol("latest_light_client_update", 0,
isLightClientRequest = true).} =
trace "Received latest LC update request", peer
let dag = peer.networkState.dag
doAssert dag.serveLightClientData
peer.awaitNonNegativeRequestQuota()
let update = dag.getLatestLightClientUpdate
if update.isSome:
await response.send(update.get)
else:
raise newException(ResourceUnavailableError,
"No light client update available")
peer.updateRequestQuota(lightClientUpdateResponseCost)
# https://github.com/ethereum/consensus-specs/blob/vFuture/specs/altair/sync-protocol.md#getoptimisticlightclientupdate
proc optimisticLightClientUpdate(
peer: Peer,
response: SingleChunkResponse[OptimisticLightClientUpdate])
{.async, libp2pProtocol("optimistic_light_client_update", 0,
isLightClientRequest = true).} =
trace "Received optimistic LC update request", peer
let dag = peer.networkState.dag
doAssert dag.serveLightClientData
peer.awaitNonNegativeRequestQuota()
let optimistic_update = dag.getOptimisticLightClientUpdate
if optimistic_update.isSome:
await response.send(optimistic_update.get)
else:
raise newException(ResourceUnavailableError,
"No optimistic light client update available")
peer.updateRequestQuota(lightClientUpdateResponseCost)
# https://github.com/ethereum/consensus-specs/blob/vFuture/specs/altair/sync-protocol.md#getlightclientbootstrap
proc lightClientBootstrap(
peer: Peer,
@ -577,13 +518,115 @@ p2pProtocol BeaconSync(version = 1,
let bootstrap = dag.getLightClientBootstrap(blockRoot)
if bootstrap.isOk:
await response.send(bootstrap.get)
let
contextEpoch = bootstrap.get.header.slot.epoch
contextBytes = dag.forkDigestAtEpoch(contextEpoch).data
await response.send(bootstrap.get, contextBytes)
else:
raise newException(ResourceUnavailableError,
"No light client bootstrap available")
peer.updateRequestQuota(lightClientEmptyResponseCost)
raise newException(
ResourceUnavailableError, "LC bootstrap unavailable")
peer.updateRequestQuota(lightClientBootstrapResponseCost)
debug "LC bootstrap request done", peer, blockRoot
# https://github.com/ethereum/consensus-specs/blob/vFuture/specs/altair/sync-protocol.md#lightclientupdatesbyrange
proc lightClientUpdatesByRange(
peer: Peer,
startPeriod: SyncCommitteePeriod,
reqCount: uint64,
response: MultipleChunksResponse[altair.LightClientUpdate])
{.async, libp2pProtocol("light_client_updates_by_range", 0,
isLightClientRequest = true).} =
trace "Received LC updates by range request", peer, startPeriod, reqCount
let dag = peer.networkState.dag
doAssert dag.serveLightClientData
let
headPeriod = dag.head.slot.sync_committee_period
# Limit number of updates in response
maxSupportedCount =
if startPeriod > headPeriod:
0'u64
else:
min(headPeriod + 1 - startPeriod, MAX_REQUEST_LIGHT_CLIENT_UPDATES)
count = min(reqCount, maxSupportedCount)
onePastPeriod = startPeriod + count
if count == 0:
peer.updateRequestQuota(lightClientEmptyResponseCost)
peer.updateRequestQuota(count.float * lightClientUpdateByRangeLookupCost)
peer.awaitNonNegativeRequestQuota()
var found = 0
for period in startPeriod..<onePastPeriod:
let update = dag.getLightClientUpdateForPeriod(period)
if update.isSome:
let
contextEpoch = update.get.attested_header.slot.epoch
contextBytes = dag.forkDigestAtEpoch(contextEpoch).data
await response.write(update.get, contextBytes)
inc found
peer.updateRequestQuota(found.float * lightClientUpdateResponseCost)
debug "LC updates by range request done", peer, startPeriod, count, found
# https://github.com/ethereum/consensus-specs/blob/vFuture/specs/altair/sync-protocol.md#getlightclientfinalityupdate
proc lightClientFinalityUpdate(
peer: Peer,
response: SingleChunkResponse[altair.LightClientFinalityUpdate])
{.async, libp2pProtocol("light_client_finality_update", 0,
isLightClientRequest = true).} =
trace "Received LC finality update request", peer
let dag = peer.networkState.dag
doAssert dag.serveLightClientData
peer.awaitNonNegativeRequestQuota()
let finality_update = dag.getLightClientFinalityUpdate
if finality_update.isSome:
let
contextEpoch = finality_update.get.attested_header.slot.epoch
contextBytes = dag.forkDigestAtEpoch(contextEpoch).data
await response.send(finality_update.get, contextBytes)
else:
peer.updateRequestQuota(lightClientEmptyResponseCost)
raise newException(ResourceUnavailableError,
"LC finality update unavailable")
peer.updateRequestQuota(lightClientFinalityUpdateResponseCost)
debug "LC finality update request done", peer
# https://github.com/ethereum/consensus-specs/blob/vFuture/specs/altair/sync-protocol.md#getlightclientoptimisticupdate
proc lightClientOptimisticUpdate(
peer: Peer,
response: SingleChunkResponse[altair.LightClientOptimisticUpdate])
{.async, libp2pProtocol("light_client_optimistic_update", 0,
isLightClientRequest = true).} =
trace "Received LC optimistic update request", peer
let dag = peer.networkState.dag
doAssert dag.serveLightClientData
peer.awaitNonNegativeRequestQuota()
let optimistic_update = dag.getLightClientOptimisticUpdate
if optimistic_update.isSome:
let
contextEpoch = optimistic_update.get.attested_header.slot.epoch
contextBytes = dag.forkDigestAtEpoch(contextEpoch).data
await response.send(optimistic_update.get, contextBytes)
else:
peer.updateRequestQuota(lightClientEmptyResponseCost)
raise newException(ResourceUnavailableError,
"LC optimistic update unavailable")
peer.updateRequestQuota(lightClientOptimisticUpdateResponseCost)
debug "LC optimistic update request done", peer
proc goodbye(peer: Peer,
reason: uint64)
{.async, libp2pProtocol("goodbye", 1).} =

View File

@ -64,6 +64,12 @@ declareHistogram beacon_sync_committee_message_sent_delay,
"Time(s) between slot start and sync committee message sent moment",
buckets = delayBuckets
declareCounter beacon_light_client_finality_updates_sent,
"Number of LC finality updates sent by this peer"
declareCounter beacon_light_client_optimistic_updates_sent,
"Number of LC optimistic updates sent by this peer"
declareCounter beacon_blocks_proposed,
"Number of beacon chain blocks sent by this peer"
@ -239,7 +245,57 @@ proc sendAttestation*(
error = res.error()
err(res.error()[1])
proc sendSyncCommitteeMessage*(
proc handleLightClientUpdates(node: BeaconNode, slot: Slot) {.async.} =
static: doAssert lightClientFinalityUpdateSlotOffset ==
lightClientOptimisticUpdateSlotOffset
let sendTime = node.beaconClock.fromNow(
slot.light_client_finality_update_time())
if sendTime.inFuture:
debug "Waiting to send LC updates", slot, delay = shortLog(sendTime.offset)
await sleepAsync(sendTime.offset)
template latest(): auto = node.dag.lightClientCache.latest
let signature_slot = latest.signature_slot
if slot != signature_slot:
return
template sync_aggregate(): auto = latest.sync_aggregate
template sync_committee_bits(): auto = sync_aggregate.sync_committee_bits
let num_active_participants = countOnes(sync_committee_bits).uint64
if num_active_participants < MIN_SYNC_COMMITTEE_PARTICIPANTS:
return
let finalized_slot = latest.finalized_header.slot
if finalized_slot > node.lightClientPool[].latestForwardedFinalitySlot:
template msg(): auto = latest
node.network.broadcastLightClientFinalityUpdate(msg)
node.lightClientPool[].latestForwardedFinalitySlot = finalized_slot
beacon_light_client_finality_updates_sent.inc()
notice "LC finality update sent", message = shortLog(msg)
let attested_slot = latest.attested_header.slot
if attested_slot > node.lightClientPool[].latestForwardedOptimisticSlot:
let msg = latest.toOptimistic
node.network.broadcastLightClientOptimisticUpdate(msg)
node.lightClientPool[].latestForwardedOptimisticSlot = attested_slot
beacon_light_client_optimistic_updates_sent.inc()
notice "LC optimistic update sent", message = shortLog(msg)
proc scheduleSendingLightClientUpdates(node: BeaconNode, slot: Slot) =
if not node.config.serveLightClientData.get:
return
if node.lightClientPool[].broadcastGossipFut != nil:
return
if slot <= node.lightClientPool[].latestBroadcastedSlot:
return
node.lightClientPool[].latestBroadcastedSlot = slot
template fut(): auto = node.lightClientPool[].broadcastGossipFut
fut = node.handleLightClientUpdates(slot)
fut.addCallback do (p: pointer) {.gcsafe.}:
fut = nil
proc sendSyncCommitteeMessage(
node: BeaconNode, msg: SyncCommitteeMessage,
subcommitteeIdx: SyncSubcommitteeIndex,
checkSignature: bool): Future[SendResult] {.async.} =
@ -254,6 +310,7 @@ proc sendSyncCommitteeMessage*(
if res.isGoodForSending:
node.network.broadcastSyncCommitteeMessage(msg, subcommitteeIdx)
beacon_sync_committee_messages_sent.inc()
node.scheduleSendingLightClientUpdates(msg.slot)
SendResult.ok()
else:
notice "Sync committee message failed validation",
@ -833,19 +890,6 @@ proc handleSyncCommitteeMessages(node: BeaconNode, head: BlockRef, slot: Slot) =
asyncSpawn createAndSendSyncCommitteeMessage(node, slot, validator,
subcommitteeIdx, head)
proc handleOptimisticLightClientUpdates(
node: BeaconNode, head: BlockRef, slot: Slot) =
if slot < node.dag.cfg.ALTAIR_FORK_EPOCH.start_slot():
return
doAssert head.parent != nil, "Newly proposed block lacks parent reference"
let msg = node.dag.lightClientCache.optimisticUpdate
if msg.attested_header.slot != head.parent.bid.slot:
notice "No optimistic light client update for proposed block",
slot = slot, block_root = shortLog(head.root)
return
node.network.broadcastOptimisticLightClientUpdate(msg)
notice "Sent optimistic light client update", message = shortLog(msg)
proc signAndSendContribution(node: BeaconNode,
validator: AttachedValidator,
contribution: SyncCommitteeContribution,
@ -1234,16 +1278,6 @@ proc handleValidatorDuties*(node: BeaconNode, lastSlot, slot: Slot) {.async.} =
handleAttestations(node, head, slot)
handleSyncCommitteeMessages(node, head, slot)
if node.config.serveLightClientData.get and didSubmitBlock:
let cutoff = node.beaconClock.fromNow(
slot.optimistic_light_client_update_time())
if cutoff.inFuture:
debug "Waiting to send optimistic light client update",
head = shortLog(head),
optimisticLightClientUpdateCutoff = shortLog(cutoff.offset)
await sleepAsync(cutoff.offset)
handleOptimisticLightClientUpdates(node, head, slot)
updateValidatorMetrics(node) # the important stuff is done, update the vanity numbers
# https://github.com/ethereum/consensus-specs/blob/v1.1.10/specs/phase0/validator.md#broadcast-aggregate
@ -1400,31 +1434,11 @@ proc sendBeaconBlock*(node: BeaconNode, forked: ForkedSignedBeaconBlock
notice "Block published",
blockRoot = shortLog(blck.root), blck = shortLog(blck.message),
signature = shortLog(blck.signature)
if node.config.serveLightClientData.get:
# The optimistic light client update is sent with a delay because it
# only validates once the new block has been processed by the peers.
# https://github.com/ethereum/consensus-specs/blob/vFuture/specs/altair/sync-protocol.md#block-proposal
proc publishOptimisticLightClientUpdate() {.async.} =
let cutoff = node.beaconClock.fromNow(
wallTime.slotOrZero.optimistic_light_client_update_time())
if cutoff.inFuture:
debug "Waiting to publish optimistic light client update",
blockRoot = shortLog(blck.root), blck = shortLog(blck.message),
signature = shortLog(blck.signature),
optimisticLightClientUpdateCutoff = shortLog(cutoff.offset)
await sleepAsync(cutoff.offset)
handleOptimisticLightClientUpdates(
node, newBlockRef.get, wallTime.slotOrZero)
asyncSpawn publishOptimisticLightClientUpdate()
true
else:
warn "Unable to add proposed block to block pool",
blockRoot = shortLog(blck.root), blck = shortLog(blck.message),
signature = shortLog(blck.signature), err = newBlockRef.error()
false
return SendBlockResult.ok(accepted)

View File

@ -60,7 +60,7 @@ func gauss(r: var Rand; mu = 0.0; sigma = 1.0): float =
cli do(slots = SLOTS_PER_EPOCH * 6,
validators = SLOTS_PER_EPOCH * 400, # One per shard is minimum
attesterRatio {.desc: "ratio of validators that attest in each round"} = 0.82,
syncCommitteeRatio {.desc: "ratio of validators that perform sync committee actions in each round"} = 0.75,
syncCommitteeRatio {.desc: "ratio of validators that perform sync committee actions in each round"} = 0.82,
blockRatio {.desc: "ratio of slots with blocks"} = 1.0,
replay = true):
let

View File

@ -19,5 +19,6 @@ import
./test_fixture_ssz_consensus_objects,
./test_fixture_state_transition_epoch,
./test_fixture_sync_protocol_light_client_sync,
./test_fixture_sync_protocol_update_ranking,
./test_fixture_sync_protocol,
./test_fixture_transition

View File

@ -114,7 +114,14 @@ suite "EF - Altair - SSZ consensus objects " & preset():
of "ForkData": checkSSZ(ForkData, path, hash)
of "HistoricalBatch": checkSSZ(HistoricalBatch, path, hash)
of "IndexedAttestation": checkSSZ(IndexedAttestation, path, hash)
of "LightClientUpdate": checkSSZ(LightClientUpdate, path, hash)
of "LightClientBootstrap":
checkSSZ(LightClientBootstrap, path, hash)
of "LightClientUpdate":
discard # Modified - checkSSZ(LightClientUpdate, path, hash)
of "LightClientFinalityUpdate":
checkSSZ(LightClientFinalityUpdate, path, hash)
of "LightClientOptimisticUpdate":
checkSSZ(LightClientOptimisticUpdate, path, hash)
of "PendingAttestation": checkSSZ(PendingAttestation, path, hash)
of "ProposerSlashing": checkSSZ(ProposerSlashing, path, hash)
of "SignedAggregateAndProof":

View File

@ -18,7 +18,7 @@ import
[beaconstate, forks, helpers, light_client_sync, signatures,
state_transition],
# Mock helpers
../../mocking/[mock_blocks, mock_genesis],
../../mocking/mock_genesis,
# Test utilities
../../testutil, ../../testblockutil
@ -26,13 +26,14 @@ import
# light client sync protocol. Conflicting release versions are not in use.
# https://github.com/ethereum/consensus-specs/pull/2802
# https://github.com/ethereum/consensus-specs/blob/v1.1.10/tests/core/pyspec/eth2spec/test/helpers/sync_committee.py#L27-L44
# https://github.com/ethereum/consensus-specs/blob/vFuture/tests/core/pyspec/eth2spec/test/helpers/light_client.py#L63-L93
proc compute_aggregate_sync_committee_signature(
cfg: RuntimeConfig,
forked: ForkedHashedBeaconState,
signature_slot: Slot,
participants: openArray[ValidatorIndex],
block_root: Eth2Digest): ValidatorSig =
template state: untyped {.inject.} = forked.altairData.data
template state(): auto = forked.altairData.data
if len(participants) == 0:
return ValidatorSig.infinity
@ -56,12 +57,74 @@ proc compute_aggregate_sync_committee_signature(
aggregateSig.aggregate(signature)
aggregateSig.finish.toValidatorSig
# https://github.com/ethereum/consensus-specs/blob/vFuture/tests/core/pyspec/eth2spec/test/helpers/light_client.py#L63-L93
proc get_sync_aggregate(
cfg: RuntimeConfig,
forked: ForkedHashedBeaconState,
num_participants = SYNC_COMMITTEE_SIZE.uint64,
signature_slot = FAR_FUTURE_SLOT): (SyncAggregate, Slot) =
template state(): auto = forked.altairData.data
let
block_root =
BeaconBlockHeader(
slot: state.latest_block_header.slot,
proposer_index: state.latest_block_header.proposer_index,
parent_root: state.latest_block_header.parent_root,
state_root: hash_tree_root(state),
body_root: state.latest_block_header.body_root
).hash_tree_root()
# By default, the sync committee signs the previous slot
sig_slot =
if signature_slot == FAR_FUTURE_SLOT:
state.slot + 1
else:
signature_slot
# Ensure correct sync committee and fork version are selected
var
signature_forked = assignClone(forked)
cache: StateCache
info: ForkedEpochInfo
template signature_state(): auto = signature_forked.altairData.data
process_slots(cfg, signature_forked[], sig_slot, cache, info, flags = {})
.expect("no failure")
# Fetch sync committee
let
all_pubkeys = signature_state.validators.mapIt(it.pubkey)
committee_indices = signature_state.current_sync_committee.pubkeys
.mapIt(all_pubkeys.find(it).ValidatorIndex)
committee_size = lenu64(committee_indices)
# By default, use full participation
doAssert committee_size == SYNC_COMMITTEE_SIZE
doAssert committee_size >= num_participants
# Compute sync aggregate
var sync_committee_bits: BitArray[SYNC_COMMITTEE_SIZE]
for i in 0 ..< num_participants:
sync_committee_bits[i] = true
let
sync_committee_signature = compute_aggregate_sync_committee_signature(
cfg,
signature_forked[],
sig_slot,
committee_indices[0 ..< num_participants],
block_root)
sync_aggregate = SyncAggregate(
sync_committee_bits: sync_committee_bits,
sync_committee_signature: sync_committee_signature
)
(sync_aggregate, sig_slot)
proc block_for_next_slot(
cfg: RuntimeConfig,
forked: var ForkedHashedBeaconState,
cache: var StateCache,
withAttestations = false): ForkedSignedBeaconBlock =
template state: untyped {.inject.} = forked.altairData.data
template state(): auto = forked.altairData.data
let attestations =
if withAttestations:
@ -98,160 +161,134 @@ suite "EF - Altair - Unittests - Sync protocol" & preset():
res
genesisState = newClone(initGenesisState(cfg = cfg))
# https://github.com/ethereum/consensus-specs/blob/vFuture/tests/core/pyspec/eth2spec/test/altair/unittests/test_sync_protocol.py#L27-L77
# https://github.com/ethereum/consensus-specs/blob/vFuture/tests/core/pyspec/eth2spec/test/altair/unittests/test_sync_protocol.py#L25-L67
test "test_process_light_client_update_not_timeout":
let forked = assignClone(genesisState[])
template state: untyped {.inject.} = forked[].altairData.data
template state(): auto = forked[].altairData.data
var store = initialize_light_client_store(state)
# Block at slot 1 doesn't increase sync committee period,
# so it won't update snapshot
var cache = StateCache()
var cache: StateCache
let
signed_block = block_for_next_slot(cfg, forked[], cache).altairData
block_header = signed_block.toBeaconBlockHeader
attested_block = block_for_next_slot(cfg, forked[], cache).altairData
attested_header = attested_block.toBeaconBlockHeader
# Sync committee signing the block_header
signature_slot = block_header.slot + 1
all_pubkeys = state.validators.mapIt(it.pubkey)
committee = state.current_sync_committee.pubkeys
.mapIt(all_pubkeys.find(it).ValidatorIndex)
sync_committee_bits = full_sync_committee_bits
sync_committee_signature = compute_aggregate_sync_committee_signature(
forked[], signature_slot, committee, block_header.hash_tree_root())
sync_aggregate = SyncAggregate(
sync_committee_bits: sync_committee_bits,
sync_committee_signature: sync_committee_signature)
template next_sync_committee(): auto = state.next_sync_committee
var next_sync_committee_branch {.noinit.}:
array[log2trunc(altair.NEXT_SYNC_COMMITTEE_INDEX), Eth2Digest]
state.build_proof(
altair.NEXT_SYNC_COMMITTEE_INDEX, next_sync_committee_branch)
# Sync committee signing the attested_header
(sync_aggregate, signature_slot) = get_sync_aggregate(cfg, forked[])
next_sync_committee = SyncCommittee()
next_sync_committee_branch =
default(array[log2trunc(altair.NEXT_SYNC_COMMITTEE_INDEX), Eth2Digest])
# Ensure that finality checkpoint is genesis
check: state.finalized_checkpoint.epoch == 0
check state.finalized_checkpoint.epoch == 0
# Finality is unchanged
let
finality_header = BeaconBlockHeader()
pre_store_finalized_header = store.finalized_header
var finality_branch:
array[log2trunc(altair.FINALIZED_ROOT_INDEX), Eth2Digest]
finality_branch =
default(array[log2trunc(altair.FINALIZED_ROOT_INDEX), Eth2Digest])
let
update = altair.LightClientUpdate(
attested_header: block_header,
attested_header: attested_header,
next_sync_committee: next_sync_committee,
next_sync_committee_branch: next_sync_committee_branch,
finalized_header: finality_header,
finality_branch: finality_branch,
sync_aggregate: sync_aggregate,
fork_version: state.fork.current_version)
signature_slot: signature_slot)
pre_store_finalized_header = store.finalized_header
res = process_light_client_update(
store, update, signature_slot, cfg, state.genesis_validators_root)
check:
res.isOk
store.current_max_active_participants > 0
store.optimistic_header == update.attested_header
store.finalized_header == pre_store_finalized_header
store.best_valid_update.get == update
store.optimistic_header == update.attested_header
store.current_max_active_participants > 0
# https://github.com/ethereum/consensus-specs/blob/vFuture/tests/core/pyspec/eth2spec/test/altair/unittests/test_sync_protocol.py#L80-L136
# https://github.com/ethereum/consensus-specs/blob/vFuture/tests/core/pyspec/eth2spec/test/altair/unittests/test_sync_protocol.py#L70-L116
test "test_process_light_client_update_at_period_boundary":
var forked = assignClone(genesisState[])
template state: untyped {.inject.} = forked[].altairData.data
template state(): auto = forked[].altairData.data
var store = initialize_light_client_store(state)
# Forward to slot before next sync committee period so that next block is final one in period
# Forward to slot before next sync committee period so that next block is
# final one in period
var
cache = StateCache()
info = ForkedEpochInfo()
cache: StateCache
info: ForkedEpochInfo
process_slots(
cfg, forked[], Slot(UPDATE_TIMEOUT - 2), cache, info, flags = {}).expect("no failure")
cfg, forked[], Slot(UPDATE_TIMEOUT - 2), cache, info, flags = {}
).expect("no failure")
let
snapshot_period = sync_committee_period(store.optimistic_header.slot)
store_period = sync_committee_period(store.optimistic_header.slot)
update_period = sync_committee_period(state.slot)
check: snapshot_period == update_period
check: store_period == update_period
let
signed_block = block_for_next_slot(cfg, forked[], cache).altairData
block_header = signed_block.toBeaconBlockHeader
attested_block = block_for_next_slot(cfg, forked[], cache).altairData
attested_header = attested_block.toBeaconBlockHeader
# Sync committee signing the block_header
signature_slot = block_header.slot + 1
all_pubkeys = state.validators.mapIt(it.pubkey)
committee = state.next_sync_committee.pubkeys
.mapIt(all_pubkeys.find(it).ValidatorIndex)
sync_committee_bits = full_sync_committee_bits
sync_committee_signature = compute_aggregate_sync_committee_signature(
forked[], signature_slot, committee, block_header.hash_tree_root())
sync_aggregate = SyncAggregate(
sync_committee_bits: sync_committee_bits,
sync_committee_signature: sync_committee_signature)
# Sync committee is omitted (signed by next sync committee)
# Sync committee signing the attested_header
(sync_aggregate, signature_slot) = get_sync_aggregate(cfg, forked[])
next_sync_committee = SyncCommittee()
var next_sync_committee_branch:
array[log2trunc(altair.NEXT_SYNC_COMMITTEE_INDEX), Eth2Digest]
next_sync_committee_branch =
default(array[log2trunc(altair.NEXT_SYNC_COMMITTEE_INDEX), Eth2Digest])
# Finality is unchanged
let
finality_header = BeaconBlockHeader()
pre_store_finalized_header = store.finalized_header
var finality_branch:
array[log2trunc(altair.FINALIZED_ROOT_INDEX), Eth2Digest]
finality_branch =
default(array[log2trunc(altair.FINALIZED_ROOT_INDEX), Eth2Digest])
let
update = altair.LightClientUpdate(
attested_header: block_header,
attested_header: attested_header,
next_sync_committee: next_sync_committee,
next_sync_committee_branch: next_sync_committee_branch,
finalized_header: finality_header,
finality_branch: finality_branch,
sync_aggregate: sync_aggregate,
fork_version: state.fork.current_version)
signature_slot: signature_slot)
pre_store_finalized_header = store.finalized_header
res = process_light_client_update(
store, update, signature_slot, cfg, state.genesis_validators_root)
check:
res.isOk
store.current_max_active_participants > 0
store.optimistic_header == update.attested_header
store.finalized_header == pre_store_finalized_header
store.best_valid_update.get == update
store.optimistic_header == update.attested_header
store.current_max_active_participants > 0
# https://github.com/ethereum/consensus-specs/blob/vFuture/tests/core/pyspec/eth2spec/test/altair/unittests/test_sync_protocol.py#L139-L193
# https://github.com/ethereum/consensus-specs/blob/vFuture/tests/core/pyspec/eth2spec/test/altair/unittests/test_sync_protocol.py#L119-L166
test "process_light_client_update_timeout":
let forked = assignClone(genesisState[])
template state: untyped {.inject.} = forked[].altairData.data
template state(): auto = forked[].altairData.data
var store = initialize_light_client_store(state)
# Forward to next sync committee period
var
cache = StateCache()
info = ForkedEpochInfo()
cache: StateCache
info: ForkedEpochInfo
process_slots(
cfg, forked[], Slot(UPDATE_TIMEOUT), cache, info, flags = {}).expect("no failure")
cfg, forked[], Slot(UPDATE_TIMEOUT), cache, info, flags = {}
).expect("no failure")
let
snapshot_period = sync_committee_period(store.optimistic_header.slot)
store_period = sync_committee_period(store.optimistic_header.slot)
update_period = sync_committee_period(state.slot)
check: snapshot_period + 1 == update_period
check: store_period + 1 == update_period
let
signed_block = block_for_next_slot(cfg, forked[], cache).altairData
block_header = signed_block.toBeaconBlockHeader
attested_block = block_for_next_slot(cfg, forked[], cache).altairData
attested_header = attested_block.toBeaconBlockHeader
# Sync committee signing the block_header
signature_slot = block_header.slot + 1
all_pubkeys = state.validators.mapIt(it.pubkey)
committee = state.current_sync_committee.pubkeys
.mapIt(all_pubkeys.find(it).ValidatorIndex)
sync_committee_bits = full_sync_committee_bits
sync_committee_signature = compute_aggregate_sync_committee_signature(
forked[], signature_slot, committee, block_header.hash_tree_root())
sync_aggregate = SyncAggregate(
sync_committee_bits: sync_committee_bits,
sync_committee_signature: sync_committee_signature)
# Sync committee signing the attested_header
(sync_aggregate, signature_slot) = get_sync_aggregate(cfg, forked[])
# Sync committee is updated
template next_sync_committee(): auto = state.next_sync_committee
@ -259,119 +296,95 @@ suite "EF - Altair - Unittests - Sync protocol" & preset():
array[log2trunc(altair.NEXT_SYNC_COMMITTEE_INDEX), Eth2Digest]
state.build_proof(
altair.NEXT_SYNC_COMMITTEE_INDEX, next_sync_committee_branch)
# Finality is unchanged
let finality_header = BeaconBlockHeader()
var finality_branch:
array[log2trunc(altair.FINALIZED_ROOT_INDEX), Eth2Digest]
# Finality is unchanged
let
finality_header = BeaconBlockHeader()
finality_branch =
default(array[log2trunc(altair.FINALIZED_ROOT_INDEX), Eth2Digest])
update = altair.LightClientUpdate(
attested_header: block_header,
attested_header: attested_header,
next_sync_committee: next_sync_committee,
next_sync_committee_branch: next_sync_committee_branch,
finalized_header: finality_header,
finality_branch: finality_branch,
sync_aggregate: sync_aggregate,
fork_version: state.fork.current_version)
signature_slot: signature_slot)
pre_store_finalized_header = store.finalized_header
res = process_light_client_update(
store, update, signature_slot, cfg, state.genesis_validators_root)
check:
res.isOk
store.previous_max_active_participants > 0
store.finalized_header == pre_store_finalized_header
store.best_valid_update.get == update
store.optimistic_header == update.attested_header
store.finalized_header == update.attested_header
store.best_valid_update.isNone
store.current_max_active_participants > 0
# https://github.com/ethereum/consensus-specs/blob/vFuture/tests/core/pyspec/eth2spec/test/altair/unittests/test_sync_protocol.py#L196-L260
# https://github.com/ethereum/consensus-specs/blob/vFuture/tests/core/pyspec/eth2spec/test/altair/unittests/test_sync_protocol.py#L169-L223
test "process_light_client_update_finality_updated":
let forked = assignClone(genesisState[])
template state: untyped {.inject.} = forked[].altairData.data
template state(): auto = forked[].altairData.data
var store = initialize_light_client_store(state)
# Change finality
var
cache = StateCache()
info = ForkedEpochInfo()
cache: StateCache
info: ForkedEpochInfo
blocks = newSeq[ForkedSignedBeaconBlock]()
process_slots(
cfg, forked[], Slot(SLOTS_PER_EPOCH * 2), cache, info, flags = {}).expect("no failure")
for slot in 0 ..< SLOTS_PER_EPOCH:
blocks.add block_for_next_slot(cfg, forked[], cache,
withAttestations = true)
let finalized = assignClone(forked[])
template finalized_state: untyped {.inject.} = finalized[].altairData.data
for slot in 0 ..< SLOTS_PER_EPOCH:
blocks.add block_for_next_slot(cfg, forked[], cache,
withAttestations = true)
for slot in 0 ..< SLOTS_PER_EPOCH:
for slot in 0 ..< 3 * SLOTS_PER_EPOCH:
blocks.add block_for_next_slot(cfg, forked[], cache,
withAttestations = true)
# Ensure that finality checkpoint has changed
check: state.finalized_checkpoint.epoch == 3
check: state.finalized_checkpoint.root ==
mockBlockForNextSlot(finalized[]).altairData.message.parent_root
# Ensure that it's same period
let
snapshot_period = sync_committee_period(store.optimistic_header.slot)
store_period = sync_committee_period(store.optimistic_header.slot)
update_period = sync_committee_period(state.slot)
check: snapshot_period == update_period
check: store_period == update_period
let
attested_block = blocks[^1].altairData
attested_header = attested_block.toBeaconBlockHeader
# Sync committee signing the attested_header
(sync_aggregate, signature_slot) = get_sync_aggregate(cfg, forked[])
# Updated sync_committee and finality
template next_sync_committee(): auto = finalized_state.next_sync_committee
var next_sync_committee_branch {.noinit.}:
array[log2trunc(altair.NEXT_SYNC_COMMITTEE_INDEX), Eth2Digest]
finalized_state.build_proof(
altair.NEXT_SYNC_COMMITTEE_INDEX, next_sync_committee_branch)
let
next_sync_committee = SyncCommittee()
next_sync_committee_branch =
default(array[log2trunc(altair.NEXT_SYNC_COMMITTEE_INDEX), Eth2Digest])
finalized_block = blocks[SLOTS_PER_EPOCH - 1].altairData
finalized_block_header = finalized_block.toBeaconBlockHeader
finalized_header = finalized_block.toBeaconBlockHeader
check:
finalized_block_header.slot ==
start_slot(state.finalized_checkpoint.epoch)
finalized_block_header.hash_tree_root() ==
state.finalized_checkpoint.root
finalized_header.slot == start_slot(state.finalized_checkpoint.epoch)
finalized_header.hash_tree_root() == state.finalized_checkpoint.root
var finality_branch {.noinit.}:
array[log2trunc(altair.FINALIZED_ROOT_INDEX), Eth2Digest]
state.build_proof(
altair.FINALIZED_ROOT_INDEX, finality_branch)
# Build block header
let
blck = mockBlock(forked[], state.slot, cfg = cfg).altairData.message
block_header = BeaconBlockHeader(
slot: blck.slot,
proposer_index: blck.proposer_index,
parent_root: blck.parent_root,
state_root: state.hash_tree_root(),
body_root: blck.body.hash_tree_root())
# Sync committee signing the finalized_block_header
signature_slot = block_header.slot + 1
all_pubkeys = state.validators.mapIt(it.pubkey)
committee = state.current_sync_committee.pubkeys
.mapIt(all_pubkeys.find(it).ValidatorIndex)
sync_committee_bits = full_sync_committee_bits
sync_committee_signature = compute_aggregate_sync_committee_signature(
forked[], signature_slot, committee, block_header.hash_tree_root())
sync_aggregate = SyncAggregate(
sync_committee_bits: sync_committee_bits,
sync_committee_signature: sync_committee_signature)
update = altair.LightClientUpdate(
attested_header: block_header,
attested_header: attested_header,
next_sync_committee: next_sync_committee,
next_sync_committee_branch: next_sync_committee_branch,
finalized_header: finalized_block_header,
finalized_header: finalized_header,
finality_branch: finality_branch,
sync_aggregate: sync_aggregate,
fork_version: state.fork.current_version)
signature_slot: signature_slot)
res = process_light_client_update(
store, update, signature_slot, cfg, state.genesis_validators_root)
check:
res.isOk
store.current_max_active_participants > 0
store.optimistic_header == update.attested_header
store.finalized_header == update.finalized_header
store.best_valid_update.isNone
store.optimistic_header == update.attested_header
store.current_max_active_participants > 0

View File

@ -33,18 +33,15 @@ type
trusted_block_root: string
TestStepKind {.pure.} = enum
ProcessSlot
ForceUpdate
ProcessUpdate
ProcessOptimisticUpdate
TestStep = object
case kind: TestStepKind
of TestStepKind.ProcessSlot:
of TestStepKind.ForceUpdate:
discard
of TestStepKind.ProcessUpdate:
update: altair.LightClientUpdate
of TestStepKind.ProcessOptimisticUpdate:
optimistic_update: OptimisticLightClientUpdate
current_slot: Slot
proc loadSteps(path: string): seq[TestStep] =
@ -53,9 +50,9 @@ proc loadSteps(path: string): seq[TestStep] =
result = @[]
for step in steps[0]:
if step.hasKey"process_slot":
let s = step["process_slot"]
result.add TestStep(kind: TestStepKind.ProcessSlot,
if step.hasKey"force_update":
let s = step["force_update"]
result.add TestStep(kind: TestStepKind.ForceUpdate,
current_slot: s["current_slot"].getInt().Slot)
elif step.hasKey"process_update":
let
@ -66,15 +63,6 @@ proc loadSteps(path: string): seq[TestStep] =
result.add TestStep(kind: TestStepKind.ProcessUpdate,
update: update,
current_slot: s["current_slot"].getInt().Slot)
elif step.hasKey"process_optimistic_update":
let
s = step["process_optimistic_update"]
filename = s["optimistic_update"].getStr()
optimistic_update = parseTest(path/filename & ".ssz_snappy", SSZ,
OptimisticLightClientUpdate)
result.add TestStep(kind: TestStepKind.ProcessOptimisticUpdate,
optimistic_update: optimistic_update,
current_slot: s["current_slot"].getInt().Slot)
else:
doAssert false, "Unreachable: " & $step
@ -114,19 +102,14 @@ proc runTest(identifier: string) =
for step in steps:
case step.kind
of TestStepKind.ProcessSlot:
process_slot_for_light_client_store(
of TestStepKind.ForceUpdate:
try_light_client_store_force_update(
store, step.current_slot)
of TestStepKind.ProcessUpdate:
let res = process_light_client_update(
store, step.update, step.current_slot,
cfg, genesis_validators_root)
check res.isOk
of TestStepKind.ProcessOptimisticUpdate:
let res = process_optimistic_light_client_update(
store, step.optimistic_update, step.current_slot,
cfg, genesis_validators_root)
check res.isOk
check:
store.finalized_header == expected_finalized_header

View File

@ -0,0 +1,88 @@
# beacon_chain
# Copyright (c) 2022 Status Research & Development GmbH
# Licensed and distributed under either of
# * MIT license (license terms in the root directory or at https://opensource.org/licenses/MIT).
# * Apache v2 license (license terms in the root directory or at https://www.apache.org/licenses/LICENSE-2.0).
# at your option. This file may not be copied, modified, or distributed except according to those terms.
{.used.}
# This implements the pre-release proposal of the libp2p based light client sync
# protocol. See https://github.com/ethereum/consensus-specs/pull/2802
import
# Standard library
std/[algorithm, os, streams],
# Status libraries
stew/base10,
# Third-party
yaml,
# Beacon chain internals
../../../beacon_chain/spec/helpers,
../../../beacon_chain/spec/datatypes/altair,
# Test utilities
../../testutil,
../fixtures_utils
const TestsDir =
SszTestsDir/const_preset/"altair"/"sync_protocol"/"update_ranking"/"pyspec_tests"
type
TestMeta = object
updates_count: uint64
proc runTest(identifier: string) =
let testDir = TestsDir / identifier
proc `testImpl _ sync_protocol_update_ranking _ identifier`() =
test identifier:
let meta = block:
var s = openFileStream(testDir/"meta.yaml")
defer: close(s)
var res: TestMeta
yaml.load(s, res)
res
var updates = newSeqOfCap[altair.LightClientUpdate](meta.updates_count)
for i in 0 ..< meta.updates_count:
updates.add parseTest(
testDir/"updates_" & Base10.toString(i) & ".ssz_snappy",
SSZ, altair.LightClientUpdate)
proc cmp(a, b: altair.LightClientUpdate): int =
if a.is_better_update(b):
check: not b.is_better_update(a)
-1
elif b.is_better_update(a):
1
else:
0
check: updates.isSorted(cmp)
`testImpl _ sync_protocol_update_ranking _ identifier`()
suite "EF - Altair - Sync protocol - Update ranking" & preset():
try:
for kind, path in walkDir(TestsDir, relative = true, checkDir = true):
runTest(path)
except OSError:
# These tests are for the pre-release proposal of the libp2p based light
# client sync protocol. Corresponding test vectors need manual integration.
# https://github.com/ethereum/consensus-specs/pull/2802
#
# To locally integrate the test vectors, clone the pre-release spec repo
# at latest commit of https://github.com/ethereum/consensus-specs/pull/2802
# and place it next to the `nimbus-eth2` repo, so that `nimbus-eth2` and
# `consensus-specs` are in the same directory.
#
# To generate the additional test vectors, from `consensus-specs`:
# $ rm -rf ../consensus-spec-tests && \
# doctoc specs && make lint && make gen_sync_protocol
#
# To integrate the additional test vectors into `nimbus-eth2`, first run
# `make test` from `nimbus-eth2` to ensure that the regular test vectors
# have been downloaded and extracted, then proceed from `nimbus-eth2` with:
# $ rsync -r ../consensus-spec-tests/tests/ \
# ../nimbus-eth2/vendor/nim-eth2-scenarios/tests-v1.1.10/
test "All tests":
skip()

View File

@ -117,7 +117,14 @@ suite "EF - Bellatrix - SSZ consensus objects " & preset():
of "ForkData": checkSSZ(ForkData, path, hash)
of "HistoricalBatch": checkSSZ(HistoricalBatch, path, hash)
of "IndexedAttestation": checkSSZ(IndexedAttestation, path, hash)
of "LightClientUpdate": checkSSZ(LightClientUpdate, path, hash)
of "LightClientBootstrap":
checkSSZ(LightClientBootstrap, path, hash)
of "LightClientUpdate":
discard # Modified - checkSSZ(LightClientUpdate, path, hash)
of "LightClientFinalityUpdate":
checkSSZ(LightClientFinalityUpdate, path, hash)
of "LightClientOptimisticUpdate":
checkSSZ(LightClientOptimisticUpdate, path, hash)
of "PendingAttestation": checkSSZ(PendingAttestation, path, hash)
of "PowBlock": checkSSZ(PowBlock, path, hash)
of "ProposerSlashing": checkSSZ(ProposerSlashing, path, hash)

View File

@ -1,3 +1,10 @@
# beacon_chain
# Copyright (c) 2020-2022 Status Research & Development GmbH
# Licensed and distributed under either of
# * MIT license (license terms in the root directory or at https://opensource.org/licenses/MIT).
# * Apache v2 license (license terms in the root directory or at https://www.apache.org/licenses/LICENSE-2.0).
# at your option. This file may not be copied, modified, or distributed except according to those terms.
{.used.}
import
@ -17,6 +24,8 @@ suite "Honest validator":
getAttesterSlashingsTopic(forkDigest) == "/eth2/00000000/attester_slashing/ssz_snappy"
getAggregateAndProofsTopic(forkDigest) == "/eth2/00000000/beacon_aggregate_and_proof/ssz_snappy"
getSyncCommitteeContributionAndProofTopic(forkDigest) == "/eth2/00000000/sync_committee_contribution_and_proof/ssz_snappy"
getLightClientFinalityUpdateTopic(forkDigest) == "/eth2/00000000/light_client_finality_update_v0/ssz_snappy"
getLightClientOptimisticUpdateTopic(forkDigest) == "/eth2/00000000/light_client_optimistic_update_v0/ssz_snappy"
test "Mainnet attestation topics":
check:

View File

@ -34,7 +34,7 @@ suite "Light client" & preset():
verifier: var BatchVerifier,
quarantine: var Quarantine,
attested = true,
syncCommitteeRatio = 0.75) =
syncCommitteeRatio = 0.82) =
var cache: StateCache
const maxAttestedSlotsPerPeriod = 3 * SLOTS_PER_EPOCH
while true:
@ -89,22 +89,25 @@ suite "Light client" & preset():
# Genesis
check:
dag.headState.kind == BeaconStateFork.Phase0
dag.getBestLightClientUpdateForPeriod(0.SyncCommitteePeriod).isNone
dag.getLatestLightClientUpdate.isNone
dag.getLightClientUpdateForPeriod(0.SyncCommitteePeriod).isNone
dag.getLightClientFinalityUpdate.isNone
dag.getLightClientOptimisticUpdate.isNone
# Advance to last slot before Altair
dag.advanceToSlot(altairStartSlot - 1, verifier, quarantine[])
check:
dag.headState.kind == BeaconStateFork.Phase0
dag.getBestLightClientUpdateForPeriod(0.SyncCommitteePeriod).isNone
dag.getLatestLightClientUpdate.isNone
dag.getLightClientUpdateForPeriod(0.SyncCommitteePeriod).isNone
dag.getLightClientFinalityUpdate.isNone
dag.getLightClientOptimisticUpdate.isNone
# Advance to Altair
dag.advanceToSlot(altairStartSlot, verifier, quarantine[])
check:
dag.headState.kind == BeaconStateFork.Altair
dag.getBestLightClientUpdateForPeriod(0.SyncCommitteePeriod).isNone
dag.getLatestLightClientUpdate.isNone
dag.getLightClientUpdateForPeriod(0.SyncCommitteePeriod).isNone
dag.getLightClientFinalityUpdate.isNone
dag.getLightClientOptimisticUpdate.isNone
test "Light client sync":
# Advance to Altair
@ -136,32 +139,35 @@ suite "Light client" & preset():
while store.finalized_header.slot.sync_committee_period + 1 < headPeriod:
let
period =
if store.next_sync_committee.isZeroMemory:
store.finalized_header.slot.sync_committee_period
else:
if store.is_next_sync_committee_known:
store.finalized_header.slot.sync_committee_period + 1
bestUpdate = dag.getBestLightClientUpdateForPeriod(period)
else:
store.finalized_header.slot.sync_committee_period
update = dag.getLightClientUpdateForPeriod(period)
res = process_light_client_update(
store, bestUpdate.get, currentSlot, cfg, genesis_validators_root)
store, update.get, currentSlot, cfg, genesis_validators_root)
check:
bestUpdate.isSome
bestUpdate.get.finalized_header.slot.sync_committee_period == period
update.isSome
update.get.finalized_header.slot.sync_committee_period == period
res.isOk
store.finalized_header == bestUpdate.get.finalized_header
if update.get.finalized_header.slot > bootstrap.get.header.slot:
store.finalized_header == update.get.finalized_header
else:
store.finalized_header == bootstrap.get.header
inc numIterations
if numIterations > 20: doAssert false # Avoid endless loop on test failure
# Sync to latest update
let
latestUpdate = dag.getLatestLightClientUpdate
finalityUpdate = dag.getLightClientFinalityUpdate
res = process_light_client_update(
store, latestUpdate.get, currentSlot, cfg, genesis_validators_root)
store, finalityUpdate.get, currentSlot, cfg, genesis_validators_root)
check:
latestUpdate.isSome
latestUpdate.get.attested_header.slot == dag.head.parent.slot
finalityUpdate.isSome
finalityUpdate.get.attested_header.slot == dag.head.parent.slot
res.isOk
store.finalized_header == latestUpdate.get.finalized_header
store.optimistic_header == latestUpdate.get.attested_header
store.finalized_header == finalityUpdate.get.finalized_header
store.optimistic_header == finalityUpdate.get.attested_header
test "Init from checkpoint":
# Fetch genesis state

View File

@ -54,7 +54,7 @@ suite "Light client processor" & preset():
doAssert added.isOk()
dag.updateHead(added[], quarantine[])
addBlocks(SLOTS_PER_EPOCH, 0.75)
addBlocks(SLOTS_PER_EPOCH, 0.82)
let
genesis_validators_root = dag.genesis_validators_root
trustedBlockRoot = dag.head.root
@ -71,9 +71,9 @@ suite "Light client processor" & preset():
cache, info, flags = {}).isOk()
let syncCommitteeRatio =
if period > lastPeriodWithSupermajority:
0.25
0.62
else:
0.75
0.82
addBlocks(numFilledEpochsPerPeriod * SLOTS_PER_EPOCH, syncCommitteeRatio)
setup:
@ -93,7 +93,7 @@ suite "Light client processor" & preset():
store, getBeaconTime, didInitializeStore)
res: Result[void, BlockError]
test "Standard sync" & preset():
test "Sync" & preset():
let bootstrap = dag.getLightClientBootstrap(trustedBlockRoot)
check bootstrap.isOk
setTimeToSlot(bootstrap.get.header.slot)
@ -104,92 +104,77 @@ suite "Light client processor" & preset():
numDidInitializeStoreCalls == 1
for period in lowPeriod .. lastPeriodWithSupermajority:
let update = dag.getBestLightClientUpdateForPeriod(period)
let update = dag.getLightClientUpdateForPeriod(period)
check update.isSome
setTimeToSlot(update.get.attested_header.slot + 1)
setTimeToSlot(update.get.signature_slot)
res = processor[].storeObject(
MsgSource.gossip, getBeaconTime(), update.get)
check:
res.isOk
store[].isSome
store[].get.finalized_header == update.get.finalized_header
store[].get.optimistic_header == update.get.attested_header
test "Forced update" & preset():
let bootstrap = dag.getLightClientBootstrap(trustedBlockRoot)
check bootstrap.isOk
setTimeToSlot(bootstrap.get.header.slot)
res = processor[].storeObject(
MsgSource.gossip, getBeaconTime(), bootstrap.get)
check:
res.isOk
numDidInitializeStoreCalls == 1
for period in lowPeriod .. lastPeriodWithSupermajority:
let update = dag.getBestLightClientUpdateForPeriod(period)
check update.isSome
setTimeToSlot(update.get.attested_header.slot + 1)
res = processor[].storeObject(
MsgSource.gossip, getBeaconTime(), update.get)
check:
res.isOk
store[].isSome
store[].get.finalized_header == update.get.finalized_header
if update.get.finalized_header.slot > bootstrap.get.header.slot:
store[].get.finalized_header == update.get.finalized_header
else:
store[].get.finalized_header == bootstrap.get.header
store[].get.optimistic_header == update.get.attested_header
for period in lastPeriodWithSupermajority + 1 .. highPeriod:
let update = dag.getBestLightClientUpdateForPeriod(period)
let update = dag.getLightClientUpdateForPeriod(period)
check update.isSome
setTimeToSlot(update.get.attested_header.slot + 1)
res = processor[].storeObject(
MsgSource.gossip, getBeaconTime(), update.get)
check:
res.isOk
store[].isSome
store[].get.best_valid_update.isSome
store[].get.best_valid_update.get == update.get
setTimeToSlot(update.get.signature_slot)
res = processor[].storeObject(
MsgSource.gossip, getBeaconTime(), update.get)
check:
res.isErr
res.error == BlockError.Duplicate
store[].isSome
store[].get.best_valid_update.isSome
store[].get.best_valid_update.get == update.get
time += chronos.minutes(15)
for i in 0 ..< 2:
res = processor[].storeObject(
MsgSource.gossip, getBeaconTime(), update.get)
check:
res.isOk
store[].isSome
store[].get.best_valid_update.isSome
store[].get.best_valid_update.get == update.get
proc applyDuplicate() = # Reduce stack size by making this a `proc`
res = processor[].storeObject(
MsgSource.gossip, getBeaconTime(), update.get)
check:
res.isErr
res.error == BlockError.Duplicate
store[].isSome
store[].get.best_valid_update.isSome
store[].get.best_valid_update.get == update.get
applyDuplicate()
time += chronos.minutes(15)
for _ in 0 ..< 150:
applyDuplicate()
time += chronos.seconds(5)
time += chronos.minutes(15)
for _ in 0 ..< 150:
time += chronos.seconds(5)
res = processor[].storeObject(
MsgSource.gossip, getBeaconTime(), update.get)
check:
res.isErr
res.error == BlockError.Duplicate
store[].isSome
store[].get.best_valid_update.isSome
store[].get.best_valid_update.get == update.get
store[].get.best_valid_update.isNone
if store[].get.finalized_header == update.get.attested_header:
break
check store[].get.finalized_header == update.get.finalized_header
check store[].get.finalized_header == update.get.attested_header
time += chronos.minutes(15)
res = processor[].storeObject(
MsgSource.gossip, getBeaconTime(), update.get)
check:
res.isErr
res.error == BlockError.Duplicate
store[].isSome
store[].get.best_valid_update.isNone
store[].get.finalized_header == update.get.finalized_header
let optimisticUpdate = dag.getOptimisticLightClientUpdate()
check optimisticUpdate.isSome
setTimeToSlot(optimisticUpdate.get.attested_header.slot + 1)
let
previousFinalized = store[].get.finalized_header
finalityUpdate = dag.getLightClientFinalityUpdate()
check finalityUpdate.isSome
setTimeToSlot(finalityUpdate.get.signature_slot)
res = processor[].storeObject(
MsgSource.gossip, getBeaconTime(), optimisticUpdate.get)
MsgSource.gossip, getBeaconTime(), finalityUpdate.get)
if res.isOk:
check:
store[].isSome
store[].get.optimistic_header == optimisticUpdate.get.attested_header
store[].get.finalized_header == previousFinalized
store[].get.best_valid_update.isSome
store[].get.best_valid_update.get.matches(finalityUpdate.get)
store[].get.optimistic_header == finalityUpdate.get.attested_header
else:
check res.error == BlockError.Duplicate
check numDidInitializeStoreCalls == 1
@ -223,9 +208,9 @@ suite "Light client processor" & preset():
numDidInitializeStoreCalls == 1
test "Missing bootstrap (update)" & preset():
let update = dag.getBestLightClientUpdateForPeriod(lowPeriod)
let update = dag.getLightClientUpdateForPeriod(lowPeriod)
check update.isSome
setTimeToSlot(update.get.attested_header.slot + 1)
setTimeToSlot(update.get.signature_slot)
res = processor[].storeObject(
MsgSource.gossip, getBeaconTime(), update.get)
check:
@ -233,10 +218,21 @@ suite "Light client processor" & preset():
res.error == BlockError.MissingParent
numDidInitializeStoreCalls == 0
test "Missing bootstrap (finality update)" & preset():
let finalityUpdate = dag.getLightClientFinalityUpdate()
check finalityUpdate.isSome
setTimeToSlot(finalityUpdate.get.signature_slot)
res = processor[].storeObject(
MsgSource.gossip, getBeaconTime(), finalityUpdate.get)
check:
res.isErr
res.error == BlockError.MissingParent
numDidInitializeStoreCalls == 0
test "Missing bootstrap (optimistic update)" & preset():
let optimisticUpdate = dag.getOptimisticLightClientUpdate()
let optimisticUpdate = dag.getLightClientOptimisticUpdate()
check optimisticUpdate.isSome
setTimeToSlot(optimisticUpdate.get.attested_header.slot + 1)
setTimeToSlot(optimisticUpdate.get.signature_slot)
res = processor[].storeObject(
MsgSource.gossip, getBeaconTime(), optimisticUpdate.get)
check:

View File

@ -326,27 +326,32 @@ proc makeSyncAggregate(
latest_block_root =
withState(state): state.latest_block_root
syncCommitteePool = newClone(SyncCommitteeMsgPool.init(keys.newRng()))
type
Aggregator = object
subcommitteeIdx: SyncSubcommitteeIndex
validatorIdx: ValidatorIndex
selectionProof: ValidatorSig
var aggregators: seq[Aggregator]
let
minActiveParticipants =
if syncCommitteeRatio >= 2.0 / 3: # Ensure supermajority is hit
(SYNC_COMMITTEE_SIZE * 2 + 2) div 3
else:
0
maxActiveParticipants = (syncCommitteeRatio * SYNC_COMMITTEE_SIZE).int
var
aggregators: seq[Aggregator]
numActiveParticipants = 0
for subcommitteeIdx in SyncSubcommitteeIndex:
let
firstKeyIdx = subcommitteeIdx.int * SYNC_SUBCOMMITTEE_SIZE
lastKeyIdx = firstKeyIdx + SYNC_SUBCOMMITTEE_SIZE - 1
var processedKeys = initHashSet[ValidatorPubKey]()
for idx, validatorKey in syncCommittee.pubkeys[firstKeyIdx .. lastKeyIdx]:
if validatorKey in processedKeys: continue
if validatorKey in processedKeys:
continue
processedKeys.incl validatorKey
if rand(1.0) > syncCommitteeRatio: continue
var positions: seq[uint64]
for pos, key in syncCommittee.pubkeys[firstKeyIdx + idx .. lastKeyIdx]:
if key == validatorKey:
positions.add (idx + pos).uint64
let
validatorIdx =
block:
@ -356,15 +361,33 @@ proc makeSyncAggregate(
res = i
break
res.ValidatorIndex
signature = get_sync_committee_message_signature(
fork, genesis_validators_root,
slot, latest_block_root,
MockPrivKeys[validatorIdx])
selectionProofSig = get_sync_committee_selection_proof(
fork, genesis_validators_root,
slot, subcommitteeIdx,
MockPrivKeys[validatorIdx])
if is_sync_committee_aggregator(selectionProofSig.toValidatorSig):
aggregators.add Aggregator(
subcommitteeIdx: subcommitteeIdx,
validatorIdx: validatorIdx,
selectionProof: selectionProofSig.toValidatorSig)
if numActiveParticipants >= minActiveParticipants and
rand(1.0) > syncCommitteeRatio:
continue
var positions: seq[uint64]
for pos, key in syncCommittee.pubkeys[firstKeyIdx + idx .. lastKeyIdx]:
if numActiveParticipants >= maxActiveParticipants:
break
if key == validatorKey:
positions.add (idx + pos).uint64
inc numActiveParticipants
if positions.len == 0:
continue
let signature = get_sync_committee_message_signature(
fork, genesis_validators_root,
slot, latest_block_root,
MockPrivKeys[validatorIdx])
syncCommitteePool[].addSyncCommitteeMessage(
slot,
latest_block_root,
@ -373,12 +396,6 @@ proc makeSyncAggregate(
subcommitteeIdx,
positions)
if is_sync_committee_aggregator(selectionProofSig.toValidatorSig):
aggregators.add Aggregator(
subcommitteeIdx: subcommitteeIdx,
validatorIdx: validatorIdx,
selectionProof: selectionProofSig.toValidatorSig)
for aggregator in aggregators:
var contribution: SyncCommitteeContribution
if syncCommitteePool[].produceContribution(