track latest `LightClientUpdate` only once fork choice selects it (#5691)

Instead of tracking the latest `LightClientUpdate` across all branches,
track the latest one on the current branch as selected by fork choice.
This commit is contained in:
Etan Kissling 2024-01-03 23:36:05 +01:00 committed by GitHub
parent fd4c493708
commit 7db95f047b
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
3 changed files with 308 additions and 218 deletions

View File

@ -1,5 +1,5 @@
# beacon_chain
# Copyright (c) 2022-2023 Status Research & Development GmbH
# Copyright (c) 2022-2024 Status Research & Development GmbH
# Licensed and distributed under either of
# * MIT license (license terms in the root directory or at https://opensource.org/licenses/MIT).
# * Apache v2 license (license terms in the root directory or at https://www.apache.org/licenses/LICENSE-2.0).
@ -40,6 +40,7 @@ type
finality_branch*: altair.FinalityBranch
current_period_best_update*: ref ForkedLightClientUpdate
latest_signature_slot*: Slot
LightClientDataCache* = object
data*: Table[BlockId, CachedLightClientData]
@ -54,6 +55,9 @@ type
tailSlot*: Slot
## The earliest slot for which light client data is imported.
recentHeaders*: OrderedTable[BlockId, ForkedLightClientHeader]
recentSyncAggregates*: OrderedTable[BlockId, SyncAggregate]
LightClientDataConfig* = object
serve*: bool
## Whether to make local light client data available or not

View File

@ -1,5 +1,5 @@
# beacon_chain
# Copyright (c) 2022-2023 Status Research & Development GmbH
# Copyright (c) 2022-2024 Status Research & Development GmbH
# Licensed and distributed under either of
# * MIT license (license terms in the root directory or at https://opensource.org/licenses/MIT).
# * Apache v2 license (license terms in the root directory or at https://www.apache.org/licenses/LICENSE-2.0).
@ -48,7 +48,7 @@ template withUpdatedExistingState(
proc getExistingBlockIdAtSlot(dag: ChainDAGRef, slot: Slot): Opt[BlockSlotId] =
## Wrapper around `getBlockIdAtSlot` for blocks expected to exist.
let bsi = dag.getBlockIdAtSlot(slot)
if bsi.isErr:
if bsi.isNone:
error "Block failed to load unexpectedly", slot, tail = dag.tail.slot
doAssert strictVerification notin dag.updateFlags
bsi
@ -56,7 +56,7 @@ proc getExistingBlockIdAtSlot(dag: ChainDAGRef, slot: Slot): Opt[BlockSlotId] =
proc existingParent(dag: ChainDAGRef, bid: BlockId): Opt[BlockId] =
## Wrapper around `parent` for parents known to exist.
let parent = dag.parent(bid)
if parent.isErr:
if parent.isNone:
error "Parent failed to load unexpectedly", bid, tail = dag.tail.slot
doAssert strictVerification notin dag.updateFlags
parent
@ -65,7 +65,7 @@ proc getExistingForkedBlock(
dag: ChainDAGRef, bid: BlockId): Opt[ForkedTrustedSignedBeaconBlock] =
## Wrapper around `getForkedBlock` for blocks expected to exist.
let bdata = dag.getForkedBlock(bid)
if bdata.isErr:
if bdata.isNone:
error "Block failed to load unexpectedly", bid, tail = dag.tail.slot
doAssert strictVerification notin dag.updateFlags
bdata
@ -76,12 +76,71 @@ proc existingCurrentSyncCommitteeForPeriod(
period: SyncCommitteePeriod): Opt[SyncCommittee] =
## Wrapper around `currentSyncCommitteeForPeriod` for states known to exist.
let syncCommittee = dag.currentSyncCommitteeForPeriod(tmpState, period)
if syncCommittee.isErr:
if syncCommittee.isNone:
error "Current sync committee failed to load unexpectedly",
period, tail = dag.tail.slot
doAssert strictVerification notin dag.updateFlags
syncCommittee
func pruneToCapacity[A, B](t: var OrderedTable[A, B], capacity: int) =
while t.len > capacity:
var key {.noinit.}: A
for k in t.keys:
key = k
break
t.del(key)
func cacheRecentLightClientHeader(
dag: ChainDAGRef, bid: BlockId, header: ForkedLightClientHeader) =
dag.lcDataStore.cache.recentHeaders[bid] = header
dag.lcDataStore.cache.recentHeaders.pruneToCapacity(10)
func cacheRecentSyncAggregate(
dag: ChainDAGRef, bid: BlockId, syncAggregate: SyncAggregate) =
dag.lcDataStore.cache.recentSyncAggregates[bid] = syncAggregate
dag.lcDataStore.cache.recentSyncAggregates.pruneToCapacity(5)
func lightClientHeader(
blck: ForkyTrustedSignedBeaconBlock): ForkedLightClientHeader =
const lcDataFork = max(
lcDataForkAtConsensusFork(typeof(blck).kind), LightClientDataFork.Altair)
ForkedLightClientHeader.init(blck.toLightClientHeader(lcDataFork))
func syncAggregate(
blck: ForkyTrustedSignedBeaconBlock): SyncAggregate =
blck.asSigned().message.body.sync_aggregate
proc getExistingLightClientHeader(
dag: ChainDAGRef, bid: BlockId): ForkedLightClientHeader =
var res = dag.lcDataStore.cache.recentHeaders.getOrDefault(bid)
if res.kind > LightClientDataFork.None:
return res
let bdata = dag.getExistingForkedBlock(bid)
if bdata.isNone:
return res
res = withBlck(bdata.get): forkyBlck.lightClientHeader()
dag.cacheRecentLightClientHeader(bid, res)
res
proc getExistingSyncAggregate(
dag: ChainDAGRef, bid: BlockId): Opt[SyncAggregate] =
if bid in dag.lcDataStore.cache.recentSyncAggregates:
return Opt.some dag.lcDataStore.cache.recentSyncAggregates.getOrDefault(bid)
let bdata = dag.getExistingForkedBlock(bid)
if bdata.isNone:
return Opt.none(SyncAggregate)
let res = withBlck(bdata.get):
when consensusFork >= ConsensusFork.Altair:
Opt.some forkyBlck.syncAggregate()
else:
return Opt.none(SyncAggregate)
dag.cacheRecentSyncAggregate(bid, res.get)
res
proc initLightClientDataStore*(
config: LightClientDataConfig,
cfg: RuntimeConfig,
@ -392,24 +451,33 @@ proc getLightClientData(
except KeyError: raiseAssert "Unreachable"
proc cacheLightClientData(
dag: ChainDAGRef, state: ForkyHashedBeaconState, bid: BlockId,
current_period_best_update: ref ForkedLightClientUpdate) =
dag: ChainDAGRef,
state: ForkyHashedBeaconState,
blck: ForkyTrustedSignedBeaconBlock,
current_period_best_update: ref ForkedLightClientUpdate,
latest_signature_slot: Slot) =
## Cache data for a given block and its post-state to speed up creating future
## `LightClientUpdate` and `LightClientBootstrap` instances that refer to this
## block and state.
let cachedData = CachedLightClientData(
current_sync_committee_branch:
state.data.build_proof(altair.CURRENT_SYNC_COMMITTEE_GINDEX).get,
next_sync_committee_branch:
state.data.build_proof(altair.NEXT_SYNC_COMMITTEE_GINDEX).get,
finalized_slot:
state.data.finalized_checkpoint.epoch.start_slot,
finality_branch:
state.data.build_proof(altair.FINALIZED_ROOT_GINDEX).get,
current_period_best_update:
current_period_best_update)
let
bid = blck.toBlockId()
cachedData = CachedLightClientData(
current_sync_committee_branch:
state.data.build_proof(altair.CURRENT_SYNC_COMMITTEE_GINDEX).get,
next_sync_committee_branch:
state.data.build_proof(altair.NEXT_SYNC_COMMITTEE_GINDEX).get,
finalized_slot:
state.data.finalized_checkpoint.epoch.start_slot,
finality_branch:
state.data.build_proof(altair.FINALIZED_ROOT_GINDEX).get,
current_period_best_update:
current_period_best_update,
latest_signature_slot:
latest_signature_slot)
if dag.lcDataStore.cache.data.hasKeyOrPut(bid, cachedData):
doAssert false, "Redundant `cacheLightClientData` call"
dag.cacheRecentLightClientHeader(bid, blck.lightClientHeader())
dag.cacheRecentSyncAggregate(bid, blck.syncAggregate())
func shouldImportLcData(dag: ChainDAGRef): bool =
dag.lcDataStore.importMode != LightClientDataImportMode.None and
@ -423,209 +491,179 @@ proc deleteLightClientData*(dag: ChainDAGRef, bid: BlockId) =
dag.lcDataStore.cache.data.del bid
template lazy_header(name: untyped): untyped {.dirty.} =
## `createLightClientUpdates` helper to lazily load a known block header.
var
`name _ ptr`: ptr[data_fork.LightClientHeader]
`name _ ok` = true
template `assign _ name`(
obj: var SomeForkyLightClientObject, bid: BlockId): untyped {.used.} =
if `name _ ptr` != nil:
obj.name = `name _ ptr`[]
elif `name _ ok`:
let bdata = dag.getExistingForkedBlock(bid)
if bdata.isErr:
dag.handleUnexpectedLightClientError(bid.slot)
`name _ ok` = false
else:
withBlck(bdata.get):
when data_fork >= lcDataForkAtConsensusFork(consensusFork):
obj.name = forkyBlck.toLightClientHeader(data_fork)
else: raiseAssert "Unreachable"
`name _ ptr` = addr obj.name
`name _ ok`
template `assign _ name _ with_migration`(
obj: var SomeForkedLightClientObject, bid: BlockId): untyped {.used.} =
if `name _ ptr` != nil:
obj.migrateToDataFork(data_fork)
obj.forky(data_fork).name = `name _ ptr`[]
elif `name _ ok`:
let bdata = dag.getExistingForkedBlock(bid)
if bdata.isErr:
dag.handleUnexpectedLightClientError(bid.slot)
`name _ ok` = false
else:
obj.migrateToDataFork(data_fork)
withBlck(bdata.get):
when data_fork >= lcDataForkAtConsensusFork(consensusFork):
obj.forky(data_fork).name = forkyBlck.toLightClientHeader(data_fork)
else: raiseAssert "Unreachable"
`name _ ptr` = addr obj.forky(data_fork).name
`name _ ok`
template lazy_bid(name: untyped): untyped {.dirty.} =
## `createLightClientUpdates` helper to lazily load a known to exist block id.
var
`name` {.noinit.}: BlockId
`name _ ok` = true
`name`.slot = FAR_FUTURE_SLOT
template `load _ name`(slot: Slot): bool =
if `name _ ok` and `name`.slot == FAR_FUTURE_SLOT:
let bsi = dag.getExistingBlockIdAtSlot(slot)
if bsi.isErr:
dag.handleUnexpectedLightClientError(slot)
`name _ ok` = false
else:
`name` = bsi.get.bid
`name _ ok`
proc createLightClientUpdates(
proc assignLightClientData(
obj: var SomeForkedLightClientUpdateWithFinality,
dag: ChainDAGRef,
state: ForkyHashedBeaconState,
blck: ForkyTrustedSignedBeaconBlock,
parent_bid: BlockId,
data_fork: static LightClientDataFork): ref ForkedLightClientUpdate =
## Create `LightClientUpdate` instances for a given block and its post-state,
## and keep track of best / latest ones. Data about the parent block's
## post-state must be cached (`cacheLightClientData`) before calling this.
## Returns the best `LightClientUpdate` for the block's sync committee period.
attested_bid: BlockId,
signature_slot: Slot,
sync_aggregate: SyncAggregate,
next_sync_committee = Opt.none(SyncCommittee)): Opt[void] {.discardable.} =
## Update `obj` based on `attested_bid` / `signature_slot` / `sync_aggregate`
## and `next_sync_committee` (for full `LightClientUpdate`).
## Return `ok` if there were changes; `err` otherwise.
# Verify attested block (parent) is recent enough and that state is available
template attested_bid(): auto = parent_bid
let attested_slot = attested_bid.slot
if attested_slot < dag.lcDataStore.cache.tailSlot:
return (ref ForkedLightClientUpdate)()
# `blck` and `parent_bid` must be in the same sync committee period
# to update the best per-period `LightClientUpdate`
let
attested_period = attested_slot.sync_committee_period
signature_slot = blck.message.slot
signature_period = signature_slot.sync_committee_period
var
attested_data = dag.getLightClientData(attested_bid)
best =
if attested_period != signature_period:
(ref ForkedLightClientUpdate)()
else:
attested_data.current_period_best_update
# Verify sync committee has sufficient participants
template sync_aggregate(): auto = blck.asSigned().message.body.sync_aggregate
let num_active_participants = sync_aggregate.num_active_participants.uint64
if num_active_participants < MIN_SYNC_COMMITTEE_PARTICIPANTS:
return best
# Lazy variables to hold historic data
lazy_header(attested_header)
lazy_bid(finalized_bid)
lazy_header(finalized_header)
# Update latest light client data
template latest(): untyped = dag.lcDataStore.cache.latest
var
newFinality = false
newOptimistic = false
let is_later = withForkyFinalityUpdate(latest):
# If `sync_aggregate` is unchanged, the rest is also unchanged
withForkyObject(obj):
when lcDataFork > LightClientDataFork.None:
if attested_slot != forkyFinalityUpdate.attested_header.beacon.slot:
attested_slot > forkyFinalityUpdate.attested_header.beacon.slot
else:
signature_slot > forkyFinalityUpdate.signature_slot
if forkyObject.signature_slot == signature_slot and
forkyObject.sync_aggregate == sync_aggregate:
return err()
# Bump `attested_header`; if it is unchanged, finality info is also unchanged
let new_attested_header = withForkyObject(obj):
when lcDataFork > LightClientDataFork.None:
template beacon_header: untyped = forkyObject.attested_header.beacon
beacon_header.slot != attested_bid.slot or
beacon_header.hash_tree_root() != attested_bid.root
else:
true
if is_later and latest.assign_attested_header_with_migration(attested_bid):
template forkyLatest: untyped = latest.forky(data_fork)
var finalized_slot = attested_data.finalized_slot
if finalized_slot == forkyLatest.finalized_header.beacon.slot:
forkyLatest.finality_branch = attested_data.finality_branch
let old_num_active_participants =
forkyLatest.sync_aggregate.num_active_participants.uint64
if not hasSupermajoritySyncParticipation(old_num_active_participants) and
hasSupermajoritySyncParticipation(num_active_participants):
newFinality = true
elif finalized_slot < dag.tail.slot or
not load_finalized_bid(finalized_slot):
forkyLatest.finalized_header.reset()
forkyLatest.finality_branch.reset()
if new_attested_header:
let att_header = dag.getExistingLightClientHeader(attested_bid)
withForkyHeader(att_header):
when lcDataFork > LightClientDataFork.None:
obj.migrateToDataFork(lcDataFork)
obj.forky(lcDataFork).attested_header = forkyHeader
else:
dag.handleUnexpectedLightClientError(attested_bid.slot)
return err()
var attested_data = dag.getLightClientData(attested_bid)
when obj is SomeForkedLightClientUpdateWithSyncCommittee:
doAssert next_sync_committee.isSome
withForkyObject(obj):
when lcDataFork > LightClientDataFork.None:
forkyObject.next_sync_committee =
next_sync_committee.get
forkyObject.next_sync_committee_branch =
attested_data.next_sync_committee_branch
else:
if finalized_bid.slot != finalized_slot:
finalized_slot = finalized_bid.slot
attested_data.finalized_slot = finalized_slot
dag.lcDataStore.cache.data[attested_bid] = attested_data
if finalized_slot == forkyLatest.finalized_header.beacon.slot:
forkyLatest.finality_branch = attested_data.finality_branch
elif finalized_slot == GENESIS_SLOT:
forkyLatest.finalized_header.reset()
forkyLatest.finality_branch = attested_data.finality_branch
elif forkyLatest.assign_finalized_header(finalized_bid):
forkyLatest.finality_branch = attested_data.finality_branch
newFinality = true
else:
forkyLatest.finalized_header.reset()
forkyLatest.finality_branch.reset()
forkyLatest.sync_aggregate = sync_aggregate
forkyLatest.signature_slot = signature_slot
newOptimistic = true
# Track best light client data for current period
if attested_period == signature_period:
let
finalized_slot = attested_data.finalized_slot
has_finality =
finalized_slot >= dag.tail.slot and load_finalized_bid(finalized_slot)
meta = LightClientUpdateMetadata(
attested_slot: attested_slot,
finalized_slot: finalized_slot,
signature_slot: signature_slot,
has_sync_committee: true,
has_finality: has_finality,
num_active_participants: num_active_participants)
is_better = is_better_data(
meta, attested_data.current_period_best_update[].toMeta())
if is_better:
best = newClone attested_data.current_period_best_update[]
if not best[].assign_attested_header_with_migration(attested_bid):
best = attested_data.current_period_best_update
else:
template forkyBest: untyped = best[].forky(data_fork)
forkyBest.next_sync_committee = state.data.next_sync_committee
forkyBest.next_sync_committee_branch =
attested_data.next_sync_committee_branch
if finalized_slot == forkyBest.finalized_header.beacon.slot:
forkyBest.finality_branch = attested_data.finality_branch
elif finalized_slot == GENESIS_SLOT:
forkyBest.finalized_header.reset()
forkyBest.finality_branch = attested_data.finality_branch
elif has_finality and
forkyBest.assign_finalized_header(finalized_bid):
forkyBest.finality_branch = attested_data.finality_branch
doAssert next_sync_committee.isNone
var finalized_slot = attested_data.finalized_slot
withForkyObject(obj):
when lcDataFork > LightClientDataFork.None:
if finalized_slot == forkyObject.finalized_header.beacon.slot:
forkyObject.finality_branch = attested_data.finality_branch
elif finalized_slot < dag.tail.slot:
forkyObject.finalized_header.reset()
forkyObject.finality_branch.reset()
else:
forkyBest.finalized_header.reset()
forkyBest.finality_branch.reset()
forkyBest.sync_aggregate = sync_aggregate
forkyBest.signature_slot = signature_slot
debug "Best LC update improved",
period = attested_period, update = forkyBest
if newFinality and dag.lcDataStore.onLightClientFinalityUpdate != nil:
dag.lcDataStore.onLightClientFinalityUpdate(latest)
if newOptimistic and dag.lcDataStore.onLightClientOptimisticUpdate != nil:
dag.lcDataStore.onLightClientOptimisticUpdate(latest.toOptimistic)
best
let finalized_bsi = dag.getExistingBlockIdAtSlot(finalized_slot)
if finalized_bsi.isNone:
dag.handleUnexpectedLightClientError(finalized_slot)
forkyObject.finalized_header.reset()
forkyObject.finality_branch.reset()
else:
let finalized_bid = finalized_bsi.get.bid
if finalized_bid.slot != finalized_slot:
# Empty slots at end of epoch, update cache for latest block slot
finalized_slot = finalized_bid.slot
attested_data.finalized_slot = finalized_slot
dag.lcDataStore.cache.data[attested_bid] = attested_data
if finalized_slot == forkyObject.finalized_header.beacon.slot:
forkyObject.finality_branch = attested_data.finality_branch
elif finalized_slot == GENESIS_SLOT:
forkyObject.finalized_header.reset()
forkyObject.finality_branch = attested_data.finality_branch
else:
var fin_header = dag.getExistingLightClientHeader(finalized_bid)
if fin_header.kind == LightClientDataFork.None:
dag.handleUnexpectedLightClientError(finalized_bid.slot)
forkyObject.finalized_header.reset()
forkyObject.finality_branch.reset()
else:
fin_header.migrateToDataFork(lcDataFork)
forkyObject.finalized_header = fin_header.forky(lcDataFork)
forkyObject.finality_branch = attested_data.finality_branch
withForkyObject(obj):
when lcDataFork > LightClientDataFork.None:
forkyObject.sync_aggregate = sync_aggregate
forkyObject.signature_slot = signature_slot
ok()
proc createLightClientUpdates(
dag: ChainDAGRef,
state: ForkyHashedBeaconState,
blck: ForkyTrustedSignedBeaconBlock,
parent_bid: BlockId) =
# Attested block (parent) determines `LightClientUpdate` fork
let best = withLcDataFork(dag.cfg.lcDataForkAtEpoch(parent_bid.slot.epoch)):
when lcDataFork > LightClientDataFork.None:
dag.createLightClientUpdates(state, blck, parent_bid, lcDataFork)
else:
## Create `LightClientUpdate` instances for a given block and its post-state,
## and keep track of best / latest ones. Data about the parent block's
## post-state must be cached (`cacheLightClientData`) before calling this.
# Verify attested block (parent) is recent enough and that state is available
template attested_bid(): auto = parent_bid
let attested_slot = attested_bid.slot
if attested_slot < dag.lcDataStore.cache.tailSlot:
dag.cacheLightClientData(state, blck,
current_period_best_update = (ref ForkedLightClientUpdate)(),
latest_signature_slot = GENESIS_SLOT)
return
# If sync committee period changed, reset `best`
let
attested_period = attested_slot.sync_committee_period
signature_slot = blck.message.slot
signature_period = signature_slot.sync_committee_period
attested_data = dag.getLightClientData(attested_bid)
var best =
if attested_period != signature_period:
(ref ForkedLightClientUpdate)()
dag.cacheLightClientData(state, blck.toBlockId(), best)
else:
attested_data.current_period_best_update
# If sync committee does not have sufficient participants, do not bump latest
template sync_aggregate(): auto = blck.asSigned().message.body.sync_aggregate
let
num_active_participants = sync_aggregate.num_active_participants.uint64
latest_signature_slot =
if num_active_participants < MIN_SYNC_COMMITTEE_PARTICIPANTS:
attested_data.latest_signature_slot
else:
signature_slot
# To update `best`, sync committee must have sufficient participants, and
# `signature_slot` must be in `attested_slot`'s sync committee period
if num_active_participants < MIN_SYNC_COMMITTEE_PARTICIPANTS or
attested_period != signature_period:
dag.cacheLightClientData(state, blck,
current_period_best_update = best,
latest_signature_slot = latest_signature_slot)
return
# Check if light client data improved
let
finalized_slot = attested_data.finalized_slot
finalized_bsi = dag.getExistingBlockIdAtSlot(finalized_slot)
has_finality =
finalized_bsi.isSome and
finalized_bsi.get.bid.slot >= dag.tail.slot
meta = LightClientUpdateMetadata(
attested_slot: attested_slot,
finalized_slot: finalized_slot,
signature_slot: signature_slot,
has_sync_committee: true,
has_finality: has_finality,
num_active_participants: num_active_participants)
is_better = is_better_data(
meta, attested_data.current_period_best_update[].toMeta())
if not is_better:
dag.cacheLightClientData(state, blck,
current_period_best_update = best,
latest_signature_slot = latest_signature_slot)
return
# Update best light client data for current sync committee period
best = newClone attested_data.current_period_best_update[]
let res = best[].assignLightClientData(
dag, attested_bid, signature_slot, sync_aggregate,
Opt.some(state.data.next_sync_committee))
if not res.isOk:
dag.cacheLightClientData(state, blck,
current_period_best_update = attested_data.current_period_best_update,
latest_signature_slot = latest_signature_slot)
return
debug "Best LC update improved", period = attested_period, update = best[]
dag.cacheLightClientData(state, blck,
current_period_best_update = best,
latest_signature_slot = latest_signature_slot)
proc initLightClientDataCache*(dag: ChainDAGRef) =
## Initialize cached light client data
@ -725,7 +763,9 @@ proc initLightClientDataCache*(dag: ChainDAGRef) =
best[].reset() # Cached data is too recent or from other branch
else:
discard # Cached data is ancestor of `bid`
dag.cacheLightClientData(forkyState, bid, best)
dag.cacheLightClientData(forkyState, forkyBlck,
current_period_best_update = best,
latest_signature_slot = GENESIS_SLOT)
else:
dag.createLightClientUpdates(
forkyState, forkyBlck, parentBid = blocks[i + 1])
@ -778,22 +818,67 @@ proc processHeadChangeForLightClient*(dag: ChainDAGRef) =
return
if dag.head.slot < dag.lcDataStore.cache.tailSlot:
return
# Commit best light client data for non-finalized periods
let
headPeriod = dag.head.slot.sync_committee_period
lowSlot = max(dag.lcDataStore.cache.tailSlot, dag.finalizedHead.slot)
lowPeriod = lowSlot.sync_committee_period
var blck = dag.head
for period in countdown(headPeriod, lowPeriod):
blck = blck.get_ancestor((period + 1).start_slot - 1)
if blck == nil:
return
if blck.slot < lowSlot:
return
if blck == nil or blck.slot < lowSlot:
break
dag.lcDataStore.db.putBestUpdate(
blck.slot.sync_committee_period,
dag.getLightClientData(blck.bid).current_period_best_update[])
# Update latest light client data
template latest(): untyped = dag.lcDataStore.cache.latest
let
head_data = dag.getLightClientData(dag.head.bid)
signature_slot = head_data.latest_signature_slot
if signature_slot <= lowSlot:
latest.reset()
return
blck = dag.head.get_ancestor(signature_slot)
if blck == nil or blck.parent == nil or blck.parent.slot < lowSlot:
# If `attested_bid` is before `finalizedHead`, we don't have cached data.
latest.reset()
return
let
signature_bid = blck.bid
attested_bid = blck.parent.bid
sync_aggregate = dag.getExistingSyncAggregate(signature_bid).valueOr:
dag.handleUnexpectedLightClientError(signature_bid.slot)
return
old_meta = latest.toMeta()
latest.assignLightClientData(
dag, attested_bid, signature_slot, sync_aggregate)
let
new_meta = latest.toMeta()
new_optimistic =
if new_meta.attested_slot != old_meta.attested_slot:
new_meta.attested_slot > old_meta.attested_slot
else:
new_meta.signature_slot > old_meta.signature_slot
new_finality =
if not new_meta.has_finality:
false
elif new_meta.finalized_slot != old_meta.finalized_slot:
new_meta.finalized_slot > old_meta.finalized_slot
else:
let
old_has_supermajority =
hasSupermajoritySyncParticipation(old_meta.num_active_participants)
new_has_supermajority =
hasSupermajoritySyncParticipation(new_meta.num_active_participants)
new_has_supermajority > old_has_supermajority
if new_finality and dag.lcDataStore.onLightClientFinalityUpdate != nil:
dag.lcDataStore.onLightClientFinalityUpdate(latest)
if new_optimistic and dag.lcDataStore.onLightClientOptimisticUpdate != nil:
dag.lcDataStore.onLightClientOptimisticUpdate(latest.toOptimistic())
proc processFinalizationForLightClient*(
dag: ChainDAGRef, oldFinalizedHead: BlockSlot) =
## Prune cached data that is no longer useful for creating future

View File

@ -1,5 +1,5 @@
# beacon_chain
# Copyright (c) 2018-2023 Status Research & Development GmbH
# Copyright (c) 2018-2024 Status Research & Development GmbH
# Licensed and distributed under either of
# * MIT license (license terms in the root directory or at https://opensource.org/licenses/MIT).
# * Apache v2 license (license terms in the root directory or at https://www.apache.org/licenses/LICENSE-2.0).
@ -294,10 +294,11 @@ func toMeta*(update: SomeForkyLightClientUpdate): LightClientUpdateMetadata =
update.sync_aggregate.num_active_participants.uint64
meta
template toMeta*(update: ForkedLightClientUpdate): LightClientUpdateMetadata =
withForkyUpdate(update):
template toMeta*(
update: SomeForkedLightClientUpdate): LightClientUpdateMetadata =
withForkyObject(update):
when lcDataFork > LightClientDataFork.None:
forkyUpdate.toMeta()
forkyObject.toMeta()
else:
default(LightClientUpdateMetadata)