add option to configure max historic LC data periods (#3799)

Adds a `--light-client-data-max-periods` option to override the number
of sync committee periods to retain light client data.
Raising it above the default enables archive nodes to serve full data.
Lowering below the default speeds up import times (still no persistence)
This commit is contained in:
Etan Kissling 2022-06-27 13:24:38 +02:00 committed by GitHub
parent b318a74d83
commit 91d543440a
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
6 changed files with 379 additions and 368 deletions

View File

@ -456,6 +456,11 @@ type
"Must be one of: none, only-new, full (slow startup), on-demand (may miss validator duties)"
name: "light-client-data-import-mode" .}: Option[LightClientDataImportMode]
lightClientDataMaxPeriods* {.
hidden
desc: "BETA: Maximum number of sync committee periods to retain light client data"
name: "light-client-data-max-periods" .}: Option[uint64]
inProcessValidators* {.
desc: "Disable the push model (the beacon node tells a signing process with the private keys of the validators what to sign and when) and load the validators in the beacon node itself"
defaultValue: true # the use of the nimbus_signing_process binary by default will be delayed until async I/O over stdin/stdout is developed for the child process.

View File

@ -77,7 +77,7 @@ type
## 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
tailSlot*: Slot
## The earliest slot for which light client data is imported.
LightClientDataStore* = object
@ -94,6 +94,8 @@ type
## Whether to make local light client data available or not
importMode*: LightClientDataImportMode
## Which classes of light client data to import
maxPeriods*: uint64
## Maximum number of sync committee periods to retain light client data
# -----------------------------------
# Callbacks

View File

@ -689,6 +689,7 @@ proc init*(T: type ChainDAGRef, cfg: RuntimeConfig, db: BeaconChainDB,
onLCOptimisticUpdateCb: OnLightClientOptimisticUpdateCallback = nil,
lightClientDataServe = false,
lightClientDataImportMode = LightClientDataImportMode.None,
lightClientDataMaxPeriods = none(uint64),
vanityLogs = default(VanityLogs)): ChainDAGRef =
cfg.checkForkConsistency()
@ -728,6 +729,7 @@ proc init*(T: type ChainDAGRef, cfg: RuntimeConfig, db: BeaconChainDB,
lcDataStore: initLightClientDataStore(
serve = lightClientDataServe,
importMode = lightClientDataImportMode,
maxPeriods = lightClientDataMaxPeriods.get(cfg.defaultLCDataMaxPeriods),
onLCFinalityUpdateCb = onLCFinalityUpdateCb,
onLCOptimisticUpdateCb = onLCOptimisticUpdateCb),

View File

@ -34,28 +34,6 @@ template nextEpochBoundarySlot(slot: Slot): Slot =
## referring to a block at given slot.
(slot + (SLOTS_PER_EPOCH - 1)).epoch.start_slot
func computeEarliestLightClientSlot(dag: ChainDAGRef): Slot =
## Compute the earliest slot for which light client data is retained.
let
minSupportedSlot = max(
dag.cfg.ALTAIR_FORK_EPOCH.start_slot,
dag.lcDataStore.cache.importTailSlot)
currentSlot = getStateField(dag.headState, slot)
if currentSlot < minSupportedSlot:
return minSupportedSlot
let
MIN_EPOCHS_FOR_BLOCK_REQUESTS =
dag.cfg.MIN_VALIDATOR_WITHDRAWABILITY_DELAY +
dag.cfg.CHURN_LIMIT_QUOTIENT div 2
MIN_SLOTS_FOR_BLOCK_REQUESTS =
MIN_EPOCHS_FOR_BLOCK_REQUESTS * SLOTS_PER_EPOCH
if currentSlot - minSupportedSlot < MIN_SLOTS_FOR_BLOCK_REQUESTS:
return minSupportedSlot
let earliestSlot = currentSlot - MIN_SLOTS_FOR_BLOCK_REQUESTS
max(earliestSlot.sync_committee_period.start_slot, minSupportedSlot)
proc updateExistingState(
dag: ChainDAGRef, state: var ForkedHashedBeaconState, bsi: BlockSlotId,
save: bool, cache: var StateCache): bool =
@ -145,347 +123,36 @@ proc syncCommitteeRootForPeriod(
else: raiseAssert "Unreachable"
do: err()
proc getLightClientData(
dag: ChainDAGRef,
bid: BlockId): CachedLightClientData =
## Fetch cached light client data about a given block.
## Data must be cached (`cacheLightClientData`) before calling this function.
try: dag.lcDataStore.cache.data[bid]
except KeyError: raiseAssert "Unreachable"
proc cacheLightClientData(
dag: ChainDAGRef, state: HashedBeaconStateWithSyncCommittee, bid: BlockId) =
## 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_INDEX).get,
next_sync_committee_branch:
state.data.build_proof(altair.NEXT_SYNC_COMMITTEE_INDEX).get,
finalized_slot:
state.data.finalized_checkpoint.epoch.start_slot,
finality_branch:
state.data.build_proof(altair.FINALIZED_ROOT_INDEX).get)
if dag.lcDataStore.cache.data.hasKeyOrPut(bid, cachedData):
doAssert false, "Redundant `cacheLightClientData` call"
proc deleteLightClientData*(dag: ChainDAGRef, bid: BlockId) =
## Delete cached light client data for a given block. This needs to be called
## when a block becomes unreachable due to finalization of a different fork.
if dag.lcDataStore.importMode == LightClientDataImportMode.None:
return
dag.lcDataStore.cache.data.del bid
func handleUnexpectedLightClientError(dag: ChainDAGRef, buggedSlot: Slot) =
## If there is an unexpected error, adjust `importTailSlot` to keep track of
## section for which complete light client data is available, and to avoid
## failed lookups of cached light client data.
doAssert verifyFinalization notin dag.updateFlags
if buggedSlot >= dag.lcDataStore.cache.importTailSlot:
dag.lcDataStore.cache.importTailSlot = buggedSlot + 1
template lazy_header(name: untyped): untyped {.dirty.} =
## `createLightClientUpdates` helper to lazily load a known block header.
var
`name _ ptr`: ptr[BeaconBlockHeader]
`name _ ok` = true
template `assign _ name`(target: var BeaconBlockHeader, bid: BlockId): bool =
if `name _ ptr` != nil:
target = `name _ ptr`[]
elif `name _ ok`:
let bdata = dag.getExistingForkedBlock(bid)
if bdata.isErr:
dag.handleUnexpectedLightClientError(bid.slot)
`name _ ok` = false
else:
target = bdata.get.toBeaconBlockHeader()
`name _ ptr` = addr target
`name _ ok`
template lazy_data(name: untyped): untyped {.dirty.} =
## `createLightClientUpdates` helper to lazily load cached light client state.
var `name` {.noinit.}: CachedLightClientData
`name`.finalized_slot = FAR_FUTURE_SLOT
template `load _ name`(bid: BlockId) =
if `name`.finalized_slot == FAR_FUTURE_SLOT:
`name` = dag.getLightClientData(bid)
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(
dag: ChainDAGRef,
state: HashedBeaconStateWithSyncCommittee,
blck: TrustedSignedBeaconBlockWithSyncAggregate,
parent_bid: BlockId) =
## 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 sync committee has sufficient participants
template sync_aggregate(): auto = blck.asSigned().message.body.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
# Verify attested block (parent) is recent enough and that state is available
template attested_bid(): auto = parent_bid
let
earliest_slot = dag.computeEarliestLightClientSlot
attested_slot = attested_bid.slot
if attested_slot < earliest_slot:
return
# Lazy variables to hold historic data
lazy_header(attested_header)
lazy_data(attested_data)
lazy_bid(finalized_bid)
lazy_header(finalized_header)
# Update latest light client data
template latest(): auto = dag.lcDataStore.cache.latest
var
newFinality = false
newOptimistic = false
let
signature_slot = blck.message.slot
is_later =
if attested_slot != latest.attested_header.slot:
attested_slot > latest.attested_header.slot
else:
signature_slot > latest.signature_slot
if is_later and latest.attested_header.assign_attested_header(attested_bid):
load_attested_data(attested_bid)
let finalized_slot = attested_data.finalized_slot
if finalized_slot == latest.finalized_header.slot:
latest.finality_branch = attested_data.finality_branch
elif finalized_slot == GENESIS_SLOT:
latest.finalized_header.reset()
latest.finality_branch = attested_data.finality_branch
elif finalized_slot >= dag.tail.slot and
load_finalized_bid(finalized_slot) and
latest.finalized_header.assign_finalized_header(finalized_bid):
latest.finality_branch = attested_data.finality_branch
newFinality = true
else:
latest.finalized_header.reset()
latest.finality_branch.reset()
latest.sync_aggregate = sync_aggregate
latest.signature_slot = signature_slot
newOptimistic = true
# Track best light client data for current period
let
attested_period = attested_slot.sync_committee_period
signature_period = signature_slot.sync_committee_period
if attested_period == signature_period:
template next_sync_committee(): auto = state.data.next_sync_committee
let isCommitteeFinalized = dag.isNextSyncCommitteeFinalized(attested_period)
var best =
if isCommitteeFinalized:
dag.lcDataStore.cache.best.getOrDefault(attested_period)
else:
let key = (attested_period, state.syncCommitteeRoot)
dag.lcDataStore.cache.pendingBest.getOrDefault(key)
load_attested_data(attested_bid)
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, best.toMeta)
if is_better and best.attested_header.assign_attested_header(attested_bid):
best.next_sync_committee = next_sync_committee
best.next_sync_committee_branch = attested_data.next_sync_committee_branch
if finalized_slot == best.finalized_header.slot:
best.finality_branch = attested_data.finality_branch
elif finalized_slot == GENESIS_SLOT:
best.finalized_header.reset()
best.finality_branch = attested_data.finality_branch
elif has_finality and
best.finalized_header.assign_finalized_header(finalized_bid):
best.finality_branch = attested_data.finality_branch
else:
best.finalized_header.reset()
best.finality_branch.reset()
best.sync_aggregate = sync_aggregate
best.signature_slot = signature_slot
if isCommitteeFinalized:
dag.lcDataStore.cache.best[attested_period] = best
debug "Best LC update for period improved",
period = attested_period, update = best
else:
let key = (attested_period, state.syncCommitteeRoot)
dag.lcDataStore.cache.pendingBest[key] = best
debug "Best LC update for period improved",
period = key, update = best
if newFinality and dag.lcDataStore.onLightClientFinalityUpdate != nil:
dag.lcDataStore.onLightClientFinalityUpdate(latest)
if newOptimistic and dag.lcDataStore.onLightClientOptimisticUpdate != nil:
dag.lcDataStore.onLightClientOptimisticUpdate(latest.toOptimistic)
proc processNewBlockForLightClient*(
dag: ChainDAGRef,
state: ForkedHashedBeaconState,
signedBlock: ForkyTrustedSignedBeaconBlock,
parentBid: BlockId) =
## Update light client data with information from a new block.
if dag.lcDataStore.importMode == LightClientDataImportMode.None:
return
if signedBlock.message.slot < dag.computeEarliestLightClientSlot:
return
when signedBlock is bellatrix.TrustedSignedBeaconBlock:
dag.cacheLightClientData(state.bellatrixData, signedBlock.toBlockId())
dag.createLightClientUpdates(state.bellatrixData, signedBlock, parentBid)
elif signedBlock is altair.TrustedSignedBeaconBlock:
dag.cacheLightClientData(state.altairData, signedBlock.toBlockId())
dag.createLightClientUpdates(state.altairData, signedBlock, parentBid)
elif signedBlock is phase0.TrustedSignedBeaconBlock:
raiseAssert "Unreachable" # `earliestSlot` cannot be before Altair
else:
{.error: "Unreachable".}
proc processHeadChangeForLightClient*(dag: ChainDAGRef) =
## Update light client data to account for a new head block.
## Note that `dag.finalizedHead` is not yet updated when this is called.
if dag.lcDataStore.importMode == LightClientDataImportMode.None:
return
let earliestSlot = dag.computeEarliestLightClientSlot
if dag.head.slot < earliestSlot:
return
# Update `best` from `pendingBest` to ensure light client data
# only refers to sync committees as selected by fork choice
let headPeriod = dag.head.slot.sync_committee_period
if not dag.isNextSyncCommitteeFinalized(headPeriod):
let
earliestPeriod = earliestSlot.sync_committee_period
lowPeriod = max(dag.firstNonFinalizedPeriod, earliestPeriod)
if headPeriod > lowPeriod:
var tmpState = assignClone(dag.headState)
for period in lowPeriod ..< headPeriod:
let
syncCommitteeRoot =
dag.syncCommitteeRootForPeriod(tmpState[], period).valueOr:
dag.handleUnexpectedLightClientError(period.start_slot)
continue
key = (period, syncCommitteeRoot)
dag.lcDataStore.cache.best[period] =
dag.lcDataStore.cache.pendingBest.getOrDefault(key)
withState(dag.headState): # Common case separate to avoid `tmpState` copy
when stateFork >= BeaconStateFork.Altair:
let key = (headPeriod, state.syncCommitteeRoot)
dag.lcDataStore.cache.best[headPeriod] =
dag.lcDataStore.cache.pendingBest.getOrDefault(key)
else: raiseAssert "Unreachable"
proc processFinalizationForLightClient*(
dag: ChainDAGRef, oldFinalizedHead: BlockSlot) =
## Prune cached data that is no longer useful for creating future
## `LightClientUpdate` and `LightClientBootstrap` instances.
## This needs to be called whenever `finalized_checkpoint` changes.
if dag.lcDataStore.importMode == LightClientDataImportMode.None:
return
let
earliestSlot = dag.computeEarliestLightClientSlot
finalizedSlot = dag.finalizedHead.slot
if finalizedSlot < earliestSlot:
return
# Cache `LightClientBootstrap` for newly finalized epoch boundary blocks
let lowSlot = max(oldFinalizedHead.slot + 1, earliestSlot)
var boundarySlot = finalizedSlot
while boundarySlot >= lowSlot:
let
bsi = dag.getExistingBlockIdAtSlot(boundarySlot).valueOr:
dag.handleUnexpectedLightClientError(boundarySlot)
break
bid = bsi.bid
if bid.slot >= lowSlot:
dag.lcDataStore.cache.bootstrap[bid.slot] =
CachedLightClientBootstrap(
current_sync_committee_branch:
dag.getLightClientData(bid).current_sync_committee_branch)
boundarySlot = bid.slot.nextEpochBoundarySlot
if boundarySlot < SLOTS_PER_EPOCH:
break
boundarySlot -= SLOTS_PER_EPOCH
# Prune light client data that is no longer referrable by future updates
var bidsToDelete: seq[BlockId]
for bid, data in dag.lcDataStore.cache.data:
if bid.slot >= dag.finalizedHead.blck.slot:
continue
bidsToDelete.add bid
for bid in bidsToDelete:
dag.lcDataStore.cache.data.del bid
# Prune bootstrap data that is no longer relevant
var slotsToDelete: seq[Slot]
for slot in dag.lcDataStore.cache.bootstrap.keys:
if slot < earliestSlot:
slotsToDelete.add slot
for slot in slotsToDelete:
dag.lcDataStore.cache.bootstrap.del slot
# Prune best `LightClientUpdate` that are no longer relevant
let earliestPeriod = earliestSlot.sync_committee_period
var periodsToDelete: seq[SyncCommitteePeriod]
for period in dag.lcDataStore.cache.best.keys:
if period < earliestPeriod:
periodsToDelete.add period
for period in periodsToDelete:
dag.lcDataStore.cache.best.del period
# Prune best `LightClientUpdate` referring to non-finalized sync committees
# that are no longer relevant, i.e., orphaned or too old
let firstNonFinalizedPeriod = dag.firstNonFinalizedPeriod
var keysToDelete: seq[(SyncCommitteePeriod, Eth2Digest)]
for (period, committeeRoot) in dag.lcDataStore.cache.pendingBest.keys:
if period < firstNonFinalizedPeriod:
keysToDelete.add (period, committeeRoot)
for key in keysToDelete:
dag.lcDataStore.cache.pendingBest.del key
func initLightClientDataStore*(
serve: bool, importMode: LightClientDataImportMode,
serve: bool, importMode: LightClientDataImportMode, maxPeriods: uint64,
onLCFinalityUpdateCb: OnLightClientFinalityUpdateCallback = nil,
onLCOptimisticUpdateCb: OnLightClientOptimisticUpdateCallback = nil
): LightClientDataStore =
## Initialize light client data collector.
## Initialize light client data store.
LightClientDataStore(
serve: serve,
importMode: importMode,
maxPeriods: maxPeriods,
onLightClientFinalityUpdate: onLCFinalityUpdateCb,
onLightClientOptimisticUpdate: onLCOptimisticUpdateCb)
func targetLightClientTailSlot(dag: ChainDAGRef): Slot =
## Earliest slot for which light client data is retained.
let
maxPeriods = dag.lcDataStore.maxPeriods
headPeriod = dag.head.slot.sync_committee_period
lowSlot = max(dag.tail.slot, dag.cfg.ALTAIR_FORK_EPOCH.start_slot)
tail = max(headPeriod + 1, maxPeriods.SyncCommitteePeriod) - maxPeriods
max(tail.start_slot, lowSlot)
func handleUnexpectedLightClientError(dag: ChainDAGRef, buggedSlot: Slot) =
## If there is an unexpected error, adjust `tailSlot` to keep track of the
## section for which complete light client data is available, and to avoid
## failed lookups of cached light client data.
doAssert verifyFinalization notin dag.updateFlags
if buggedSlot >= dag.lcDataStore.cache.tailSlot:
dag.lcDataStore.cache.tailSlot = buggedSlot + 1
proc initLightClientBootstrapForPeriod(
dag: ChainDAGRef,
period: SyncCommitteePeriod) =
@ -493,22 +160,18 @@ proc initLightClientBootstrapForPeriod(
## epoch boundary blocks within a given sync committee period.
if not dag.isNextSyncCommitteeFinalized(period):
return
let
earliestSlot = dag.computeEarliestLightClientSlot
periodStartSlot = period.start_slot
periodEndSlot = periodStartSlot + SLOTS_PER_SYNC_COMMITTEE_PERIOD - 1
if periodEndSlot < earliestSlot:
return
let startTick = Moment.now()
debug "Caching LC bootstrap data for period", period
debug "Caching historic LC bootstrap data", period
defer:
let endTick = Moment.now()
debug "LC bootstrap data for period cached", period,
debug "Historic LC bootstrap data cached", period,
cacheDur = endTick - startTick
let
lowSlot = max(periodStartSlot, earliestSlot)
periodStartSlot = period.start_slot
periodEndSlot = periodStartSlot + SLOTS_PER_SYNC_COMMITTEE_PERIOD - 1
lowSlot = max(periodStartSlot, dag.targetLightClientTailSlot)
highSlot = min(periodEndSlot, dag.finalizedHead.blck.slot)
lowBoundarySlot = lowSlot.nextEpochBoundarySlot
highBoundarySlot = highSlot.nextEpochBoundarySlot
@ -544,22 +207,16 @@ proc initLightClientUpdateForPeriod(
## Non-finalized blocks are processed incrementally.
if not dag.isNextSyncCommitteeFinalized(period):
return
let
earliestSlot = dag.computeEarliestLightClientSlot
periodStartSlot = period.start_slot
periodEndSlot = periodStartSlot + SLOTS_PER_SYNC_COMMITTEE_PERIOD - 1
if periodEndSlot < earliestSlot:
return
if dag.lcDataStore.cache.best.hasKey(period):
return
let startTick = Moment.now()
debug "Computing best LC update for period", period
debug "Computing best historic LC update", period
proc logBest(endTick = Moment.now()) =
# Using a helper function reduces code size as the `defer` beneath is
# replicated on every `return`, and the log statement allocates another
# copy of the arguments on the stack for each instantiation (~1 MB stack!)
debug "Best LC update for period computed",
debug "Best historic LC update computed",
period, update = dag.lcDataStore.cache.best.getOrDefault(period),
computeDur = endTick - startTick
defer: logBest()
@ -603,7 +260,9 @@ proc initLightClientUpdateForPeriod(
# Determine the block in the period with highest sync committee participation
let
lowSlot = max(periodStartSlot, earliestSlot)
periodStartSlot = period.start_slot
periodEndSlot = periodStartSlot + SLOTS_PER_SYNC_COMMITTEE_PERIOD - 1
lowSlot = max(periodStartSlot, dag.targetLightClientTailSlot)
highSlot = min(periodEndSlot, dag.finalizedHead.blck.slot)
highBsi = dag.getExistingBlockIdAtSlot(highSlot).valueOr:
dag.handleUnexpectedLightClientError(highSlot)
@ -703,51 +362,243 @@ proc initLightClientUpdateForPeriod(
update.signature_slot = signatureBid.slot
dag.lcDataStore.cache.best[period] = update
proc getLightClientData(
dag: ChainDAGRef,
bid: BlockId): CachedLightClientData =
## Fetch cached light client data about a given block.
## Data must be cached (`cacheLightClientData`) before calling this function.
try: dag.lcDataStore.cache.data[bid]
except KeyError: raiseAssert "Unreachable"
proc cacheLightClientData(
dag: ChainDAGRef, state: HashedBeaconStateWithSyncCommittee, bid: BlockId) =
## 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_INDEX).get,
next_sync_committee_branch:
state.data.build_proof(altair.NEXT_SYNC_COMMITTEE_INDEX).get,
finalized_slot:
state.data.finalized_checkpoint.epoch.start_slot,
finality_branch:
state.data.build_proof(altair.FINALIZED_ROOT_INDEX).get)
if dag.lcDataStore.cache.data.hasKeyOrPut(bid, cachedData):
doAssert false, "Redundant `cacheLightClientData` call"
proc deleteLightClientData*(dag: ChainDAGRef, bid: BlockId) =
## Delete cached light client data for a given block. This needs to be called
## when a block becomes unreachable due to finalization of a different fork.
if dag.lcDataStore.importMode == LightClientDataImportMode.None:
return
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[BeaconBlockHeader]
`name _ ok` = true
template `assign _ name`(target: var BeaconBlockHeader, bid: BlockId): bool =
if `name _ ptr` != nil:
target = `name _ ptr`[]
elif `name _ ok`:
let bdata = dag.getExistingForkedBlock(bid)
if bdata.isErr:
dag.handleUnexpectedLightClientError(bid.slot)
`name _ ok` = false
else:
target = bdata.get.toBeaconBlockHeader()
`name _ ptr` = addr target
`name _ ok`
template lazy_data(name: untyped): untyped {.dirty.} =
## `createLightClientUpdates` helper to lazily load cached light client state.
var `name` {.noinit.}: CachedLightClientData
`name`.finalized_slot = FAR_FUTURE_SLOT
template `load _ name`(bid: BlockId) =
if `name`.finalized_slot == FAR_FUTURE_SLOT:
`name` = dag.getLightClientData(bid)
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(
dag: ChainDAGRef,
state: HashedBeaconStateWithSyncCommittee,
blck: TrustedSignedBeaconBlockWithSyncAggregate,
parent_bid: BlockId) =
## 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 sync committee has sufficient participants
template sync_aggregate(): auto = blck.asSigned().message.body.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
# 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
# Lazy variables to hold historic data
lazy_header(attested_header)
lazy_data(attested_data)
lazy_bid(finalized_bid)
lazy_header(finalized_header)
# Update latest light client data
template latest(): auto = dag.lcDataStore.cache.latest
var
newFinality = false
newOptimistic = false
let
signature_slot = blck.message.slot
is_later =
if attested_slot != latest.attested_header.slot:
attested_slot > latest.attested_header.slot
else:
signature_slot > latest.signature_slot
if is_later and latest.attested_header.assign_attested_header(attested_bid):
load_attested_data(attested_bid)
let finalized_slot = attested_data.finalized_slot
if finalized_slot == latest.finalized_header.slot:
latest.finality_branch = attested_data.finality_branch
elif finalized_slot == GENESIS_SLOT:
latest.finalized_header.reset()
latest.finality_branch = attested_data.finality_branch
elif finalized_slot >= dag.tail.slot and
load_finalized_bid(finalized_slot) and
latest.finalized_header.assign_finalized_header(finalized_bid):
latest.finality_branch = attested_data.finality_branch
newFinality = true
else:
latest.finalized_header.reset()
latest.finality_branch.reset()
latest.sync_aggregate = sync_aggregate
latest.signature_slot = signature_slot
newOptimistic = true
# Track best light client data for current period
let
attested_period = attested_slot.sync_committee_period
signature_period = signature_slot.sync_committee_period
if attested_period == signature_period:
template next_sync_committee(): auto = state.data.next_sync_committee
let isCommitteeFinalized = dag.isNextSyncCommitteeFinalized(attested_period)
var best =
if isCommitteeFinalized:
dag.lcDataStore.cache.best.getOrDefault(attested_period)
else:
let key = (attested_period, state.syncCommitteeRoot)
dag.lcDataStore.cache.pendingBest.getOrDefault(key)
load_attested_data(attested_bid)
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, best.toMeta)
if is_better and best.attested_header.assign_attested_header(attested_bid):
best.next_sync_committee = next_sync_committee
best.next_sync_committee_branch = attested_data.next_sync_committee_branch
if finalized_slot == best.finalized_header.slot:
best.finality_branch = attested_data.finality_branch
elif finalized_slot == GENESIS_SLOT:
best.finalized_header.reset()
best.finality_branch = attested_data.finality_branch
elif has_finality and
best.finalized_header.assign_finalized_header(finalized_bid):
best.finality_branch = attested_data.finality_branch
else:
best.finalized_header.reset()
best.finality_branch.reset()
best.sync_aggregate = sync_aggregate
best.signature_slot = signature_slot
if isCommitteeFinalized:
dag.lcDataStore.cache.best[attested_period] = best
debug "Best LC update improved", period = attested_period, update = best
else:
let key = (attested_period, state.syncCommitteeRoot)
dag.lcDataStore.cache.pendingBest[key] = best
debug "Best LC update improved", period = key, update = best
if newFinality and dag.lcDataStore.onLightClientFinalityUpdate != nil:
dag.lcDataStore.onLightClientFinalityUpdate(latest)
if newOptimistic and dag.lcDataStore.onLightClientOptimisticUpdate != nil:
dag.lcDataStore.onLightClientOptimisticUpdate(latest.toOptimistic)
proc initLightClientDataCache*(dag: ChainDAGRef) =
## Initialize cached light client data
if dag.lcDataStore.importMode == LightClientDataImportMode.None:
return
dag.lcDataStore.cache.importTailSlot = dag.tail.slot
# Initialize tail slot
let targetTailSlot = dag.targetLightClientTailSlot
dag.lcDataStore.cache.tailSlot = max(dag.head.slot, targetTailSlot)
# Import head state
if dag.head.slot < dag.lcDataStore.cache.tailSlot:
return
withState(dag.headState):
when stateFork >= BeaconStateFork.Altair:
dag.cacheLightClientData(state, dag.head.bid)
else: raiseAssert "Unreachable" # `tailSlot` cannot be before Altair
if dag.lcDataStore.importMode == LightClientDataImportMode.OnlyNew:
dag.lcDataStore.cache.importTailSlot = dag.head.slot
var earliestSlot = dag.computeEarliestLightClientSlot
if dag.head.slot < earliestSlot:
return
# Import light client data for finalized period through finalized head
let
finalizedSlot = dag.finalizedHead.slot
finalizedPeriod = finalizedSlot.sync_committee_period
let finalizedSlot = max(dag.finalizedHead.blck.slot, targetTailSlot)
if finalizedSlot >= dag.lcDataStore.cache.tailSlot:
return
dag.lcDataStore.cache.tailSlot = finalizedSlot
let finalizedPeriod = finalizedSlot.sync_committee_period
dag.initLightClientBootstrapForPeriod(finalizedPeriod)
dag.initLightClientUpdateForPeriod(finalizedPeriod)
let lightClientStartTick = Moment.now()
debug "Initializing cached light client data"
template handleUnexpectedError(buggedBid: BlockId) =
# Light client data is expected to be available from `earliestSlot` onward.
# If there is an error, adjust `importTailSlot` to avoid failed lookups of
# cached light client data. For new blocks / states, the caches can always
# be updated incrementally, because those blocks / states are passed in
# directly. It is only historical blocks (or sync committees) that depend
# on a potentially corrupted database.
doAssert buggedBid.slot > dag.lcDataStore.cache.importTailSlot
dag.handleUnexpectedLightClientError(buggedBid.slot)
earliestSlot = dag.computeEarliestLightClientSlot
logScope: lightClientDataMaxPeriods = dag.lcDataStore.maxPeriods
debug "Initializing cached LC data"
# Build list of block to process.
# As it is slow to load states in descending order,
# build a reverse todo list to then process them in ascending order
let lowSlot = max(finalizedSlot, earliestSlot)
var
blocks = newSeqOfCap[BlockId](dag.head.slot - lowSlot + 1)
blocks = newSeqOfCap[BlockId](dag.head.slot - finalizedSlot + 1)
bid = dag.head.bid
while bid.slot > lowSlot:
while bid.slot > finalizedSlot:
blocks.add bid
bid = dag.existingParent(bid).valueOr:
handleUnexpectedError(bid)
dag.handleUnexpectedLightClientError(bid.slot)
break
if bid.slot == lowSlot:
if bid.slot == finalizedSlot:
blocks.add bid
# Process blocks (reuses `dag.headState`, but restores it to the current head)
@ -760,31 +611,173 @@ proc initLightClientDataCache*(dag: ChainDAGRef) =
bid = blocks[i]
if not dag.updateExistingState(
dag.headState, bid.atSlot, save = false, cache):
handleUnexpectedError(bid)
dag.handleUnexpectedLightClientError(bid.slot)
continue
let bdata = dag.getExistingForkedBlock(bid).valueOr:
handleUnexpectedError(bid)
dag.handleUnexpectedLightClientError(bid.slot)
continue
withStateAndBlck(dag.headState, bdata):
when stateFork >= BeaconStateFork.Altair:
# Cache light client data (non-finalized blocks may refer to this)
dag.cacheLightClientData(state, blck.toBlockId())
if i != blocks.low:
dag.cacheLightClientData(state, bid) # `dag.head` already cached
# Create `LightClientUpdate` instances
if bid.slot != lowSlot:
if i < blocks.high:
dag.createLightClientUpdates(state, blck, parentBid = blocks[i + 1])
else: raiseAssert "Unreachable"
let lightClientEndTick = Moment.now()
debug "Initialized cached light client data",
debug "Initialized cached LC data",
initDur = lightClientEndTick - lightClientStartTick
if dag.lcDataStore.importMode == LightClientDataImportMode.OnDemand:
return
# Import historic data
if dag.lcDataStore.importMode == LightClientDataImportMode.Full:
let earliestPeriod = earliestSlot.sync_committee_period
for period in earliestPeriod ..< finalizedPeriod:
dag.lcDataStore.cache.tailSlot = targetTailSlot
let targetTailPeriod = targetTailSlot.sync_committee_period
if targetTailPeriod < finalizedPeriod:
# `countdown` through 0 fails on distinct `uint64`
# https://github.com/nim-lang/Nim/pull/19926
var period = finalizedPeriod - 1
while period >= targetTailPeriod:
dag.initLightClientBootstrapForPeriod(period)
dag.initLightClientUpdateForPeriod(period)
if period <= targetTailPeriod:
break
dec period
if dag.lcDataStore.cache.tailSlot == targetTailSlot:
debug "Historic LC data imported"
else:
# `handleUnexpectedLightClientError` updates `tailSlot`
warn "Error while importing historic LC data",
errorSlot = dag.lcDataStore.cache.tailSlot - 1, targetTailSlot
proc processNewBlockForLightClient*(
dag: ChainDAGRef,
state: ForkedHashedBeaconState,
signedBlock: ForkyTrustedSignedBeaconBlock,
parentBid: BlockId) =
## Update light client data with information from a new block.
if dag.lcDataStore.importMode == LightClientDataImportMode.None:
return
if signedBlock.message.slot < dag.lcDataStore.cache.tailSlot:
return
when signedBlock is bellatrix.TrustedSignedBeaconBlock:
dag.cacheLightClientData(state.bellatrixData, signedBlock.toBlockId())
dag.createLightClientUpdates(state.bellatrixData, signedBlock, parentBid)
elif signedBlock is altair.TrustedSignedBeaconBlock:
dag.cacheLightClientData(state.altairData, signedBlock.toBlockId())
dag.createLightClientUpdates(state.altairData, signedBlock, parentBid)
elif signedBlock is phase0.TrustedSignedBeaconBlock:
raiseAssert "Unreachable" # `tailSlot` cannot be before Altair
else:
{.error: "Unreachable".}
proc processHeadChangeForLightClient*(dag: ChainDAGRef) =
## Update light client data to account for a new head block.
## Note that `dag.finalizedHead` is not yet updated when this is called.
if dag.lcDataStore.importMode == LightClientDataImportMode.None:
return
if dag.head.slot < dag.lcDataStore.cache.tailSlot:
return
# Update `best` from `pendingBest` to ensure light client data
# only refers to sync committees as selected by fork choice
let headPeriod = dag.head.slot.sync_committee_period
if not dag.isNextSyncCommitteeFinalized(headPeriod):
let
tailPeriod = dag.lcDataStore.cache.tailSlot.sync_committee_period
lowPeriod = max(dag.firstNonFinalizedPeriod, tailPeriod)
if headPeriod > lowPeriod:
var tmpState = assignClone(dag.headState)
for period in lowPeriod ..< headPeriod:
let
syncCommitteeRoot =
dag.syncCommitteeRootForPeriod(tmpState[], period).valueOr:
dag.handleUnexpectedLightClientError(period.start_slot)
continue
key = (period, syncCommitteeRoot)
dag.lcDataStore.cache.best[period] =
dag.lcDataStore.cache.pendingBest.getOrDefault(key)
withState(dag.headState): # Common case separate to avoid `tmpState` copy
when stateFork >= BeaconStateFork.Altair:
let key = (headPeriod, state.syncCommitteeRoot)
dag.lcDataStore.cache.best[headPeriod] =
dag.lcDataStore.cache.pendingBest.getOrDefault(key)
else: raiseAssert "Unreachable" # `tailSlot` cannot be before Altair
proc processFinalizationForLightClient*(
dag: ChainDAGRef, oldFinalizedHead: BlockSlot) =
## Prune cached data that is no longer useful for creating future
## `LightClientUpdate` and `LightClientBootstrap` instances.
## This needs to be called whenever `finalized_checkpoint` changes.
if dag.lcDataStore.importMode == LightClientDataImportMode.None:
return
let finalizedSlot = dag.finalizedHead.slot
if finalizedSlot < dag.lcDataStore.cache.tailSlot:
return
# Cache `LightClientBootstrap` for newly finalized epoch boundary blocks
let
firstNewSlot = oldFinalizedHead.slot + 1
lowSlot = max(firstNewSlot, dag.lcDataStore.cache.tailSlot)
var boundarySlot = finalizedSlot
while boundarySlot >= lowSlot:
let
bsi = dag.getExistingBlockIdAtSlot(boundarySlot).valueOr:
dag.handleUnexpectedLightClientError(boundarySlot)
break
bid = bsi.bid
if bid.slot >= lowSlot:
dag.lcDataStore.cache.bootstrap[bid.slot] =
CachedLightClientBootstrap(
current_sync_committee_branch:
dag.getLightClientData(bid).current_sync_committee_branch)
boundarySlot = bid.slot.nextEpochBoundarySlot
if boundarySlot < SLOTS_PER_EPOCH:
break
boundarySlot -= SLOTS_PER_EPOCH
# Prune light client data that is no longer referrable by future updates
var bidsToDelete: seq[BlockId]
for bid, data in dag.lcDataStore.cache.data:
if bid.slot >= dag.finalizedHead.blck.slot:
continue
bidsToDelete.add bid
for bid in bidsToDelete:
dag.lcDataStore.cache.data.del bid
let
targetTailSlot = dag.targetLightClientTailSlot
targetTailPeriod = targetTailSlot.sync_committee_period
# Prune bootstrap data that is no longer relevant
var slotsToDelete: seq[Slot]
for slot in dag.lcDataStore.cache.bootstrap.keys:
if slot < targetTailSlot:
slotsToDelete.add slot
for slot in slotsToDelete:
dag.lcDataStore.cache.bootstrap.del slot
# Prune best `LightClientUpdate` that are no longer relevant
var periodsToDelete: seq[SyncCommitteePeriod]
for period in dag.lcDataStore.cache.best.keys:
if period < targetTailPeriod:
periodsToDelete.add period
for period in periodsToDelete:
dag.lcDataStore.cache.best.del period
# Prune best `LightClientUpdate` referring to non-finalized sync committees
# that are no longer relevant, i.e., orphaned or too old
let firstNonFinalizedPeriod = dag.firstNonFinalizedPeriod
var keysToDelete: seq[(SyncCommitteePeriod, Eth2Digest)]
for (period, committeeRoot) in dag.lcDataStore.cache.pendingBest.keys:
if period < firstNonFinalizedPeriod:
keysToDelete.add (period, committeeRoot)
for key in keysToDelete:
dag.lcDataStore.cache.pendingBest.del key
proc getLightClientBootstrap*(
dag: ChainDAGRef,
@ -799,8 +792,7 @@ proc getLightClientBootstrap*(
withBlck(bdata):
let slot = blck.message.slot
when stateFork >= BeaconStateFork.Altair:
let earliestSlot = dag.computeEarliestLightClientSlot
if slot < earliestSlot:
if slot < dag.lcDataStore.cache.tailSlot:
debug "LC bootstrap unavailable: Block too old", slot
return err()
if slot > dag.finalizedHead.blck.slot:

View File

@ -194,6 +194,7 @@ proc loadChainDag(
onLCOptimisticUpdateCb = onLightClientOptimisticUpdateCb,
lightClientDataServe = config.lightClientDataServe.get,
lightClientDataImportMode = config.lightClientDataImportMode.get,
lightClientDataMaxPeriods = config.lightClientDataMaxPeriods,
vanityLogs = getPandas(detectTTY(config.logStdout)))
let

View File

@ -479,3 +479,12 @@ template name*(cfg: RuntimeConfig): string =
cfg.CONFIG_NAME
else:
const_preset
# https://github.com/ethereum/consensus-specs/blob/v1.2.0-rc.1/specs/phase0/p2p-interface.md#configuration
func MIN_EPOCHS_FOR_BLOCK_REQUESTS*(cfg: RuntimeConfig): uint64 =
cfg.MIN_VALIDATOR_WITHDRAWABILITY_DELAY + cfg.CHURN_LIMIT_QUOTIENT div 2
func defaultLCDataMaxPeriods*(cfg: RuntimeConfig): uint64 =
const epochsPerPeriod = EPOCHS_PER_SYNC_COMMITTEE_PERIOD
let maxEpochs = cfg.MIN_EPOCHS_FOR_BLOCK_REQUESTS
(maxEpochs + epochsPerPeriod - 1) div epochsPerPeriod