set upper bound on EpochRef cache (#2403)

* set upper bound on EpochRef cache

* max 32 EpochRef instances
* less memory waste in BlockRef by removing EpochRef seq that is mostly
unused (~20mb)
* less memory waste in dag block lookup by not keeping an extra copy of
digest (~70mb)
* fix `==` and `$` for Eth2Digest
* remove `ChainDAG.tmpState` (~50mb?)

all in all, this branch cuts mainnet memory usage by ~160-180mb and puts
limits on EpochRef cache usage - where normally it hovered around 950mb
before, it's now sitting at 600-700mb on my machine.

* docs
This commit is contained in:
Jacek Sieka 2021-03-17 11:17:15 +01:00 committed by GitHub
parent 5e3770e994
commit 3cb31e66b4
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
12 changed files with 179 additions and 111 deletions

View File

@ -91,17 +91,15 @@ proc addResolvedBlock(
link(parent, blockRef)
var epochRef = blockRef.findEpochRef(blockEpoch)
var epochRef = dag.findEpochRef(parent, blockEpoch)
if epochRef == nil:
let prevEpochRef = blockRef.findEpochRef(blockEpoch - 1)
let prevEpochRef =
if blockEpoch < 1: nil else: dag.findEpochRef(parent, blockEpoch - 1)
epochRef = EpochRef.init(state.data.data, cache, prevEpochRef)
let ancestor = blockRef.epochAncestor(blockEpoch)
epochRef.updateKeyStores(ancestor.blck.parent, dag.finalizedHead.blck)
dag.addEpochRef(blockRef, epochRef)
ancestor.blck.epochRefs.add epochRef
dag.blocks[blockRoot] = blockRef
dag.blocks.incl(KeyedBlockRef.init(blockRef))
trace "Populating block dag", key = blockRoot, val = blockRef
# Resolved blocks should be stored in database
@ -147,7 +145,8 @@ proc addResolvedBlock(
entries = quarantine.orphans.len # keep going while quarantine is shrinking
var resolved: seq[SignedBeaconBlock]
for _, v in quarantine.orphans:
if v.message.parent_root in dag.blocks: resolved.add(v)
if v.message.parent_root in dag:
resolved.add(v)
for v in resolved:
discard addRawBlock(dag, quarantine, v, onBlockAdded)
@ -299,7 +298,7 @@ proc addRawBlock*(
template blck(): untyped = signedBlock.message # shortcuts without copy
template blockRoot(): untyped = signedBlock.root
if blockRoot in dag.blocks:
if blockRoot in dag:
debug "Block already exists"
# We should not call the block added callback for blocks that already
@ -323,7 +322,7 @@ proc addRawBlock*(
# be used, but certainly would be IGNORE.
return err((ValidationResult.Ignore, Unviable))
let parent = dag.blocks.getOrDefault(blck.parent_root)
let parent = dag.getRef(blck.parent_root)
if parent != nil:
return addRawBlockKnownParent(dag, quarantine, signedBlock, parent, onBlockAdded)

View File

@ -9,7 +9,7 @@
import
# Standard library
std/[deques, strformat, tables, hashes],
std/[deques, strformat, sets, tables, hashes],
# Status libraries
stew/[endians2, byteutils], chronicles,
eth/keys,
@ -18,7 +18,8 @@ import
../beacon_chain_db, ../extras
from libp2p/protocols/pubsub/pubsub import ValidationResult
export ValidationResult
export ValidationResult, sets, tables
# #############################################
#
@ -79,6 +80,13 @@ type
FetchRecord* = object
root*: Eth2Digest
KeyedBlockRef* = object
# Special wrapper for BlockRef used in ChainDAG.blocks that allows lookup
# by root without keeping a Table that keeps a separate copy of the digest
# At the time of writing, a Table[Eth2Digest, BlockRef] adds about 100mb of
# unnecessary overhead.
data: BlockRef
ChainDAGRef* = ref object
## Pool of blocks responsible for keeping a DAG of resolved blocks.
##
@ -105,7 +113,7 @@ type
# -----------------------------------
# ChainDAGRef - DAG of candidate chains
blocks*: Table[Eth2Digest, BlockRef] ##\
blocks*: HashSet[KeyedBlockRef] ##\
## Directed acyclic graph of blocks pointing back to a finalized block on the chain we're
## interested in - we call that block the tail
@ -144,12 +152,15 @@ type
## Cached state used during block clearance - must only be used in
## clearance module
tmpState*: StateData ## Scratchpad - may be any state
updateFlags*: UpdateFlags
runtimePreset*: RuntimePreset
epochRefs*: array[32, (BlockRef, EpochRef)] ##\
## Cached information about a particular epoch ending with the given
## block - we limit the number of held EpochRefs to put a cap on
## memory usage
EpochRef* = ref object
epoch*: Epoch
current_justified_checkpoint*: Checkpoint
@ -180,16 +191,6 @@ type
slot*: Slot # could calculate this by walking to root, but..
epochRefs*: seq[EpochRef] ##\
## Cached information about the epochs starting at this block.
## Could be multiple, since blocks could skip slots, but usually, not many
## Even if competing forks happen later during this epoch, potential empty
## slots beforehand must all be from this fork. find/getEpochRef() are the
## only supported way of accesssing these.
## In particular, epoch refs are only stored with the last block of the
## parent epoch - this way, it's easy to find them from any block in the
## epoch - including when there are forks that skip the epoch slot.
BlockData* = object
## Body and graph in one
@ -253,5 +254,18 @@ func shortLog*(v: EpochRef): string =
chronicles.formatIt BlockSlot: shortLog(it)
chronicles.formatIt BlockRef: shortLog(it)
func hash*(blockRef: BlockRef): Hash =
hash(blockRef.root)
func hash*(key: KeyedBlockRef): Hash =
hash(key.data.root)
func `==`*(a, b: KeyedBlockRef): bool =
a.data.root == b.data.root
func asLookupKey*(T: type KeyedBlockRef, root: Eth2Digest): KeyedBlockRef =
# Create a special, temporary BlockRef instance that just has the key set
KeyedBlockRef(data: BlockRef(root: root))
func init*(T: type KeyedBlockRef, blck: BlockRef): KeyedBlockRef =
KeyedBlockRef(data: blck)
func blockRef*(key: KeyedBlockRef): BlockRef =
key.data

View File

@ -52,6 +52,16 @@ proc updateStateData*(
dag: ChainDAGRef, state: var StateData, bs: BlockSlot, save: bool,
cache: var StateCache) {.gcsafe.}
template withStateVars*(stateData: var StateData, body: untyped): untyped =
## Inject a few more descriptive names for the members of `stateData` -
## the stateData instance may get mutated through these names as well
template hashedState(): HashedBeaconState {.inject, used.} = stateData.data
template state(): BeaconState {.inject, used.} = stateData.data.data
template blck(): BlockRef {.inject, used.} = stateData.blck
template root(): Eth2Digest {.inject, used.} = stateData.data.root
body
template withState*(
dag: ChainDAGRef, stateData: var StateData, blockSlot: BlockSlot,
body: untyped): untyped =
@ -63,12 +73,8 @@ template withState*(
var cache {.inject.} = StateCache()
updateStateData(dag, stateData, blockSlot, false, cache)
template hashedState(): HashedBeaconState {.inject, used.} = stateData.data
template state(): BeaconState {.inject, used.} = stateData.data.data
template blck(): BlockRef {.inject, used.} = stateData.blck
template root(): Eth2Digest {.inject, used.} = stateData.data.root
body
withStateVars(stateData):
body
func parent*(bs: BlockSlot): BlockSlot =
## Return a blockslot representing the previous slot, using the parent block
@ -180,16 +186,6 @@ func effective_balances*(epochRef: EpochRef): seq[Gwei] =
except CatchableError as exc:
raiseAssert exc.msg
func updateKeyStores*(epochRef: EpochRef, blck: BlockRef, finalized: BlockRef) =
# Because key stores are additive lists, we can use a newer list whereever an
# older list is expected - all indices in the new list will be valid for the
# old list also
var blck = blck
while blck != nil and blck.slot >= finalized.slot:
for e in blck.epochRefs:
e.validator_key_store = epochRef.validator_key_store
blck = blck.parent
func link*(parent, child: BlockRef) =
doAssert (not (parent.root == Eth2Digest() or child.root == Eth2Digest())),
"blocks missing root!"
@ -265,31 +261,32 @@ func epochAncestor*(blck: BlockRef, epoch: Epoch): BlockSlot =
##
## This function returns a BlockSlot pointing to that epoch boundary, ie the
## boundary where the last block has been applied to the state and epoch
## processing has been done - we will store epoch caches in that particular
## block so that any block in the dag that needs it can find it easily. In
## particular, if empty slot processing is done, there may be multiple epoch
## caches found there.
## processing has been done.
var blck = blck
while blck.slot.epoch >= epoch and not blck.parent.isNil:
blck = blck.parent
blck.atEpochStart(epoch)
func findEpochRef*(blck: BlockRef, epoch: Epoch): EpochRef = # may return nil!
func findEpochRef*(
dag: ChainDAGRef, blck: BlockRef, epoch: Epoch): EpochRef = # may return nil!
let ancestor = blck.epochAncestor(epoch)
doAssert ancestor.blck != nil
for epochRef in ancestor.blck.epochRefs:
if epochRef.epoch == epoch:
return epochRef
for i in 0..<dag.epochRefs.len:
if dag.epochRefs[i][0] == ancestor.blck and dag.epochRefs[i][1].epoch == epoch:
return dag.epochRefs[i][1]
proc loadStateCache*(cache: var StateCache, blck: BlockRef, epoch: Epoch) =
return nil
proc loadStateCache*(
dag: ChainDAGRef, cache: var StateCache, blck: BlockRef, epoch: Epoch) =
# When creating a state cache, we want the current and the previous epoch
# information to be preloaded as both of these are used in state transition
# functions
template load(e: Epoch) =
if epoch notin cache.shuffled_active_validator_indices:
let epochRef = blck.findEpochRef(epoch)
let epochRef = dag.findEpochRef(blck, epoch)
if epochRef != nil:
cache.shuffled_active_validator_indices[epochRef.epoch] =
epochRef.shuffled_active_validator_indices
@ -314,6 +311,9 @@ func init(T: type BlockRef, root: Eth2Digest, slot: Slot): BlockRef =
func init*(T: type BlockRef, root: Eth2Digest, blck: SomeBeaconBlock): BlockRef =
BlockRef.init(root, blck.slot)
func contains*(dag: ChainDAGRef, root: Eth2Digest): bool =
KeyedBlockRef.asLookupKey(root) in dag.blocks
proc init*(T: type ChainDAGRef,
preset: RuntimePreset,
db: BeaconChainDB,
@ -344,11 +344,13 @@ proc init*(T: type ChainDAGRef,
BlockRef.init(genesisBlockRoot, genesisBlock.message)
var
blocks = {tailRef.root: tailRef}.toTable()
blocks: HashSet[KeyedBlockRef]
headRef: BlockRef
blocks.incl(KeyedBlockRef.init(tailRef))
if genesisRef != tailRef:
blocks[genesisRef.root] = genesisRef
blocks.incl(KeyedBlockRef.init(genesisRef))
if headRoot != tailRoot:
var curRef: BlockRef
@ -367,7 +369,7 @@ proc init*(T: type ChainDAGRef,
else:
link(newRef, curRef)
curRef = curRef.parent
blocks[curRef.root] = curRef
blocks.incl(KeyedBlockRef.init(curRef))
trace "Populating block dag", key = curRef.root, val = curRef
doAssert curRef == tailRef,
@ -414,7 +416,6 @@ proc init*(T: type ChainDAGRef,
headState: tmpState[],
epochRefState: tmpState[],
clearanceState: tmpState[],
tmpState: tmpState[],
# The only allowed flag right now is verifyFinalization, as the others all
# allow skipping some validation.
@ -450,8 +451,38 @@ proc init*(T: type ChainDAGRef,
res
proc addEpochRef*(dag: ChainDAGRef, blck: BlockRef, epochRef: EpochRef) =
# Because we put a cap on the number of epochRefs we store, we want to
# prune the least useful state - for now, we'll assume that to be the oldest
# epochRef we know about.
var
oldest = 0
ancestor = blck.epochAncestor(epochRef.epoch)
for x in 0..<dag.epochRefs.len:
let candidate = dag.epochRefs[x]
if candidate[1] == nil:
oldest = x
break
if candidate[1].epoch < dag.epochRefs[oldest][1].epoch:
oldest = x
dag.epochRefs[oldest] = (ancestor.blck, epochRef)
# Because key stores are additive lists, we can use a newer list whereever an
# older list is expected - all indices in the new list will be valid for the
# old list also
if epochRef.epoch > 0:
var cur = ancestor.blck.epochAncestor(epochRef.epoch - 1)
while cur.slot >= dag.finalizedHead.slot:
let er = dag.findEpochRef(cur.blck, cur.slot.epoch)
if er != nil:
er.validator_key_store = epochRef.validator_key_store
if cur.slot.epoch == 0:
break
cur = cur.blck.epochAncestor(cur.slot.epoch - 1)
proc getEpochRef*(dag: ChainDAGRef, blck: BlockRef, epoch: Epoch): EpochRef =
let epochRef = blck.findEpochRef(epoch)
let epochRef = dag.findEpochRef(blck, epoch)
if epochRef != nil:
beacon_state_data_cache_hits.inc
return epochRef
@ -463,16 +494,14 @@ proc getEpochRef*(dag: ChainDAGRef, blck: BlockRef, epoch: Epoch): EpochRef =
dag.withState(dag.epochRefState, ancestor):
let
prevEpochRef = if dag.tail.slot.epoch >= epoch: nil
else: blck.findEpochRef(epoch - 1)
prevEpochRef = if epoch < 1: nil
else: dag.findEpochRef(blck, epoch - 1)
newEpochRef = EpochRef.init(state, cache, prevEpochRef)
# TODO consider constraining the number of epochrefs per state
if ancestor.blck.slot >= dag.finalizedHead.blck.slot:
if epoch >= dag.finalizedHead.slot.epoch():
# Only cache epoch information for unfinalized blocks - earlier states
# are seldomly used (ie RPC), so no need to cache
ancestor.blck.epochRefs.add newEpochRef
newEpochRef.updateKeyStores(blck.parent, dag.finalizedHead.blck)
dag.addEpochRef(blck, newEpochRef)
newEpochRef
proc getFinalizedEpochRef*(dag: ChainDAGRef): EpochRef =
@ -484,7 +513,7 @@ proc getState(
let restoreAddr =
# Any restore point will do as long as it's not the object being updated
if unsafeAddr(state) == unsafeAddr(dag.headState):
unsafeAddr dag.tmpState
unsafeAddr dag.clearanceState
else:
unsafeAddr dag.headState
@ -561,7 +590,14 @@ proc putState*(dag: ChainDAGRef, state: var StateData) =
func getRef*(dag: ChainDAGRef, root: Eth2Digest): BlockRef =
## Retrieve a resolved block reference, if available
dag.blocks.getOrDefault(root, nil)
let key = KeyedBlockRef.asLookupKey(root)
# HashSet lacks the api to do check-and-get in one lookup - `[]` will return
# the copy of the instance in the set which has more fields than `root` set!
if key in dag.blocks:
try: dag.blocks[key].blockRef()
except KeyError: raiseAssert "contains"
else:
nil
func getBlockRange*(
dag: ChainDAGRef, startSlot: Slot, skipStep: uint64,
@ -675,7 +711,7 @@ proc applyBlock(
doAssert (addr(statePtr.data) == addr v)
statePtr[] = dag.headState
loadStateCache(cache, blck.refs, blck.data.message.slot.epoch)
loadStateCache(dag, cache, blck.refs, blck.data.message.slot.epoch)
let ok = state_transition(
dag.runtimePreset, state.data, blck.data,
@ -800,7 +836,7 @@ proc updateStateData*(
dag.applyBlock(state, dag.get(ancestors[i]), {}, cache)
doAssert ok, "Blocks in database should never fail to apply.."
loadStateCache(cache, bs.blck, bs.slot.epoch)
loadStateCache(dag, cache, bs.blck, bs.slot.epoch)
# ...and make sure to process empty slots as requested
dag.advanceSlots(state, bs.slot, save, cache)
@ -869,7 +905,7 @@ proc pruneBlocksDAG(dag: ChainDAGRef) =
dag.delState(cur) # TODO: should we move that disk I/O to `onSlotEnd`
if cur.blck.slot == cur.slot:
dag.blocks.del(cur.blck.root)
dag.blocks.excl(KeyedBlockRef.init(cur.blck))
dag.db.delBlock(cur.blck.root)
if cur.blck.parent.isNil:
@ -921,13 +957,10 @@ proc pruneStateCachesDAG*(dag: ChainDAGRef) =
block: # Clean up old EpochRef instances
# After finalization, we can clear up the epoch cache and save memory -
# it will be recomputed if needed
# TODO don't store recomputed pre-finalization epoch refs
var tmp = dag.finalizedHead.blck
while tmp != dag.lastPrunePoint.blck:
# leave the epoch cache in the last block of the epoch..
tmp = tmp.parent
if tmp.parent != nil:
tmp.parent.epochRefs = @[]
for i in 0..<dag.epochRefs.len:
if dag.epochRefs[i][1] != nil and
dag.epochRefs[i][1].epoch < dag.finalizedHead.slot.epoch:
dag.epochRefs[i] = (nil, nil)
let stopEpochRef = getTime()
let durEpochRef = stopEpochRef - startEpochRef

View File

@ -118,7 +118,7 @@ proc blockValidator*(
let delay = wallTime - signedBlock.message.slot.toBeaconTime
if signedBlock.root in self.chainDag.blocks:
if signedBlock.root in self.chainDag:
# The gossip algorithm itself already does one round of hashing to find
# already-seen data, but it is fairly aggressive about forgetting about
# what it has seen already

View File

@ -96,7 +96,8 @@ proc installNimbusApiHandlers*(rpcServer: RpcServer, node: BeaconNode) =
wallSlot = node.beaconClock.now.slotOrZero
head = node.doChecksAndGetCurrentHead(wallSlot)
node.chainDag.withState(node.chainDag.tmpState, head.atSlot(wallSlot)):
let proposalState = assignClone(node.chainDag.headState)
node.chainDag.withState(proposalState[], head.atSlot(wallSlot)):
return node.getBlockProposalEth1Data(state)
rpcServer.rpc("debug_getChronosFutures") do () -> seq[FutureInfo]:

View File

@ -8,10 +8,20 @@ import
export blockchain_dag
template withStateForStateId*(stateId: string, body: untyped): untyped =
# TODO this can be optimized for the "head" case since that should be most common
node.chainDag.withState(node.chainDag.tmpState,
node.stateIdToBlockSlot(stateId)):
body
let
bs = node.stateIdToBlockSlot(stateId)
template isState(state: StateData): bool =
state.blck.atSlot(state.data.data.slot) == bs
if isState(node.chainDag.headState):
withStateVars(node.chainDag.headState):
var cache {.inject.}: StateCache
body
else:
let rpcState = assignClone(node.chainDag.headState)
node.chainDag.withState(rpcState[], bs):
body
proc toBlockSlot*(blckRef: BlockRef): BlockSlot =
blckRef.atSlot(blckRef.slot)

View File

@ -32,7 +32,9 @@ import
blscurve
export
hash.`$`, sha2, readValue, writeValue
# Exports from sha2 / hash are explicit to avoid exporting upper-case `$` and
# constant-time `==`
sha2.update, hash.fromHex, readValue, writeValue
type
Eth2Digest* = MDigest[32 * 8] ## `hash32` from spec
@ -43,6 +45,9 @@ when BLS_BACKEND == BLST:
else:
type Eth2DigestCtx* = sha2.sha256
func `$`*(x: Eth2Digest): string =
x.data.toHex()
func shortLog*(x: Eth2Digest): string =
x.data.toOpenArray(0, 3).toHex()

View File

@ -248,7 +248,12 @@ proc makeBeaconBlockForHeadAndSlot*(node: BeaconNode,
head: BlockRef,
slot: Slot): Option[BeaconBlock] =
# Advance state to the slot that we're proposing for
node.chainDag.withState(node.chainDag.tmpState, head.atSlot(slot)):
let
proposalState = assignClone(node.chainDag.headState)
proposalStateAddr = unsafeAddr proposalState[]
node.chainDag.withState(proposalState[], head.atSlot(slot)):
let
eth1Proposal = node.getBlockProposalEth1Data(state)
poolPtr = unsafeAddr node.chainDag # safe because restore is short-lived
@ -261,8 +266,8 @@ proc makeBeaconBlockForHeadAndSlot*(node: BeaconNode,
# TODO address this ugly workaround - there should probably be a
# `state_transition` that takes a `StateData` instead and updates
# the block as well
doAssert v.addr == addr poolPtr.tmpState.data
assign(poolPtr.tmpState, poolPtr.headState)
doAssert v.addr == addr proposalStateAddr.data
assign(proposalStateAddr[], poolPtr.headState)
makeBeaconBlock(
node.runtimePreset,

View File

@ -316,7 +316,8 @@ proc cmdRewindState(conf: DbConf, preset: RuntimePreset) =
echo "Block not found in database"
return
dag.withState(dag.tmpState, blckRef.atSlot(Slot(conf.slot))):
let tmpState = assignClone(dag.headState)
dag.withState(tmpState[], blckRef.atSlot(Slot(conf.slot))):
echo "Writing state..."
dump("./", hashedState, blck)
@ -338,6 +339,8 @@ proc cmdExportEra(conf: DbConf, preset: RuntimePreset) =
let
dag = init(ChainDAGRef, preset, db)
let tmpState = assignClone(dag.headState)
for era in conf.era..<conf.era + conf.eraCount:
let
firstSlot = if era == 0: Slot(0) else: Slot((era - 1) * SLOTS_PER_HISTORICAL_ROOT)
@ -353,7 +356,7 @@ proc cmdExportEra(conf: DbConf, preset: RuntimePreset) =
var e2s = E2Store.open(".", name, firstSlot).get()
defer: e2s.close()
dag.withState(dag.tmpState, canonical):
dag.withState(tmpState[], canonical):
e2s.appendRecord(state).get()
var

View File

@ -78,6 +78,7 @@ cli do(slots = SLOTS_PER_EPOCH * 6,
timers: array[Timers, RunningStat]
attesters: RunningStat
r = initRand(1)
tmpState = assignClone(chainDag.headState)
eth1Chain.addBlock Eth1Block(
number: Eth1BlockNumber 1,
@ -92,7 +93,7 @@ cli do(slots = SLOTS_PER_EPOCH * 6,
let
attestationHead = chainDag.head.atSlot(slot)
chainDag.withState(chainDag.tmpState, attestationHead):
chainDag.withState(tmpState[], attestationHead):
let committees_per_slot =
get_committee_count_per_slot(state, slot.epoch, cache)
@ -126,7 +127,7 @@ cli do(slots = SLOTS_PER_EPOCH * 6,
let
head = chainDag.head
chainDag.withState(chainDag.tmpState, head.atSlot(slot)):
chainDag.withState(tmpState[], head.atSlot(slot)):
let
finalizedEpochRef = chainDag.getFinalizedEpochRef()
proposerIdx = get_beacon_proposer_index(state, cache).get()

View File

@ -169,10 +169,10 @@ suiteReport "Block pool processing" & preset():
b2Add[].root == b2Get.get().refs.root
dag.heads.len == 1
dag.heads[0] == b2Add[]
not b1Add[].findEpochRef(b1Add[].slot.epoch).isNil
b1Add[].findEpochRef(b1Add[].slot.epoch) ==
b2Add[].findEpochRef(b2Add[].slot.epoch)
b1Add[].findEpochRef(b1Add[].slot.epoch + 1).isNil
not dag.findEpochRef(b1Add[], b1Add[].slot.epoch).isNil
dag.findEpochRef(b1Add[], b1Add[].slot.epoch) ==
dag.findEpochRef(b2Add[], b2Add[].slot.epoch)
dag.findEpochRef(b1Add[], b1Add[].slot.epoch + 1).isNil
# Skip one slot to get a gap
check:
@ -385,36 +385,31 @@ suiteReport "chain DAG finalization tests" & preset():
dag.heads.len() == 1
let
headER = dag.heads[0].findEpochRef(dag.heads[0].slot.epoch)
finalER = dag.finalizedHead.blck.findEpochRef(dag.finalizedHead.slot.epoch)
headER = dag.findEpochRef(dag.heads[0], dag.heads[0].slot.epoch)
finalER = dag.findEpochRef(dag.finalizedHead.blck, dag.finalizedHead.slot.epoch)
check:
# Epochrefs should share validator key set when the validator set is
# stable
not headER.isNil
not dag.heads[0].findEpochRef(dag.heads[0].slot.epoch - 1).isNil
not dag.findEpochRef(dag.heads[0], dag.heads[0].slot.epoch - 1).isNil
headER !=
dag.heads[0].findEpochRef(dag.heads[0].slot.epoch - 1)
dag.findEpochRef(dag.heads[0], dag.heads[0].slot.epoch - 1)
headER.validator_key_store[1] ==
dag.heads[0].findEpochRef(dag.heads[0].slot.epoch - 1).validator_key_store[1]
dag.findEpochRef(dag.heads[0], dag.heads[0].slot.epoch - 1).validator_key_store[1]
# The EpochRef for the finalized block is needed for eth1 voting, so we
# should never drop it!
not finalER.isNil
block:
var cur = dag.heads[0]
while cur != nil:
if cur.slot < dag.finalizedHead.blck.parent.slot:
# Cache should be cleaned on finalization
check: cur.epochRefs.len == 0
else:
for er in dag.epochRefs:
check: er[1] == nil or er[1].epoch >= dag.finalizedHead.slot.epoch
if er[1] != nil:
# EpochRef validator keystores should back-propagate to all previous
# epochs
for e in cur.epochRefs:
check (addr headER.validator_keys) == (addr e.validator_keys)
cur = cur.parent
check (addr headER.validator_keys) == (addr er[1].validator_keys)
block:
# The late block is a block whose parent was finalized long ago and thus
# is no longer a viable head candidate

View File

@ -8,7 +8,9 @@
{.used.}
import
unittest, options, json_serialization,
std/[unittest, options],
nimcrypto/hash,
json_serialization,
serialization/testing/generic_suite,
./testutil,
../beacon_chain/spec/[datatypes, digest],
@ -94,7 +96,7 @@ suiteReport "SSZ navigator":
leaves.add b
leaves.add c
let root = hash_tree_root(leaves)
check $root == "5248085B588FAB1DD1E03F3CD62201602B12E6560665935964F46E805977E8C5"
check $root == "5248085b588fab1dd1e03f3cd62201602b12e6560665935964f46e805977e8c5"
while leaves.len < 1 shl 3:
leaves.add c