forkedbeaconstate_helpers -> forks (#2772)
Simpler module name for stuff that covers forks * check that runtime config matches database state * also include some assorted altair cleanups * use "standard" genesis fork in local testnet to work around missing runtime config support
This commit is contained in:
parent
92aba71604
commit
9697b73e71
|
@ -15,7 +15,7 @@ import
|
|||
chronicles, stew/byteutils, json_serialization/std/sets as jsonSets,
|
||||
# Internal
|
||||
../spec/[
|
||||
beaconstate, crypto, digest, forkedbeaconstate_helpers,
|
||||
beaconstate, crypto, digest, forks,
|
||||
validator],
|
||||
../spec/datatypes/[phase0, altair],
|
||||
../ssz/[merkleization, types],
|
||||
|
@ -298,7 +298,7 @@ proc addAttestation*(pool: var AttestationPool,
|
|||
return
|
||||
do:
|
||||
if not addAttestation(
|
||||
pool.candidates[candidateIdx.get()].mGetOrPut(
|
||||
pool.candidates[candidateIdx.get()].mgetOrPut(
|
||||
attestation_data_root,
|
||||
AttestationEntry(
|
||||
data: attestation.data,
|
||||
|
|
|
@ -14,7 +14,7 @@ import
|
|||
eth/keys,
|
||||
../extras, ../beacon_clock,
|
||||
../spec/[
|
||||
crypto, digest, forkedbeaconstate_helpers, helpers, signatures,
|
||||
crypto, digest, forks, helpers, signatures,
|
||||
signatures_batch, state_transition],
|
||||
../spec/datatypes/[phase0, altair],
|
||||
./block_pools_types, ./blockchain_dag, ./block_quarantine
|
||||
|
|
|
@ -14,7 +14,7 @@ import
|
|||
stew/endians2, chronicles,
|
||||
eth/keys,
|
||||
# Internals
|
||||
../spec/[crypto, digest, signatures_batch, forkedbeaconstate_helpers],
|
||||
../spec/[crypto, digest, signatures_batch, forks],
|
||||
../spec/datatypes/[phase0, altair],
|
||||
".."/[beacon_chain_db, beacon_clock, extras]
|
||||
|
||||
|
@ -192,7 +192,6 @@ type
|
|||
eth1_deposit_index*: uint64
|
||||
beacon_proposers*: array[SLOTS_PER_EPOCH, Option[ValidatorIndex]]
|
||||
shuffled_active_validator_indices*: seq[ValidatorIndex]
|
||||
|
||||
# balances, as used in fork choice
|
||||
effective_balances_bytes*: seq[byte]
|
||||
|
||||
|
|
|
@ -14,7 +14,7 @@ import
|
|||
../ssz/[ssz_serialization, merkleization], ../beacon_chain_db, ../extras,
|
||||
../spec/[
|
||||
crypto, digest, helpers, validator, state_transition,
|
||||
beaconstate, forkedbeaconstate_helpers],
|
||||
beaconstate, forks],
|
||||
../spec/datatypes/[phase0, altair],
|
||||
../beacon_clock,
|
||||
"."/[block_pools_types, block_quarantine, forkedbeaconstate_dbhelpers]
|
||||
|
@ -404,6 +404,20 @@ proc init*(T: type ChainDAGRef,
|
|||
# would be a good recovery model?
|
||||
raiseAssert "No state found in head history, database corrupt?"
|
||||
|
||||
case tmpState.data.beaconStateFork
|
||||
of forkPhase0:
|
||||
if tmpState.data.hbsPhase0.data.fork != genesisFork(cfg):
|
||||
error "State from database does not match network, check --network parameter",
|
||||
stateFork = tmpState.data.hbsPhase0.data.fork,
|
||||
configFork = genesisFork(cfg)
|
||||
quit 1
|
||||
of forkAltair:
|
||||
if tmpState.data.hbsAltair.data.fork != altairFork(cfg):
|
||||
error "State from database does not match network, check --network parameter",
|
||||
stateFork = tmpState.data.hbsAltair.data.fork,
|
||||
configFork = altairFork(cfg)
|
||||
quit 1
|
||||
|
||||
let dag = ChainDAGRef(
|
||||
blocks: blocks,
|
||||
tail: tailRef,
|
||||
|
|
|
@ -13,7 +13,7 @@ import
|
|||
# Status libraries
|
||||
chronicles,
|
||||
# Internal
|
||||
../spec/[crypto, forkedbeaconstate_helpers, helpers],
|
||||
../spec/[crypto, forks, helpers],
|
||||
../spec/datatypes/base,
|
||||
"."/[blockchain_dag, block_quarantine],
|
||||
../beacon_node_types
|
||||
|
|
|
@ -8,7 +8,7 @@
|
|||
{.push raises: [Defect].}
|
||||
|
||||
import
|
||||
../spec/forkedbeaconstate_helpers,
|
||||
../spec/forks,
|
||||
../beacon_chain_db
|
||||
|
||||
proc putState*(db: BeaconChainDB, state: ForkedHashedBeaconState) =
|
||||
|
|
|
@ -15,7 +15,7 @@ import
|
|||
web3, web3/ethtypes as web3Types, web3/ethhexstrings, eth/common/eth_types,
|
||||
eth/async_utils, stew/byteutils,
|
||||
# Local modules:
|
||||
../spec/[digest, crypto, forkedbeaconstate_helpers, helpers],
|
||||
../spec/[digest, crypto, forks, helpers],
|
||||
../spec/datatypes/base,
|
||||
../networking/network_metadata,
|
||||
../consensus_object_pools/block_pools_types,
|
||||
|
|
|
@ -12,7 +12,7 @@ import
|
|||
stew/results,
|
||||
chronicles, chronos, metrics,
|
||||
../spec/datatypes/[phase0, altair],
|
||||
../spec/[crypto, digest, forkedbeaconstate_helpers],
|
||||
../spec/[crypto, digest, forks],
|
||||
../consensus_object_pools/[block_clearance, blockchain_dag, attestation_pool],
|
||||
./consensus_manager,
|
||||
".."/[beacon_clock, beacon_node_types],
|
||||
|
|
|
@ -11,7 +11,7 @@ import
|
|||
std/tables,
|
||||
stew/results,
|
||||
chronicles, chronos, metrics,
|
||||
../spec/[crypto, digest, forkedbeaconstate_helpers],
|
||||
../spec/[crypto, digest, forks],
|
||||
../spec/datatypes/[altair, phase0],
|
||||
../consensus_object_pools/[block_clearance, blockchain_dag, exit_pool, attestation_pool],
|
||||
./gossip_validation, ./block_processor,
|
||||
|
|
|
@ -17,7 +17,7 @@ import
|
|||
../spec/datatypes/[phase0, altair],
|
||||
../spec/[
|
||||
beaconstate, state_transition_block,
|
||||
crypto, digest, forkedbeaconstate_helpers, helpers, network,
|
||||
crypto, digest, forks, helpers, network,
|
||||
signatures],
|
||||
../consensus_object_pools/[
|
||||
spec_cache, blockchain_dag, block_quarantine, spec_cache,
|
||||
|
@ -269,7 +269,7 @@ proc validateAttestation*(
|
|||
"validateAttestation: number of aggregation bits and committee size mismatch"))
|
||||
|
||||
let
|
||||
fork = getStateField(pool.dag.headState.data, fork)
|
||||
fork = pool.dag.forkAtEpoch(attestation.data.slot.epoch)
|
||||
genesis_validators_root =
|
||||
getStateField(pool.dag.headState.data, genesis_validators_root)
|
||||
attesting_index = get_attesting_indices_one(
|
||||
|
@ -448,7 +448,7 @@ proc validateAggregate*(
|
|||
# 3. [REJECT] The signature of aggregate is valid.
|
||||
|
||||
let
|
||||
fork = getStateField(pool.dag.headState.data, fork)
|
||||
fork = pool.dag.forkAtEpoch(aggregate.data.slot.epoch)
|
||||
genesis_validators_root =
|
||||
getStateField(pool.dag.headState.data, genesis_validators_root)
|
||||
|
||||
|
@ -650,7 +650,7 @@ proc isValidBeaconBlock*(
|
|||
# [REJECT] The proposer signature, signed_beacon_block.signature, is valid
|
||||
# with respect to the proposer_index pubkey.
|
||||
if not verify_block_signature(
|
||||
getStateField(dag.headState.data, fork),
|
||||
dag.forkAtEpoch(signed_beacon_block.message.slot.epoch),
|
||||
getStateField(dag.headState.data, genesis_validators_root),
|
||||
signed_beacon_block.message.slot,
|
||||
signed_beacon_block.message,
|
||||
|
|
|
@ -35,14 +35,11 @@ import
|
|||
".."/[
|
||||
version, conf,
|
||||
ssz/ssz_serialization, beacon_clock],
|
||||
../spec/datatypes/base,
|
||||
../spec/[digest, network, helpers, forkedbeaconstate_helpers],
|
||||
../spec/datatypes/[phase0, altair],
|
||||
../spec/[digest, network, helpers, forks],
|
||||
../validators/keystore_management,
|
||||
./eth2_discovery, ./peer_pool, ./libp2p_json_serialization
|
||||
|
||||
from ../spec/datatypes/phase0 import nil
|
||||
from ../spec/datatypes/altair import nil
|
||||
|
||||
when chronicles.enabledLogLevel == LogLevel.TRACE:
|
||||
import std/sequtils
|
||||
|
||||
|
@ -343,7 +340,7 @@ proc getPeer*(node: Eth2Node, peerId: PeerID): Peer =
|
|||
return peer[]
|
||||
do:
|
||||
let peer = Peer.init(node, PeerInfo.init(peerId))
|
||||
return node.peers.mGetOrPut(peerId, peer)
|
||||
return node.peers.mgetOrPut(peerId, peer)
|
||||
|
||||
proc peerFromStream(network: Eth2Node, conn: Connection): Peer =
|
||||
result = network.getPeer(conn.peerInfo.peerId)
|
||||
|
@ -984,15 +981,16 @@ proc getPersistentNetMetadata*(config: BeaconNodeConf): altair.MetaData
|
|||
{.raises: [Defect, IOError, SerializationError].} =
|
||||
let metadataPath = config.dataDir / nodeMetadataFilename
|
||||
if not fileExists(metadataPath):
|
||||
result = altair.MetaData()
|
||||
var res: altair.MetaData
|
||||
for i in 0 ..< ATTESTATION_SUBNET_COUNT:
|
||||
# TODO:
|
||||
# Persistent (stability) subnets should be stored with their expiration
|
||||
# epochs. For now, indicate that we participate in no persistent subnets.
|
||||
result.attnets[i] = false
|
||||
Json.saveFile(metadataPath, result)
|
||||
res.attnets[i] = false
|
||||
Json.saveFile(metadataPath, res)
|
||||
res
|
||||
else:
|
||||
result = Json.loadFile(metadataPath, altair.MetaData)
|
||||
Json.loadFile(metadataPath, altair.MetaData)
|
||||
|
||||
proc resolvePeer(peer: Peer) =
|
||||
# Resolve task which performs searching of peer's public key and recovery of
|
||||
|
@ -1589,7 +1587,7 @@ proc createEth2Node*(rng: ref BrHmacDrbgContext,
|
|||
wallEpoch: Epoch,
|
||||
genesisValidatorsRoot: Eth2Digest): Eth2Node
|
||||
{.raises: [Defect, CatchableError].} =
|
||||
var
|
||||
let
|
||||
enrForkId = getENRForkID(cfg, wallEpoch, genesisValidatorsRoot)
|
||||
|
||||
(extIp, extTcpPort, extUdpPort) = try: setupAddress(
|
||||
|
@ -1657,7 +1655,7 @@ proc createEth2Node*(rng: ref BrHmacDrbgContext,
|
|||
maddress = MultiAddress.init(s).tryGet()
|
||||
mpeerId = maddress[multiCodec("p2p")].tryGet()
|
||||
peerId = PeerID.init(mpeerId.protoAddress().tryGet()).tryGet()
|
||||
res.mGetOrPut(peerId, @[]).add(maddress)
|
||||
res.mgetOrPut(peerId, @[]).add(maddress)
|
||||
info "Adding priviledged direct peer", peerId, address = maddress
|
||||
res
|
||||
)
|
||||
|
|
|
@ -37,7 +37,7 @@ import
|
|||
./rpc/[rest_api, rpc_api],
|
||||
./spec/datatypes/[altair, phase0],
|
||||
./spec/[
|
||||
digest, crypto, forkedbeaconstate_helpers, beaconstate,
|
||||
digest, crypto, forks, beaconstate,
|
||||
eth2_apis/rpc_beacon_client, helpers, network, presets,
|
||||
weak_subjectivity, signatures],
|
||||
./consensus_object_pools/[
|
||||
|
|
|
@ -12,7 +12,7 @@ import
|
|||
../consensus_object_pools/[blockchain_dag, exit_pool],
|
||||
../gossip_processing/gossip_validation,
|
||||
../validators/validator_duties,
|
||||
../spec/[crypto, digest, forkedbeaconstate_helpers, network],
|
||||
../spec/[crypto, digest, forks, network],
|
||||
../spec/datatypes/[phase0, altair],
|
||||
../ssz/merkleization,
|
||||
./rest_utils
|
||||
|
|
|
@ -11,7 +11,7 @@ import
|
|||
nimcrypto/utils as ncrutils,
|
||||
../beacon_node_common, ../eth1/eth1_monitor,
|
||||
../spec/datatypes/base,
|
||||
../spec/[digest, forkedbeaconstate_helpers, presets],
|
||||
../spec/[digest, forks, presets],
|
||||
./rest_utils
|
||||
|
||||
logScope: topics = "rest_config"
|
||||
|
|
|
@ -14,7 +14,7 @@ import
|
|||
./rest_utils,
|
||||
../eth1/eth1_monitor,
|
||||
../validators/validator_duties,
|
||||
../spec/forkedbeaconstate_helpers,
|
||||
../spec/forks,
|
||||
../beacon_node_common, ../nimbus_binary_common
|
||||
|
||||
logScope: topics = "rest_nimbusapi"
|
||||
|
|
|
@ -1,6 +1,6 @@
|
|||
import presto, presto/client as presto_client,
|
||||
nimcrypto/utils as ncrutils,
|
||||
../spec/[crypto, datatypes, digest, forkedbeaconstate_helpers],
|
||||
../spec/[crypto, datatypes, digest, forks],
|
||||
../spec/eth2_apis/[rest_types, eth2_rest_serialization],
|
||||
../beacon_node_common,
|
||||
../consensus_object_pools/[block_pools_types, blockchain_dag]
|
||||
|
|
|
@ -12,7 +12,7 @@ import
|
|||
../consensus_object_pools/[blockchain_dag, spec_cache, attestation_pool],
|
||||
../gossip_processing/gossip_validation,
|
||||
../validators/validator_duties,
|
||||
../spec/[crypto, digest, forkedbeaconstate_helpers, network],
|
||||
../spec/[crypto, digest, forks, network],
|
||||
../spec/datatypes/[base, phase0],
|
||||
../ssz/merkleization,
|
||||
./rest_utils
|
||||
|
|
|
@ -18,7 +18,7 @@ import
|
|||
../validators/validator_duties,
|
||||
../gossip_processing/gossip_validation,
|
||||
../consensus_object_pools/blockchain_dag,
|
||||
../spec/[crypto, datatypes/phase0, digest, forkedbeaconstate_helpers, network],
|
||||
../spec/[crypto, datatypes/phase0, digest, forks, network],
|
||||
../ssz/merkleization,
|
||||
./rpc_utils
|
||||
|
||||
|
|
|
@ -14,7 +14,7 @@ import
|
|||
nimcrypto/utils as ncrutils,
|
||||
../beacon_node_common,
|
||||
../eth1/eth1_monitor,
|
||||
../spec/[datatypes, digest, forkedbeaconstate_helpers, presets]
|
||||
../spec/[datatypes, digest, forks, presets]
|
||||
|
||||
logScope: topics = "configapi"
|
||||
|
||||
|
|
|
@ -18,7 +18,7 @@ import
|
|||
beacon_node_common, nimbus_binary_common, networking/eth2_network,
|
||||
eth1/eth1_monitor, validators/validator_duties],
|
||||
../spec/datatypes/base,
|
||||
../spec/[digest, forkedbeaconstate_helpers, presets],
|
||||
../spec/[digest, forks, presets],
|
||||
./rpc_utils
|
||||
|
||||
|
||||
|
|
|
@ -13,7 +13,7 @@ import
|
|||
../beacon_node_common, ../validators/validator_duties,
|
||||
../consensus_object_pools/[block_pools_types, blockchain_dag],
|
||||
../spec/datatypes/base,
|
||||
../spec/[digest, forkedbeaconstate_helpers, helpers],
|
||||
../spec/[digest, forks, helpers],
|
||||
../spec/eth2_apis/[rpc_types, eth2_json_rpc_serialization]
|
||||
|
||||
export rpc_types, eth2_json_rpc_serialization, blockchain_dag
|
||||
|
|
|
@ -17,7 +17,7 @@ import
|
|||
chronicles,
|
||||
|
||||
# Local modules
|
||||
../spec/[crypto, digest, forkedbeaconstate_helpers, helpers, network, signatures],
|
||||
../spec/[crypto, digest, forks, helpers, network, signatures],
|
||||
../spec/datatypes/phase0,
|
||||
../spec/eth2_apis/rpc_types,
|
||||
../consensus_object_pools/[blockchain_dag, spec_cache, attestation_pool], ../ssz/merkleization,
|
||||
|
|
|
@ -239,10 +239,7 @@ proc initialize_beacon_state_from_eth1*(
|
|||
doAssert deposits.lenu64 >= SLOTS_PER_EPOCH
|
||||
|
||||
var state = phase0.BeaconStateRef(
|
||||
fork: Fork(
|
||||
previous_version: cfg.GENESIS_FORK_VERSION,
|
||||
current_version: cfg.GENESIS_FORK_VERSION,
|
||||
epoch: GENESIS_EPOCH),
|
||||
fork: genesisFork(cfg),
|
||||
genesis_time: genesis_time_from_eth1_timestamp(cfg, eth1_timestamp),
|
||||
eth1_data:
|
||||
Eth1Data(block_hash: eth1_block_hash, deposit_count: uint64(len(deposits))),
|
||||
|
@ -795,12 +792,7 @@ func translate_participation(
|
|||
state.previous_epoch_participation[index] =
|
||||
add_flag(state.previous_epoch_participation[index], flag_index)
|
||||
|
||||
proc upgrade_to_altair*(pre: phase0.BeaconState): ref altair.BeaconState =
|
||||
let epoch = get_current_epoch(pre)
|
||||
|
||||
# https://github.com/ethereum/eth2.0-specs/blob/v1.1.0-alpha.8/specs/altair/fork.md#configuration
|
||||
const ALTAIR_FORK_VERSION = Version [byte 1, 0, 0, 0]
|
||||
|
||||
proc upgrade_to_altair*(cfg: RuntimeConfig, pre: phase0.BeaconState): ref altair.BeaconState =
|
||||
var
|
||||
empty_participation =
|
||||
HashList[ParticipationFlags, Limit VALIDATOR_REGISTRY_LIMIT]()
|
||||
|
@ -816,11 +808,7 @@ proc upgrade_to_altair*(pre: phase0.BeaconState): ref altair.BeaconState =
|
|||
genesis_time: pre.genesis_time,
|
||||
genesis_validators_root: pre.genesis_validators_root,
|
||||
slot: pre.slot,
|
||||
fork: Fork(
|
||||
previous_version: pre.fork.current_version,
|
||||
current_version: ALTAIR_FORK_VERSION,
|
||||
epoch: epoch
|
||||
),
|
||||
fork: altairFork(cfg),
|
||||
|
||||
# History
|
||||
latest_block_header: pre.latest_block_header,
|
||||
|
|
|
@ -8,7 +8,7 @@
|
|||
{.push raises: [Defect].}
|
||||
|
||||
import
|
||||
"."/[digest, helpers, forkedbeaconstate_helpers],
|
||||
"."/[digest, helpers, forks],
|
||||
"."/datatypes/base
|
||||
|
||||
const
|
||||
|
|
|
@ -15,7 +15,7 @@ import
|
|||
../ssz/merkleization,
|
||||
"."/[
|
||||
crypto, helpers, presets, beaconstate, digest,
|
||||
forkedbeaconstate_helpers],
|
||||
forks],
|
||||
"."/datatypes/[altair, phase0]
|
||||
|
||||
# Otherwise, error.
|
||||
|
|
|
@ -46,7 +46,7 @@ import
|
|||
stew/results,
|
||||
../extras, ../ssz/merkleization, metrics,
|
||||
./datatypes/[phase0, altair], ./crypto, ./digest, ./helpers, ./signatures, ./validator, ./beaconstate,
|
||||
./state_transition_block, ./state_transition_epoch, forkedbeaconstate_helpers,
|
||||
./state_transition_block, ./state_transition_epoch, forks,
|
||||
../../nbench/bench_lab
|
||||
|
||||
# TODO why need anything except the first two?
|
||||
|
@ -177,7 +177,7 @@ proc maybeUpgradeStateToAltair*(
|
|||
# once by checking for existing fork.
|
||||
if getStateField(state, slot).epoch == cfg.ALTAIR_FORK_EPOCH and
|
||||
state.beaconStateFork == forkPhase0:
|
||||
var newState = upgrade_to_altair(state.hbsPhase0.data)
|
||||
var newState = upgrade_to_altair(cfg, state.hbsPhase0.data)
|
||||
state = (ref ForkedHashedBeaconState)(
|
||||
beaconStateFork: forkAltair,
|
||||
hbsAltair: altair.HashedBeaconState(
|
||||
|
|
|
@ -8,7 +8,7 @@
|
|||
{.push raises: [Defect].}
|
||||
|
||||
import
|
||||
./datatypes/base, ./digest, ./forkedbeaconstate_helpers, ./helpers
|
||||
./datatypes/base, ./digest, ./forks, ./helpers
|
||||
|
||||
const
|
||||
SAFETY_DECAY* = 10'u64
|
||||
|
|
|
@ -265,6 +265,9 @@ func isZeros*(x: BitSeq): bool =
|
|||
if w != 0: return false
|
||||
return true
|
||||
|
||||
func isZeros*(x: BitArray): bool =
|
||||
x == default(type(x))
|
||||
|
||||
func countOnes*(x: BitSeq): int =
|
||||
# Count the number of set bits
|
||||
var res = 0
|
||||
|
|
|
@ -10,11 +10,10 @@
|
|||
import
|
||||
os, strformat, chronicles,
|
||||
./ssz_serialization,
|
||||
../spec/[crypto, datatypes/base, datatypes/altair, datatypes/phase0, digest],
|
||||
../spec/datatypes/[phase0, altair],
|
||||
../spec/[crypto, digest],
|
||||
../consensus_object_pools/block_pools_types
|
||||
|
||||
from ../spec/datatypes/altair import SyncCommitteeMessage
|
||||
|
||||
# Dump errors are generally not fatal where used currently - the code calling
|
||||
# these functions, like most code, is not exception safe
|
||||
template logErrors(body: untyped) =
|
||||
|
|
|
@ -9,7 +9,7 @@
|
|||
|
||||
import options, sequtils, strutils
|
||||
import chronos, chronicles
|
||||
import ../spec/[datatypes/phase0, datatypes/altair, digest, forkedbeaconstate_helpers],
|
||||
import ../spec/[datatypes/phase0, datatypes/altair, digest, forks],
|
||||
../networking/eth2_network,
|
||||
../beacon_node_types,
|
||||
../ssz/merkleization,
|
||||
|
|
|
@ -10,7 +10,7 @@
|
|||
import chronicles
|
||||
import options, deques, heapqueue, tables, strutils, sequtils, math, algorithm
|
||||
import stew/results, chronos, chronicles
|
||||
import ../spec/[datatypes/phase0, datatypes/altair, digest, helpers, eth2_apis/rpc_types, forkedbeaconstate_helpers],
|
||||
import ../spec/[datatypes/phase0, datatypes/altair, digest, helpers, eth2_apis/rpc_types, forks],
|
||||
../networking/[peer_pool, eth2_network]
|
||||
|
||||
import ../gossip_processing/block_processor
|
||||
|
|
|
@ -10,13 +10,12 @@
|
|||
import
|
||||
options, tables, sets, macros,
|
||||
chronicles, chronos, stew/ranges/bitranges, libp2p/switch,
|
||||
../spec/[crypto, datatypes, digest, forkedbeaconstate_helpers, network],
|
||||
../spec/datatypes/[phase0, altair],
|
||||
../spec/[crypto, datatypes, digest, forks, network],
|
||||
../beacon_node_types, ../beacon_clock,
|
||||
../networking/eth2_network,
|
||||
../consensus_object_pools/blockchain_dag
|
||||
|
||||
from ../spec/datatypes/altair import nil
|
||||
|
||||
logScope:
|
||||
topics = "sync"
|
||||
|
||||
|
@ -46,8 +45,6 @@ type
|
|||
else:
|
||||
index: uint32
|
||||
|
||||
BeaconBlockCallback* = proc(signedBlock: SignedBeaconBlock) {.gcsafe, raises: [Defect].}
|
||||
|
||||
BeaconSyncNetworkState* = ref object
|
||||
dag*: ChainDAGRef
|
||||
getTime*: GetTimeFn
|
||||
|
@ -62,8 +59,6 @@ type
|
|||
|
||||
BlockRootsList* = List[Eth2Digest, Limit MAX_REQUEST_BLOCKS]
|
||||
|
||||
AltairSignedBeaconBlock* = altair.SignedBeaconBlock
|
||||
|
||||
proc readChunkPayload*(conn: Connection, peer: Peer,
|
||||
MsgType: type ForkedSignedBeaconBlock): Future[NetRes[ForkedSignedBeaconBlock]] {.async.} =
|
||||
var contextBytes: ForkDigest
|
||||
|
@ -73,7 +68,7 @@ proc readChunkPayload*(conn: Connection, peer: Peer,
|
|||
return neterr UnexpectedEOF
|
||||
|
||||
if contextBytes == peer.network.forkDigests.phase0:
|
||||
let res = await readChunkPayload(conn, peer, SignedBeaconBlock)
|
||||
let res = await readChunkPayload(conn, peer, phase0.SignedBeaconBlock)
|
||||
if res.isOk:
|
||||
return ok ForkedSignedBeaconBlock(
|
||||
kind: BeaconBlockFork.Phase0,
|
||||
|
@ -81,7 +76,7 @@ proc readChunkPayload*(conn: Connection, peer: Peer,
|
|||
else:
|
||||
return err(res.error)
|
||||
elif contextBytes == peer.network.forkDigests.altair:
|
||||
let res = await readChunkPayload(conn, peer, AltairSignedBeaconBlock)
|
||||
let res = await readChunkPayload(conn, peer, altair.SignedBeaconBlock)
|
||||
if res.isOk:
|
||||
return ok ForkedSignedBeaconBlock(
|
||||
kind: BeaconBlockFork.Altair,
|
||||
|
@ -191,9 +186,9 @@ p2pProtocol BeaconSync(version = 1,
|
|||
{.libp2pProtocol("ping", 1).} =
|
||||
return peer.network.metadata.seq_number
|
||||
|
||||
proc getMetaData(peer: Peer): MetaData
|
||||
proc getMetaData(peer: Peer): phase0.MetaData
|
||||
{.libp2pProtocol("metadata", 1).} =
|
||||
return peer.network.phase0Metadata
|
||||
return peer.network.phase0metadata
|
||||
|
||||
proc getMetadata_v2(peer: Peer): altair.MetaData
|
||||
{.libp2pProtocol("metadata", 2).} =
|
||||
|
@ -204,7 +199,7 @@ p2pProtocol BeaconSync(version = 1,
|
|||
startSlot: Slot,
|
||||
reqCount: uint64,
|
||||
reqStep: uint64,
|
||||
response: MultipleChunksResponse[SignedBeaconBlock])
|
||||
response: MultipleChunksResponse[phase0.SignedBeaconBlock])
|
||||
{.async, libp2pProtocol("beacon_blocks_by_range", 1).} =
|
||||
trace "got range request", peer, startSlot,
|
||||
count = reqCount, step = reqStep
|
||||
|
@ -252,7 +247,7 @@ p2pProtocol BeaconSync(version = 1,
|
|||
# Please note that the SSZ list here ensures that the
|
||||
# spec constant MAX_REQUEST_BLOCKS is enforced:
|
||||
blockRoots: BlockRootsList,
|
||||
response: MultipleChunksResponse[SignedBeaconBlock])
|
||||
response: MultipleChunksResponse[phase0.SignedBeaconBlock])
|
||||
{.async, libp2pProtocol("beacon_blocks_by_root", 1).} =
|
||||
if blockRoots.len == 0:
|
||||
raise newException(InvalidInputsError, "No blocks requested")
|
||||
|
|
|
@ -22,7 +22,7 @@ import
|
|||
# Local modules
|
||||
../spec/[
|
||||
datatypes/phase0, datatypes/altair, digest, crypto,
|
||||
forkedbeaconstate_helpers, helpers, network, signatures, state_transition],
|
||||
forks, helpers, network, signatures, state_transition],
|
||||
../conf, ../beacon_clock,
|
||||
../consensus_object_pools/[
|
||||
spec_cache, blockchain_dag, block_clearance,
|
||||
|
|
|
@ -12,7 +12,7 @@ import
|
|||
confutils/defs, serialization, chronicles,
|
||||
# Beacon-chain
|
||||
../beacon_chain/spec/[
|
||||
beaconstate, crypto, datatypes, forkedbeaconstate_helpers, helpers,
|
||||
beaconstate, crypto, datatypes, forks, helpers,
|
||||
presets, state_transition, state_transition_block],
|
||||
../beacon_chain/extras,
|
||||
../beacon_chain/ssz/[merkleization, ssz_serialization],
|
||||
|
|
|
@ -4,7 +4,7 @@ import
|
|||
stew/byteutils,
|
||||
../research/simutils,
|
||||
../beacon_chain/spec/[
|
||||
crypto, datatypes, digest, forkedbeaconstate_helpers, helpers, presets,
|
||||
crypto, datatypes, digest, forks, helpers, presets,
|
||||
state_transition],
|
||||
../beacon_chain/extras,
|
||||
../beacon_chain/networking/network_metadata,
|
||||
|
|
|
@ -6,7 +6,7 @@ import
|
|||
../beacon_chain/consensus_object_pools/[
|
||||
blockchain_dag, forkedbeaconstate_dbhelpers],
|
||||
../beacon_chain/spec/[
|
||||
crypto, datatypes/phase0, digest, forkedbeaconstate_helpers, helpers,
|
||||
crypto, datatypes/phase0, digest, forks, helpers,
|
||||
state_transition, state_transition_epoch, presets],
|
||||
../beacon_chain/ssz, ../beacon_chain/ssz/sszdump,
|
||||
../research/simutils, ./e2store
|
||||
|
|
|
@ -5,7 +5,7 @@ import
|
|||
stew/ptrops, stew/ranges/ptr_arith, chronicles,
|
||||
../beacon_chain/extras,
|
||||
../beacon_chain/spec/[
|
||||
beaconstate, crypto, datatypes, digest, forkedbeaconstate_helpers, presets,
|
||||
beaconstate, crypto, datatypes, digest, forks, presets,
|
||||
validator, state_transition, state_transition_block],
|
||||
../beacon_chain/ssz/[merkleization, ssz_serialization]
|
||||
|
||||
|
|
|
@ -21,7 +21,7 @@ import
|
|||
eth/keys,
|
||||
../tests/testblockutil,
|
||||
../beacon_chain/spec/[beaconstate, crypto, digest,
|
||||
forkedbeaconstate_helpers, presets,
|
||||
forks, presets,
|
||||
helpers, signatures, state_transition],
|
||||
../beacon_chain/spec/datatypes/[phase0, altair],
|
||||
../beacon_chain/[beacon_node_types, beacon_chain_db, extras],
|
||||
|
|
|
@ -4,7 +4,7 @@ import
|
|||
../beacon_chain/[extras, beacon_chain_db],
|
||||
../beacon_chain/ssz/[merkleization, ssz_serialization],
|
||||
../beacon_chain/spec/[
|
||||
beaconstate, crypto, datatypes, digest, forkedbeaconstate_helpers,
|
||||
beaconstate, crypto, datatypes, digest, forks,
|
||||
helpers, presets],
|
||||
../beacon_chain/consensus_object_pools/[blockchain_dag, block_pools_types],
|
||||
../beacon_chain/eth1/eth1_monitor
|
||||
|
|
|
@ -14,7 +14,7 @@ import
|
|||
options, sequtils, random, tables,
|
||||
../tests/testblockutil,
|
||||
../beacon_chain/spec/[
|
||||
beaconstate, crypto, datatypes, digest, forkedbeaconstate_helpers, helpers],
|
||||
beaconstate, crypto, datatypes, digest, forks, helpers],
|
||||
../beacon_chain/extras,
|
||||
../beacon_chain/ssz/[merkleization, ssz_serialization],
|
||||
./simutils
|
||||
|
|
|
@ -330,12 +330,12 @@ fi
|
|||
# but it can be considered non-critical
|
||||
echo Wrote $RUNTIME_CONFIG_FILE:
|
||||
|
||||
# TODO the runtime config file should be used during deposit generation as well!
|
||||
tee "$RUNTIME_CONFIG_FILE" <<EOF
|
||||
PRESET_BASE: ${CONST_PRESET}
|
||||
MIN_GENESIS_ACTIVE_VALIDATOR_COUNT: ${TOTAL_VALIDATORS}
|
||||
MIN_GENESIS_TIME: 0
|
||||
GENESIS_DELAY: 10
|
||||
GENESIS_FORK_VERSION: 0x00000000
|
||||
DEPOSIT_CONTRACT_ADDRESS: ${DEPOSIT_CONTRACT_ADDRESS}
|
||||
ETH1_FOLLOW_DISTANCE: 1
|
||||
EOF
|
||||
|
|
|
@ -10,7 +10,7 @@
|
|||
|
||||
import
|
||||
# Specs
|
||||
../../beacon_chain/spec/[forkedbeaconstate_helpers, presets, state_transition],
|
||||
../../beacon_chain/spec/[forks, presets, state_transition],
|
||||
../../beacon_chain/spec/datatypes/base
|
||||
|
||||
proc nextEpoch*(state: var ForkedHashedBeaconState) =
|
||||
|
|
|
@ -11,7 +11,7 @@ import
|
|||
# Standard library
|
||||
os,
|
||||
# Beacon chain internals
|
||||
../../../beacon_chain/spec/beaconstate,
|
||||
../../../beacon_chain/spec/[beaconstate, helpers],
|
||||
../../../beacon_chain/spec/datatypes/[phase0, altair],
|
||||
../../../beacon_chain/ssz,
|
||||
# Test utilities
|
||||
|
@ -36,7 +36,12 @@ proc runTest(identifier: string) =
|
|||
parseTest(testDir/"pre.ssz_snappy", SSZ, phase0.BeaconState))
|
||||
postState = newClone(
|
||||
parseTest(testDir/"post.ssz_snappy", SSZ, altair.BeaconState))
|
||||
upgradedState = upgrade_to_altair(preState[])
|
||||
|
||||
var cfg = defaultRuntimeConfig
|
||||
cfg.ALTAIR_FORK_EPOCH = preState[].slot.epoch
|
||||
|
||||
let
|
||||
upgradedState = upgrade_to_altair(cfg, preState[])
|
||||
check: upgradedState[].hash_tree_root() == postState[].hash_tree_root()
|
||||
reportDiff(upgradedState, postState)
|
||||
|
||||
|
|
|
@ -12,7 +12,7 @@ import
|
|||
os, sequtils, chronicles,
|
||||
# Beacon chain internals
|
||||
../../../beacon_chain/spec/[
|
||||
crypto, forkedbeaconstate_helpers, presets, state_transition],
|
||||
crypto, forks, presets, state_transition],
|
||||
../../../beacon_chain/spec/datatypes/altair,
|
||||
../../../beacon_chain/ssz,
|
||||
# Test utilities
|
||||
|
|
|
@ -12,7 +12,7 @@ import
|
|||
# Standard library
|
||||
os, strutils,
|
||||
# Beacon chain internals
|
||||
../../../beacon_chain/spec/[forkedbeaconstate_helpers, state_transition],
|
||||
../../../beacon_chain/spec/[forks, state_transition],
|
||||
../../../beacon_chain/spec/datatypes/altair,
|
||||
# Test utilities
|
||||
../../testutil,
|
||||
|
|
|
@ -15,7 +15,7 @@ import
|
|||
faststreams, streams,
|
||||
# Beacon chain internals
|
||||
../../../beacon_chain/spec/[
|
||||
crypto, state_transition, presets, forkedbeaconstate_helpers, helpers],
|
||||
crypto, state_transition, presets, forks, helpers],
|
||||
../../../beacon_chain/spec/datatypes/[phase0, altair],
|
||||
../../../beacon_chain/[extras, ssz],
|
||||
# Test utilities
|
||||
|
|
|
@ -12,7 +12,7 @@ import
|
|||
os, sequtils, chronicles,
|
||||
# Beacon chain internals
|
||||
../../../beacon_chain/spec/[
|
||||
crypto, forkedbeaconstate_helpers, presets, state_transition],
|
||||
crypto, forks, presets, state_transition],
|
||||
../../../beacon_chain/spec/datatypes/phase0,
|
||||
../../../beacon_chain/ssz,
|
||||
# Test utilities
|
||||
|
|
|
@ -11,7 +11,7 @@ import
|
|||
# Standard library
|
||||
os, strutils,
|
||||
# Beacon chain internals
|
||||
../../../beacon_chain/spec/[forkedbeaconstate_helpers, state_transition],
|
||||
../../../beacon_chain/spec/[forks, state_transition],
|
||||
../../../beacon_chain/spec/datatypes/phase0,
|
||||
# Test utilities
|
||||
../../testutil,
|
||||
|
|
|
@ -14,7 +14,7 @@
|
|||
import
|
||||
stew/results,
|
||||
# Specs
|
||||
../../beacon_chain/spec/[beaconstate, forkedbeaconstate_helpers, helpers],
|
||||
../../beacon_chain/spec/[beaconstate, forks, helpers],
|
||||
../../beacon_chain/spec/datatypes/base,
|
||||
# Mock helpers
|
||||
../mocking/[mock_genesis, mock_attestations, mock_state],
|
||||
|
|
|
@ -8,7 +8,7 @@
|
|||
import
|
||||
# Specs
|
||||
../../beacon_chain/spec/[
|
||||
forkedbeaconstate_helpers, presets, state_transition, state_transition_epoch],
|
||||
forks, presets, state_transition, state_transition_epoch],
|
||||
../../beacon_chain/spec/datatypes/base
|
||||
|
||||
proc processSlotsUntilEndCurrentEpoch(state: var ForkedHashedBeaconState) =
|
||||
|
|
|
@ -13,7 +13,7 @@ import
|
|||
stew/bitops2,
|
||||
# Specs
|
||||
../../beacon_chain/spec/datatypes/base,
|
||||
../../beacon_chain/spec/forkedbeaconstate_helpers,
|
||||
../../beacon_chain/spec/forks,
|
||||
# Test helpers
|
||||
../mocking/mock_genesis,
|
||||
./epoch_utils,
|
||||
|
|
|
@ -21,7 +21,7 @@ import
|
|||
../beacon_chain/consensus_object_pools/[
|
||||
block_quarantine, blockchain_dag, block_clearance, attestation_pool],
|
||||
../beacon_chain/ssz/merkleization,
|
||||
../beacon_chain/spec/[crypto, datatypes, digest, forkedbeaconstate_helpers,
|
||||
../beacon_chain/spec/[crypto, datatypes, digest, forks,
|
||||
state_transition, helpers, presets],
|
||||
# Test utilities
|
||||
./testutil, ./testdbutil, ./testblockutil
|
||||
|
|
|
@ -12,7 +12,7 @@ import
|
|||
unittest2,
|
||||
../beacon_chain/[beacon_chain_db, extras, interop, ssz],
|
||||
../beacon_chain/spec/[
|
||||
beaconstate, crypto, digest, forkedbeaconstate_helpers, presets,
|
||||
beaconstate, crypto, digest, forks, presets,
|
||||
state_transition],
|
||||
../beacon_chain/spec/datatypes/[phase0, altair],
|
||||
../beacon_chain/consensus_object_pools/blockchain_dag,
|
||||
|
|
|
@ -138,3 +138,26 @@ suite "Bit fields":
|
|||
not a.overlaps(b)
|
||||
not b.overlaps(a)
|
||||
a.countOverlap(b) == 0
|
||||
|
||||
test "isZeros":
|
||||
template carryOutTests(N: static int) =
|
||||
var a = BitArray[N]()
|
||||
check a.isZeros()
|
||||
|
||||
for i in 0 ..< N:
|
||||
var b = a
|
||||
b.setBit(i)
|
||||
check(not b.isZeros())
|
||||
|
||||
carryOutTests(1)
|
||||
carryOutTests(10)
|
||||
carryOutTests(31)
|
||||
carryOutTests(32)
|
||||
carryOutTests(63)
|
||||
carryOutTests(64)
|
||||
carryOutTests(65)
|
||||
carryOutTests(95)
|
||||
carryOutTests(96)
|
||||
carryOutTests(97)
|
||||
carryOutTests(12494)
|
||||
|
||||
|
|
|
@ -15,7 +15,7 @@ import
|
|||
eth/keys,
|
||||
../beacon_chain/spec/datatypes/base,
|
||||
../beacon_chain/spec/[
|
||||
beaconstate, digest, forkedbeaconstate_helpers, helpers, state_transition,
|
||||
beaconstate, digest, forks, helpers, state_transition,
|
||||
presets],
|
||||
../beacon_chain/beacon_node_types,
|
||||
../beacon_chain/[beacon_chain_db, ssz, extras],
|
||||
|
|
|
@ -4,6 +4,7 @@ import
|
|||
unittest2,
|
||||
chronos, stew/shims/net, eth/keys, eth/p2p/discoveryv5/enr,
|
||||
../beacon_chain/spec/datatypes/base,
|
||||
../beacon_chain/spec/network,
|
||||
../beacon_chain/networking/[eth2_network, eth2_discovery],
|
||||
./testutil
|
||||
|
||||
|
@ -43,8 +44,10 @@ suite "Eth2 specific discovery tests":
|
|||
|
||||
let
|
||||
node1 = generateNode(rng, Port(5000))
|
||||
node2 = generateNode(rng, Port(5001),
|
||||
{"eth2": SSZ.encode(enrForkId), "attnets": SSZ.encode(attnets)})
|
||||
node2 = generateNode(rng, Port(5001), {
|
||||
enrForkIdField: SSZ.encode(enrForkId),
|
||||
enrAttestationSubnetsField: SSZ.encode(attnets)
|
||||
})
|
||||
|
||||
node1.open()
|
||||
node2.open()
|
||||
|
@ -73,10 +76,14 @@ suite "Eth2 specific discovery tests":
|
|||
|
||||
let
|
||||
node1 = generateNode(rng, Port(5000))
|
||||
node2 = generateNode(rng, Port(5001),
|
||||
{"eth2": SSZ.encode(enrForkId), "attnets": SSZ.encode(invalidAttnets)})
|
||||
node3 = generateNode(rng, Port(5002),
|
||||
{"eth2": SSZ.encode(enrForkId), "attnets": SSZ.encode(attnets)})
|
||||
node2 = generateNode(rng, Port(5001), {
|
||||
enrForkIdField: SSZ.encode(enrForkId),
|
||||
enrAttestationSubnetsField: SSZ.encode(invalidAttnets)
|
||||
})
|
||||
node3 = generateNode(rng, Port(5002), {
|
||||
enrForkIdField: SSZ.encode(enrForkId),
|
||||
enrAttestationSubnetsField: SSZ.encode(attnets)
|
||||
})
|
||||
|
||||
node1.open()
|
||||
node2.open()
|
||||
|
@ -102,8 +109,10 @@ suite "Eth2 specific discovery tests":
|
|||
|
||||
let
|
||||
node1 = generateNode(rng, Port(5000))
|
||||
node2 = generateNode(rng, Port(5001),
|
||||
{"eth2": SSZ.encode(enrForkId), "attnets": SSZ.encode(attnets)})
|
||||
node2 = generateNode(rng, Port(5001), {
|
||||
enrForkIdField: SSZ.encode(enrForkId),
|
||||
enrAttestationSubnetsField: SSZ.encode(attnets)
|
||||
})
|
||||
|
||||
node1.open()
|
||||
node2.open()
|
||||
|
@ -119,8 +128,10 @@ suite "Eth2 specific discovery tests":
|
|||
|
||||
block:
|
||||
attnets.setBit(2)
|
||||
check node2.updateRecord(
|
||||
{"eth2": SSZ.encode(enrForkId), "attnets": SSZ.encode(attnets)}).isOk()
|
||||
check node2.updateRecord({
|
||||
enrForkIdField: SSZ.encode(enrForkId),
|
||||
enrAttestationSubnetsField: SSZ.encode(attnets)
|
||||
}).isOk()
|
||||
|
||||
let nodes = await node1.findNode(node2.localNode, @[0'u16])
|
||||
check nodes.isOk() and nodes[].len > 0
|
||||
|
|
|
@ -19,7 +19,7 @@ import
|
|||
../beacon_chain/consensus_object_pools/[
|
||||
block_quarantine, blockchain_dag, block_clearance, attestation_pool],
|
||||
../beacon_chain/ssz/merkleization,
|
||||
../beacon_chain/spec/[crypto, datatypes, digest, forkedbeaconstate_helpers,
|
||||
../beacon_chain/spec/[crypto, datatypes, digest, forks,
|
||||
state_transition, helpers, presets, network],
|
||||
# Test utilities
|
||||
./testutil, ./testdbutil, ./testblockutil
|
||||
|
|
|
@ -12,7 +12,7 @@ import
|
|||
unittest2,
|
||||
./testutil, ./testdbutil, ./teststateutil,
|
||||
../beacon_chain/spec/[
|
||||
datatypes, digest, forkedbeaconstate_helpers, helpers, presets],
|
||||
datatypes, digest, forks, helpers, presets],
|
||||
../beacon_chain/[beacon_node_types, statediff],
|
||||
../beacon_chain/ssz,
|
||||
../beacon_chain/consensus_object_pools/[blockchain_dag, block_quarantine]
|
||||
|
|
|
@ -6,7 +6,7 @@ import chronos
|
|||
import ../beacon_chain/gossip_processing/block_processor,
|
||||
../beacon_chain/sync/sync_manager,
|
||||
../beacon_chain/spec/datatypes/phase0,
|
||||
../beacon_chain/spec/forkedbeaconstate_helpers
|
||||
../beacon_chain/spec/forks
|
||||
|
||||
type
|
||||
SomeTPeer = ref object
|
||||
|
|
|
@ -12,7 +12,7 @@ import
|
|||
../beacon_chain/validators/validator_pool,
|
||||
../beacon_chain/ssz/merkleization,
|
||||
../beacon_chain/spec/[crypto, datatypes, digest, presets, helpers,
|
||||
signatures, state_transition, forkedbeaconstate_helpers],
|
||||
signatures, state_transition, forks],
|
||||
../beacon_chain/consensus_object_pools/attestation_pool
|
||||
|
||||
func makeFakeValidatorPrivKey(i: int): ValidatorPrivKey =
|
||||
|
|
|
@ -13,7 +13,7 @@ import
|
|||
./helpers/math_helpers,
|
||||
../beacon_chain/ssz/merkleization,
|
||||
../beacon_chain/spec/[
|
||||
crypto, datatypes, forkedbeaconstate_helpers, helpers,
|
||||
crypto, datatypes, forks, helpers,
|
||||
presets, state_transition, state_transition_block]
|
||||
|
||||
proc valid_deposit[T](state: var T) =
|
||||
|
|
|
@ -51,7 +51,7 @@ func toFloatSeconds(duration: Duration): float =
|
|||
|
||||
method testEnded*(formatter: TimingCollector, testResult: TestResult) =
|
||||
{.gcsafe.}: # Lie!
|
||||
status.mGetOrPut(testResult.suiteName, initOrderedTable[string, Status]())[testResult.testName] =
|
||||
status.mgetOrPut(testResult.suiteName, initOrderedTable[string, Status]())[testResult.testName] =
|
||||
case testResult.status
|
||||
of TestStatus.OK: Status.OK
|
||||
of TestStatus.FAILED: Status.Fail
|
||||
|
|
Loading…
Reference in New Issue