encapsulate LC data variables into single structure (#3777)

Combines the LC data configuration options (serve / importMode), the
callbacks (finality / optimistic LC update) as well as the cache storing
light client data, into a new `LightClientDataStore` structure.
Also moves the structure into a light client specific file.
This commit is contained in:
Etan Kissling 2022-06-24 16:57:50 +02:00 committed by GitHub
parent c45c017349
commit 2e98c7722f
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
12 changed files with 187 additions and 129 deletions

View File

@ -185,12 +185,6 @@ type
cfg*: RuntimeConfig
lightClientDataServe*: bool
## Whether to make local light client data available or not
lightClientDataImportMode*: LightClientDataImportMode
## Which classes of light client data to import
epochRefs*: array[32, EpochRef]
## Cached information about a particular epoch ending with the given
## block - we limit the number of held EpochRefs to put a cap on
@ -209,9 +203,10 @@ type
## called several times.
# -----------------------------------
# Data to enable light clients to stay in sync with the network
# Light client data
lightClientCache*: LightClientCache
lcDataStore*: LightClientDataStore
# Data store to enable light clients to sync with the network
# -----------------------------------
# Callbacks
@ -224,10 +219,6 @@ type
## On beacon chain reorganization
onFinHappened*: OnFinalizedCallback
## On finalization 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,11 +18,6 @@ import
./block_dag
type
OnLightClientFinalityUpdateCallback* =
proc(data: altair.LightClientFinalityUpdate) {.gcsafe, raises: [Defect].}
OnLightClientOptimisticUpdateCallback* =
proc(data: altair.LightClientOptimisticUpdate) {.gcsafe, raises: [Defect].}
LightClientDataImportMode* {.pure.} = enum
## Controls which classes of light client data are imported.
None = "none"
@ -34,6 +29,11 @@ type
OnDemand = "on-demand"
## Don't precompute historic data. Slow, may miss validator duties.
OnLightClientFinalityUpdateCallback* =
proc(data: altair.LightClientFinalityUpdate) {.gcsafe, raises: [Defect].}
OnLightClientOptimisticUpdateCallback* =
proc(data: altair.LightClientOptimisticUpdate) {.gcsafe, raises: [Defect].}
CachedLightClientData* = object
## Cached data from historical non-finalized states to improve speed when
## creating future `LightClientUpdate` and `LightClientBootstrap` instances.
@ -52,7 +52,7 @@ type
current_sync_committee_branch*:
array[log2trunc(altair.CURRENT_SYNC_COMMITTEE_INDEX), Eth2Digest]
LightClientCache* = object
LightClientDataCache* = object
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.
@ -79,3 +79,26 @@ type
importTailSlot*: Slot
## The earliest slot for which light client data is imported.
LightClientDataStore* = object
# -----------------------------------
# Light client data
cache*: LightClientDataCache
## Cached data to accelerate serving light client data
# -----------------------------------
# Config
serve*: bool
## Whether to make local light client data available or not
importMode*: LightClientDataImportMode
## Which classes of light client data to import
# -----------------------------------
# Callbacks
onLightClientFinalityUpdate*: OnLightClientFinalityUpdateCallback
## On new `LightClientFinalityUpdate` callback
onLightClientOptimisticUpdate*: OnLightClientOptimisticUpdateCallback
## On new `LightClientOptimisticUpdate` callback

View File

@ -692,7 +692,7 @@ proc init*(T: type ChainDAGRef, cfg: RuntimeConfig, db: BeaconChainDB,
vanityLogs = default(VanityLogs)): ChainDAGRef =
cfg.checkForkConsistency()
doAssert updateFlags in [{}, {verifyFinalization}],
doAssert updateFlags - {verifyFinalization, enableTestFeatures} == {},
"Other flags not supported in ChainDAG"
# TODO we require that the db contains both a head and a tail block -
@ -725,15 +725,16 @@ proc init*(T: type ChainDAGRef, cfg: RuntimeConfig, db: BeaconChainDB,
vanityLogs: vanityLogs,
lightClientDataServe: lightClientDataServe,
lightClientDataImportMode: lightClientDataImportMode,
lcDataStore: initLightClientDataStore(
serve = lightClientDataServe,
importMode = lightClientDataImportMode,
onLCFinalityUpdateCb = onLCFinalityUpdateCb,
onLCOptimisticUpdateCb = onLCOptimisticUpdateCb),
onBlockAdded: onBlockCb,
onHeadChanged: onHeadCb,
onReorgHappened: onReorgCb,
onFinHappened: onFinCb,
onLightClientFinalityUpdate: onLCFinalityUpdateCb,
onLightClientOptimisticUpdate: onLCOptimisticUpdateCb
onFinHappened: onFinCb
)
loadTick = Moment.now()
@ -953,7 +954,7 @@ proc init*(T: type ChainDAGRef, cfg: RuntimeConfig, db: BeaconChainDB,
frontfillDur = frontfillTick - finalizedTick,
keysDur = Moment.now() - frontfillTick
dag.initLightClientCache()
dag.initLightClientDataCache()
dag

View File

@ -39,7 +39,7 @@ func computeEarliestLightClientSlot(dag: ChainDAGRef): Slot =
let
minSupportedSlot = max(
dag.cfg.ALTAIR_FORK_EPOCH.start_slot,
dag.lightClientCache.importTailSlot)
dag.lcDataStore.cache.importTailSlot)
currentSlot = getStateField(dag.headState, slot)
if currentSlot < minSupportedSlot:
return minSupportedSlot
@ -150,7 +150,7 @@ proc getLightClientData(
bid: BlockId): CachedLightClientData =
## Fetch cached light client data about a given block.
## Data must be cached (`cacheLightClientData`) before calling this function.
try: dag.lightClientCache.data[bid]
try: dag.lcDataStore.cache.data[bid]
except KeyError: raiseAssert "Unreachable"
proc cacheLightClientData(
@ -170,24 +170,24 @@ proc cacheLightClientData(
state.data.build_proof(
altair.FINALIZED_ROOT_INDEX,
cachedData.finality_branch)
if dag.lightClientCache.data.hasKeyOrPut(bid, cachedData):
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.lightClientDataImportMode == LightClientDataImportMode.None:
if dag.lcDataStore.importMode == LightClientDataImportMode.None:
return
dag.lightClientCache.data.del bid
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.lightClientCache.importTailSlot:
dag.lightClientCache.importTailSlot = buggedSlot + 1
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.
@ -262,7 +262,7 @@ proc createLightClientUpdates(
lazy_header(finalized_header)
# Update latest light client data
template latest(): auto = dag.lightClientCache.latest
template latest(): auto = dag.lcDataStore.cache.latest
var
newFinality = false
newOptimistic = false
@ -303,10 +303,10 @@ proc createLightClientUpdates(
let isCommitteeFinalized = dag.isNextSyncCommitteeFinalized(attested_period)
var best =
if isCommitteeFinalized:
dag.lightClientCache.best.getOrDefault(attested_period)
dag.lcDataStore.cache.best.getOrDefault(attested_period)
else:
let key = (attested_period, state.syncCommitteeRoot)
dag.lightClientCache.pendingBest.getOrDefault(key)
dag.lcDataStore.cache.pendingBest.getOrDefault(key)
load_attested_data(attested_bid)
let
@ -339,19 +339,19 @@ proc createLightClientUpdates(
best.signature_slot = signature_slot
if isCommitteeFinalized:
dag.lightClientCache.best[attested_period] = best
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.lightClientCache.pendingBest[key] = best
dag.lcDataStore.cache.pendingBest[key] = best
debug "Best LC update for period improved",
period = key, update = best
if newFinality and dag.onLightClientFinalityUpdate != nil:
dag.onLightClientFinalityUpdate(latest)
if newOptimistic and dag.onLightClientOptimisticUpdate != nil:
dag.onLightClientOptimisticUpdate(latest.toOptimistic)
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,
@ -359,7 +359,7 @@ proc processNewBlockForLightClient*(
signedBlock: ForkyTrustedSignedBeaconBlock,
parentBid: BlockId) =
## Update light client data with information from a new block.
if dag.lightClientDataImportMode == LightClientDataImportMode.None:
if dag.lcDataStore.importMode == LightClientDataImportMode.None:
return
if signedBlock.message.slot < dag.computeEarliestLightClientSlot:
return
@ -378,7 +378,7 @@ proc processNewBlockForLightClient*(
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.lightClientDataImportMode == LightClientDataImportMode.None:
if dag.lcDataStore.importMode == LightClientDataImportMode.None:
return
let earliestSlot = dag.computeEarliestLightClientSlot
if dag.head.slot < earliestSlot:
@ -400,13 +400,13 @@ proc processHeadChangeForLightClient*(dag: ChainDAGRef) =
dag.handleUnexpectedLightClientError(period.start_slot)
continue
key = (period, syncCommitteeRoot)
dag.lightClientCache.best[period] =
dag.lightClientCache.pendingBest.getOrDefault(key)
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.lightClientCache.best[headPeriod] =
dag.lightClientCache.pendingBest.getOrDefault(key)
dag.lcDataStore.cache.best[headPeriod] =
dag.lcDataStore.cache.pendingBest.getOrDefault(key)
else: raiseAssert "Unreachable"
proc processFinalizationForLightClient*(
@ -414,7 +414,7 @@ proc processFinalizationForLightClient*(
## 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.lightClientDataImportMode == LightClientDataImportMode.None:
if dag.lcDataStore.importMode == LightClientDataImportMode.None:
return
let
earliestSlot = dag.computeEarliestLightClientSlot
@ -432,7 +432,7 @@ proc processFinalizationForLightClient*(
break
bid = bsi.bid
if bid.slot >= lowSlot:
dag.lightClientCache.bootstrap[bid.slot] =
dag.lcDataStore.cache.bootstrap[bid.slot] =
CachedLightClientBootstrap(
current_sync_committee_branch:
dag.getLightClientData(bid).current_sync_committee_branch)
@ -443,39 +443,51 @@ proc processFinalizationForLightClient*(
# Prune light client data that is no longer referrable by future updates
var bidsToDelete: seq[BlockId]
for bid, data in dag.lightClientCache.data:
for bid, data in dag.lcDataStore.cache.data:
if bid.slot >= dag.finalizedHead.blck.slot:
continue
bidsToDelete.add bid
for bid in bidsToDelete:
dag.lightClientCache.data.del bid
dag.lcDataStore.cache.data.del bid
# Prune bootstrap data that is no longer relevant
var slotsToDelete: seq[Slot]
for slot in dag.lightClientCache.bootstrap.keys:
for slot in dag.lcDataStore.cache.bootstrap.keys:
if slot < earliestSlot:
slotsToDelete.add slot
for slot in slotsToDelete:
dag.lightClientCache.bootstrap.del slot
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.lightClientCache.best.keys:
for period in dag.lcDataStore.cache.best.keys:
if period < earliestPeriod:
periodsToDelete.add period
for period in periodsToDelete:
dag.lightClientCache.best.del period
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.lightClientCache.pendingBest.keys:
for (period, committeeRoot) in dag.lcDataStore.cache.pendingBest.keys:
if period < firstNonFinalizedPeriod:
keysToDelete.add (period, committeeRoot)
for key in keysToDelete:
dag.lightClientCache.pendingBest.del key
dag.lcDataStore.cache.pendingBest.del key
func initLightClientDataStore*(
serve: bool, importMode: LightClientDataImportMode,
onLCFinalityUpdateCb: OnLightClientFinalityUpdateCallback = nil,
onLCOptimisticUpdateCb: OnLightClientOptimisticUpdateCallback = nil
): LightClientDataStore =
## Initialize light client data collector.
LightClientDataStore(
serve: serve,
importMode: importMode,
onLightClientFinalityUpdate: onLCFinalityUpdateCb,
onLightClientOptimisticUpdate: onLCOptimisticUpdateCb)
proc initLightClientBootstrapForPeriod(
dag: ChainDAGRef,
@ -516,7 +528,7 @@ proc initLightClientBootstrapForPeriod(
bid = bsi.bid
boundarySlot = bid.slot.nextEpochBoundarySlot
if boundarySlot == nextBoundarySlot and bid.slot >= lowSlot and
not dag.lightClientCache.bootstrap.hasKey(bid.slot):
not dag.lcDataStore.cache.bootstrap.hasKey(bid.slot):
var cachedBootstrap {.noinit.}: CachedLightClientBootstrap
if not dag.updateExistingState(
tmpState[], bid.atSlot, save = false, tmpCache):
@ -528,7 +540,7 @@ proc initLightClientBootstrapForPeriod(
altair.CURRENT_SYNC_COMMITTEE_INDEX,
cachedBootstrap.current_sync_committee_branch)
else: raiseAssert "Unreachable"
dag.lightClientCache.bootstrap[bid.slot] = cachedBootstrap
dag.lcDataStore.cache.bootstrap[bid.slot] = cachedBootstrap
proc initLightClientUpdateForPeriod(
dag: ChainDAGRef, period: SyncCommitteePeriod) =
@ -543,7 +555,7 @@ proc initLightClientUpdateForPeriod(
periodEndSlot = periodStartSlot + SLOTS_PER_SYNC_COMMITTEE_PERIOD - 1
if periodEndSlot < earliestSlot:
return
if dag.lightClientCache.best.hasKey(period):
if dag.lcDataStore.cache.best.hasKey(period):
return
let startTick = Moment.now()
@ -553,7 +565,7 @@ proc initLightClientUpdateForPeriod(
# 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",
period, update = dag.lightClientCache.best.getOrDefault(period),
period, update = dag.lcDataStore.cache.best.getOrDefault(period),
computeDur = endTick - startTick
defer: logBest()
@ -605,7 +617,7 @@ proc initLightClientUpdateForPeriod(
maxParticipantsRes = dag.maxParticipantsBlock(highBid, lowSlot)
maxParticipantsBid = maxParticipantsRes.bid.valueOr:
if maxParticipantsRes.ok: # No single valid block exists in the period
dag.lightClientCache.best[period] = default(altair.LightClientUpdate)
dag.lcDataStore.cache.best[period] = default(altair.LightClientUpdate)
return
# The block with highest participation may refer to a `finalized_checkpoint`
@ -696,15 +708,15 @@ proc initLightClientUpdateForPeriod(
update.sync_aggregate = blck.asSigned().message.body.sync_aggregate
else: raiseAssert "Unreachable"
update.signature_slot = signatureBid.slot
dag.lightClientCache.best[period] = update
dag.lcDataStore.cache.best[period] = update
proc initLightClientCache*(dag: ChainDAGRef) =
proc initLightClientDataCache*(dag: ChainDAGRef) =
## Initialize cached light client data
if dag.lightClientDataImportMode == LightClientDataImportMode.None:
if dag.lcDataStore.importMode == LightClientDataImportMode.None:
return
dag.lightClientCache.importTailSlot = dag.tail.slot
if dag.lightClientDataImportMode == LightClientDataImportMode.OnlyNew:
dag.lightClientCache.importTailSlot = dag.head.slot
dag.lcDataStore.cache.importTailSlot = dag.tail.slot
if dag.lcDataStore.importMode == LightClientDataImportMode.OnlyNew:
dag.lcDataStore.cache.importTailSlot = dag.head.slot
var earliestSlot = dag.computeEarliestLightClientSlot
if dag.head.slot < earliestSlot:
return
@ -726,7 +738,7 @@ proc initLightClientCache*(dag: ChainDAGRef) =
# 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.lightClientCache.importTailSlot
doAssert buggedBid.slot > dag.lcDataStore.cache.importTailSlot
dag.handleUnexpectedLightClientError(buggedBid.slot)
earliestSlot = dag.computeEarliestLightClientSlot
@ -775,7 +787,7 @@ proc initLightClientCache*(dag: ChainDAGRef) =
initDur = lightClientEndTick - lightClientStartTick
# Import historic data
if dag.lightClientDataImportMode == LightClientDataImportMode.Full:
if dag.lcDataStore.importMode == LightClientDataImportMode.Full:
let earliestPeriod = earliestSlot.sync_committee_period
for period in earliestPeriod ..< finalizedPeriod:
dag.initLightClientBootstrapForPeriod(period)
@ -784,7 +796,7 @@ proc initLightClientCache*(dag: ChainDAGRef) =
proc getLightClientBootstrap*(
dag: ChainDAGRef,
blockRoot: Eth2Digest): Opt[altair.LightClientBootstrap] =
if not dag.lightClientDataServe:
if not dag.lcDataStore.serve:
return err()
let bdata = dag.getForkedBlock(blockRoot).valueOr:
@ -801,9 +813,9 @@ proc getLightClientBootstrap*(
if slot > dag.finalizedHead.blck.slot:
debug "LC bootstrap unavailable: Not finalized", blockRoot
return err()
var cachedBootstrap = dag.lightClientCache.bootstrap.getOrDefault(slot)
var cachedBootstrap = dag.lcDataStore.cache.bootstrap.getOrDefault(slot)
if cachedBootstrap.current_sync_committee_branch.isZeroMemory:
if dag.lightClientDataImportMode == LightClientDataImportMode.OnDemand:
if dag.lcDataStore.importMode == LightClientDataImportMode.OnDemand:
let bsi = ? dag.getExistingBlockIdAtSlot(slot)
var tmpState = assignClone(dag.headState)
dag.withUpdatedExistingState(tmpState[], bsi) do:
@ -814,7 +826,7 @@ proc getLightClientBootstrap*(
cachedBootstrap.current_sync_committee_branch)
else: raiseAssert "Unreachable"
do: return err()
dag.lightClientCache.bootstrap[slot] = cachedBootstrap
dag.lcDataStore.cache.bootstrap[slot] = cachedBootstrap
else:
debug "LC bootstrap unavailable: Data not cached", slot
return err()
@ -836,32 +848,32 @@ proc getLightClientBootstrap*(
proc getLightClientUpdateForPeriod*(
dag: ChainDAGRef,
period: SyncCommitteePeriod): Option[altair.LightClientUpdate] =
if not dag.lightClientDataServe:
if not dag.lcDataStore.serve:
return
if dag.lightClientDataImportMode == LightClientDataImportMode.OnDemand:
if dag.lcDataStore.importMode == LightClientDataImportMode.OnDemand:
dag.initLightClientUpdateForPeriod(period)
result = some(dag.lightClientCache.best.getOrDefault(period))
result = some(dag.lcDataStore.cache.best.getOrDefault(period))
let numParticipants = countOnes(result.get.sync_aggregate.sync_committee_bits)
if numParticipants < MIN_SYNC_COMMITTEE_PARTICIPANTS:
result.reset()
proc getLightClientFinalityUpdate*(
dag: ChainDAGRef): Option[altair.LightClientFinalityUpdate] =
if not dag.lightClientDataServe:
if not dag.lcDataStore.serve:
return
result = some(dag.lightClientCache.latest)
result = some(dag.lcDataStore.cache.latest)
let numParticipants = countOnes(result.get.sync_aggregate.sync_committee_bits)
if numParticipants < MIN_SYNC_COMMITTEE_PARTICIPANTS:
result.reset()
proc getLightClientOptimisticUpdate*(
dag: ChainDAGRef): Option[altair.LightClientOptimisticUpdate] =
if not dag.lightClientDataServe:
if not dag.lcDataStore.serve:
return
result = some(dag.lightClientCache.latest.toOptimistic)
result = some(dag.lcDataStore.cache.latest.toOptimistic)
let numParticipants = countOnes(result.get.sync_aggregate.sync_committee_bits)
if numParticipants < MIN_SYNC_COMMITTEE_PARTICIPANTS:
result.reset()

View File

@ -1,5 +1,5 @@
# beacon_chain
# Copyright (c) 2018-2021 Status Research & Development GmbH
# Copyright (c) 2018-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).
@ -34,5 +34,7 @@ type
## When process_slots() is being called as part of a state_transition(),
## the hash_tree_root() from the block will fill in the state.root so it
## should skip calculating that last state root.
enableTestFeatures ##\
## Whether to enable extra features for testing.
UpdateFlags* = set[UpdateFlag]

View File

@ -1057,7 +1057,7 @@ proc validateLightClientFinalityUpdate*(
# `signature_slot` was given enough time to propagate through the network.
return errIgnore("LightClientFinalityUpdate: received too early")
if finality_update != dag.lightClientCache.latest:
if finality_update != dag.lcDataStore.cache.latest:
# [IGNORE] The received `finality_update` matches the locally computed one
# exactly.
return errIgnore("LightClientFinalityUpdate: not matching local")
@ -1085,7 +1085,7 @@ proc validateLightClientOptimisticUpdate*(
# `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):
if not optimistic_update.matches(dag.lcDataStore.cache.latest):
# [IGNORE] The received `optimistic_update` matches the locally computed one
# exactly.
return errIgnore("LightClientOptimisticUpdate: not matching local")

View File

@ -87,6 +87,56 @@ type
else:
incompatibilityDesc*: string
type DeploymentPhase* {.pure.} = enum
None,
Devnet,
Testnet,
Mainnet
func deploymentPhase*(genesisData: string): DeploymentPhase =
# SSZ processing at compile time does not work well.
#
# `BeaconState` layout:
# ```
# - genesis_time: uint64
# - genesis_validators_root: Eth2Digest
# - ...
# ```
#
# Comparing the first 40 bytes covers those two fields,
# which should identify the network with high likelihood.
# ''.join('%02X'%b for b in open("network_name/genesis.ssz", "rb").read()[:40])
if genesisData.len < 40:
return DeploymentPhase.None
const
mainnets = [
# Mainnet
"5730C65F000000004B363DB94E286120D76EB905340FDD4E54BFE9F06BF33FF6CF5AD27F511BFE95",
]
testnets = [
# Kiln
"0C572B620000000099B09FCD43E5905236C370F184056BEC6E6638CFC31A323B304FC4AA789CB4AD",
# Ropsten
"F0DB94620000000044F1E56283CA88B35C789F7F449E52339BC1FEFE3A45913A43A6D16EDCD33CF1",
# Prater
"60F4596000000000043DB0D9A83813551EE2F33450D23797757D430911A9320530AD8A0EABC43EFB",
# Sepolia
"607DB06200000000D8EA171F3C94AEA21EBC42A1ED61052ACF3F9209C00E4EFBAADDAC09ED9B8078",
]
devnets = [
"placeholder",
]
let data = (genesisData[0 ..< 40].toHex())
if data in mainnets:
return DeploymentPhase.Mainnet
if data in testnets:
return DeploymentPhase.Testnet
if data in devnets:
return DeploymentPhase.Devnet
DeploymentPhase.None
const
eth2NetworksDir = currentSourcePath.parentDir.replace('\\', '/') & "/../../vendor/eth2-networks"
mergeTestnetsDir = currentSourcePath.parentDir.replace('\\', '/') & "/../../vendor/merge-testnets"
@ -166,42 +216,16 @@ proc loadEth2NetworkMetadata*(path: string, eth1Network = none(Eth1Network)): Et
else:
""
shouldSupportLightClient =
if genesisData.len >= 40:
# SSZ processing at compile time does not work well.
#
# `BeaconState` layout:
# ```
# - genesis_time: uint64
# - genesis_validators_root: Eth2Digest
# - ...
# ```
#
# Comparing the first 40 bytes covers those two fields,
# which should identify the network with high likelihood.
# ''.join('%02X'%b for b in open("network_name/genesis.ssz", "rb").read()[:40])
let data = (genesisData[0 ..< 40].toHex())
data in [
# Kiln
"0C572B620000000099B09FCD43E5905236C370F184056BEC6E6638CFC31A323B304FC4AA789CB4AD",
# Ropsten
"F0DB94620000000044F1E56283CA88B35C789F7F449E52339BC1FEFE3A45913A43A6D16EDCD33CF1",
# Prater
"60F4596000000000043DB0D9A83813551EE2F33450D23797757D430911A9320530AD8A0EABC43EFB",
# Sepolia
"607DB06200000000D8EA171F3C94AEA21EBC42A1ED61052ACF3F9209C00E4EFBAADDAC09ED9B8078",
]
else:
false
deploymentPhase = genesisData.deploymentPhase
configDefaults =
Eth2NetworkConfigDefaults(
lightClientEnable:
false, # Only produces debug logs so far
lightClientDataServe:
shouldSupportLightClient,
deploymentPhase <= DeploymentPhase.Testnet,
lightClientDataImportMode:
if shouldSupportLightClient:
if deploymentPhase <= DeploymentPhase.Testnet:
LightClientDataImportMode.OnlyNew
else:
LightClientDataImportMode.None

View File

@ -151,7 +151,8 @@ proc loadChainDag(
db: BeaconChainDB,
eventBus: EventBus,
validatorMonitor: ref ValidatorMonitor,
networkGenesisValidatorsRoot: Option[Eth2Digest]): ChainDAGRef =
networkGenesisValidatorsRoot: Option[Eth2Digest],
shouldEnableTestFeatures: bool): ChainDAGRef =
var dag: ChainDAGRef
info "Loading block DAG from database", path = config.databaseDir
@ -173,6 +174,9 @@ proc loadChainDag(
eventBus.optUpdateQueue.emit(data)
let
extraFlags =
if shouldEnableTestFeatures: {enableTestFeatures}
else: {}
chainDagFlags =
if config.verifyFinalization: {verifyFinalization}
else: {}
@ -184,7 +188,7 @@ proc loadChainDag(
else: nil
dag = ChainDAGRef.init(
cfg, db, validatorMonitor, chainDagFlags, config.eraDir,
cfg, db, validatorMonitor, extraFlags + chainDagFlags, config.eraDir,
onBlockAdded, onHeadChanged, onChainReorg,
onLCFinalityUpdateCb = onLightClientFinalityUpdateCb,
onLCOptimisticUpdateCb = onLightClientOptimisticUpdateCb,
@ -580,7 +584,8 @@ proc init*(T: type BeaconNode,
none(Eth2Digest)
dag = loadChainDag(
config, cfg, db, eventBus,
validatorMonitor, networkGenesisValidatorsRoot)
validatorMonitor, networkGenesisValidatorsRoot,
genesisStateContents.deploymentPhase <= DeploymentPhase.Devnet)
genesisTime = getStateField(dag.headState, genesis_time)
beaconClock = BeaconClock.init(genesisTime)
getBeaconTime = beaconClock.getBeaconTimeFn()
@ -1329,7 +1334,7 @@ proc installRestHandlers(restServer: RestServerRef, node: BeaconNode) =
restServer.router.installNimbusApiHandlers(node)
restServer.router.installNodeApiHandlers(node)
restServer.router.installValidatorApiHandlers(node)
if node.dag.lightClientDataServe:
if node.dag.lcDataStore.serve:
restServer.router.installLightClientApiHandlers(node)
proc installMessageValidators(node: BeaconNode) =

View File

@ -92,7 +92,7 @@ proc installEventApiHandlers*(router: var RestRouter, node: BeaconNode) =
return RestApiResponse.jsonError(Http400, "Invalid topics value",
$topics.error())
let res = validateEventTopics(topics.get(),
node.dag.lightClientDataServe)
node.dag.lcDataStore.serve)
if res.isErr():
return RestApiResponse.jsonError(Http400, "Invalid topics value",
$res.error())
@ -145,12 +145,12 @@ proc installEventApiHandlers*(router: var RestRouter, node: BeaconNode) =
"contribution_and_proof")
res.add(handler)
if EventTopic.LightClientFinalityUpdate in eventTopics:
doAssert node.dag.lightClientDataServe
doAssert node.dag.lcDataStore.serve
let handler = response.eventHandler(node.eventBus.finUpdateQueue,
"light_client_finality_update_v0")
res.add(handler)
if EventTopic.LightClientOptimisticUpdate in eventTopics:
doAssert node.dag.lightClientDataServe
doAssert node.dag.lcDataStore.serve
let handler = response.eventHandler(node.eventBus.optUpdateQueue,
"light_client_optimistic_update_v0")
res.add(handler)

View File

@ -18,7 +18,7 @@ proc installLightClientApiHandlers*(router: var RestRouter, node: BeaconNode) =
router.api(MethodGet,
"/eth/v0/beacon/light_client/bootstrap/{block_root}") do (
block_root: Eth2Digest) -> RestApiResponse:
doAssert node.dag.lightClientDataServe
doAssert node.dag.lcDataStore.serve
let vroot = block:
if block_root.isErr():
return RestApiResponse.jsonError(Http400, InvalidBlockRootValueError,
@ -36,7 +36,7 @@ proc installLightClientApiHandlers*(router: var RestRouter, node: BeaconNode) =
"/eth/v0/beacon/light_client/updates") do (
start_period: Option[SyncCommitteePeriod], count: Option[uint64]
) -> RestApiResponse:
doAssert node.dag.lightClientDataServe
doAssert node.dag.lcDataStore.serve
let vstart = block:
if start_period.isNone():
return RestApiResponse.jsonError(Http400, MissingStartPeriodValueError)
@ -75,7 +75,7 @@ proc installLightClientApiHandlers*(router: var RestRouter, node: BeaconNode) =
router.api(MethodGet,
"/eth/v0/beacon/light_client/finality_update") do (
) -> RestApiResponse:
doAssert node.dag.lightClientDataServe
doAssert node.dag.lcDataStore.serve
let finality_update = node.dag.getLightClientFinalityUpdate()
if finality_update.isSome:
return RestApiResponse.jsonResponse(finality_update)
@ -86,7 +86,7 @@ proc installLightClientApiHandlers*(router: var RestRouter, node: BeaconNode) =
router.api(MethodGet,
"/eth/v0/beacon/light_client/optimistic_update") do (
) -> RestApiResponse:
doAssert node.dag.lightClientDataServe
doAssert node.dag.lcDataStore.serve
let optimistic_update = node.dag.getLightClientOptimisticUpdate()
if optimistic_update.isSome:
return RestApiResponse.jsonResponse(optimistic_update)

View File

@ -536,7 +536,7 @@ p2pProtocol BeaconSync(version = 1,
isLightClientRequest = true).} =
trace "Received LC bootstrap request", peer, blockRoot
let dag = peer.networkState.dag
doAssert dag.lightClientDataServe
doAssert dag.lcDataStore.serve
peer.updateRequestQuota(lightClientBootstrapLookupCost)
peer.awaitNonNegativeRequestQuota()
@ -565,7 +565,7 @@ p2pProtocol BeaconSync(version = 1,
isLightClientRequest = true).} =
trace "Received LC updates by range request", peer, startPeriod, reqCount
let dag = peer.networkState.dag
doAssert dag.lightClientDataServe
doAssert dag.lcDataStore.serve
let
headPeriod = dag.head.slot.sync_committee_period
@ -605,7 +605,7 @@ p2pProtocol BeaconSync(version = 1,
isLightClientRequest = true).} =
trace "Received LC finality update request", peer
let dag = peer.networkState.dag
doAssert dag.lightClientDataServe
doAssert dag.lcDataStore.serve
peer.awaitNonNegativeRequestQuota()
@ -631,7 +631,7 @@ p2pProtocol BeaconSync(version = 1,
isLightClientRequest = true).} =
trace "Received LC optimistic update request", peer
let dag = peer.networkState.dag
doAssert dag.lightClientDataServe
doAssert dag.lcDataStore.serve
peer.awaitNonNegativeRequestQuota()

View File

@ -255,7 +255,7 @@ proc handleLightClientUpdates(node: BeaconNode, slot: Slot) {.async.} =
debug "Waiting to send LC updates", slot, delay = shortLog(sendTime.offset)
await sleepAsync(sendTime.offset)
template latest(): auto = node.dag.lightClientCache.latest
template latest(): auto = node.dag.lcDataStore.cache.latest
let signature_slot = latest.signature_slot
if slot != signature_slot:
return