beacon state updates

* add `on_startup` genesis state constructor
* update state transition to use ssz
* update validator adder
This commit is contained in:
Jacek Sieka 2018-11-28 23:23:40 -06:00
parent 19d885165c
commit 4d6b068697
No known key found for this signature in database
GPG Key ID: 6299FEB3EB6FA465
11 changed files with 242 additions and 127 deletions

View File

@ -135,8 +135,8 @@ proc scheduleCycleActions(node: BeaconNode) =
# Schedule block proposals
let
slot = cycleStart + i
proposerIdx = get_beacon_proposer_idx(node.beaconState, slot)
attachedValidator = node.getAttachedValidator(proposerIdx)
proposerIdx = get_beacon_proposer_index(node.beaconState, slot.uint64)
attachedValidator = node.getAttachedValidator(proposerIdx.int)
if attachedValidator != nil:
# TODO:
@ -148,7 +148,7 @@ proc scheduleCycleActions(node: BeaconNode) =
# Schedule attestations
let
committeesIdx = get_shard_and_committees_idx(node.beaconState, slot)
committeesIdx = get_shard_and_committees_index(node.beaconState, slot.uint64)
for shard in node.beaconState.shard_and_committee_for_slots[committees_idx]:
for validatorIdx in shard.committee:

21
beacon_chain/extras.nim Normal file
View File

@ -0,0 +1,21 @@
# 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.
# Temporary dumping ground for extra types and helpers that could make it into
# the spec potentially
import
./spec/[crypto, digest]
type
InitialValidator* = object
## Eth1 validator registration contract output
pubkey*: ValidatorPubKey
deposit_size*: uint64
proof_of_possession*: seq[byte]
withdrawal_credentials*: Eth2Digest
randao_commitment*: Eth2Digest

View File

