remove some copies

* in makeBeaconBlock - use rollback instead
* in tests - this helps state_sim give more accurate data and makes it
30% faster
* fix some usages of raw BeaconState
This commit is contained in:
Jacek Sieka 2020-05-22 16:21:22 +02:00 committed by tersec
parent f16fb31060
commit f06df1cea6
12 changed files with 143 additions and 129 deletions

View File

@ -135,7 +135,8 @@ proc preInit*(
signedBlock: SignedBeaconBlock) = signedBlock: SignedBeaconBlock) =
preInit(CandidateChains, db, state, signedBlock) preInit(CandidateChains, db, state, signedBlock)
proc getProposer*(pool: BlockPool, head: BlockRef, slot: Slot): Option[ValidatorPubKey] = proc getProposer*(pool: BlockPool, head: BlockRef, slot: Slot):
Option[(ValidatorIndex, ValidatorPubKey)] =
getProposer(pool.dag, head, slot) getProposer(pool.dag, head, slot)
# Rewinder / State transitions # Rewinder / State transitions

View File

@ -822,7 +822,9 @@ proc preInit*(
db.putHeadBlock(blockRoot) db.putHeadBlock(blockRoot)
db.putStateRoot(blockRoot, state.slot, signedBlock.message.state_root) db.putStateRoot(blockRoot, state.slot, signedBlock.message.state_root)
proc getProposer*(dag: CandidateChains, head: BlockRef, slot: Slot): Option[ValidatorPubKey] = proc getProposer*(
dag: CandidateChains, head: BlockRef, slot: Slot):
Option[(ValidatorIndex, ValidatorPubKey)] =
dag.withState(dag.tmpState, head.atSlot(slot)): dag.withState(dag.tmpState, head.atSlot(slot)):
var cache = get_empty_per_epoch_cache() var cache = get_empty_per_epoch_cache()
@ -838,4 +840,4 @@ proc getProposer*(dag: CandidateChains, head: BlockRef, slot: Slot): Option[Vali
balances=state.balances balances=state.balances
return return
return some(state.validators[proposerIdx.get()].pubkey) return some((proposerIdx.get(), state.validators[proposerIdx.get()].pubkey))

View File

@ -447,51 +447,6 @@ proc process_block*(
true true
# https://github.com/ethereum/eth2.0-specs/blob/v0.11.1/specs/phase0/validator.md
# TODO There's more to do here - the spec has helpers that deal set up some of
# the fields in here!
proc makeBeaconBlock*(
state: BeaconState,
parent_root: Eth2Digest,
randao_reveal: ValidatorSig,
eth1_data: Eth1Data,
graffiti: Eth2Digest,
attestations: seq[Attestation],
deposits: seq[Deposit]): Option[BeaconBlock] =
## Create a block for the given state. The last block applied to it must be
## the one identified by parent_root and process_slots must be called up to
## the slot for which a block is to be created.
var cache = get_empty_per_epoch_cache()
let proposer_index = get_beacon_proposer_index(state, cache)
if proposer_index.isNone:
warn "Unable to get proposer index when proposing!"
return
# To create a block, we'll first apply a partial block to the state, skipping
# some validations.
var blck = BeaconBlock(
slot: state.slot,
proposer_index: proposer_index.get().uint64,
parent_root: parent_root,
body: BeaconBlockBody(
randao_reveal: randao_reveal,
eth1_data: eth1data,
graffiti: graffiti,
attestations: List[Attestation, MAX_ATTESTATIONS](attestations),
deposits: List[Deposit, MAX_DEPOSITS](deposits)))
let tmpState = newClone(state)
let ok = process_block(tmpState[], blck, {skipBlsValidation}, cache)
if not ok:
warn "Unable to apply new block to state", blck = shortLog(blck)
return
blck.state_root = hash_tree_root(tmpState[])
some(blck)
# https://github.com/ethereum/eth2.0-specs/blob/v0.11.1/specs/phase0/validator.md#aggregation-selection # https://github.com/ethereum/eth2.0-specs/blob/v0.11.1/specs/phase0/validator.md#aggregation-selection
func get_slot_signature*( func get_slot_signature*(
fork: Fork, genesis_validators_root: Eth2Digest, slot: Slot, fork: Fork, genesis_validators_root: Eth2Digest, slot: Slot,

View File

@ -244,3 +244,47 @@ proc state_transition*(
rollback(state) rollback(state)
false false
# https://github.com/ethereum/eth2.0-specs/blob/v0.11.1/specs/phase0/validator.md
# TODO There's more to do here - the spec has helpers that deal set up some of
# the fields in here!
proc makeBeaconBlock*(
state: var HashedBeaconState,
proposer_index: ValidatorIndex,
parent_root: Eth2Digest,
randao_reveal: ValidatorSig,
eth1_data: Eth1Data,
graffiti: Eth2Digest,
attestations: seq[Attestation],
deposits: seq[Deposit],
rollback: RollbackHashedProc): Option[BeaconBlock] =
## Create a block for the given state. The last block applied to it must be
## the one identified by parent_root and process_slots must be called up to
## the slot for which a block is to be created.
var cache = get_empty_per_epoch_cache()
# To create a block, we'll first apply a partial block to the state, skipping
# some validations.
var blck = BeaconBlock(
slot: state.data.slot,
proposer_index: proposer_index.uint64,
parent_root: parent_root,
body: BeaconBlockBody(
randao_reveal: randao_reveal,
eth1_data: eth1data,
graffiti: graffiti,
attestations: List[Attestation, MAX_ATTESTATIONS](attestations),
deposits: List[Deposit, MAX_DEPOSITS](deposits)))
let ok = process_block(state.data, blck, {skipBlsValidation}, cache)
if not ok:
warn "Unable to apply new block to state", blck = shortLog(blck)
rollback(state)
return
state.root = hash_tree_root(state.data)
blck.state_root = state.root
some(blck)

View File

@ -18,9 +18,8 @@ import
eth/[keys, async_utils], eth/p2p/discoveryv5/[protocol, enr], eth/[keys, async_utils], eth/p2p/discoveryv5/[protocol, enr],
# Local modules # Local modules
spec/[datatypes, digest, crypto, beaconstate, helpers, validator, network, spec/[datatypes, digest, crypto, beaconstate, helpers, validator, network],
state_transition_block], conf, time, validator_pool, state_transition,
conf, time, validator_pool,
attestation_pool, block_pool, eth2_network, attestation_pool, block_pool, eth2_network,
beacon_node_common, beacon_node_types, beacon_node_common, beacon_node_types,
mainchain_monitor, version, ssz, interop, mainchain_monitor, version, ssz, interop,
@ -139,6 +138,7 @@ proc sendAttestation(node: BeaconNode,
proc proposeBlock(node: BeaconNode, proc proposeBlock(node: BeaconNode,
validator: AttachedValidator, validator: AttachedValidator,
validator_index: ValidatorIndex,
head: BlockRef, head: BlockRef,
slot: Slot): Future[BlockRef] {.async.} = slot: Slot): Future[BlockRef] {.async.} =
logScope: pcs = "block_proposal" logScope: pcs = "block_proposal"
@ -164,17 +164,36 @@ proc proposeBlock(node: BeaconNode,
else: else:
node.mainchainMonitor.getBlockProposalData(state) node.mainchainMonitor.getBlockProposalData(state)
let
poolPtr = unsafeAddr node.blockPool.dag # safe because restore is short-lived
func restore(v: var HashedBeaconState) =
# 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
poolPtr.tmpState = poolPtr.headState
let message = makeBeaconBlock( let message = makeBeaconBlock(
state, hashedState,
validator_index,
head.root, head.root,
validator.genRandaoReveal(state.fork, state.genesis_validators_root, slot), validator.genRandaoReveal(state.fork, state.genesis_validators_root, slot),
eth1data, eth1data,
Eth2Digest(), Eth2Digest(),
node.attestationPool.getAttestationsForBlock(state), node.attestationPool.getAttestationsForBlock(state),
deposits) deposits,
restore)
if not message.isSome(): if not message.isSome():
return head # already logged elsewhere! return head # already logged elsewhere!
# TODO this restore is needed because otherwise tmpState will be internally
# inconsistent - it's blck will not be pointing to the block that
# created this state - we have to reset it here before `await` to avoid
# races.
restore(poolPtr.tmpState.data)
var var
newBlock = SignedBeaconBlock( newBlock = SignedBeaconBlock(
message: message.get() message: message.get()
@ -289,25 +308,25 @@ proc handleProposal(node: BeaconNode, head: BlockRef, slot: Slot):
# proposing for it - basically, we're selecting proposer based on an # proposing for it - basically, we're selecting proposer based on an
# empty slot # empty slot
let proposerKey = node.blockPool.getProposer(head, slot) let proposer = node.blockPool.getProposer(head, slot)
if proposerKey.isNone(): if proposer.isNone():
return head return head
let validator = node.attachedValidators.getValidator(proposerKey.get()) let validator = node.attachedValidators.getValidator(proposer.get()[1])
if validator != nil: if validator != nil:
return await proposeBlock(node, validator, head, slot) return await proposeBlock(node, validator, proposer.get()[0], head, slot)
debug "Expecting block proposal", debug "Expecting block proposal",
headRoot = shortLog(head.root), headRoot = shortLog(head.root),
slot = shortLog(slot), slot = shortLog(slot),
proposer = shortLog(proposerKey.get()), proposer_index = proposer.get()[0],
proposer = shortLog(proposer.get()[1]),
cat = "consensus", cat = "consensus",
pcs = "wait_for_proposal" pcs = "wait_for_proposal"
return head return head
proc broadcastAggregatedAttestations( proc broadcastAggregatedAttestations(
node: BeaconNode, aggregationHead: BlockRef, aggregationSlot: Slot, node: BeaconNode, aggregationHead: BlockRef, aggregationSlot: Slot,
trailing_distance: uint64) = trailing_distance: uint64) =

View File

@ -24,7 +24,7 @@ import
state_transition_block], state_transition_block],
../beacon_chain/[ ../beacon_chain/[
attestation_pool, block_pool, beacon_node_types, beacon_chain_db, attestation_pool, block_pool, beacon_node_types, beacon_chain_db,
interop, ssz, validator_pool], interop, ssz, state_transition, validator_pool],
eth/db/[kvstore, kvstore_sqlite3], eth/db/[kvstore, kvstore_sqlite3],
./simutils ./simutils
@ -44,14 +44,14 @@ cli do(slots = SLOTS_PER_EPOCH * 6,
replay = true): replay = true):
let let
state = loadGenesis(validators, true) state = loadGenesis(validators, true)
genesisBlock = get_initial_beacon_block(state[]) genesisBlock = get_initial_beacon_block(state[].data)
echo "Starting simulation..." echo "Starting simulation..."
let let
db = BeaconChainDB.init(kvStore SqStoreRef.init(".", "block_sim").tryGet()) db = BeaconChainDB.init(kvStore SqStoreRef.init(".", "block_sim").tryGet())
BlockPool.preInit(db, state[], genesisBlock) BlockPool.preInit(db, state[].data, genesisBlock)
var var
blockPool = BlockPool.init(db) blockPool = BlockPool.init(db)
@ -108,13 +108,15 @@ cli do(slots = SLOTS_PER_EPOCH * 6,
eth1data = get_eth1data_stub( eth1data = get_eth1data_stub(
state.eth1_deposit_index, slot.compute_epoch_at_slot()) state.eth1_deposit_index, slot.compute_epoch_at_slot())
message = makeBeaconBlock( message = makeBeaconBlock(
state, hashedState,
proposerIdx,
head.root, head.root,
privKey.genRandaoReveal(state.fork, state.genesis_validators_root, slot), privKey.genRandaoReveal(state.fork, state.genesis_validators_root, slot),
eth1data, eth1data,
Eth2Digest(), Eth2Digest(),
attPool.getAttestationsForBlock(state), attPool.getAttestationsForBlock(state),
@[]) @[],
noRollback)
var var
newBlock = SignedBeaconBlock( newBlock = SignedBeaconBlock(
@ -131,6 +133,7 @@ cli do(slots = SLOTS_PER_EPOCH * 6,
blockRoot, privKey) blockRoot, privKey)
let added = blockPool.add(blockRoot, newBlock).tryGet() let added = blockPool.add(blockRoot, newBlock).tryGet()
blck() = added
blockPool.updateHead(added) blockPool.updateHead(added)
for i in 0..<slots: for i in 0..<slots:

View File

@ -39,16 +39,18 @@ func verifyConsensus*(state: BeaconState, attesterRatio: auto) =
if current_epoch >= 4: if current_epoch >= 4:
doAssert state.finalized_checkpoint.epoch + 2 >= current_epoch doAssert state.finalized_checkpoint.epoch + 2 >= current_epoch
proc loadGenesis*(validators: int, validate: bool): ref BeaconState = proc loadGenesis*(validators: int, validate: bool): ref HashedBeaconState =
let fn = &"genesim_{const_preset}_{validators}.ssz" let fn = &"genesim_{const_preset}_{validators}.ssz"
let res = (ref HashedBeaconState)()
if fileExists(fn): if fileExists(fn):
let res = newClone(SSZ.loadFile(fn, BeaconState)) res.data = SSZ.loadFile(fn, BeaconState)
if res.slot != GENESIS_SLOT: res.root = hash_tree_root(res.data)
if res.data.slot != GENESIS_SLOT:
echo "Can only start from genesis state" echo "Can only start from genesis state"
quit 1 quit 1
if res.validators.len != validators: if res.data.validators.len != validators:
echo &"Supplied genesis file has {res.validators.len} validators, while {validators} where requested, running anyway" echo &"Supplied genesis file has {res.data.validators.len} validators, while {validators} where requested, running anyway"
echo &"Loaded {fn}..." echo &"Loaded {fn}..."
# TODO check that the private keys are interop keys # TODO check that the private keys are interop keys
@ -63,12 +65,13 @@ proc loadGenesis*(validators: int, validate: bool): ref BeaconState =
echo "Generating Genesis..." echo "Generating Genesis..."
let state = res.data =
initialize_beacon_state_from_eth1(Eth2Digest(), 0, deposits, flags) initialize_beacon_state_from_eth1(Eth2Digest(), 0, deposits, flags)[]
res.root = hash_tree_root(res.data)
echo &"Saving to {fn}..." echo &"Saving to {fn}..."
SSZ.saveFile(fn, state[]) SSZ.saveFile(fn, res.data)
state res
proc printTimers*[Timers: enum]( proc printTimers*[Timers: enum](
state: BeaconState, attesters: RunningStat, validate: bool, state: BeaconState, attesters: RunningStat, validate: bool,

View File

@ -42,7 +42,7 @@ cli do(slots = SLOTS_PER_EPOCH * 6,
let let
flags = if validate: {} else: {skipBlsValidation} flags = if validate: {} else: {skipBlsValidation}
state = loadGenesis(validators, validate) state = loadGenesis(validators, validate)
genesisBlock = get_initial_beacon_block(state[]) genesisBlock = get_initial_beacon_block(state.data)
echo "Starting simulation..." echo "Starting simulation..."
@ -57,7 +57,7 @@ cli do(slots = SLOTS_PER_EPOCH * 6,
proc maybeWrite(last: bool) = proc maybeWrite(last: bool) =
if write_last_json: if write_last_json:
if state.slot mod json_interval.uint64 == 0: if state[].data.slot mod json_interval.uint64 == 0:
write(stdout, ":") write(stdout, ":")
else: else:
write(stdout, ".") write(stdout, ".")
@ -65,8 +65,8 @@ cli do(slots = SLOTS_PER_EPOCH * 6,
if last: if last:
writeJson("state.json", state[]) writeJson("state.json", state[])
else: else:
if state[].slot mod json_interval.uint64 == 0: if state[].data.slot mod json_interval.uint64 == 0:
writeJson(jsonName(prefix, state.slot), state[]) writeJson(jsonName(prefix, state[].data.slot), state[].data)
write(stdout, ":") write(stdout, ":")
else: else:
write(stdout, ".") write(stdout, ".")
@ -77,10 +77,10 @@ cli do(slots = SLOTS_PER_EPOCH * 6,
for i in 0..<slots: for i in 0..<slots:
maybeWrite(false) maybeWrite(false)
verifyConsensus(state[], attesterRatio) verifyConsensus(state[].data, attesterRatio)
let let
attestations_idx = state.slot attestations_idx = state[].data.slot
blockAttestations = attestations.getOrDefault(attestations_idx) blockAttestations = attestations.getOrDefault(attestations_idx)
attestations.del attestations_idx attestations.del attestations_idx
@ -88,8 +88,8 @@ cli do(slots = SLOTS_PER_EPOCH * 6,
(SLOTS_PER_EPOCH.int + MIN_ATTESTATION_INCLUSION_DELAY.int) (SLOTS_PER_EPOCH.int + MIN_ATTESTATION_INCLUSION_DELAY.int)
let t = let t =
if (state.slot > GENESIS_SLOT and if (state[].data.slot > GENESIS_SLOT and
(state.slot + 1).isEpoch): tEpoch (state[].data.slot + 1).isEpoch): tEpoch
else: tBlock else: tBlock
withTimer(timers[t]): withTimer(timers[t]):
@ -103,11 +103,11 @@ cli do(slots = SLOTS_PER_EPOCH * 6,
# work for every slot - we'll randomize it deterministically to give # work for every slot - we'll randomize it deterministically to give
# some variation # some variation
let let
target_slot = state.slot + MIN_ATTESTATION_INCLUSION_DELAY - 1 target_slot = state[].data.slot + MIN_ATTESTATION_INCLUSION_DELAY - 1
scass = withTimerRet(timers[tShuffle]): scass = withTimerRet(timers[tShuffle]):
mapIt( mapIt(
0'u64 ..< get_committee_count_at_slot(state[], target_slot), 0'u64 ..< get_committee_count_at_slot(state[].data, target_slot),
get_beacon_committee(state[], target_slot, it.CommitteeIndex, cache)) get_beacon_committee(state[].data, target_slot, it.CommitteeIndex, cache))
for i, scas in scass: for i, scas in scass:
var var
@ -121,12 +121,12 @@ cli do(slots = SLOTS_PER_EPOCH * 6,
if (rand(r, high(int)).float * attesterRatio).int <= high(int): if (rand(r, high(int)).float * attesterRatio).int <= high(int):
if first: if first:
attestation = attestation =
makeAttestation(state[], latest_block_root, scas, target_slot, makeAttestation(state[].data, latest_block_root, scas, target_slot,
i.uint64, v, cache, flags) i.uint64, v, cache, flags)
first = false first = false
else: else:
attestation.combine( attestation.combine(
makeAttestation(state[], latest_block_root, scas, target_slot, makeAttestation(state[].data, latest_block_root, scas, target_slot,
i.uint64, v, cache, flags), i.uint64, v, cache, flags),
flags) flags)
@ -145,13 +145,13 @@ cli do(slots = SLOTS_PER_EPOCH * 6,
flushFile(stdout) flushFile(stdout)
if (state.slot) mod SLOTS_PER_EPOCH == 0: if (state[].data.slot) mod SLOTS_PER_EPOCH == 0:
echo &" slot: {shortLog(state.slot)} ", echo &" slot: {shortLog(state[].data.slot)} ",
&"epoch: {shortLog(state.slot.compute_epoch_at_slot)}" &"epoch: {shortLog(state[].data.slot.compute_epoch_at_slot)}"
maybeWrite(true) # catch that last state as well.. maybeWrite(true) # catch that last state as well..
echo "Done!" echo "Done!"
printTimers(state[], attesters, validate, timers) printTimers(state[].data, attesters, validate, timers)

View File

@ -150,7 +150,7 @@ suiteReport "Attestation pool processing" & preset():
timedTest "Fork choice returns latest block with no attestations": timedTest "Fork choice returns latest block with no attestations":
let let
b1 = addTestBlock(state.data.data, blockPool.tail.root) b1 = addTestBlock(state.data, blockPool.tail.root)
b1Root = hash_tree_root(b1.message) b1Root = hash_tree_root(b1.message)
b1Add = blockPool.add(b1Root, b1)[] b1Add = blockPool.add(b1Root, b1)[]
head = pool.selectHead() head = pool.selectHead()
@ -159,7 +159,7 @@ suiteReport "Attestation pool processing" & preset():
head == b1Add head == b1Add
let let
b2 = addTestBlock(state.data.data, b1Root) b2 = addTestBlock(state.data, b1Root)
b2Root = hash_tree_root(b2.message) b2Root = hash_tree_root(b2.message)
b2Add = blockPool.add(b2Root, b2)[] b2Add = blockPool.add(b2Root, b2)[]
head2 = pool.selectHead() head2 = pool.selectHead()
@ -170,7 +170,7 @@ suiteReport "Attestation pool processing" & preset():
timedTest "Fork choice returns block with attestation": timedTest "Fork choice returns block with attestation":
var cache = get_empty_per_epoch_cache() var cache = get_empty_per_epoch_cache()
let let
b10 = makeTestBlock(state.data.data, blockPool.tail.root) b10 = makeTestBlock(state.data, blockPool.tail.root)
b10Root = hash_tree_root(b10.message) b10Root = hash_tree_root(b10.message)
b10Add = blockPool.add(b10Root, b10)[] b10Add = blockPool.add(b10Root, b10)[]
head = pool.selectHead() head = pool.selectHead()
@ -179,7 +179,7 @@ suiteReport "Attestation pool processing" & preset():
head == b10Add head == b10Add
let let
b11 = makeTestBlock(state.data.data, blockPool.tail.root, b11 = makeTestBlock(state.data, blockPool.tail.root,
graffiti = Eth2Digest(data: [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]) graffiti = Eth2Digest(data: [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])
) )
b11Root = hash_tree_root(b11.message) b11Root = hash_tree_root(b11.message)

View File

@ -88,15 +88,11 @@ suiteReport "Block pool processing" & preset():
db = makeTestDB(SLOTS_PER_EPOCH) db = makeTestDB(SLOTS_PER_EPOCH)
pool = BlockPool.init(db) pool = BlockPool.init(db)
stateData = newClone(pool.loadTailState()) stateData = newClone(pool.loadTailState())
b1 = addTestBlock(stateData.data.data, pool.tail.root) b1 = addTestBlock(stateData.data, pool.tail.root)
b1Root = hash_tree_root(b1.message) b1Root = hash_tree_root(b1.message)
b2 = addTestBlock(stateData.data.data, b1Root) b2 = addTestBlock(stateData.data, b1Root)
b2Root {.used.} = hash_tree_root(b2.message) b2Root {.used.} = hash_tree_root(b2.message)
# addTestBlock(...) operates on BeaconState, so doesn't update root
# TODO fix addTestBlock to work on HashedBeaconState directly
stateData.data.root = hash_tree_root(stateData.data.data)
timedTest "getRef returns nil for missing blocks": timedTest "getRef returns nil for missing blocks":
check: check:
pool.getRef(default Eth2Digest) == nil pool.getRef(default Eth2Digest) == nil
@ -136,11 +132,9 @@ suiteReport "Block pool processing" & preset():
process_slots(stateData.data, stateData.data.data.slot + 1) process_slots(stateData.data, stateData.data.data.slot + 1)
let let
b4 = addTestBlock(stateData.data.data, b2Root) b4 = addTestBlock(stateData.data, b2Root)
b4Root = hash_tree_root(b4.message) b4Root = hash_tree_root(b4.message)
b4Add = pool.add(b4Root, b4)[] b4Add = pool.add(b4Root, b4)[]
# TODO fix addTestBlock to work on HashedBeaconState
stateData.data.root = hash_tree_root(stateData.data.data)
check: check:
b4Add.parent == b2Add b4Add.parent == b2Add
@ -295,7 +289,7 @@ when const_preset == "minimal": # These require some minutes in mainnet
block: block:
# Create a fork that will not be taken # Create a fork that will not be taken
var var
blck = makeTestBlock(pool.headState.data.data, pool.head.blck.root) blck = makeTestBlock(pool.headState.data, pool.head.blck.root)
check: pool.add(hash_tree_root(blck.message), blck).isOk check: pool.add(hash_tree_root(blck.message), blck).isOk
for i in 0 ..< (SLOTS_PER_EPOCH * 6): for i in 0 ..< (SLOTS_PER_EPOCH * 6):
@ -307,7 +301,7 @@ when const_preset == "minimal": # These require some minutes in mainnet
var var
cache = get_empty_per_epoch_cache() cache = get_empty_per_epoch_cache()
blck = makeTestBlock( blck = makeTestBlock(
pool.headState.data.data, pool.head.blck.root, pool.headState.data, pool.head.blck.root,
attestations = makeFullAttestations( attestations = makeFullAttestations(
pool.headState.data.data, pool.head.blck.root, pool.headState.data.data, pool.head.blck.root,
pool.headState.data.data.slot, cache, {})) pool.headState.data.data.slot, cache, {}))
@ -338,7 +332,7 @@ when const_preset == "minimal": # These require some minutes in mainnet
for i in 0 ..< (SLOTS_PER_EPOCH * 6 - 2): for i in 0 ..< (SLOTS_PER_EPOCH * 6 - 2):
var var
blck = makeTestBlock( blck = makeTestBlock(
pool.headState.data.data, pool.head.blck.root, pool.headState.data, pool.head.blck.root,
attestations = makeFullAttestations( attestations = makeFullAttestations(
pool.headState.data.data, pool.head.blck.root, pool.headState.data.data, pool.head.blck.root,
pool.headState.data.data.slot, cache, {})) pool.headState.data.data.slot, cache, {}))
@ -351,7 +345,7 @@ when const_preset == "minimal": # These require some minutes in mainnet
pool.headState.data, Slot(SLOTS_PER_EPOCH * 6 + 2) ) pool.headState.data, Slot(SLOTS_PER_EPOCH * 6 + 2) )
var blck = makeTestBlock( var blck = makeTestBlock(
pool.headState.data.data, pool.head.blck.root, pool.headState.data, pool.head.blck.root,
attestations = makeFullAttestations( attestations = makeFullAttestations(
pool.headState.data.data, pool.head.blck.root, pool.headState.data.data, pool.head.blck.root,
pool.headState.data.data.slot, cache, {})) pool.headState.data.data.slot, cache, {}))

View File

@ -36,7 +36,7 @@ suiteReport "Block processing" & preset():
timedTest "Passes from genesis state, empty block" & preset(): timedTest "Passes from genesis state, empty block" & preset():
var var
previous_block_root = hash_tree_root(genesisBlock.message) previous_block_root = hash_tree_root(genesisBlock.message)
new_block = makeTestBlock(state.data, previous_block_root) new_block = makeTestBlock(state[], previous_block_root)
let block_ok = state_transition(state[], new_block, {}, noRollback) let block_ok = state_transition(state[], new_block, {}, noRollback)
@ -55,7 +55,7 @@ suiteReport "Block processing" & preset():
previous_block_root = genesisRoot previous_block_root = genesisRoot
for i in 1..SLOTS_PER_EPOCH.int: for i in 1..SLOTS_PER_EPOCH.int:
let new_block = makeTestBlock(state.data, previous_block_root) let new_block = makeTestBlock(state[], previous_block_root)
let block_ok = state_transition(state[], new_block, {}, noRollback) let block_ok = state_transition(state[], new_block, {}, noRollback)
@ -90,7 +90,7 @@ suiteReport "Block processing" & preset():
state[], GENESIS_SLOT + MIN_ATTESTATION_INCLUSION_DELAY + 1) state[], GENESIS_SLOT + MIN_ATTESTATION_INCLUSION_DELAY + 1)
let let
new_block = makeTestBlock(state.data, previous_block_root, new_block = makeTestBlock(state[], previous_block_root,
attestations = @[attestation] attestations = @[attestation]
) )
check state_transition(state[], new_block, {}, noRollback) check state_transition(state[], new_block, {}, noRollback)

View File

@ -86,7 +86,7 @@ func signBlock*(
) )
proc addTestBlock*( proc addTestBlock*(
state: var BeaconState, state: var HashedBeaconState,
parent_root: Eth2Digest, parent_root: Eth2Digest,
eth1_data = Eth1Data(), eth1_data = Eth1Data(),
attestations = newSeq[Attestation](), attestations = newSeq[Attestation](),
@ -94,53 +94,46 @@ proc addTestBlock*(
graffiti = Eth2Digest(), graffiti = Eth2Digest(),
flags: set[UpdateFlag] = {}): SignedBeaconBlock = flags: set[UpdateFlag] = {}): SignedBeaconBlock =
# Create and add a block to state - state will advance by one slot! # Create and add a block to state - state will advance by one slot!
advance_slot(state, err(Opt[Eth2Digest]), flags)
# TODO workaround, disable when this works directly
var hashedState = HashedBeaconState(data: state, root: hash_tree_root(state))
doAssert process_slots(hashedState, hashedState.data.slot + 1)
state = hashedState.data
var cache = get_empty_per_epoch_cache() var cache = get_empty_per_epoch_cache()
let proposer_index = get_beacon_proposer_index(state, cache)
let let
# Index from the new state, but registry from the old state.. hmm... proposer_index = get_beacon_proposer_index(state.data, cache)
# In tests, let this throw privKey = hackPrivKey(state.data.validators[proposer_index.get])
proposer = state.validators[proposer_index.get]
privKey = hackPrivKey(proposer)
randao_reveal = randao_reveal =
if skipBlsValidation notin flags: if skipBlsValidation notin flags:
privKey.genRandaoReveal( privKey.genRandaoReveal(
state.fork, state.genesis_validators_root, state.slot) state.data.fork, state.data.genesis_validators_root, state.data.slot)
else: else:
ValidatorSig() ValidatorSig()
let let
message = makeBeaconBlock( message = makeBeaconBlock(
state, state,
proposer_index.get(),
parent_root, parent_root,
randao_reveal, randao_reveal,
# Keep deposit counts internally consistent. # Keep deposit counts internally consistent.
Eth1Data( Eth1Data(
deposit_root: eth1_data.deposit_root, deposit_root: eth1_data.deposit_root,
deposit_count: state.eth1_deposit_index + deposits.len.uint64, deposit_count: state.data.eth1_deposit_index + deposits.len.uint64,
block_hash: eth1_data.block_hash), block_hash: eth1_data.block_hash),
graffiti, graffiti,
attestations, attestations,
deposits) deposits,
noRollback)
doAssert message.isSome(), "Should have created a valid block!" doAssert message.isSome(), "Should have created a valid block!"
let let
new_block = signBlock( new_block = signBlock(
state.fork, state.genesis_validators_root, message.get(), privKey, flags) state.data.fork,
ok = process_block(state, new_block.message, flags, cache) state.data.genesis_validators_root, message.get(), privKey, flags)
doAssert ok, "adding block after producing it should work"
new_block new_block
proc makeTestBlock*( proc makeTestBlock*(
state: BeaconState, state: HashedBeaconState,
parent_root: Eth2Digest, parent_root: Eth2Digest,
eth1_data = Eth1Data(), eth1_data = Eth1Data(),
attestations = newSeq[Attestation](), attestations = newSeq[Attestation](),