v0.8.1 tests refactor (#326)

* Introduce new mocking proc to replace:
- makeFakeValidatorPrivKey
- hackPrivKey
- getNextBeaconProposerIndex
- addBlock
- makeBlock

* Add comments on datastructure unsynced with the spec

* Add merkle tree constructor and initial mocking for deposits (missing merkle proofs)

* [Mock] Implement sparse merkle tree and merkle proof builder

* [Mocking] Genesis deposits

* Add compact_committees_roots init + mock genesis state

* [Tests] Add first deposit test using the new mocking procedures

* [Tests -deposits] add at and over 32 ETH deposit tests

* [Tests - deposits] Add test for validator top-up

* [Tests -deposits] Mention the TODO to test for invalid conditions

* [Tests] Add stub to test "is_valid_genesis_state"

* [Merkle proofs] Implement round-trip checks

* Deactivate roundtrips test

* SSZ - use EF convention for hash_tree_root / hashTreeRoot

* [Tests - Attestation] Attestation mocking + initial test

* Add mocking + 3 new tests for valid attestations + mention future invalid attestation tests

* Add crosslinks test (1 failing to attestations in block being duplicated in state transition)

* Single attestation crosslink test - workaround https://github.com/status-im/nim-beacon-chain/issues/361

* Add test for failed crosslink penalty

* Rebase fixes + add refactored tests to test suite

* justif-finalization helpers first batch

* Add 234 finalization tests

* Fix justif test, Rule I 234 finalization does not happen with sufficient support.
(Also unittest check template does not fail properly in some cases)

* Add tests for all finalization rules

* Properly delete nim-byteutils following c91727e7e5 (diff-7c3613dba5171cb6027c67835dd3b9d4)

* use digest helper for deposit root
This commit is contained in:
Mamy Ratsimbazafy 2019-08-28 14:07:00 +02:00 committed by Dustin Brody
parent 9cb90b734c
commit ca4f29caca
28 changed files with 1673 additions and 26 deletions

View File

@ -241,11 +241,13 @@ func initialize_beacon_state_from_eth1*(
validator.activation_eligibility_epoch = GENESIS_EPOCH
validator.activation_epoch = GENESIS_EPOCH
let genesis_active_index_root = hash_tree_root(
# Populate active_index_roots and compact_committees_roots
let active_index_root = hash_tree_root(
get_active_validator_indices(state, GENESIS_EPOCH))
let committee_root = get_compact_committees_root(state, GENESIS_EPOCH)
for index in 0 ..< EPOCHS_PER_HISTORICAL_VECTOR:
state.active_index_roots[index] = genesis_active_index_root
state.active_index_roots[index] = active_index_root
state.compact_committees_roots[index] = committee_root
state
proc initialize_beacon_state_from_eth1*(eth1_block_hash: Eth2Digest,

View File

@ -243,9 +243,7 @@ type
BeaconBlockBody* = object
randao_reveal*: ValidatorSig
eth1_data*: Eth1Data
graffiti*: Eth2Digest
# Each of these is a length-bounded list, but enforcing that's elsewhere
graffiti*: Eth2Digest # TODO make that raw bytes
proposer_slashings*: seq[ProposerSlashing]
attester_slashings*: seq[AttesterSlashing]
attestations*: seq[Attestation]
@ -370,6 +368,8 @@ type
# https://github.com/ethereum/eth2.0-specs/blob/v0.8.3/specs/core/0_beacon-chain.md#fork
Fork* = object
# TODO: Spec introduced an alias for Version = array[4, byte]
# and a default parameter to compute_domain
previous_version*: array[4, byte]
current_version*: array[4, byte]
@ -554,6 +554,9 @@ template overlaps*(a, b: BitList): bool = overlaps(BitSeq(a), BitSeq(b))
template combine*(a: var BitList, b: BitList) = combine(BitSeq(a), BitSeq(b))
template isSubsetOf*(a, b: BitList): bool = isSubsetOf(BitSeq(a), BitSeq(b))
template `$`*(a: BitList): string = $(BitSeq(a))
iterator items*(x: BitList): bool =
for i in 0 ..< x.len:
yield x[i]
when useListType:
template len*[T; N](x: List[T, N]): auto = len(seq[T](x))

View File

@ -46,14 +46,18 @@ func eth2hash*(v: openArray[byte]): Eth2Digest {.inline.} =
ctx.update(v)
ctx.finish()
proc update*(ctx: var Sha2Context; digest: Eth2Digest) =
ctx.update digest.data
template withEth2Hash*(body: untyped): Eth2Digest =
## This little helper will init the hash function and return the sliced
## hash:
## let hashOfData = withHash: h.update(data)
var h {.inject.}: sha256
h.init()
var h {.inject.}: sha256
init(h)
body
h.finish()
var res = finish(h)
res
func hash*(x: Eth2Digest): Hash =
## Hash for digests for Nim hash tables

View File

@ -145,7 +145,7 @@ func int_to_bytes4*(x: uint64): array[4, byte] =
result[3] = ((x shr 24) and 0xff).byte
# https://github.com/ethereum/eth2.0-specs/blob/v0.8.3/specs/core/0_beacon-chain.md#compute_domain
func compute_domain(domain_type: DomainType, fork_version: array[4, byte]):
func compute_domain*(domain_type: DomainType, fork_version: array[4, byte]):
uint64 =
var buf: array[8, byte]
buf[0..3] = int_to_bytes4(domain_type.uint64)

View File

@ -92,7 +92,7 @@ const
# Unchanged
MIN_VALIDATOR_WITHDRAWABILITY_DELAY* = 2'u64^8
PERSISTENT_COMMITTEE_PERIOD* = 2'u64^11
MAX_EPOCHS_PER_CROSSLINK* = 2'u64^6
MAX_EPOCHS_PER_CROSSLINK* = 4
MIN_EPOCHS_TO_INACTIVITY_PENALTY* = 2'u64^2
# State list lengths

View File

@ -39,7 +39,7 @@ import # TODO - cleanup imports
beaconstate, crypto, datatypes, digest, helpers, validator
# https://github.com/ethereum/eth2.0-specs/blob/v0.8.3/specs/core/0_beacon-chain.md#get_total_active_balance
func get_total_active_balance(state: BeaconState): Gwei =
func get_total_active_balance*(state: BeaconState): Gwei =
return get_total_balance(
state,
get_active_validator_indices(state, get_current_epoch(state)))
@ -187,7 +187,7 @@ func get_winning_crosslink_and_attesting_indices(
get_unslashed_attesting_indices(state, winning_attestations, stateCache))
# https://github.com/ethereum/eth2.0-specs/blob/v0.8.3/specs/core/0_beacon-chain.md#justification-and-finalization
proc process_justification_and_finalization(
proc process_justification_and_finalization*(
state: var BeaconState, stateCache: var StateCache) =
if get_current_epoch(state) <= GENESIS_EPOCH + 1:
return
@ -284,7 +284,7 @@ proc process_justification_and_finalization(
state.finalized_checkpoint = old_current_justified_checkpoint
# https://github.com/ethereum/eth2.0-specs/blob/v0.8.3/specs/core/0_beacon-chain.md#crosslinks
func process_crosslinks(state: var BeaconState, stateCache: var StateCache) =
func process_crosslinks*(state: var BeaconState, stateCache: var StateCache) =
state.previous_crosslinks = state.current_crosslinks
for epoch in @[get_previous_epoch(state), get_current_epoch(state)]:
@ -403,7 +403,7 @@ func get_attestation_deltas(state: BeaconState, stateCache: var StateCache):
(rewards, penalties)
# https://github.com/ethereum/eth2.0-specs/blob/v0.8.3/specs/core/0_beacon-chain.md#rewards-and-penalties-1
func get_crosslink_deltas(state: BeaconState, cache: var StateCache):
func get_crosslink_deltas*(state: BeaconState, cache: var StateCache):
tuple[a: seq[Gwei], b: seq[Gwei]] =
var

View File

@ -434,13 +434,13 @@ func merkelizeSerializedChunks(merkelizer: SszChunksMerkelizer,
func merkelizeSerializedChunks(obj: auto): Eth2Digest =
merkelizeSerializedChunks(SszChunksMerkelizer(), obj)
func hashTreeRoot*(x: auto): Eth2Digest {.gcsafe.}
func hash_tree_root*(x: auto): Eth2Digest {.gcsafe.}
template merkelizeFields(body: untyped): Eth2Digest {.dirty.} =
var merkelizer {.inject.} = SszChunksMerkelizer()
template addField(field) =
let hash = hashTreeRoot(field)
let hash = hash_tree_root(field)
trs "MERKLEIZING FIELD ", astToStr(field), " = ", hash
addChunk(merkelizer, hash.data)
trs "CHUNK ADDED"
@ -546,7 +546,7 @@ func maxChunksCount(T: type, maxLen: static int64): int64 {.compileTime.} =
else:
unsupported T # This should never happen
func hashTreeRoot*(x: auto): Eth2Digest =
func hash_tree_root*(x: auto): Eth2Digest =
trs "STARTING HASH TREE ROOT FOR TYPE ", name(type(x))
mixin toSszType
when x is TypeWithMaxLen:
@ -563,7 +563,7 @@ func hashTreeRoot*(x: auto): Eth2Digest =
merkelizeSerializedChunks(merkelizer, valueOf(x))
else:
for elem in valueOf(x):
let elemHash = hashTreeRoot(elem)
let elemHash = hash_tree_root(elem)
merkelizer.addChunk(elemHash.data)
merkelizer.getFinalHash()
result = mixInLength(contentsHash, valueOf(x).len)
@ -587,4 +587,3 @@ func signingRoot*(obj: object): Eth2Digest =
obj.enumInstanceSerializedFields(fieldName, field):
when fieldName != lastField:
addField2 field

View File

@ -40,7 +40,7 @@ import
# ---------------------------------------------------------------
# https://github.com/ethereum/eth2.0-specs/blob/v0.8.3/specs/core/0_beacon-chain.md#beacon-chain-state-transition-function
func process_slot(state: var BeaconState) =
func process_slot*(state: var BeaconState) =
# Cache state root
let previous_state_root = hash_tree_root(state)
state.state_roots[state.slot mod SLOTS_PER_HISTORICAL_ROOT] =
@ -126,6 +126,8 @@ proc state_transition*(
# state we arrive at is what the block producer thought it would be -
# meaning that potentially, it could fail verification
if skipValidation in flags or verifyStateRoot(state, blck):
# TODO: allow skipping just verifyStateRoot for mocking
# instead of both processBlock and verifyStateRoot
# State root is what it should be - we're done!
return true

View File

@ -17,6 +17,12 @@ import # Unit test
./test_sync_protocol
# ./test_validator # Empty!
import # Refactor state transition unit tests
./spec_block_processing/test_genesis,
./spec_block_processing/test_process_deposits,
./spec_block_processing/test_process_attestation,
./spec_epoch_processing/test_process_crosslinks
import # Official fixtures
./official/test_fixture_shuffling,
./official/test_fixture_bls,

View File

@ -0,0 +1,14 @@
# beacon_chain
# Copyright (c) 2018-2019 Status Research & Development GmbH
# Licensed and distributed under either of
# * MIT license (license terms in the root directory or at http://opensource.org/licenses/MIT).
# * Apache v2 license (license terms in the root directory or at http://www.apache.org/licenses/LICENSE-2.0).
# at your option. This file may not be copied, modified, or distributed except according to those terms.
import
../../beacon_chain/spec/digest
proc `*`*(a: static array[1, byte], n: static int): static Eth2Digest =
assert n == 32
for mbyte in result.data.mitems:
mbyte = a[0]

View File

@ -0,0 +1,14 @@
# beacon_chain
# Copyright (c) 2018-2019 Status Research & Development GmbH
# Licensed and distributed under either of
# * MIT license (license terms in the root directory or at http://opensource.org/licenses/MIT).
# * Apache v2 license (license terms in the root directory or at http://www.apache.org/licenses/LICENSE-2.0).
# at your option. This file may not be copied, modified, or distributed except according to those terms.
func round_multiple_down*(x: uint64, n: uint64): uint64 {.inline.} =
## Round the input to the previous multiple of "n"
result = x - x mod n
func round_multiple_up*(x: uint64, n: uint64): uint64 {.inline.} =
## Round the input to the next multiple of "n"
result = ((x + n - 1) div n) * n

View File

@ -0,0 +1,187 @@
# beacon_chain
# Copyright (c) 2018-2019 Status Research & Development GmbH
# Licensed and distributed under either of
# * MIT license (license terms in the root directory or at http://opensource.org/licenses/MIT).
# * Apache v2 license (license terms in the root directory or at http://www.apache.org/licenses/LICENSE-2.0).
# at your option. This file may not be copied, modified, or distributed except according to those terms.
# Merkle tree helpers
# ---------------------------------------------------------------
import
# Specs
../../beacon_chain/spec/[datatypes, digest, beaconstate, helpers],
../../beacon_chain/ssz,
# shims
stew/objects
func round_step_down*(x: Natural, step: static Natural): int {.inline.} =
## Round the input to the previous multiple of "step"
when (step and (step - 1)) == 0:
# Step is a power of 2. (If compiler cannot prove that x>0 it does not make the optim)
result = x and not(step - 1)
else:
result = x - x mod step
let ZeroHashes = block:
# hashes for a merkle tree full of zeros for leafs
var zh = @[Eth2Digest()]
for i in 1 ..< DEPOSIT_CONTRACT_TREE_DEPTH:
let nodehash = withEth2Hash:
h.update zh[i-1]
h.update zh[i-1]
zh.add nodehash
zh
type SparseMerkleTree*[Depth: static int] = object
## Sparse Merkle tree
# There is an extra "depth" layer to store leaf nodes
# This stores leaves at depth = 0
# and the root hash at the last depth
nnznodes: array[Depth+1, seq[Eth2Digest]] # nodes that leads to non-zero leaves
proc merkleTreeFromLeaves*(
values: openarray[Eth2Digest],
Depth: static[int] = DEPOSIT_CONTRACT_TREE_DEPTH
): SparseMerkleTree[Depth] =
## Depth should be the same as
## verify_merkle_branch / is_valid_merkle_branch
result.nnznodes[0] = @values
for depth in 1 .. Depth: # Inclusive range
let prev_depth_len = result.nnznodes[depth-1].len
let stop = round_step_down(prev_depth_len, 2)
for i in countup(0, stop-1, 2):
# hash by pair of previous nodes
let nodeHash = withEth2Hash:
h.update result.nnznodes[depth-1][i]
h.update result.nnznodes[depth-1][i+1]
result.nnznodes[depth].add nodeHash
if prev_depth_len != stop:
# If length is odd, the last one was skipped,
# we need to combine it
# with the zeroHash corresponding to the current depth
let nodeHash = withEth2Hash:
h.update result.nnznodes[depth-1][^1]
h.update ZeroHashes[depth-1]
result.nnznodes[depth].add nodeHash
proc getMerkleProof*[Depth: static int](
tree: SparseMerkleTree[Depth],
index: int,
): array[Depth, Eth2Digest] =
# Descend down the tree according to the bit representation
# of the index:
# - 0 --> go left
# - 1 --> go right
let path = uint32(index)
for depth in 0 ..< Depth:
let nodeIdx = int((path shr depth) xor 1)
if nodeIdx < tree.nnznodes[depth].len:
result[depth] = tree.nnznodes[depth][nodeIdx]
else:
result[depth] = ZeroHashes[depth]
when isMainModule: # Checks
import strutils, macros, bitops
proc toDigest[N: static int](x: array[N, byte]): Eth2Digest =
result.data[0 .. N-1] = x
let a = [byte 0x01, 0x02, 0x03].toDigest
let b = [byte 0x04, 0x05, 0x06].toDigest
let c = [byte 0x07, 0x08, 0x09].toDigest
block: # SSZ Sanity checks vs Python impl
block: # 3 leaves
let leaves = sszList(@[a, b, c], 3'i64)
let root = hash_tree_root(leaves)
doAssert $root == "9ff412e827b7c9d40fc7df2725021fd579ab762581d1ff5c270316682868456e".toUpperAscii
block: # 2^3 leaves
let leaves = sszList(@[a, b, c], int64(1 shl 3))
let root = hash_tree_root(leaves)
doAssert $root == "5248085b588fab1dd1e03f3cd62201602b12e6560665935964f46e805977e8c5".toUpperAscii
block: # 2^10 leaves
let leaves = sszList(@[a, b, c], int64(1 shl 10))
let root = hash_tree_root(leaves)
doAssert $root == "9fb7d518368dc14e8cc588fb3fd2749beef9f493fef70ae34af5721543c67173".toUpperAscii
block: # Round-trips
# TODO: there is an issue (also in EF specs?)
# using hash_tree_root([a, b, c])
# doesn't give the same hash as
# - hash_tree_root(@[a, b, c])
# - sszList(@[a, b, c], int64(nleaves))
# which both have the same hash.
#
# hash_tree_root([a, b, c]) gives the same hash as
# the last hash of merkleTreeFromLeaves
#
# Running tests with hash_tree_root([a, b, c])
# works for depth 2 (3 or 4 leaves)
when false:
macro roundTrips(): untyped =
result = newStmtList()
# Unsure why sszList ident is undeclared in "quote do"
let list = bindSym"sszList"
# compile-time unrolled test
for nleaves in [3, 4, 5, 7, 8, 1 shl 10, 1 shl 32]:
let depth = fastLog2(nleaves-1) + 1
result.add quote do:
block:
let tree = merkleTreeFromLeaves([a, b, c], Depth = `depth`)
echo "Tree: ", tree
let leaves = `list`(@[a, b, c], int64(`nleaves`))
let root = hash_tree_root(leaves)
echo "Root: ", root
block: # proof for a
let index = 0
let proof = getMerkleProof(tree, index)
echo "Proof: ", proof
doAssert verify_merkle_branch(
a, get_merkle_proof(tree, index = index),
depth = `depth`,
index = index.uint64,
root = root
), "Failed (depth: " & $`depth` &
", nleaves: " & $`nleaves` & ')'
block: # proof for b
let index = 1
let proof = getMerkleProof(tree, index)
# echo "Proof: ", proof
doAssert verify_merkle_branch(
b, get_merkle_proof(tree, index = index),
depth = `depth`,
index = index.uint64,
root = root
), "Failed (depth: " & $`depth` &
", nleaves: " & $`nleaves` & ')'
block: # proof for c
let index = 2
let proof = getMerkleProof(tree, index)
# echo "Proof: ", proof
doAssert verify_merkle_branch(
c, get_merkle_proof(tree, index = index),
depth = `depth`,
index = index.uint64,
root = root
), "Failed (depth: " & $`depth` &
", nleaves: " & $`nleaves` & ')'
roundTrips()

View File

@ -0,0 +1,171 @@
# beacon_chain
# Copyright (c) 2018-2019 Status Research & Development GmbH
# Licensed and distributed under either of
# * MIT license (license terms in the root directory or at http://opensource.org/licenses/MIT).
# * Apache v2 license (license terms in the root directory or at http://www.apache.org/licenses/LICENSE-2.0).
# at your option. This file may not be copied, modified, or distributed except according to those terms.
# Mocking attestations
# ---------------------------------------------------------------
import
# Standard library
sets,
# 0.19.6 shims
# Specs
../../beacon_chain/spec/[datatypes, beaconstate, helpers, validator, crypto],
# Internals
../../beacon_chain/[ssz, extras, state_transition],
# Mocking procs
./mock_blocks,
./mock_validator_keys
proc mockAttestationData(
state: BeaconState,
slot: Slot,
shard: Shard): AttestationData =
doAssert state.slot >= slot
if slot == state.slot:
result.beacon_block_root = mockBlockForNextSlot(state).parent_root
else:
result.beacon_block_root = get_block_root_at_slot(state, slot)
let current_epoch_start_slot = state.get_current_epoch().compute_start_slot_of_epoch()
let epoch_boundary_root = block:
if slot < current_epoch_start_slot:
get_block_root(state, get_previous_epoch(state))
elif slot == current_epoch_start_slot:
result.beacon_block_root
else:
get_block_root(state, get_current_epoch(state))
if slot < current_epoch_start_slot:
result.source = state.previous_justified_checkpoint
else:
result.source = state.current_justified_checkpoint
let target_epoch = compute_epoch_of_slot(slot)
let parent_crosslink = block:
if target_epoch == get_current_epoch(state):
state.current_crosslinks[shard]
else:
state.previous_crosslinks[shard]
result.target = Checkpoint(
epoch: target_epoch, root: epoch_boundary_root
)
result.crosslink = Crosslink(
shard: shard,
start_epoch: parent_crosslink.end_epoch,
end_epoch: min(target_epoch, parent_crosslink.end_epoch + MAX_EPOCHS_PER_CROSSLINK),
parent_root: hash_tree_root(parent_crosslink)
)
proc get_attestation_signature(
state: BeaconState,
attestation_data: AttestationData,
privkey: ValidatorPrivKey
): ValidatorSig =
let msg = AttestationDataAndCustodyBit(
data: attestation_data,
custody_bit: false
).hash_tree_root()
return bls_sign(
key = privkey,
msg = msg.data,
domain = get_domain(
state = state,
domain_type = DOMAIN_ATTESTATION,
message_epoch = attestation_data.target.epoch
)
)
proc signMockAttestation*(state: BeaconState, attestation: var Attestation) =
var cache = get_empty_per_epoch_cache()
let participants = get_attesting_indices(
state,
attestation.data,
attestation.aggregation_bits,
cache
)
var first_iter = true # Can't do while loop on hashset
for validator_index in participants:
let sig = get_attestation_signature(
state, attestation.data, MockPrivKeys[validator_index]
)
if first_iter:
attestation.signature = sig
first_iter = false
else:
combine(attestation.signature, sig)
proc mockAttestationImpl(
state: BeaconState,
slot: Slot,
flags: UpdateFlags): Attestation =
var cache = get_empty_per_epoch_cache()
let
epoch = compute_epoch_of_slot(slot)
epoch_start_shard = get_start_shard(state, epoch)
committees_per_slot = get_committee_count(state, epoch) div SLOTS_PER_EPOCH
shard = (
epoch_start_shard +
committees_per_slot * (slot mod SLOTS_PER_EPOCH)
) mod SHARD_COUNT
crosslink_committee = get_crosslink_committee(
state,
result.data.target.epoch,
result.data.crosslink.shard,
cache
)
committee_size = crosslink_committee.len
result.data = mockAttestationData(state, slot, shard)
result.aggregation_bits = init(CommitteeValidatorsBits, committee_size)
result.custody_bits = init(CommitteeValidatorsBits, committee_size)
# fillAggregateAttestation
for i in 0 ..< crosslink_committee.len:
result.aggregation_bits[i] = true
if skipValidation notin flags:
signMockAttestation(state, result)
proc mockAttestation*(
state: BeaconState,
flags: UpdateFlags = {}): Attestation {.inline.}=
mockAttestationImpl(state, state.slot, flags)
proc mockAttestation*(
state: BeaconState,
slot: Slot,
flags: UpdateFlags = {}): Attestation {.inline.}=
mockAttestationImpl(state, slot, flags)
proc fillAggregateAttestation*(state: BeaconState, attestation: var Attestation) =
var cache = get_empty_per_epoch_cache()
let crosslink_committee = get_crosslink_committee(
state,
attestation.data.target.epoch,
attestation.data.crosslink.shard,
cache
)
for i in 0 ..< crosslink_committee.len:
attestation.aggregation_bits[i] = true
proc add*(state: var BeaconState, attestation: Attestation, slot: Slot) =
var blck = mockBlockForNextSlot(state)
blck.slot = slot
blck.body.attestations.add attestation
process_slots(state, slot)
signMockBlock(state, blck)
# TODO: we can skip just VerifyStateRoot
doAssert state_transition(state, blck, flags = {skipValidation})

View File

@ -0,0 +1,105 @@
# beacon_chain
# Copyright (c) 2018-2019 Status Research & Development GmbH
# Licensed and distributed under either of
# * MIT license (license terms in the root directory or at http://opensource.org/licenses/MIT).
# * Apache v2 license (license terms in the root directory or at http://www.apache.org/licenses/LICENSE-2.0).
# at your option. This file may not be copied, modified, or distributed except according to those terms.
import
# Specs
../../beacon_chain/spec/[datatypes, crypto, helpers, validator],
# Internals
../../beacon_chain/[ssz, extras, state_transition],
# Mock helpers
./mock_validator_keys
# Routines for mocking blocks
# ---------------------------------------------------------------
proc signMockBlockImpl(
state: BeaconState,
blck: var BeaconBlock,
proposer_index: ValidatorIndex
) =
doAssert state.slot <= blck.slot
let privkey = MockPrivKeys[proposer_index]
blck.body.randao_reveal = bls_sign(
key = privkey,
msg = blck.slot
.compute_epoch_of_slot()
.hash_tree_root()
.data,
domain = get_domain(
state,
DOMAIN_RANDAO,
message_epoch = blck.slot.compute_epoch_of_slot(),
)
)
blck.signature = bls_sign(
key = privkey,
msg = blck.signing_root().data,
domain = get_domain(
state,
DOMAIN_BEACON_PROPOSER,
message_epoch = blck.slot.compute_epoch_of_slot(),
)
)
proc signMockBlock*(
state: BeaconState,
blck: var BeaconBlock,
proposer_index: ValidatorIndex
) =
signMockBlockImpl(state, blck, proposer_index)
proc signMockBlock*(
state: BeaconState,
blck: var BeaconBlock
) =
var proposer_index: ValidatorIndex
var emptyCache = get_empty_per_epoch_cache()
if blck.slot == state.slot:
proposer_index = get_beacon_proposer_index(state, emptyCache)
else:
# Stub to get proposer index of future slot
# Note: this relies on ``let`` deep-copying the state
# i.e. BeaconState should have value semantics
# and not contain ref objects or pointers
var stubState = state
process_slots(stub_state, blck.slot)
proposer_index = get_beacon_proposer_index(stub_state, emptyCache)
signMockBlockImpl(state, blck, proposer_index)
proc mockBlock*(
state: BeaconState,
slot: Slot,
flags: UpdateFlags = {}): BeaconBlock =
## Mock a BeaconBlock for the specific slot
## Add skipValidation if block should not be signed
result.slot = slot
result.body.eth1_data.deposit_count = state.eth1_deposit_index
var previous_block_header = state.latest_block_header
if previous_block_header.state_root == ZERO_HASH:
previous_block_header.state_root = state.hash_tree_root()
result.parent_root = previous_block_header.signing_root()
if skipValidation notin flags:
signMockBlock(state, result)
proc mockBlockForNextSlot*(state: BeaconState, flags: UpdateFlags = {}): BeaconBlock =
mockBlock(state, state.slot + 1, flags)
proc applyEmptyBlock*(state: var BeaconState) =
## Do a state transition with an empty signed block
## on the current slot
let blck = mockBlock(state, state.slot, flags = {})
# TODO: we only need to skip verifyStateRoot validation
# processBlock validation should work
doAssert state_transition(state, blck, {skipValidation})

View File

@ -0,0 +1,222 @@
# beacon_chain
# Copyright (c) 2018-2019 Status Research & Development GmbH
# Licensed and distributed under either of
# * MIT license (license terms in the root directory or at http://opensource.org/licenses/MIT).
# * Apache v2 license (license terms in the root directory or at http://www.apache.org/licenses/LICENSE-2.0).
# at your option. This file may not be copied, modified, or distributed except according to those terms.
# Mocking deposits and genesis deposits
# ---------------------------------------------------------------
import
# Standard library
math, random,
# 0.19.6 shims
stew/objects, # import default
# Specs
../../beacon_chain/spec/[datatypes, crypto, helpers, digest, beaconstate],
# Internals
../../beacon_chain/[ssz, extras],
# Mocking procs
./merkle_minimal, ./mock_validator_keys
func signMockDepositData(
deposit_data: var DepositData,
privkey: ValidatorPrivKey
) =
# No state --> Genesis
deposit_data.signature = bls_sign(
key = privkey,
msg = deposit_data.signing_root().data,
domain = compute_domain(
DOMAIN_DEPOSIT,
default(array[4, byte]) # Genesis is fork_version 0
)
)
func signMockDepositData(
deposit_data: var DepositData,
privkey: ValidatorPrivKey,
state: BeaconState
) =
deposit_data.signature = bls_sign(
key = privkey,
msg = deposit_data.signing_root().data,
domain = get_domain(
state,
DOMAIN_DEPOSIT
)
)
func mockDepositData(
deposit_data: var DepositData,
pubkey: ValidatorPubKey,
amount: uint64,
# withdrawal_credentials: Eth2Digest
) =
deposit_data.pubkey = pubkey
deposit_data.amount = amount
# Insecurely use pubkey as withdrawal key
deposit_data.withdrawal_credentials.data[0] = byte BLS_WITHDRAWAL_PREFIX
deposit_data.withdrawal_credentials.data[1..^1] = pubkey.getBytes()
.eth2hash()
.data
.toOpenArray(1, 31)
func mockDepositData(
deposit_data: var DepositData,
pubkey: ValidatorPubKey,
privkey: ValidatorPrivKey,
amount: uint64,
# withdrawal_credentials: Eth2Digest,
flags: UpdateFlags = {}
) =
mockDepositData(deposit_data, pubkey, amount)
if skipValidation notin flags:
signMockDepositData(deposit_data, privkey)
func mockDepositData(
deposit_data: var DepositData,
pubkey: ValidatorPubKey,
privkey: ValidatorPrivKey,
amount: uint64,
# withdrawal_credentials: Eth2Digest,
state: BeaconState,
flags: UpdateFlags = {}
) =
mockDepositData(deposit_data, pubkey, amount)
if skipValidation notin flags:
signMockDepositData(deposit_data, privkey, state)
template mockGenesisDepositsImpl(
result: seq[Deposit],
validatorCount: uint64,
amount: untyped,
flags: UpdateFlags = {},
updateAmount: untyped,
) =
# Genesis deposits with varying amounts
if skipValidation in flags:
# 1st loop - build deposit data
for valIdx in 0 ..< validatorCount.int:
# Directly build the Deposit in-place for speed
result.setLen(valIdx + 1)
updateAmount
# DepositData
mockDepositData(
result[valIdx].data,
MockPubKeys[valIdx],
amount
)
else: # With signing
var depositsDataHash: seq[Eth2Digest]
var depositsData: seq[DepositData]
# 1st loop - build deposit data
for valIdx in 0 ..< validatorCount.int:
# Directly build the Deposit in-place for speed
result.setLen(valIdx + 1)
updateAmount
# DepositData
mockDepositData(
result[valIdx].data,
MockPubKeys[valIdx],
MockPrivKeys[valIdx],
amount,
flags
)
depositsData.add result[valIdx].data
depositsDataHash.add hash_tree_root(result[valIdx].data)
# 2nd & 3rd loops - build hashes and proofs
let root = hash_tree_root(depositsData)
let tree = merkleTreeFromLeaves(depositsDataHash)
# 4th loop - append proof
for valIdx in 0 ..< validatorCount.int:
when false: # TODO
result[valIdx].proof[0..31] = tree.getMerkleProof(valIdx)
result[valIdx].proof[32] = int_to_bytes32(index + 1)
doAssert:
verify_merkle_branch(
depositsDataHash[valIdx],
result[valIdx].proof,
DEPOSIT_CONTRACT_TREE_DEPTH,
valIdx,
root
)
proc mockGenesisBalancedDeposits*(
validatorCount: uint64,
amountInEth: Positive,
flags: UpdateFlags = {}
): seq[Deposit] =
## The amount should be strictly positive
## - 1 is the minimum deposit amount (MIN_DEPOSIT_AMOUNT)
## - 16 is the ejection balance (EJECTION_BALANCE)
## - 32 is the max effective balance (MAX_EFFECTIVE_BALANCE)
## ETH beyond do not contribute more for staking.
##
## Only validators with 32 ETH will be active at genesis
let amount = amountInEth.uint64 * 10'u64^9
mockGenesisDepositsImpl(result, validatorCount,amount,flags):
discard
proc mockGenesisUnBalancedDeposits*(
validatorCount: uint64,
amountRangeInEth: Slice[int], # TODO: use "Positive", Nim range bug
flags: UpdateFlags = {}
): seq[Deposit] =
## The range of deposit amount should be strictly positive
## - 1 is the minimum deposit amount (MIN_DEPOSIT_AMOUNT)
## - 16 is the ejection balance (EJECTION_BALANCE)
## - 32 is the max effective balance (MAX_EFFECTIVE_BALANCE)
## ETH beyond do not contribute more for staking.
##
## Only validators with 32 ETH will be active at genesis
var rng {.global.} = initRand(0x42) # Fixed seed for reproducibility
var amount: uint64
mockGenesisDepositsImpl(result, validatorCount, amount, flags):
amount = rng.rand(amountRangeInEth).uint64 * 10'u64^9
proc mockUpdateStateForNewDeposit*(
state: var BeaconState,
validator_index: uint64,
amount: uint64,
# withdrawal_credentials: Eth2Digest
flags: UpdateFlags
): Deposit =
# TODO withdrawal credentials
mockDepositData(
result.data,
MockPubKeys[validator_index],
MockPrivKeys[validator_index],
amount,
# withdrawal_credentials: Eth2Digest
flags
)
let tree = merkleTreeFromLeaves([hash_tree_root(result.data)])
when false: # TODO
result[valIdx].proof[0..31] = tree.getMerkleProof(0)
result[valIdx].proof[32] = int_to_bytes32(0 + 1)
# doAssert: verify_merkle_branch(...)
# TODO: this logic from the eth2.0-specs test suite seems strange
# but confirmed by running it
state.eth1_deposit_index = 0
state.eth1_data.deposit_root = hash_tree_root(result.data)
state.eth1_data.deposit_count = 1

View File

@ -0,0 +1,49 @@
# beacon_chain
# Copyright (c) 2018-2019 Status Research & Development GmbH
# Licensed and distributed under either of
# * MIT license (license terms in the root directory or at http://opensource.org/licenses/MIT).
# * Apache v2 license (license terms in the root directory or at http://www.apache.org/licenses/LICENSE-2.0).
# at your option. This file may not be copied, modified, or distributed except according to those terms.
# Mocking a genesis state
# ---------------------------------------------------------------
import
# Specs
../../beacon_chain/spec/[datatypes, beaconstate, digest],
# Internals
../../beacon_chain/extras,
# Mocking procs
./mock_deposits,
# Helpers
../helpers/digest_helpers
proc initGenesisState*(num_validators: uint64, genesis_time: uint64 = 0): BeaconState =
# EF magic number (similar to https://en.wikipedia.org/wiki/Magic_number_(programming))
const deposit_root = [byte 0x42] * 32
let eth1_data = Eth1Data(
deposit_root: deposit_root,
deposit_count: num_validators,
block_hash: ZERO_HASH
)
let deposits = mockGenesisBalancedDeposits(
validatorCount = num_validators,
amountInEth = 32, # We create canonical validators with 32 Eth
flags = {skipValidation}
)
result = initialize_beacon_state_from_eth1(
genesis_validator_deposits = deposits,
genesis_time = 0,
genesis_eth1_data = eth1_data,
flags = {skipValidation}
)
when isMainModule:
# Smoke test
let state = initGenesisState(num_validators = SLOTS_PER_EPOCH)
doAssert state.validators.len == SLOTS_PER_EPOCH

View File

@ -0,0 +1,24 @@
# beacon_chain
# Copyright (c) 2018-2019 Status Research & Development GmbH
# Licensed and distributed under either of
# * MIT license (license terms in the root directory or at http://opensource.org/licenses/MIT).
# * Apache v2 license (license terms in the root directory or at http://www.apache.org/licenses/LICENSE-2.0).
# at your option. This file may not be copied, modified, or distributed except according to those terms.
# Mocking helpers for BeaconState
# ---------------------------------------------------------------
import
# Specs
../../beacon_chain/spec/[datatypes],
# Internals
../../beacon_chain/state_transition
proc nextEpoch*(state: var BeaconState) =
## Transition to the start of the next epoch
let slot = state.slot + SLOTS_PER_EPOCH - (state.slot mod SLOTS_PER_EPOCH)
process_slots(state, slot)
proc nextSlot*(state: var BeaconState) =
## Transition to the next slot
process_slots(state, state.slot + 1)

View File

@ -0,0 +1,30 @@
# beacon_chain
# Copyright (c) 2018-2019 Status Research & Development GmbH
# Licensed and distributed under either of
# * MIT license (license terms in the root directory or at http://opensource.org/licenses/MIT).
# * Apache v2 license (license terms in the root directory or at http://www.apache.org/licenses/LICENSE-2.0).
# at your option. This file may not be copied, modified, or distributed except according to those terms.
# Mocking validator public and private keys
# ---------------------------------------------------------------
import
# Specs
../../beacon_chain/spec/[datatypes, crypto]
let MockPrivKeys* = block:
var privkeys: array[SLOTS_PER_EPOCH * 16, ValidatorPrivKey]
for pk in privkeys.mitems():
pk = newPrivKey()
privkeys
let MockPubKeys* = block:
var pubkeys: array[SLOTS_PER_EPOCH * 16, ValidatorPubKey]
for idx, privkey in MockPrivKeys:
pubkeys[idx] = pubkey(privkey)
pubkeys
type MockKey = ValidatorPrivKey or ValidatorPubKey
template `[]`*[N: static int](a: array[N, MockKey], idx: ValidatorIndex): MockKey =
a[idx.int]

View File

@ -108,7 +108,7 @@ proc testerImpl[T](path: string, sszTest: SszStaticTest) {.cdecl, gcsafe.} =
sszTest.expectedBytes
execTest "root hash check",
hashTreeRoot(obj.obj[]),
hash_tree_root(obj.obj[]),
sszTest.expectedRootHash
when hasSigningRoot(T):
@ -213,4 +213,3 @@ for kind, path in walkDir(testsDir):
if kind notin {pcFile, pcLinkToFile}: continue
if const_preset in path:
executeSuite path

View File

@ -0,0 +1,56 @@
# beacon_chain
# Copyright (c) 2018 Status Research & Development GmbH
# Licensed and distributed under either of
# * MIT license (license terms in the root directory or at http://opensource.org/licenses/MIT).
# * Apache v2 license (license terms in the root directory or at http://www.apache.org/licenses/LICENSE-2.0).
# at your option. This file may not be copied, modified, or distributed except according to those terms.
# initialize_beacon_state_from_eth1 (beaconstate.nim)
# https://github.com/ethereum/eth2.0-specs/blob/v0.8.1/specs/core/0_beacon-chain.md#genesis
# ---------------------------------------------------------------
import
# Standard library
unittest,
# Specs
../../beacon_chain/spec/[beaconstate, datatypes, helpers, validator, digest],
# Internals
../../beacon_chain/[ssz, extras, state_transition],
# Mock helpers
../mocking/[mock_deposits, mock_genesis],
../testutil
# TODO:
# - MIN_GENESIS_ACTIVE_VALIDATOR_COUNT is not implemented
# - MIN_GENESIS_TIME is not implemented
# - is_valid_genesis_state is not implemented
suite "[Unit - Spec - Genesis] Genesis block checks " & preset():
test "is_valid_genesis_state for a valid state":
let state = initGenesisState(
num_validators = MIN_GENESIS_ACTIVE_VALIDATOR_COUNT,
genesis_time = MIN_GENESIS_TIME
)
discard "TODO"
test "Invalid genesis time":
let state = initGenesisState(
num_validators = MIN_GENESIS_ACTIVE_VALIDATOR_COUNT,
genesis_time = MIN_GENESIS_TIME.uint64 - 1
)
discard "TODO"
test "Not enough validators":
let state = initGenesisState(
num_validators = MIN_GENESIS_ACTIVE_VALIDATOR_COUNT.uint64 - 1,
genesis_time = MIN_GENESIS_TIME.uint64 - 1
)
discard "TODO"
test "Validators with more than 32 ETH":
discard "TODO"
test "More validators than minimum":
discard "TODO"

View File

@ -0,0 +1,117 @@
# beacon_chain
# Copyright (c) 2018 Status Research & Development GmbH
# Licensed and distributed under either of
# * MIT license (license terms in the root directory or at http://opensource.org/licenses/MIT).
# * Apache v2 license (license terms in the root directory or at http://www.apache.org/licenses/LICENSE-2.0).
# at your option. This file may not be copied, modified, or distributed except according to those terms.
# process_attestation (beaconstate.nim)
# https://github.com/ethereum/eth2.0-specs/blob/v0.8.1/specs/core/0_beacon-chain.md#attestations
# ---------------------------------------------------------------
import
# Standard library
unittest, math,
# shims 0.19.6
stew/objects, # import default
# Specs
../../beacon_chain/spec/[beaconstate, datatypes, helpers, validator],
# Internals
../../beacon_chain/[state_transition],
# Mock helpers
../mocking/[mock_genesis, mock_attestations, mock_state, mock_blocks],
../testutil
suite "[Unit - Spec - Block processing] Attestations " & preset():
const NumValidators = uint64(8) * SLOTS_PER_EPOCH
let genesisState = initGenesisState(NumValidators)
doAssert genesisState.validators.len == int NumValidators
template valid_attestation(name: string, body: untyped): untyped {.dirty.}=
# Process a valid attestation
#
# The BeaconState is exposed as "state" in the calling context
# The attestation to process must be named "attestation" in the calling context
test name:
var state{.inject.}: BeaconState
deepCopy(state, genesisState)
# Attestation setup body
# ----------------------------------------
body
# Params for sanity checks
# ----------------------------------------
let
current_epoch_count = state.current_epoch_attestations.len
previous_epoch_count = state.previous_epoch_attestations.len
# State transition
# ----------------------------------------
var cache = get_empty_per_epoch_cache()
check process_attestation(
state, attestation, flags = {}, cache
)
# Check that the attestation was processed
if attestation.data.target.epoch == state.get_current_epoch():
check(state.current_epoch_attestations.len == current_epoch_count + 1)
else:
check(state.previous_epoch_attestations.len == previous_epoch_count + 1)
valid_attestation("Valid attestation"):
let attestation = mockAttestation(state)
state.slot += MIN_ATTESTATION_INCLUSION_DELAY
valid_attestation("Valid attestation from previous epoch"):
let attestation = mockAttestation(state)
state.slot = Slot(SLOTS_PER_EPOCH - 1)
nextEpoch(state)
applyEmptyBlock(state)
when MAX_EPOCHS_PER_CROSSLINK > 4'u64:
test "Valid attestation since max epochs per crosslinks [Skipped for preset: " & const_preset & ']':
discard
else:
valid_attestation("Valid attestation since max epochs per crosslinks"):
for _ in 0 ..< MAX_EPOCHS_PER_CROSSLINK + 2:
nextEpoch(state)
applyEmptyBlock(state)
let attestation = mockAttestation(state)
check: attestation.data.crosslink.end_epoch - attestation.data.crosslink.start_epoch == MAX_EPOCHS_PER_CROSSLINK
for _ in 0 ..< MIN_ATTESTATION_INCLUSION_DELAY:
nextSlot(state)
valid_attestation("Empty aggregation bit"):
var attestation = mockAttestation(state)
state.slot += MIN_ATTESTATION_INCLUSION_DELAY
# Overwrite committee
attestation.aggregation_bits = init(CommitteeValidatorsBits, attestation.aggregation_bits.len)
signMockAttestation(state, attestation)
# TODO - invalid attestations
# - Wrong end epoch
# - Invalid signature
# - Before inclusion delay
# - past last inclusion slot
# - before oldest known source epoch
# - wrong shard
# - invalid shard
# - target epoch too old
# - target epoch too far in the future
# - source epoch in the future
# - invalid current source root
# - bad source root
# - non-zero crosslink data root
# - bad parent crosslink
# - bad crosslink start epoch
# - bad crosslink end epoch
# - inconsistent custody bits length
# - non-empty custody bits in phase 0

View File

@ -0,0 +1,119 @@
# beacon_chain
# Copyright (c) 2018 Status Research & Development GmbH
# Licensed and distributed under either of
# * MIT license (license terms in the root directory or at http://opensource.org/licenses/MIT).
# * Apache v2 license (license terms in the root directory or at http://www.apache.org/licenses/LICENSE-2.0).
# at your option. This file may not be copied, modified, or distributed except according to those terms.
# process_deposit (beaconstate.nim)
# https://github.com/ethereum/eth2.0-specs/blob/v0.8.1/specs/core/0_beacon-chain.md#deposits
# ---------------------------------------------------------------
import
# Standard library
unittest, math,
# Specs
../../beacon_chain/spec/[beaconstate, datatypes, crypto, helpers, validator],
# Internals
../../beacon_chain/[ssz, extras, state_transition],
# Mock helpers
../mocking/[mock_deposits, mock_genesis],
../testutil, ../helpers/math_helpers
suite "[Unit - Spec - Block processing] Deposits " & preset():
const NumValidators = uint64 5 * SLOTS_PER_EPOCH
let genesisState = initGenesisState(NumValidators)
doAssert genesisState.validators.len == int NumValidators
template valid_deposit(deposit_amount: uint64, name: string): untyped =
# TODO: BLS signature
test "Deposit " & name & " MAX_EFFECTIVE_BALANCE balance (" &
$(MAX_EFFECTIVE_BALANCE div 10'u64^9) & " ETH)":
var state: BeaconState
deepCopy(state, genesisState)
# Test configuration
# ----------------------------------------
let validator_index = state.validators.len
let deposit = mockUpdateStateForNewDeposit(
state,
uint64 validator_index,
deposit_amount,
flags = {skipValidation}
)
# Params for sanity checks
# ----------------------------------------
let pre_val_count = state.validators.len
let pre_balance = if validator_index < pre_val_count:
state.balances[validator_index]
else:
0
# State transition
# ----------------------------------------
check: state.process_deposit(deposit, {skipValidation})
# Check invariants
# ----------------------------------------
check:
state.validators.len == pre_val_count + 1
state.balances.len == pre_val_count + 1
state.balances[validator_index] == pre_balance + deposit.data.amount
state.validators[validator_index].effective_balance ==
round_multiple_down(
min(MAX_EFFECTIVE_BALANCE, state.balances[validator_index]),
EFFECTIVE_BALANCE_INCREMENT
)
valid_deposit(MAX_EFFECTIVE_BALANCE - 1, "under")
valid_deposit(MAX_EFFECTIVE_BALANCE, "at")
valid_deposit(MAX_EFFECTIVE_BALANCE + 1, "over")
test "Validator top-up":
var state: BeaconState
deepCopy(state, genesisState)
# Test configuration
# ----------------------------------------
let validator_index = 0
let deposit_amount = MAX_EFFECTIVE_BALANCE div 4
let deposit = mockUpdateStateForNewDeposit(
state,
uint64 validator_index,
deposit_amount,
flags = {skipValidation}
)
# Params for sanity checks
# ----------------------------------------
let pre_val_count = state.validators.len
let pre_balance = if validator_index < pre_val_count:
state.balances[validator_index]
else:
0
# State transition
# ----------------------------------------
check: state.process_deposit(deposit, {skipValidation})
# Check invariants
# ----------------------------------------
check:
state.validators.len == pre_val_count
state.balances.len == pre_val_count
state.balances[validator_index] == pre_balance + deposit.data.amount
state.validators[validator_index].effective_balance ==
round_multiple_down(
min(MAX_EFFECTIVE_BALANCE, state.balances[validator_index]),
EFFECTIVE_BALANCE_INCREMENT
)
# TODO, tests with:
# - invalid BLS signature
# - invalid withdrawal credential
# - invalid deposit root
# - invalid merkle proof

View File

@ -0,0 +1,43 @@
# beacon_chain
# Copyright (c) 2018 Status Research & Development GmbH
# Licensed and distributed under either of
# * MIT license (license terms in the root directory or at http://opensource.org/licenses/MIT).
# * Apache v2 license (license terms in the root directory or at http://www.apache.org/licenses/LICENSE-2.0).
# at your option. This file may not be copied, modified, or distributed except according to those terms.
import
# Specs
../../beacon_chain/spec/[datatypes, state_transition_epoch, validator, helpers],
# Internals
../../beacon_chain/[state_transition]
proc processSlotsUntilEndCurrentEpoch(state: var BeaconState) =
# Process all slots until the end of the last slot of the current epoch
let slot = state.slot + SLOTS_PER_EPOCH - (state.slot mod SLOTS_PER_EPOCH)
# Transition to slot before the epoch state transition
process_slots(state, slot - 1)
# For the last slot of the epoch,
# only process_slot without process_epoch
# (see process_slots())
process_slot(state)
proc transitionEpochUntilJustificationFinalization*(state: var BeaconState) =
# Process slots and do the epoch transition until crosslinks
processSlotsUntilEndCurrentEpoch(state)
# From process_epoch()
var per_epoch_cache = get_empty_per_epoch_cache()
process_justification_and_finalization(state, per_epoch_cache)
proc transitionEpochUntilCrosslinks*(state: var BeaconState) =
# Process slots and do the epoch transition until crosslinks
processSlotsUntilEndCurrentEpoch(state)
# From process_epoch()
var per_epoch_cache = get_empty_per_epoch_cache()
process_justification_and_finalization(state, per_epoch_cache)
process_crosslinks(state, per_epoch_cache)

Binary file not shown.

After

Width:  |  Height:  |  Size: 201 KiB

View File

@ -0,0 +1,105 @@
# beacon_chain
# Copyright (c) 2018 Status Research & Development GmbH
# Licensed and distributed under either of
# * MIT license (license terms in the root directory or at http://opensource.org/licenses/MIT).
# * Apache v2 license (license terms in the root directory or at http://www.apache.org/licenses/LICENSE-2.0).
# at your option. This file may not be copied, modified, or distributed except according to those terms.
import
# Standard library
strformat,
# Specs
../../beacon_chain/spec/[datatypes, state_transition_epoch, validator, helpers],
# Internals
../../beacon_chain/[state_transition],
# Test helpers
../helpers/digest_helpers
# Justification and finalization utils
# ---------------------------------------------------------------
iterator getShardsForSlot(state: BeaconState, slot: Slot): Shard =
let
epoch = compute_epoch_of_slot(slot)
epoch_start_shard = get_start_shard(state, epoch)
committees_per_slot = get_committee_count(state, epoch) div SLOTS_PER_EPOCH
shard = epoch_start_shard + committees_per_slot * (slot mod SLOTS_PER_EPOCH)
for i in 0 ..< committees_per_slot.int:
yield shard + Shard(i)
proc addMockAttestations*(
state: BeaconState, epoch: Epoch,
source, target: Checkpoint,
sufficient_support = false
) =
# We must be at the end of the epoch
doAssert (state.slot + 1) mod SLOTS_PER_EPOCH == 0
var attestations: seq[PendingAttestation]
if state.get_current_epoch() == epoch:
attestations = state.current_epoch_attestations
elif state.get_previous_epoch() == epoch:
attestations = state.previous_epoch_attestations
else:
raise newException(ValueError, &"Cannot include attestations from epoch {state.get_current_epoch()} in epoch {epoch}")
# TODO: Working with an unsigned Gwei balance is a recipe for underflows to happen
var remaining_balance = state.get_total_active_balance().int64 * 2 div 3
let start_slot = compute_start_slot_of_epoch(epoch)
# for-loop of distinct type is broken: https://github.com/nim-lang/Nim/issues/12074
for slot in start_slot.uint64 ..< start_slot.uint64 + SLOTS_PER_EPOCH:
for shard in getShardsForSlot(state, slot.Slot):
# TODO: can we move cache out of the loops
var cache = get_empty_per_epoch_cache()
let committee = get_crosslink_committee(
state, slot.Slot.compute_epoch_of_slot(),
shard, cache
)
# Create a bitfield filled with the given count per attestation,
# exactly on the right-most part of the committee field.
var aggregation_bits = init(CommitteeValidatorsBits, committee.len)
for v in 0 ..< committee.len * 2 div 3 + 1:
if remaining_balance > 0:
# Beware of the underflows, use int
remaining_balance -= state.validators[v].effective_balance.int64
aggregation_bits[v] = true
else:
break
# Remove just one attester to make the marginal support insufficient
if not sufficient_support:
# Find the first attester if any
let idx = aggregation_bits.find(true)
if idx != -1:
aggregation_bits[idx] = false
attestations.add PendingAttestation(
aggregation_bits: aggregation_bits,
data: AttestationData(
beacon_block_root: [byte 0xFF] * 32, # Irrelevant for testing
source: source,
target: target,
crosslink: Crosslink(shard: shard)
),
inclusion_delay: 1
)
proc getCheckpoints*(epoch: Epoch): tuple[c1, c2, c3, c4, c5: Checkpoint] =
if epoch >= 1: result.c1 = Checkpoint(epoch: epoch - 1, root: [byte 0xAA] * 32)
if epoch >= 2: result.c2 = Checkpoint(epoch: epoch - 2, root: [byte 0xBB] * 32)
if epoch >= 3: result.c3 = Checkpoint(epoch: epoch - 3, root: [byte 0xCC] * 32)
if epoch >= 4: result.c4 = Checkpoint(epoch: epoch - 4, root: [byte 0xDD] * 32)
if epoch >= 5: result.c5 = Checkpoint(epoch: epoch - 5, root: [byte 0xEE] * 32)
proc putCheckpointsInBlockRoots*(
state: var BeaconState,
checkpoints: openarray[Checkpoint]) =
for c in checkpoints:
let idx = c.epoch.compute_start_slot_of_epoch() mod SLOTS_PER_HISTORICAL_ROOT
state.block_roots[idx] = c.root

View File

@ -0,0 +1,120 @@
# beacon_chain
# Copyright (c) 2018 Status Research & Development GmbH
# Licensed and distributed under either of
# * MIT license (license terms in the root directory or at http://opensource.org/licenses/MIT).
# * Apache v2 license (license terms in the root directory or at http://www.apache.org/licenses/LICENSE-2.0).
# at your option. This file may not be copied, modified, or distributed except according to those terms.
# process_crosslinks (state_transition_epoch.nim)
# https://github.com/ethereum/eth2.0-specs/blob/v0.8.2/specs/core/0_beacon-chain.md#crosslinks
# ---------------------------------------------------------------
import
# Standard library
unittest,
# Specs
../../beacon_chain/spec/[beaconstate, datatypes, validator, helpers, state_transition_epoch],
# Internals
# Mock helpers
../mocking/[mock_genesis, mock_attestations, mock_state, mock_blocks],
./epoch_utils,
../testutil
suite "[Unit - Spec - Epoch processing] Crosslinks " & preset():
const NumValidators = uint64(8) * SLOTS_PER_EPOCH
let genesisState = initGenesisState(NumValidators)
doAssert genesisState.validators.len == int NumValidators
var state: BeaconState
template resetState: untyped =
deepCopy(state, genesisState)
test "No attestations":
resetState()
transitionEpochUntilCrosslinks(state)
for shard in 0 ..< SHARD_COUNT:
check state.previous_crosslinks[shard] == state.current_crosslinks[shard]
test "Single crosslink update from current epoch":
resetState()
nextEpoch(state)
var attestation = mockAttestation(state)
fillAggregateAttestation(state, attestation)
state.add(attestation, state.slot + MIN_ATTESTATION_INCLUSION_DELAY)
# TODO: all attestations are duplicated at the moment
# pending fix of https://github.com/status-im/nim-beacon-chain/issues/361
check: state.current_epoch_attestations.len == 2
# For sanity checks
let shard = attestation.data.crosslink.shard
let pre_crosslink = state.current_crosslinks[shard]
transitionEpochUntilCrosslinks(state)
check:
state.previous_crosslinks[shard] != state.current_crosslinks[shard]
pre_crosslink != state.current_crosslinks[shard]
test "Double late crosslink":
resetState()
if get_committee_count(state, get_current_epoch(state)) < SHARD_COUNT:
echo " [Warning] Skipping Double-late crosslink test: Committee.len < SHARD_COUNT for preset " & const_preset
else:
nextEpoch(state)
state.slot += 4
var attestation_1 = mockAttestation(state)
fillAggregateAttestation(state, attestation_1)
# Add attestation_1 to next epoch
nextEpoch(state)
state.add(attestation_1, state.slot + 1)
var attestation_2: Attestation
for _ in 0 ..< SLOTS_PER_EPOCH:
attestation_2 = mockAttestation(state)
if attestation_2.data.crosslink.shard == attestation_1.data.crosslink.shard:
signMockAttestation(state, attestation_2)
break
nextSlot(state)
applyEmptyBlock(state)
fillAggregateAttestation(state, attestation_2)
# Add attestation_2 in the next epoch after attestation_1 has already
# updated the relevant crosslink
nextEpoch(state)
state.add(attestation_2, state.slot + 1)
# TODO: all attestations are duplicated at the moment
# pending fix of https://github.com/status-im/nim-beacon-chain/issues/361
check: state.previous_epoch_attestations.len == 2
check: state.current_epoch_attestations.len == 0
var cache = get_empty_per_epoch_cache()
let crosslink_deltas = get_crosslink_deltas(state, cache)
transitionEpochUntilCrosslinks(state)
let shard = attestation_2.data.crosslink.shard
# ensure that the current crosslinks were not updated by the second attestation
check: state.previous_crosslinks[shard] == state.current_crosslinks[shard]
# ensure no reward, only penalties for the failed crosslink
for index in get_crosslink_committee(
state,
attestation_2.data.target.epoch,
attestation_2.data.crosslink.shard,
cache
):
check:
crosslink_deltas[0][index] == 0.Gwei
crosslink_deltas[1][index] > 0.Gwei

View File

@ -0,0 +1,257 @@
# beacon_chain
# Copyright (c) 2018 Status Research & Development GmbH
# Licensed and distributed under either of
# * MIT license (license terms in the root directory or at http://opensource.org/licenses/MIT).
# * Apache v2 license (license terms in the root directory or at http://www.apache.org/licenses/LICENSE-2.0).
# at your option. This file may not be copied, modified, or distributed except according to those terms.
import
# Standard library
strformat, unittest,
# Vendored packages
stew/bitops2,
# Specs
../../beacon_chain/spec/[datatypes, state_transition_epoch, validator, helpers],
# Internals
../../beacon_chain/[state_transition],
# Test helpers
../mocking/[mock_genesis],
./epoch_utils,
./justification_finalization_helpers,
../testutil
# See diagram: eth2-finalization.png
# (source) https://github.com/protolambda/eth2-docs#justification-and-finalization
# for a visualization of finalization rules
proc finalizeOn234(state: var BeaconState, epoch: Epoch, sufficient_support: bool) =
## Check finalization on rule 1 "234"
doAssert epoch > 4
state.slot = Slot((epoch * SLOTS_PER_EPOCH) - 1) # Skip ahead to just before epoch
# 43210 -- epochs ago
# 3210x -- justification bitfields indices
# 11*0. -- justification bitfield contents. . = this epoch, * is being justified now
# checkpoints for epochs ago
let (c1, c2, c3, c4, _) = getCheckpoints(epoch)
putCheckpointsInBlockRoots(state, [c1, c2, c3, c4])
# Save for final checks
let old_finalized = state.finalized_checkpoint
# Mock the state
state.previous_justified_checkpoint = c4
state.current_justified_checkpoint = c3
state.justification_bits = 0'u8 # Bitvector of length 4
# mock 3rd and 4th latest epochs as justified
# indices are pre-shift
state.justification_bits.raiseBit 1
state.justification_bits.raiseBit 2
# mock the 2nd latest epoch as justifiable, with 4th as the source
addMockAttestations(
state,
epoch = epoch - 1,
source = c4,
target = c2,
sufficient_support = sufficient_support
)
# State transition
transitionEpochUntilJustificationFinalization(state)
# Checks
doAssert state.previous_justified_checkpoint == c3 # changed to old current
if sufficient_support:
doAssert state.current_justified_checkpoint == c2 # changed to second latest
doAssert state.finalized_checkpoint == c4 # finalized old previous justified epoch
else:
doAssert state.current_justified_checkpoint == c3 # still old current
doAssert state.finalized_checkpoint == old_finalized # no new finalized checkpoint
proc finalizeOn23(state: var BeaconState, epoch: Epoch, sufficient_support: bool) =
## Check finalization on rule 2 "23"
doAssert epoch > 3
state.slot = Slot((epoch * SLOTS_PER_EPOCH) - 1) # Skip ahead to just before epoch
# 43210 -- epochs ago
# 210xx -- justification bitfields indices preshift
# 3210x -- justification bitfield indices postshift
# 01*0. -- justification bitfield contents. . = this epoch, * is being justified now
# checkpoints for epochs ago
let (c1, c2, c3, _, _) = getCheckpoints(epoch)
putCheckpointsInBlockRoots(state, [c1, c2, c3])
# Save for final checks
let old_finalized = state.finalized_checkpoint
# Mock the state
state.previous_justified_checkpoint = c3
state.current_justified_checkpoint = c3
state.justification_bits = 0'u8 # Bitvector of length 4
# mock 3rd as justified
# indices are pre-shift
state.justification_bits.raiseBit 1
# mock the 2nd latest epoch as justifiable, with 3rd as the source
addMockAttestations(
state,
epoch = epoch - 2,
source = c3,
target = c2,
sufficient_support = sufficient_support
)
# State transition
transitionEpochUntilJustificationFinalization(state)
# Checks
doAssert state.previous_justified_checkpoint == c3 # changed to old current
if sufficient_support:
doAssert state.current_justified_checkpoint == c2 # changed to second latest
doAssert state.finalized_checkpoint == c3 # finalized old previous justified epoch
else:
doAssert state.current_justified_checkpoint == c3 # still old current
doAssert state.finalized_checkpoint == old_finalized # no new finalized checkpoint
proc finalizeOn123(state: var BeaconState, epoch: Epoch, sufficient_support: bool) =
## Check finalization on rule 2 "23"
doAssert epoch > 5
state.slot = Slot((epoch * SLOTS_PER_EPOCH) - 1) # Skip ahead to just before epoch
# 43210 -- epochs ago
# 210xx -- justification bitfields indices preshift
# 3210x -- justification bitfield indices postshift
# 0110*. -- justification bitfield contents. . = this epoch, * is being justified now
# checkpoints for epochs ago
let (c1, c2, c3, c4, c5) = getCheckpoints(epoch)
putCheckpointsInBlockRoots(state, [c1, c2, c3, c4, c5])
# Save for final checks
let old_finalized = state.finalized_checkpoint
# Mock the state
state.previous_justified_checkpoint = c5
state.current_justified_checkpoint = c3
state.justification_bits = 0'u8 # Bitvector of length 4
# mock 3rd as justified
# indices are pre-shift
state.justification_bits.raiseBit 1
# mock the 2nd latest epoch as justifiable, with 5th as the source
addMockAttestations(
state,
epoch = epoch - 2,
source = c5,
target = c2,
sufficient_support = sufficient_support
)
# mock the 1st latest epoch as justifiable with 3rd as source
addMockAttestations(
state,
epoch = epoch - 1,
source = c3,
target = c1,
sufficient_support = sufficient_support
)
# State transition
transitionEpochUntilJustificationFinalization(state)
# Checks
doAssert state.previous_justified_checkpoint == c3 # changed to old current
if sufficient_support:
doAssert state.current_justified_checkpoint == c1 # changed to second latest
doAssert state.finalized_checkpoint == c3 # finalized old previous justified epoch
else:
doAssert state.current_justified_checkpoint == c3 # still old current
doAssert state.finalized_checkpoint == old_finalized # no new finalized checkpoint
proc finalizeOn12(state: var BeaconState, epoch: Epoch, sufficient_support: bool) =
## Check finalization on rule 2 "23"
doAssert epoch > 2
state.slot = Slot((epoch * SLOTS_PER_EPOCH) - 1) # Skip ahead to just before epoch
# 43210 -- epochs ago
# 210xx -- justification bitfields indices preshift
# 3210x -- justification bitfield indices postshift
# 01*0. -- justification bitfield contents. . = this epoch, * is being justified now
# checkpoints for epochs ago
let (c1, c2, _, _, _) = getCheckpoints(epoch)
putCheckpointsInBlockRoots(state, [c1, c2])
# Save for final checks
let old_finalized = state.finalized_checkpoint
# Mock the state
state.previous_justified_checkpoint = c2
state.current_justified_checkpoint = c2
state.justification_bits = 0'u8 # Bitvector of length 4
# mock 3rd as justified
# indices are pre-shift
state.justification_bits.raiseBit 0
# mock the 2nd latest epoch as justifiable, with 3rd as the source
addMockAttestations(
state,
epoch = epoch - 1,
source = c2,
target = c1,
sufficient_support = sufficient_support
)
# State transition
transitionEpochUntilJustificationFinalization(state)
# Checks
doAssert state.previous_justified_checkpoint == c2 # changed to old current
if sufficient_support:
doAssert state.current_justified_checkpoint == c1 # changed to second latest
doAssert state.finalized_checkpoint == c2 # finalized old previous justified epoch
else:
doAssert state.current_justified_checkpoint == c2 # still old current
doAssert state.finalized_checkpoint == old_finalized # no new finalized checkpoint
suite "[Unit - Spec - Epoch processing] Justification and Finalization " & preset():
echo " Finalization rules are detailed at https://github.com/protolambda/eth2-docs#justification-and-finalization"
const NumValidators = uint64(8) * SLOTS_PER_EPOCH
let genesisState = initGenesisState(NumValidators)
doAssert genesisState.validators.len == int NumValidators
var state: BeaconState
template resetState: untyped =
deepCopy(state, genesisState)
test " Rule I - 234 finalization with enough support":
resetState()
finalizeOn234(state, Epoch 5, sufficient_support = true)
test " Rule I - 234 finalization without support":
resetState()
finalizeOn234(state, Epoch 5, sufficient_support = false)
test " Rule II - 23 finalization with enough support":
resetState()
finalizeOn23(state, Epoch 4, sufficient_support = true)
test " Rule II - 23 finalization without support":
resetState()
finalizeOn23(state, Epoch 4, sufficient_support = false)
test " Rule III - 123 finalization with enough support":
resetState()
finalizeOn123(state, Epoch 6, sufficient_support = true)
test " Rule III - 123 finalization without support":
resetState()
finalizeOn123(state, Epoch 6, sufficient_support = false)
test " Rule IV - 12 finalization with enough support":
resetState()
finalizeOn12(state, Epoch 3, sufficient_support = true)
test " Rule IV - 12 finalization without support":
resetState()
finalizeOn12(state, Epoch 3, sufficient_support = false)

View File

@ -90,10 +90,9 @@ suite "SSZ Navigation":
let c = [byte 0x07, 0x08, 0x09].toDigest
let leaves = sszList(@[a, b, c], int64(1 shl 3))
let root = hashTreeRoot(leaves)
let root = hash_tree_root(leaves)
check $root == "5248085B588FAB1DD1E03F3CD62201602B12E6560665935964F46E805977E8C5"
let leaves2 = sszList(@[a, b, c], int64(1 shl 10))
let root2 = hashTreeRoot(leaves2)
let root2 = hash_tree_root(leaves2)
check $root2 == "9FB7D518368DC14E8CC588FB3FD2749BEEF9F493FEF70AE34AF5721543C67173"