@ -6,24 +6,82 @@
# at your option. This file may not be copied, modified, or distributed except according to those terms.
import
math, sequtils,
../extras,
./datatypes, ./digest, ./helpers, ./validator
func mod_get[T](arr: openarray[T], pos: Natural): T =
arr[pos mod arr.len]
func get_shard_and_committees_idx*(state: BeaconState, slot: int): int =
# This replaces `get_shards_and_committees_for_slot` from the spec
# since in Nim, it's not currently efficient to create read-only
# accessors to expensive-to-copy members (such as sequences).
let earliest_slot_in_array = state.last_state_recalculation_slot.int - CYCLE_LENGTH
func on_startup*(initial_validator_entries: openArray[InitialValidator],
genesis_time: int,
processed_pow_receipt_root: Eth2Digest): BeaconState =
## BeaconState constructor
##
## Before the beacon chain starts, validators will register in the Eth1 chain
## and deposit ETH. When enough many validators have registered, a
## `ChainStart` log will be emitted and the beacon chain can start beaconing.
##
## Because the state root hash is part of the genesis block, the beacon state
## must be calculated before creating the genesis block.
#
# Induct validators
var validators: seq[ValidatorRecord]
for v in initial_validator_entries:
validators = get_new_validators(
validators,
ForkData(
pre_fork_version: 0,
post_fork_version: 0,
fork_slot_number: 0xffffffffffffffff'u64
),
v.pubkey,
v.deposit_size,
v.proof_of_possession,
v.withdrawal_credentials,
v.randao_commitment,
ACTIVE,
0
).validators
# Setup state
let
x = get_new_shuffling(Eth2Digest(), validators, 0)
# x + x in spec, but more ugly
var tmp: array[2 * CYCLE_LENGTH, seq[ShardAndCommittee]]
for i, n in x:
tmp[i] = n
tmp[CYCLE_LENGTH + i] = n
# The spec says to use validators, but it's actually indices..
let validator_indices = get_active_validator_indices(validators)
BeaconState(
validators: validators,
shard_and_committee_for_slots: tmp,
persistent_committees: split(
shuffle(validator_indices, Eth2Digest()), SHARD_COUNT),
fork_data: ForkData(
pre_fork_version: INITIAL_FORK_VERSION,
post_fork_version: INITIAL_FORK_VERSION
)
)
func get_shard_and_committees_index*(state: BeaconState, slot: uint64): uint64 =
let earliest_slot_in_array =
state.last_state_recalculation_slot - CYCLE_LENGTH
doAssert earliest_slot_in_array <= slot and
slot < earliest_slot_in_array + CYCLE_LENGTH * 2
return int(slot - earliest_slot_in_array)
slot - earliest_slot_in_array
proc get_shards_and_committees_for_slot*(state: BeaconState, slot: int): seq[ShardAndCommittee] =
return state.shard_and_committee_for_slots[state.get_shard_and_committees_idx(slot)]
proc get_shards_and_committees_for_slot*(
state: BeaconState, slot: uint64): seq[ShardAndCommittee] =
let index = state.get_shard_and_committees_index(slot)
state.shard_and_committee_for_slots[index]
func get_beacon_proposer_idx*(state: BeaconState, slot: int): int =
func get_beacon_proposer_index*(state: BeaconState, slot: uint64): uint64 =
## From Casper RPJ mini-spec:
## When slot i begins, validator Vidx is expected
## to create ("propose") a block, which contains a pointer to some parent block
@ -33,19 +91,39 @@ func get_beacon_proposer_idx*(state: BeaconState, slot: int): int =
##
## idx in Vidx == p(i mod N), pi being a random permutation of validators indices (i.e. a committee)
# This replaces `get_beacon_proposer` from the spec since in Nim,
# it's not currently efficient to create read-only accessors to
# expensive-to-copy members (such as ValidatorRecord).
let idx = get_shard_and_committees_idx(state, slot)
return state.shard_and_committee_for_slots[idx][0].committee.mod_get(slot)
let idx = get_shard_and_committees_index(state, slot)
state.shard_and_committee_for_slots[idx][0].committee.mod_get(slot)
func get_block_hash*(state: BeaconState,
current_block: BeaconBlock,
slot: int): Eth2Digest =
let earliest_slot_in_array = current_block.slot.int - state.recent_block_hashes.len
let earliest_slot_in_array =
current_block.slot.int - state.recent_block_hashes.len
assert earliest_slot_in_array <= slot
assert slot < current_block.slot.int
return state.recent_block_hashes[slot - earliest_slot_in_array]
state.recent_block_hashes[slot - earliest_slot_in_array]
func append_to_recent_block_hashes*(old_block_hashes: seq[Eth2Digest],
parent_slot, current_slot: uint64,
parent_hash: Eth2Digest): seq[Eth2Digest] =
let d = current_slot - parent_slot
result = old_block_hashes
result.add repeat(parent_hash, d)
proc get_attestation_participants*(state: BeaconState,
attestation_data: AttestationSignedData,
attester_bitfield: seq[byte]): seq[int] =
let
sncs_for_slot = get_shards_and_committees_for_slot(
state, attestation_data.slot)
for snc in sncs_for_slot:
if snc.shard == attestation_data.shard:
assert len(attester_bitfield) == ceil_div8(len(snc.committee))
for i, vindex in snc.committee:
let
bit = (attester_bitfield[i div 8] shr (7 - (i mod 8))) mod 2
if bit == 1:
result.add(vindex)
return

View File

@ -99,7 +99,7 @@ type
BeaconState* = object
validator_set_change_slot*: uint64 # Slot of last validator set change
validators*: seq[ValidatorRecord] # List of validators
crosslinks*: seq[CrosslinkRecord] # Most recent crosslink for each shard
crosslinks*: array[SHARD_COUNT, CrosslinkRecord] # Most recent crosslink for each shard
last_state_recalculation_slot*: uint64 # Last cycle-boundary state recalculation
last_finalized_slot*: uint64 # Last finalized slot
justification_source*: uint64 # Justification source
@ -119,7 +119,7 @@ type
candidate_pow_receipt_roots*: seq[CandidatePoWReceiptRootRecord] #
fork_data*: ForkData # Parameters relevant to hard forks / versioning.
# Should be updated only by hard forks.
pending_attestations*: seq[AttestationRecord] # Attestations not yet processed
pending_attestations*: seq[ProcessedAttestation] # Attestations not yet processed
recent_block_hashes*: seq[Eth2Digest] # recent beacon block hashes needed to process attestations, older to newer
randao_mix*: Eth2Digest # RANDAO state

View File

