renamed CandidateChains to ChainDagRef and made the Quarantine type a ref type so there is a single instance in the beacon node (#1407)

This commit is contained in:
Viktor Kirilov 2020-07-31 17:49:06 +03:00 committed by GitHub
parent e0a6f58abe
commit 0a96e5f564
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
18 changed files with 102 additions and 102 deletions

View File

@ -12,7 +12,7 @@ import
./spec/[
beaconstate, datatypes, crypto, digest, helpers, network, validator,
signatures],
./block_pools/[spec_cache, candidate_chains, quarantine], ./attestation_pool,
./block_pools/[spec_cache, chain_dag, quarantine], ./attestation_pool,
./beacon_node_types, ./ssz
logScope:

View File

@ -14,14 +14,14 @@ import
chronicles, stew/[byteutils], json_serialization/std/sets as jsonSets,
# Internal
./spec/[beaconstate, datatypes, crypto, digest, helpers],
./block_pools/[spec_cache, candidate_chains, clearance], ./beacon_node_types,
./block_pools/[spec_cache, chain_dag, clearance], ./beacon_node_types,
./fork_choice/fork_choice
export beacon_node_types, sets
logScope: topics = "attpool"
proc init*(T: type AttestationPool, chainDag: CandidateChains, quarantine: Quarantine): T =
proc init*(T: type AttestationPool, chainDag: ChainDAGRef, quarantine: QuarantineRef): T =
## Initialize an AttestationPool from the chainDag `headState`
## The `finalized_root` works around the finalized_checkpoint of the genesis block
## holding a zero_root.

View File

@ -26,7 +26,7 @@ import
conf, time, beacon_chain_db, validator_pool, extras,
attestation_pool, eth2_network, eth2_discovery,
beacon_node_common, beacon_node_types,
block_pools/[spec_cache, candidate_chains, quarantine, clearance, block_pools_types],
block_pools/[spec_cache, chain_dag, quarantine, clearance, block_pools_types],
nimbus_binary_common, network_metadata,
mainchain_monitor, version, ssz/[merkleization], sszdump, merkle_minimal,
sync_protocol, request_manager, keystore_management, interop, statusbar,
@ -150,7 +150,7 @@ proc init*(T: type BeaconNode,
var mainchainMonitor: MainchainMonitor
if not CandidateChains.isInitialized(db):
if not ChainDAGRef.isInitialized(db):
# Fresh start - need to load a genesis state from somewhere
var genesisState = conf.getStateFromSnapshot()
@ -215,8 +215,8 @@ proc init*(T: type BeaconNode,
let tailBlock = get_initial_beacon_block(genesisState[])
try:
CandidateChains.preInit(db, genesisState[], tailBlock)
doAssert CandidateChains.isInitialized(db), "preInit should have initialized db"
ChainDAGRef.preInit(db, genesisState[], tailBlock)
doAssert ChainDAGRef.isInitialized(db), "preInit should have initialized db"
except CatchableError as e:
error "Failed to initialize database", err = e.msg
quit 1
@ -229,8 +229,8 @@ proc init*(T: type BeaconNode,
let
chainDagFlags = if conf.verifyFinalization: {verifyFinalization}
else: {}
chainDag = init(CandidateChains, conf.runtimePreset, db, chainDagFlags)
quarantine = Quarantine()
chainDag = init(ChainDAGRef, conf.runtimePreset, db, chainDagFlags)
quarantine = QuarantineRef()
if mainchainMonitor.isNil and
conf.web3Url.len > 0 and

View File

@ -19,7 +19,7 @@ import
spec/[datatypes, crypto, digest],
conf, time, beacon_chain_db,
attestation_pool, eth2_network,
block_pools/[candidate_chains, quarantine],
block_pools/[chain_dag, quarantine],
beacon_node_types, mainchain_monitor, request_manager,
sync_manager
@ -40,8 +40,8 @@ type
db*: BeaconChainDB
config*: BeaconNodeConf
attachedValidators*: ValidatorPool
chainDag*: CandidateChains
quarantine*: Quarantine
chainDag*: ChainDAGRef
quarantine*: QuarantineRef
attestationPool*: AttestationPool
mainchainMonitor*: MainchainMonitor
beaconClock*: BeaconClock

View File

@ -63,8 +63,8 @@ type
## Generally, we keep attestations only until a slot has been finalized -
## after that, they may no longer affect fork choice.
chainDag*: CandidateChains
quarantine*: Quarantine
chainDag*: ChainDAGRef
quarantine*: QuarantineRef
unresolved*: Table[Eth2Digest, UnresolvedAttestation]

View File

@ -21,7 +21,7 @@ import
# #############################################
#
# The Quarantine and DagChain data structures
# keeps track respectively of unsafe blocks coming from the network
# keep track respectively of unsafe blocks coming from the network
# and blocks that underwent verification and have a resolved path to
# the last finalized block known.
@ -42,12 +42,12 @@ type
Old
Duplicate
Quarantine* = object
QuarantineRef* = ref object
## Keeps track of unsafe blocks coming from the network
## and that cannot be added to the chain
##
## This only stores valid blocks that cannot be linked to the
## CandidateChains DAG due to missing ancestor(s).
## ChainDAGRef DAG due to missing ancestor(s).
##
## Invalid blocks are dropped immediately.
@ -68,7 +68,7 @@ type
FetchRecord* = object
root*: Eth2Digest
CandidateChains* = ref object
ChainDAGRef* = ref object
## Pool of blocks responsible for keeping a DAG of resolved blocks.
##
## It is responsible for the following
@ -92,7 +92,7 @@ type
## ColdDB - Stores the canonical chain
# -----------------------------------
# CandidateChains - DAG of candidate chains
# ChainDAGRef - DAG of candidate chains
blocks*: Table[Eth2Digest, BlockRef] ##\
## Directed acyclic graph of blocks pointing back to a finalized block on the chain we're

View File

@ -28,15 +28,15 @@ declareCounter beacon_state_data_cache_misses, "dag.cachedStates misses"
logScope: topics = "hotdb"
proc putBlock*(
dag: var CandidateChains, signedBlock: SignedBeaconBlock) =
dag: var ChainDAGRef, signedBlock: SignedBeaconBlock) =
dag.db.putBlock(signedBlock)
proc updateStateData*(
dag: CandidateChains, state: var StateData, bs: BlockSlot,
dag: ChainDAGRef, state: var StateData, bs: BlockSlot,
matchEpoch: bool = false) {.gcsafe.}
template withState*(
dag: CandidateChains, cache: var StateData, blockSlot: BlockSlot, body: untyped): untyped =
dag: ChainDAGRef, cache: var StateData, blockSlot: BlockSlot, body: untyped): untyped =
## Helper template that updates state to a particular BlockSlot - usage of
## cache is unsafe outside of block.
## TODO async transformations will lead to a race where cache gets updated
@ -186,7 +186,7 @@ func getEpochInfo*(blck: BlockRef, state: BeaconState): EpochRef =
if (compute_activation_exit_epoch(blck.slot.compute_epoch_at_slot) >
state_epoch):
blck.epochsInfo.add(epochInfo)
trace "candidate_chains.getEpochInfo: back-filling parent.epochInfo",
trace "chain_dag.getEpochInfo: back-filling parent.epochInfo",
state_slot = state.slot
epochInfo
elif matching_epochinfo.len == 1:
@ -222,10 +222,10 @@ 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)
proc init*(T: type CandidateChains,
proc init*(T: type ChainDAGRef,
preset: RuntimePreset,
db: BeaconChainDB,
updateFlags: UpdateFlags = {}): CandidateChains =
updateFlags: UpdateFlags = {}): ChainDAGRef =
# TODO we require that the db contains both a head and a tail block -
# asserting here doesn't seem like the right way to go about it however..
@ -305,7 +305,7 @@ proc init*(T: type CandidateChains,
finalizedHead = headRef.atEpochStart(
tmpState.data.data.finalized_checkpoint.epoch)
let res = CandidateChains(
let res = ChainDAGRef(
blocks: blocks,
tail: tailRef,
head: headRef,
@ -337,7 +337,7 @@ proc init*(T: type CandidateChains,
res
proc getEpochRef*(dag: CandidateChains, blck: BlockRef, epoch: Epoch): EpochRef =
proc getEpochRef*(dag: ChainDAGRef, blck: BlockRef, epoch: Epoch): EpochRef =
var bs = blck.atEpochEnd(epoch)
while true:
@ -354,7 +354,7 @@ proc getEpochRef*(dag: CandidateChains, blck: BlockRef, epoch: Epoch): EpochRef
getEpochInfo(blck, state)
proc getState(
dag: CandidateChains, db: BeaconChainDB, stateRoot: Eth2Digest, blck: BlockRef,
dag: ChainDAGRef, db: BeaconChainDB, stateRoot: Eth2Digest, blck: BlockRef,
output: var StateData): bool =
let outputAddr = unsafeAddr output # local scope
func restore(v: var BeaconState) =
@ -377,7 +377,7 @@ proc getState(
true
func getStateCacheIndex(
dag: CandidateChains, blockRoot: Eth2Digest, slot: Slot, matchEpoch: bool):
dag: ChainDAGRef, blockRoot: Eth2Digest, slot: Slot, matchEpoch: bool):
int =
for i, cachedState in dag.cachedStates:
let (cacheBlockRoot, cacheSlot, _) = cachedState
@ -389,7 +389,7 @@ func getStateCacheIndex(
-1
func putStateCache*(
dag: CandidateChains, state: HashedBeaconState, blck: BlockRef) =
dag: ChainDAGRef, state: HashedBeaconState, blck: BlockRef) =
# Efficiently access states for both attestation aggregation and to process
# block proposals going back to the last finalized slot.
let stateCacheIndex =
@ -406,10 +406,10 @@ func putStateCache*(
assign(entry[], state)
insert(dag.cachedStates, (blck.root, state.data.slot, entry))
trace "CandidateChains.putState(): state cache updated",
trace "ChainDAGRef.putState(): state cache updated",
cacheLen, root = shortLog(blck.root), slot = state.data.slot
proc putState*(dag: CandidateChains, state: HashedBeaconState, blck: BlockRef) =
proc putState*(dag: ChainDAGRef, state: HashedBeaconState, blck: BlockRef) =
# TODO we save state at every epoch start but never remove them - we also
# potentially save multiple states per slot if reorgs happen, meaning
# we could easily see a state explosion
@ -437,12 +437,12 @@ proc putState*(dag: CandidateChains, state: HashedBeaconState, blck: BlockRef) =
if state.data.slot mod 2 == 0:
putStateCache(dag, state, blck)
func getRef*(dag: CandidateChains, root: Eth2Digest): BlockRef =
func getRef*(dag: ChainDAGRef, root: Eth2Digest): BlockRef =
## Retrieve a resolved block reference, if available
dag.blocks.getOrDefault(root, nil)
func getBlockRange*(
dag: CandidateChains, startSlot: Slot, skipStep: Natural,
dag: ChainDAGRef, startSlot: Slot, skipStep: Natural,
output: var openArray[BlockRef]): Natural =
## This function populates an `output` buffer of blocks
## with a slots ranging from `startSlot` up to, but not including,
@ -479,18 +479,18 @@ func getBlockRange*(
o # Return the index of the first non-nil item in the output
func getBlockBySlot*(dag: CandidateChains, slot: Slot): BlockRef =
func getBlockBySlot*(dag: ChainDAGRef, slot: Slot): BlockRef =
## Retrieves the first block in the current canonical chain
## with slot number less or equal to `slot`.
dag.head.atSlot(slot).blck
func getBlockByPreciseSlot*(dag: CandidateChains, slot: Slot): BlockRef =
func getBlockByPreciseSlot*(dag: ChainDAGRef, slot: Slot): BlockRef =
## Retrieves a block from the canonical chain with a slot
## number equal to `slot`.
let found = dag.getBlockBySlot(slot)
if found.slot != slot: found else: nil
proc get*(dag: CandidateChains, blck: BlockRef): BlockData =
proc get*(dag: ChainDAGRef, blck: BlockRef): BlockData =
## Retrieve the associated block body of a block reference
doAssert (not blck.isNil), "Trying to get nil BlockRef"
@ -499,7 +499,7 @@ proc get*(dag: CandidateChains, blck: BlockRef): BlockData =
BlockData(data: data.get(), refs: blck)
proc get*(dag: CandidateChains, root: Eth2Digest): Option[BlockData] =
proc get*(dag: ChainDAGRef, root: Eth2Digest): Option[BlockData] =
## Retrieve a resolved block reference and its associated body, if available
let refs = dag.getRef(root)
@ -509,7 +509,7 @@ proc get*(dag: CandidateChains, root: Eth2Digest): Option[BlockData] =
none(BlockData)
proc skipAndUpdateState(
dag: CandidateChains,
dag: ChainDAGRef,
state: var HashedBeaconState, blck: BlockRef, slot: Slot, save: bool) =
while state.data.slot < slot:
# Process slots one at a time in case afterUpdate needs to see empty states
@ -520,7 +520,7 @@ proc skipAndUpdateState(
dag.putState(state, blck)
proc skipAndUpdateState(
dag: CandidateChains,
dag: ChainDAGRef,
state: var StateData, blck: BlockData, flags: UpdateFlags, save: bool): bool =
dag.skipAndUpdateState(
@ -542,7 +542,7 @@ proc skipAndUpdateState(
ok
proc rewindState(
dag: CandidateChains, state: var StateData, bs: BlockSlot,
dag: ChainDAGRef, state: var StateData, bs: BlockSlot,
matchEpoch: bool): seq[BlockRef] =
logScope:
blockSlot = shortLog(bs)
@ -630,7 +630,7 @@ proc rewindState(
ancestors
proc getStateDataCached(
dag: CandidateChains, state: var StateData, bs: BlockSlot,
dag: ChainDAGRef, state: var StateData, bs: BlockSlot,
matchEpoch: bool): bool =
# This pointedly does not run rewindState or state_transition, but otherwise
# mostly matches updateStateData(...), because it's too expensive to run the
@ -657,7 +657,7 @@ proc getStateDataCached(
false
template withEpochState*(
dag: CandidateChains, cache: var StateData, blockSlot: BlockSlot,
dag: ChainDAGRef, cache: var StateData, blockSlot: BlockSlot,
body: untyped): untyped =
## Helper template that updates state to a particular BlockSlot - usage of
## cache is unsafe outside of block.
@ -669,7 +669,7 @@ template withEpochState*(
body
proc updateStateData*(
dag: CandidateChains, state: var StateData, bs: BlockSlot,
dag: ChainDAGRef, state: var StateData, bs: BlockSlot,
matchEpoch: bool = false) =
## Rewind or advance state such that it matches the given block and slot -
## this may include replaying from an earlier snapshot if blck is on a
@ -718,19 +718,19 @@ proc updateStateData*(
dag.putStateCache(state.data, bs.blck)
proc loadTailState*(dag: CandidateChains): StateData =
proc loadTailState*(dag: ChainDAGRef): StateData =
## Load the state associated with the current tail in the dag
let stateRoot = dag.db.getBlock(dag.tail.root).get().message.state_root
let found = dag.getState(dag.db, stateRoot, dag.tail, result)
# TODO turn into regular error, this can happen
doAssert found, "Failed to load tail state, database corrupt?"
proc delState(dag: CandidateChains, bs: BlockSlot) =
proc delState(dag: ChainDAGRef, bs: BlockSlot) =
# Delete state state and mapping for a particular block+slot
if (let root = dag.db.getStateRoot(bs.blck.root, bs.slot); root.isSome()):
dag.db.delState(root.get())
proc updateHead*(dag: CandidateChains, newHead: BlockRef) =
proc updateHead*(dag: ChainDAGRef, newHead: BlockRef) =
## Update what we consider to be the current head, as given by the fork
## choice.
## The choice of head affects the choice of finalization point - the order
@ -840,7 +840,7 @@ proc updateHead*(dag: CandidateChains, newHead: BlockRef) =
finalizedHead = shortLog(finalizedHead),
heads = dag.heads.len
proc isInitialized*(T: type CandidateChains, db: BeaconChainDB): bool =
proc isInitialized*(T: type ChainDAGRef, db: BeaconChainDB): bool =
let
headBlockRoot = db.getHeadBlock()
tailBlockRoot = db.getTailBlock()
@ -861,9 +861,9 @@ proc isInitialized*(T: type CandidateChains, db: BeaconChainDB): bool =
true
proc preInit*(
T: type CandidateChains, db: BeaconChainDB, state: BeaconState,
T: type ChainDAGRef, db: BeaconChainDB, state: BeaconState,
signedBlock: SignedBeaconBlock) =
# write a genesis state, the way the CandidateChains expects it to be stored in
# write a genesis state, the way the ChainDAGRef expects it to be stored in
# database
# TODO probably should just init a block pool with the freshly written
# state - but there's more refactoring needed to make it nice - doing
@ -882,7 +882,7 @@ proc preInit*(
db.putStateRoot(signedBlock.root, state.slot, signedBlock.message.state_root)
proc getProposer*(
dag: CandidateChains, head: BlockRef, slot: Slot):
dag: ChainDAGRef, head: BlockRef, slot: Slot):
Option[(ValidatorIndex, ValidatorPubKey)] =
dag.withState(dag.tmpState, head.atSlot(slot)):
var cache = StateCache()

View File

@ -13,7 +13,7 @@ import
metrics, stew/results,
../extras,
../spec/[crypto, datatypes, digest, helpers, signatures, state_transition],
./block_pools_types, ./candidate_chains, ./quarantine
./block_pools_types, ./chain_dag, ./quarantine
export results
@ -27,7 +27,7 @@ export results
logScope:
topics = "clearance"
func getOrResolve*(dag: CandidateChains, quarantine: var Quarantine, root: Eth2Digest): BlockRef =
func getOrResolve*(dag: ChainDAGRef, quarantine: var QuarantineRef, root: Eth2Digest): BlockRef =
## Fetch a block ref, or nil if not found (will be added to list of
## blocks-to-resolve)
result = dag.getRef(root)
@ -36,12 +36,12 @@ func getOrResolve*(dag: CandidateChains, quarantine: var Quarantine, root: Eth2D
quarantine.missing[root] = MissingBlock()
proc addRawBlock*(
dag: var CandidateChains, quarantine: var Quarantine,
dag: var ChainDAGRef, quarantine: var QuarantineRef,
signedBlock: SignedBeaconBlock, onBlockAdded: OnBlockAdded
): Result[BlockRef, BlockError]
proc addResolvedBlock(
dag: var CandidateChains, quarantine: var Quarantine,
dag: var ChainDAGRef, quarantine: var QuarantineRef,
state: HashedBeaconState, signedBlock: SignedBeaconBlock,
parent: BlockRef, cache: StateCache,
onBlockAdded: OnBlockAdded
@ -117,7 +117,7 @@ proc addResolvedBlock(
blockRef
proc addRawBlock*(
dag: var CandidateChains, quarantine: var Quarantine,
dag: var ChainDAGRef, quarantine: var QuarantineRef,
signedBlock: SignedBeaconBlock,
onBlockAdded: OnBlockAdded
): Result[BlockRef, BlockError] =
@ -255,7 +255,7 @@ proc addRawBlock*(
# https://github.com/ethereum/eth2.0-specs/blob/v0.11.1/specs/phase0/p2p-interface.md#global-topics
proc isValidBeaconBlock*(
dag: CandidateChains, quarantine: var Quarantine,
dag: ChainDAGRef, quarantine: var QuarantineRef,
signed_beacon_block: SignedBeaconBlock, current_slot: Slot,
flags: UpdateFlags): Result[void, BlockError] =
logScope:
@ -338,9 +338,9 @@ proc isValidBeaconBlock*(
# the future this block moves from pending to being resolved, consider if
# it's worth broadcasting it then.
# Pending dag gets checked via `CandidateChains.add(...)` later, and relevant
# Pending dag gets checked via `ChainDAGRef.add(...)` later, and relevant
# checks are performed there. In usual paths beacon_node adds blocks via
# CandidateChains.add(...) directly, with no additional validity checks. TODO,
# ChainDAGRef.add(...) directly, with no additional validity checks. TODO,
# not specific to this, but by the pending dag keying on the htr of the
# BeaconBlock, not SignedBeaconBlock, opens up certain spoofing attacks.
debug "parent unknown, putting block in quarantine"

View File

@ -19,7 +19,7 @@ logScope:
{.push raises: [Defect].}
func checkMissing*(quarantine: var Quarantine): seq[FetchRecord] =
func checkMissing*(quarantine: var QuarantineRef): seq[FetchRecord] =
## Return a list of blocks that we should try to resolve from other client -
## to be called periodically but not too often (once per slot?)
var done: seq[Eth2Digest]
@ -40,10 +40,10 @@ func checkMissing*(quarantine: var Quarantine): seq[FetchRecord] =
if countOnes(v.tries.uint64) == 1:
result.add(FetchRecord(root: k))
func addMissing*(quarantine: var Quarantine, broot: Eth2Digest) {.inline.} =
func addMissing*(quarantine: var QuarantineRef, broot: Eth2Digest) {.inline.} =
discard quarantine.missing.hasKeyOrPut(broot, MissingBlock())
func add*(quarantine: var Quarantine, dag: CandidateChains,
func add*(quarantine: var QuarantineRef, dag: ChainDAGRef,
sblck: SignedBeaconBlock) =
## Adds block to quarantine's `orphans` and `missing` lists.
quarantine.orphans[sblck.root] = sblck

View File

@ -16,7 +16,7 @@ import
../spec/[beaconstate, datatypes, digest, helpers],
# Fork choice
./fork_choice_types, ./proto_array,
../block_pools/[spec_cache, candidate_chains]
../block_pools/[spec_cache, chain_dag]
export sets, results, fork_choice_types
@ -160,12 +160,12 @@ func contains*(self: ForkChoiceBackend, block_root: Eth2Digest): bool =
## In particular, before adding a block, its parent must be known to the fork choice
self.proto_array.indices.contains(block_root)
proc get_balances_for_block(self: var Checkpoints, blck: BlockSlot, dag: CandidateChains): seq[Gwei] =
proc get_balances_for_block(self: var Checkpoints, blck: BlockSlot, dag: ChainDAGRef): seq[Gwei] =
dag.withState(dag.balanceState, blck):
get_effective_balances(state)
proc process_state(self: var Checkpoints,
dag: CandidateChains,
dag: ChainDAGRef,
state: BeaconState,
blck: BlockRef) =
trace "Processing state",
@ -248,7 +248,7 @@ proc process_block*(self: var ForkChoiceBackend,
justified_epoch, finalized_epoch)
proc process_block*(self: var ForkChoice,
dag: CandidateChains,
dag: ChainDAGRef,
state: BeaconState,
blckRef: BlockRef,
blck: SomeBeaconBlock,

View File

@ -3,7 +3,7 @@ import
chronicles, chronos, stew/ranges/bitranges, libp2p/switch,
spec/[datatypes, crypto, digest],
beacon_node_types, eth2_network,
block_pools/candidate_chains
block_pools/chain_dag
logScope:
topics = "sync"
@ -36,7 +36,7 @@ type
BeaconBlockCallback* = proc(signedBlock: SignedBeaconBlock) {.gcsafe.}
BeaconSyncNetworkState* = ref object
chainDag*: CandidateChains
chainDag*: ChainDAGRef
forkDigest*: ForkDigest
BeaconSyncPeerState* = ref object
@ -230,7 +230,7 @@ proc handleStatus(peer: Peer,
peer.setStatusMsg(theirStatus)
proc initBeaconSync*(network: Eth2Node, chainDag: CandidateChains,
proc initBeaconSync*(network: Eth2Node, chainDag: ChainDAGRef,
forkDigest: ForkDigest) =
var networkState = network.protocolState(BeaconSync)
networkState.chainDag = chainDag

View File

@ -16,7 +16,7 @@ import
# Local modules
spec/[datatypes, digest, crypto, validator, beaconstate, helpers],
block_pools/candidate_chains, ssz/merkleization,
block_pools/chain_dag, ssz/merkleization,
beacon_node_common, beacon_node_types,
validator_duties, eth2_network,
spec/eth2_apis/callsigs_types,

View File

@ -21,7 +21,7 @@ import
spec/[datatypes, digest, crypto, beaconstate, helpers, validator, network],
spec/state_transition,
conf, time, validator_pool,
attestation_pool, block_pools/[spec_cache, candidate_chains, clearance],
attestation_pool, block_pools/[spec_cache, chain_dag, clearance],
eth2_network, keystore_management, beacon_node_common, beacon_node_types,
nimbus_binary_common, mainchain_monitor, version, ssz/merkleization, interop,
attestation_aggregation, sync_manager, sszdump

View File

@ -2,7 +2,7 @@
confutils, stats, chronicles, strformat, tables,
stew/byteutils,
../beacon_chain/[beacon_chain_db, extras],
../beacon_chain/block_pools/[candidate_chains],
../beacon_chain/block_pools/[chain_dag],
../beacon_chain/spec/[crypto, datatypes, digest, helpers,
state_transition, presets],
../beacon_chain/sszdump, ../research/simutils,
@ -67,13 +67,13 @@ proc cmdBench(conf: DbConf) =
db = BeaconChainDB.init(
kvStore SqStoreRef.init(conf.databaseDir.string, "nbc").tryGet())
if not CandidateChains.isInitialized(db):
if not ChainDAGRef.isInitialized(db):
echo "Database not initialized"
quit 1
echo "Initializing block pool..."
let pool = withTimerRet(timers[tInit]):
CandidateChains.init(defaultRuntimePreset, db, {})
ChainDAGRef.init(defaultRuntimePreset, db, {})
echo &"Loaded {pool.blocks.len} blocks, head slot {pool.head.slot}"
@ -149,12 +149,12 @@ proc cmdRewindState(conf: DbConf) =
db = BeaconChainDB.init(
kvStore SqStoreRef.init(conf.databaseDir.string, "nbc").tryGet())
if not CandidateChains.isInitialized(db):
if not ChainDAGRef.isInitialized(db):
echo "Database not initialized"
quit 1
echo "Initializing block pool..."
let dag = init(CandidateChains, defaultRuntimePreset, db)
let dag = init(ChainDAGRef, defaultRuntimePreset, db)
let blckRef = dag.getRef(fromHex(Eth2Digest, conf.blockRoot))
if blckRef == nil:

View File

@ -25,7 +25,7 @@ import
attestation_pool, beacon_node_types, beacon_chain_db,
interop, validator_pool],
../beacon_chain/block_pools/[
spec_cache, candidate_chains, quarantine, clearance],
spec_cache, chain_dag, quarantine, clearance],
eth/db/[kvstore, kvstore_sqlite3],
../beacon_chain/ssz/[merkleization, ssz_serialization],
./simutils
@ -53,11 +53,11 @@ cli do(slots = SLOTS_PER_EPOCH * 6,
let
db = BeaconChainDB.init(kvStore SqStoreRef.init(".", "block_sim").tryGet())
CandidateChains.preInit(db, state[].data, genesisBlock)
ChainDAGRef.preInit(db, state[].data, genesisBlock)
var
chainDag = init(CandidateChains, defaultRuntimePreset, db)
quarantine = Quarantine()
chainDag = init(ChainDAGRef, defaultRuntimePreset, db)
quarantine = QuarantineRef()
attPool = AttestationPool.init(chainDag, quarantine)
timers: array[Timers, RunningStat]
attesters: RunningStat

View File

@ -16,7 +16,7 @@ import
helpers, beaconstate, presets],
../beacon_chain/[beacon_node_types, attestation_pool, extras],
../beacon_chain/fork_choice/[fork_choice_types, fork_choice],
../beacon_chain/block_pools/[candidate_chains, clearance]
../beacon_chain/block_pools/[chain_dag, clearance]
template wrappedTimedTest(name: string, body: untyped) =
# `check` macro takes a copy of whatever it's checking, on the stack!
@ -33,9 +33,9 @@ suiteReport "Attestation pool processing" & preset():
setup:
# Genesis state that results in 3 members per committee
var
chainDag = newClone(init(CandidateChains, defaultRuntimePreset, makeTestDB(SLOTS_PER_EPOCH * 3)))
quarantine = newClone(Quarantine())
pool = newClone(AttestationPool.init(chainDag, quarantine[]))
chainDag = newClone(init(ChainDAGRef, defaultRuntimePreset, makeTestDB(SLOTS_PER_EPOCH * 3)))
quarantine = newClone(QuarantineRef())
pool = newClone(AttestationPool.init(chainDag, quarantine))
state = newClone(loadTailState(chainDag))
# Slot 0 is a finalized slot - won't be making attestations for it..
check:
@ -164,7 +164,7 @@ suiteReport "Attestation pool processing" & preset():
var cache = StateCache()
let
b1 = addTestBlock(state.data, chainDag.tail.root, cache)
b1Add = chainDag.addRawBlock(quarantine[], b1) do (
b1Add = chainDag.addRawBlock(quarantine, b1) do (
blckRef: BlockRef, signedBlock: SignedBeaconBlock,
state: HashedBeaconState):
# Callback add to fork choice if valid
@ -177,7 +177,7 @@ suiteReport "Attestation pool processing" & preset():
let
b2 = addTestBlock(state.data, b1.root, cache)
b2Add = chainDag.addRawBlock(quarantine[], b2) do (
b2Add = chainDag.addRawBlock(quarantine, b2) do (
blckRef: BlockRef, signedBlock: SignedBeaconBlock,
state: HashedBeaconState):
# Callback add to fork choice if valid
@ -192,7 +192,7 @@ suiteReport "Attestation pool processing" & preset():
var cache = StateCache()
let
b10 = makeTestBlock(state.data, chainDag.tail.root, cache)
b10Add = chainDag.addRawBlock(quarantine[], b10) do (
b10Add = chainDag.addRawBlock(quarantine, b10) do (
blckRef: BlockRef, signedBlock: SignedBeaconBlock,
state: HashedBeaconState):
# Callback add to fork choice if valid
@ -207,7 +207,7 @@ suiteReport "Attestation pool processing" & preset():
b11 = makeTestBlock(state.data, chainDag.tail.root, cache,
graffiti = GraffitiBytes [1'u8, 0, 0, 0 ,0 ,0 ,0 ,0 ,0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0]
)
b11Add = chainDag.addRawBlock(quarantine[], b11) do (
b11Add = chainDag.addRawBlock(quarantine, b11) do (
blckRef: BlockRef, signedBlock: SignedBeaconBlock,
state: HashedBeaconState):
# Callback add to fork choice if valid
@ -249,7 +249,7 @@ suiteReport "Attestation pool processing" & preset():
var cache = StateCache()
let
b10 = makeTestBlock(state.data, chainDag.tail.root, cache)
b10Add = chainDag.addRawBlock(quarantine[], b10) do (
b10Add = chainDag.addRawBlock(quarantine, b10) do (
blckRef: BlockRef, signedBlock: SignedBeaconBlock,
state: HashedBeaconState):
# Callback add to fork choice if valid
@ -263,7 +263,7 @@ suiteReport "Attestation pool processing" & preset():
# -------------------------------------------------------------
# Add back the old block to ensure we have a duplicate error
let b10_clone = b10 # Assumes deep copy
let b10Add_clone = chainDag.addRawBlock(quarantine[], b10_clone) do (
let b10Add_clone = chainDag.addRawBlock(quarantine, b10_clone) do (
blckRef: BlockRef, signedBlock: SignedBeaconBlock,
state: HashedBeaconState):
# Callback add to fork choice if valid
@ -277,7 +277,7 @@ suiteReport "Attestation pool processing" & preset():
var cache = StateCache()
let
b10 = makeTestBlock(state.data, chainDag.tail.root, cache)
b10Add = chainDag.addRawBlock(quarantine[], b10) do (
b10Add = chainDag.addRawBlock(quarantine, b10) do (
blckRef: BlockRef, signedBlock: SignedBeaconBlock,
state: HashedBeaconState):
# Callback add to fork choice if valid
@ -311,7 +311,7 @@ suiteReport "Attestation pool processing" & preset():
doAssert: block_ok
block_root = new_block.root
let blockRef = chainDag.addRawBlock(quarantine[], new_block) do (
let blockRef = chainDag.addRawBlock(quarantine, new_block) do (
blckRef: BlockRef, signedBlock: SignedBeaconBlock,
state: HashedBeaconState):
# Callback add to fork choice if valid
@ -352,7 +352,7 @@ suiteReport "Attestation pool processing" & preset():
doAssert: b10.root notin pool.forkChoice.backend
# Add back the old block to ensure we have a duplicate error
let b10Add_clone = chainDag.addRawBlock(quarantine[], b10_clone) do (
let b10Add_clone = chainDag.addRawBlock(quarantine, b10_clone) do (
blckRef: BlockRef, signedBlock: SignedBeaconBlock,
state: HashedBeaconState):
# Callback add to fork choice if valid

View File

@ -12,7 +12,7 @@ import
./testutil, ./testblockutil,
../beacon_chain/spec/[datatypes, digest, state_transition, presets],
../beacon_chain/[beacon_node_types, ssz],
../beacon_chain/block_pools/[candidate_chains, quarantine, clearance]
../beacon_chain/block_pools/[chain_dag, quarantine, clearance]
when isMainModule:
import chronicles # or some random compile error happens...
@ -90,8 +90,8 @@ suiteReport "Block pool processing" & preset():
setup:
var
db = makeTestDB(SLOTS_PER_EPOCH)
dag = init(CandidateChains, defaultRuntimePreset, db)
quarantine: Quarantine
dag = init(ChainDAGRef, defaultRuntimePreset, db)
quarantine = QuarantineRef()
stateData = newClone(dag.loadTailState())
cache = StateCache()
b1 = addTestBlock(stateData.data, dag.tail.root, cache)
@ -205,7 +205,7 @@ suiteReport "Block pool processing" & preset():
# check that init also reloads block graph
var
dag2 = init(CandidateChains, defaultRuntimePreset, db)
dag2 = init(ChainDAGRef, defaultRuntimePreset, db)
check:
# ensure we loaded the correct head state
@ -287,8 +287,8 @@ suiteReport "chain DAG finalization tests" & preset():
setup:
var
db = makeTestDB(SLOTS_PER_EPOCH)
dag = init(CandidateChains, defaultRuntimePreset, db)
quarantine: Quarantine
dag = init(ChainDAGRef, defaultRuntimePreset, db)
quarantine = QuarantineRef()
cache = StateCache()
timedTest "prune heads on finalization" & preset():
@ -330,7 +330,7 @@ suiteReport "chain DAG finalization tests" & preset():
check: status.error == Unviable
let
dag2 = init(CandidateChains, defaultRuntimePreset, db)
dag2 = init(ChainDAGRef, defaultRuntimePreset, db)
# check that the state reloaded from database resembles what we had before
check:

View File

@ -10,7 +10,7 @@ import
testutils/markdown_reports, chronicles,
../beacon_chain/[beacon_chain_db, extras, ssz],
../beacon_chain/spec/[digest, beaconstate, datatypes, presets],
../beacon_chain/block_pools/candidate_chains,
../beacon_chain/block_pools/chain_dag,
eth/db/kvstore,
testblockutil
@ -99,7 +99,7 @@ template timedTest*(name, body) =
proc makeTestDB*(tailState: BeaconState, tailBlock: SignedBeaconBlock): BeaconChainDB =
result = init(BeaconChainDB, kvStore MemStoreRef.init())
CandidateChains.preInit(result, tailState, tailBlock)
ChainDAGRef.preInit(result, tailState, tailBlock)
proc makeTestDB*(validators: Natural): BeaconChainDB =
let