@ -76,3 +76,4 @@ func get_new_recent_block_hashes*(old_block_hashes: seq[Eth2Digest],
for _ in 0 ..< min(d, old_block_hashes.len):
result.add parent_hash
func ceil_div8*(v: int): int = (v + 7) div 8

View File

@ -16,22 +16,39 @@ func min_empty_validator(validators: seq[ValidatorRecord], current_slot: uint64)
if v.status == WITHDRAWN and v.last_status_change_slot + DELETION_PERIOD.uint64 <= current_slot:
return some(i)
func add_validator*(validators: var seq[ValidatorRecord],
pubkey: ValidatorPubKey,
proof_of_possession: seq[byte],
withdrawal_credentials: Eth2Digest,
randao_commitment: Eth2Digest,
status: ValidatorStatusCodes,
current_slot: uint64
): int =
func get_new_validators*(current_validators: seq[ValidatorRecord],
fork_data: ForkData,
pubkey: ValidatorPubKey,
deposit_size: uint64,
proof_of_possession: seq[byte],
withdrawal_credentials: Eth2Digest,
randao_commitment: Eth2Digest,
status: ValidatorStatusCodes,
current_slot: uint64
): tuple[validators: seq[ValidatorRecord], index: int] =
# Check that validator really did register
# let signed_message = as_bytes32(pubkey) + as_bytes2(withdrawal_shard) + withdrawal_address + randao_commitment
# let signed_message = signed_message = bytes32(pubkey) + withdrawal_credentials + randao_commitment
# assert BLSVerify(pub=pubkey,
# msg=hash(signed_message),
# sig=proof_of_possession)
# sig=proof_of_possession,
# domain=get_domain(
# fork_data,
# current_slot,
# DOMAIN_DEPOSIT
# ))
# Pubkey uniqueness
# assert pubkey not in [v.pubkey for v in validators]
var new_validators = current_validators
for index, val in new_validators.mpairs():
if val.pubkey == pubkey:
# assert deposit_size >= MIN_TOPUP_SIZE
# assert val.status != WITHDRAWN
# assert val.withdrawal_credentials == withdrawal_credentials
val.balance.inc(deposit_size.int)
return (new_validators, index)
# new validator
let
rec = ValidatorRecord(
pubkey: pubkey,
@ -44,17 +61,16 @@ func add_validator*(validators: var seq[ValidatorRecord],
exit_seq: 0
)
let index = min_empty_validator(validators, current_slot)
let index = min_empty_validator(new_validators, current_slot)
if index.isNone:
validators.add(rec)
return len(validators) - 1
new_validators.add(rec)
(new_validators, len(new_validators) - 1)
else:
validators[index.get()] = rec
return index.get()
new_validators[index.get()] = rec
(new_validators, index.get())
func get_active_validator_indices(validators: openArray[ValidatorRecord]): seq[Uint24] =
func get_active_validator_indices*(validators: openArray[ValidatorRecord]): seq[Uint24] =
## Select the active validators
result = @[]
for idx, val in validators:
if val.status == ACTIVE:
result.add idx.Uint24
@ -86,7 +102,8 @@ func get_new_shuffling*(seed: Eth2Digest,
var committees = newSeq[ShardAndCommittee](shard_indices.len)
for shard_position, indices in shard_indices:
committees[shard_position].shard = (shard_id_start + shard_position).uint16 mod SHARD_COUNT
committees[shard_position].shard =
uint64(shard_id_start + shard_position) mod SHARD_COUNT
committees[shard_position].committee = indices
result[slot] = committees

View File

@ -5,106 +5,79 @@
# * 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.
# Note: this is also inspired by https://github.com/ethereum/beacon_chain/blob/master/beacon_chain/state/state_transition.py
# The official spec at https://notes.ethereum.org/SCIg8AH5SA-O4C1G1LYZHQ# is not fully
# defining the state transitions.
# A imcomplete implementation of the state transition function, as described
# under "Per-block processing" in https://github.com/ethereum/eth2.0-specs/blob/master/specs/core/0_beacon-chain.md
#
# Note that the ethresearch impl is using "block_vote_cache" field, which is a dictionary mapping hashes
# to the following sub-dictionary:
# {
# 'voter_indices': set(),
# 'total_voter_deposits': 0
# }
# It should not be needed anymore with the new AttestationRecord type
{.warning: "The official spec at https://notes.ethereum.org/SCIg8AH5SA-O4C1G1LYZHQ# is not fully defining state transitions.".}
# The code is here mainly to verify the data types and get an idea about
# missing pieces - needs testing throughout
import
options,
./spec/[beaconstate, crypto, datatypes, digest, helpers],
intsets, endians, nimcrypto,
./ssz,
milagro_crypto # nimble install https://github.com/status-im/nim-milagro-crypto@#master
func process_block*(active_state: BeaconState, crystallized_state: BeaconState, blck: BeaconBlock, slot: uint64) =
# TODO: non-attestation verification parts of per-block processing
func checkAttestations(state: BeaconState, blck: BeaconBlock):
seq[ProcessedAttestation] =
discard
#let parent_hash = blck.ancestor_hashes[0]
func process_block*(state: BeaconState, blck: BeaconBlock): Option[BeaconState] =
## When a new block is received, all participants must verify that the block
## makes sense and update their state accordingly. This function will return
## the new state, unless something breaks along the way
# XXX: simplistic way to be able to rollback state
var state = state
let
parent_hash = blck.ancestor_hashes[0]
slot = blck.slot
parent_slot = slot - 1 # XXX Not!! can skip slots...
# TODO actually get parent block, which means fixing up BeaconState refs above;
# there's no distinction between active/crystallized state anymore, etc.
let parent_slot = 0'u64
state.recent_block_hashes =
append_to_recent_block_hashes(state.recent_block_hashes, parent_slot, slot,
parent_hash)
state.pending_attestations.add checkAttestations(state, blck)
doAssert blck.attestations.len <= MAX_ATTESTATION_COUNT
for attestation in blck.attestations:
doAssert attestation.data.slot <= blck.slot - MIN_ATTESTATION_INCLUSION_DELAY
doAssert attestation.data.slot >= max(parent_slot - CYCLE_LENGTH + 1, 0)
if attestation.data.slot <= blck.slot - MIN_ATTESTATION_INCLUSION_DELAY:
return
if attestation.data.slot >= max(parent_slot - CYCLE_LENGTH + 1, 0):
return
#doAssert attestation.data.justified_slot == justification_source if attestation.data.slot >= state.last_state_recalculation_slot else prev_cycle_justification_source
# doAssert attestation.data.justified_block_hash == get_block_hash(state, block, attestation.data.justified_slot).
# doAssert either attestation.data.last_crosslink_hash or attestation.data.shard_block_hash equals state.crosslinks[shard].shard_block_hash.
# Let attestation_indices be get_shards_and_committees_for_slot(crystallized_state, slot)[x], choosing x so that attestation_indices.shard_id equals the shard_id value provided to find the set of validators that is creating this attestation record.
let attestation_indices = block:
let shard_and_committees = get_shards_and_committees_for_slot(crystallized_state, slot.int)
var
x = 1
record_creator = shard_and_committees[0]
while record_creator.shard != attestation.data.shard:
record_creator = shard_and_committees[x]
inc x
record_creator
let attestation_participants = get_attestation_participants(
state, attestation.data, attestation.attester_bitfield)
# TODO: Verify that len(attester_bitfield) == ceil_div8(len(attestation_indices)), where ceil_div8 = (x + 7) // 8. Verify that bits len(attestation_indices).... and higher, if present (i.e. len(attestation_indices) is not a multiple of 8), are all zero
var
agg_pubkey: ValidatorPubKey
empty = true
# Derive a group public key by adding the public keys of all of the attesters in attestation_indices for whom the corresponding bit in attester_bitfield (the ith bit is (attester_bitfield[i // 8] >> (7 - (i %8))) % 2) equals 1
var agg_pubkey: ValidatorPubKey
var empty = true
for attester_idx in attestation_indices.committee:
# TODO re-enable, but currently this whole function's a nonfunctional stub
# because state's lacking.
#if attester_idx in attestation.attester_bitfield:
let validator = crystallized_state.validators[attester_idx]
if empty:
agg_pubkey = validator.pubkey
empty = false
else:
agg_pubkey.combine(validator.pubkey)
for attester_idx in attestation_participants:
let validator = state.validators[attester_idx]
if empty:
agg_pubkey = validator.pubkey
empty = false
else:
agg_pubkey.combine(validator.pubkey)
# Verify that aggregate_sig verifies using the group pubkey generated and hash((slot % CYCLE_LENGTH).to_bytes(8, 'big') + parent_hashes + shard_id + shard_block_hash) as the message.
var msg: array[32, byte]
block:
var ctx: blake2_512 # Context for streaming blake2b computation
ctx.init()
var be_slot: array[8, byte]
bigEndian64(be_slot[0].addr, attestation.data.slot.unsafeAddr)
ctx.update be_slot
# TODO: re-enable these, but parent_hashes isn't in new spec, so
# this needs more substantial adjustment.
# let size_p_hashes = uint parent_hashes.len * sizeof(Eth2Digest)
# ctx.update(cast[ptr byte](parent_hashes[0].addr), size_p_hashes)
var be_shard_id: array[2, byte] # Unsure, spec doesn't mention big-endian representation
bigEndian16(be_shard_id.addr, attestation.data.shard.unsafeAddr)
ctx.update be_shard_id
ctx.update attestation.data.shard_block_hash.data
var be_justified_slot: array[8, byte]
bigEndian64(be_justified_slot[0].addr, attestation.data.justified_slot.unsafeAddr)
ctx.update be_justified_slot
let h = ctx.finish() # Full hash (Blake2b-512)
msg[0 ..< 32] = h.data.toOpenArray(0, 32) # Keep only the first 32 bytes - https://github.com/ethereum/beacon_chain/issues/60
ctx.clear() # Cleanup context/memory
# Verify that aggregate_sig verifies using the group pubkey.
let msg = hashSSZ(attestation.data)
# For now only check compilation
# doAssert attestation.aggregate_sig.verifyMessage(msg, agg_pubkey)
debugEcho "Aggregate sig verify message: ", attestation.aggregate_sig.verifyMessage(msg, agg_pubkey)
return some(state)
# Extend the list of AttestationRecord objects in the active_state, ordering the new additions in the same order as they came in the block.
# TODO
# Verify that the slot % len(get_indices_for_slot(crystallized_state, slot-1)[0])'th attester in get_indices_for_slot(crystallized_state, slot-1)[0]is part of at least one of the AttestationRecord objects; this attester can be considered to be the proposer of the block.
# Verify that the slot % len(get_indices_for_slot(state, slot-1)[0])'th attester in get_indices_for_slot(state, slot-1)[0]is part of at least one of the AttestationRecord objects; this attester can be considered to be the proposer of the block.
# TODO

View File

@ -6,6 +6,7 @@
# at your option. This file may not be copied, modified, or distributed except according to those terms.
import
./test_beaconstate,
./test_block_processing,
./test_ssz,
./test_validator

View File

@ -0,0 +1,18 @@
# 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
unittest,
../beacon_chain/extras,
../beacon_chain/spec/[beaconstate, datatypes, digest]
suite "Beacon state":
# Smoke test
test "Smoke on_startup":
let state = on_startup([InitialValidator()], 0, Eth2Digest())
check: state.validators.len == 1

View File

@ -6,17 +6,19 @@
# at your option. This file may not be copied, modified, or distributed except according to those terms.
import
unittest,
../beacon_chain/spec/datatypes,
options, unittest,
../beacon_chain/spec/[datatypes, digest],
../beacon_chain/state_transition
suite "Block processing":
## For now just test that we can compile and execute block processing with mock data.
test "Mock process_block":
let actState = BeaconState()
let crystState = BeaconState()
let blck = BeaconBlock()
let slot = 10'u
actState.process_block(crystState, blck, slot)
let
state = BeaconState()
blck = BeaconBlock(
ancestor_hashes: @[Eth2Digest()]
)
newState = process_block(state, blck).get()
check:
newState.genesis_time == state.genesis_time

View File

@ -22,10 +22,14 @@ suite "Validators":
validators = repeat(
ValidatorRecord(
status: ACTIVE
), 1024)
), 32*1024)
# XXX the shuffling looks really odd, probably buggy
let s = get_new_shuffling(Eth2Digest(), validators, 0)
check:
s.len == CYCLE_LENGTH
# 32k validators means 2 shards validated per slot - the aim is to get
# TARGET_COMMITTEE_SIZE validators in each shard and there are
# CYCLE_LENGTH slots which each will crosslink a different shard
s[0].len == 32 * 1024 div (TARGET_COMMITTEE_SIZE * CYCLE_LENGTH)
sumCommittees(s) == validators.len() # all validators accounted for