Merge pull request #33 from sigp/validation

Add block and attestation validation
This commit is contained in:
Paul Hauner 2018-10-18 10:07:06 +11:00 committed by GitHub
commit 2fb9dfbf14
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
18 changed files with 2284 additions and 3 deletions

View File

@ -41,5 +41,6 @@ members = [
"beacon_chain/utils/shuffling",
"beacon_chain/utils/ssz",
"beacon_chain/utils/ssz_helpers",
"beacon_chain/validation",
"lighthouse/db",
]

View File

@ -0,0 +1,13 @@
[package]
name = "validation"
version = "0.1.0"
authors = ["Paul Hauner <paul@paulhauner.com>"]
[dependencies]
bls = { path = "../utils/bls" }
db = { path = "../../lighthouse/db" }
hashing = { path = "../utils/hashing" }
rayon = "1.0.2"
ssz = { path = "../utils/ssz" }
ssz_helpers = { path = "../utils/ssz_helpers" }
types = { path = "../types" }

View File

@ -0,0 +1,226 @@
use super::types::Hash256;
#[derive(Debug)]
pub enum ParentHashesError {
BadCurrentHashes,
BadObliqueHashes,
SlotTooHigh,
SlotTooLow,
IntWrapping,
}
/// This function is used to select the hashes used in
/// the signing of an AttestationRecord.
///
/// It either returns Result with a vector of length `cycle_length,` or
/// returns an Error.
///
/// This function corresponds to the `get_signed_parent_hashes` function
/// in the Python reference implentation.
///
/// See this slide for more information:
/// https://tinyurl.com/ybzn2spw
pub fn attestation_parent_hashes(
cycle_length: u8,
block_slot: u64,
attestation_slot: u64,
current_hashes: &[Hash256],
oblique_hashes: &[Hash256])
-> Result<Vec<Hash256>, ParentHashesError>
{
// This cast places a limit on cycle_length. If you change it, check math
// for overflow.
let cycle_length: u64 = u64::from(cycle_length);
if current_hashes.len() as u64 != (cycle_length * 2) {
return Err(ParentHashesError::BadCurrentHashes);
}
if oblique_hashes.len() as u64 > cycle_length {
return Err(ParentHashesError::BadObliqueHashes);
}
if attestation_slot >= block_slot {
return Err(ParentHashesError::SlotTooHigh);
}
/*
* Cannot underflow as block_slot cannot be less
* than attestation_slot.
*/
let attestation_distance = block_slot - attestation_slot;
if attestation_distance > cycle_length {
return Err(ParentHashesError::SlotTooLow);
}
/*
* Cannot underflow because attestation_distance cannot
* be larger than cycle_length.
*/
let start = cycle_length - attestation_distance;
/*
* Overflow is potentially impossible, but proof is complicated
* enough to just use checked math.
*
* Arithmetic is:
* start + cycle_length - oblique_hashes.len()
*/
let end = start.checked_add(cycle_length)
.and_then(|x| x.checked_sub(oblique_hashes.len() as u64))
.ok_or(ParentHashesError::IntWrapping)?;
let mut hashes = Vec::new();
hashes.extend_from_slice(
&current_hashes[(start as usize)..(end as usize)]);
hashes.extend_from_slice(oblique_hashes);
Ok(hashes)
}
#[cfg(test)]
mod tests {
use super::*;
fn get_range_of_hashes(from: usize, to: usize) -> Vec<Hash256> {
(from..to).map(|i| get_hash(&vec![i as u8])).collect()
}
fn get_hash(value: &[u8]) -> Hash256 {
Hash256::from_slice(value)
}
#[test]
fn test_get_signed_hashes_oblique_scenario_1() {
/*
* Two oblique hashes.
*/
let cycle_length: u8 = 8;
let block_slot: u64 = 19;
let attestation_slot: u64 = 15;
let current_hashes = get_range_of_hashes(3, 19);
let oblique_hashes = get_range_of_hashes(100, 102);
let result = attestation_parent_hashes(
cycle_length,
block_slot,
attestation_slot,
&current_hashes,
&oblique_hashes);
assert!(result.is_ok());
let result = result.unwrap();
assert_eq!(result.len(), cycle_length as usize);
let mut expected_result = get_range_of_hashes(7, 13);
expected_result.append(&mut get_range_of_hashes(100, 102));
assert_eq!(result, expected_result);
}
#[test]
fn test_get_signed_hashes_oblique_scenario_2() {
/*
* All oblique hashes.
*/
let cycle_length: u8 = 8;
let block_slot: u64 = 19;
let attestation_slot: u64 = 15;
let current_hashes = get_range_of_hashes(3, 19);
let oblique_hashes = get_range_of_hashes(100, 108);
let result = attestation_parent_hashes(
cycle_length,
block_slot,
attestation_slot,
&current_hashes,
&oblique_hashes);
assert!(result.is_ok());
let result = result.unwrap();
assert_eq!(result.len(), cycle_length as usize);
let expected_result = get_range_of_hashes(100, 108);
assert_eq!(result, expected_result);
}
#[test]
fn test_get_signed_hashes_scenario_1() {
/*
* Google Slides example.
* https://tinyurl.com/ybzn2spw
*/
let cycle_length: u8 = 8;
let block_slot: u64 = 19;
let attestation_slot: u64 = 15;
let current_hashes = get_range_of_hashes(3, 19);
let oblique_hashes = vec![];
let result = attestation_parent_hashes(
cycle_length,
block_slot,
attestation_slot,
&current_hashes,
&oblique_hashes);
assert!(result.is_ok());
let result = result.unwrap();
assert_eq!(result.len(), cycle_length as usize);
let expected_result = get_range_of_hashes(7, 15);
assert_eq!(result, expected_result);
}
#[test]
fn test_get_signed_hashes_scenario_2() {
/*
* Block 1, attestation 0.
*/
let cycle_length: u8 = 8;
let block_slot: u64 = 1;
let attestation_slot: u64 = 0;
let current_hashes = get_range_of_hashes(0, 16);
let oblique_hashes = vec![];
let result = attestation_parent_hashes(
cycle_length,
block_slot,
attestation_slot,
&current_hashes,
&oblique_hashes);
let result = result.unwrap();
assert_eq!(result.len(), cycle_length as usize);
let expected_result = get_range_of_hashes(7, 15);
assert_eq!(result, expected_result);
}
#[test]
fn test_get_signed_hashes_scenario_3() {
/*
* attestation_slot too large
*/
let cycle_length: u8 = 8;
let block_slot: u64 = 100;
let attestation_slot: u64 = 100;
let current_hashes = get_range_of_hashes(0, 16);
let oblique_hashes = vec![];
let result = attestation_parent_hashes(
cycle_length,
block_slot,
attestation_slot,
&current_hashes,
&oblique_hashes);
assert!(result.is_err());
}
#[test]
fn test_get_signed_hashes_scenario_4() {
/*
* Current hashes too small
*/
let cycle_length: u8 = 8;
let block_slot: u64 = 100;
let attestation_slot: u64 = 99;
let current_hashes = get_range_of_hashes(0, 15);
let oblique_hashes = vec![];
let result = attestation_parent_hashes(
cycle_length,
block_slot,
attestation_slot,
&current_hashes,
&oblique_hashes);
assert!(result.is_err());
}
}

View File

@ -0,0 +1,256 @@
use std::collections::HashSet;
use std::sync::Arc;
use super::types::{
AttestationRecord,
AttesterMap,
};
use super::attestation_parent_hashes::{
attestation_parent_hashes,
ParentHashesError,
};
use super::db::{
ClientDB,
DBError
};
use super::db::stores::{
BlockStore,
BlockAtSlotError,
ValidatorStore,
};
use super::types::{
Hash256,
};
use super::message_generation::generate_signed_message;
use super::signature_verification::{
verify_aggregate_signature_for_indices,
SignatureVerificationError,
};
#[derive(Debug,PartialEq)]
pub enum AttestationValidationError {
ParentSlotTooHigh,
ParentSlotTooLow,
BlockSlotTooHigh,
BlockSlotTooLow,
JustifiedSlotIncorrect,
InvalidJustifiedBlockHash,
TooManyObliqueHashes,
BadCurrentHashes,
BadObliqueHashes,
BadAttesterMap,
IntWrapping,
PublicKeyCorrupt,
NoPublicKeyForValidator,
BadBitfieldLength,
InvalidBitfield,
InvalidBitfieldEndBits,
NoSignatures,
NonZeroTrailingBits,
BadAggregateSignature,
DBError(String),
}
/// The context against which some attestation should be validated.
pub struct AttestationValidationContext<T>
where T: ClientDB + Sized
{
/// The slot as determined by the system time.
pub block_slot: u64,
/// The slot of the parent of the block that contained this attestation.
pub parent_block_slot: u64,
/// The cycle_length as determined by the chain configuration.
pub cycle_length: u8,
/// The last justified slot as per the client's view of the canonical chain.
pub last_justified_slot: u64,
/// A vec of the hashes of the blocks preceeding the present slot.
pub parent_hashes: Arc<Vec<Hash256>>,
/// The store containing block information.
pub block_store: Arc<BlockStore<T>>,
/// The store containing validator information.
pub validator_store: Arc<ValidatorStore<T>>,
/// A map of (slot, shard_id) to the attestation set of validation indices.
pub attester_map: Arc<AttesterMap>,
}
impl<T> AttestationValidationContext<T>
where T: ClientDB
{
/// Validate a (fully deserialized) AttestationRecord against this context.
///
/// The function will return a HashSet of validator indices (canonical validator indices not
/// attestation indices) if the validation passed successfully, or an error otherwise.
///
/// The attestation's aggregate signature will be verified, therefore the function must able to
/// access all required validation public keys via the `validator_store`.
pub fn validate_attestation(&self, a: &AttestationRecord)
-> Result<HashSet<usize>, AttestationValidationError>
{
/*
* The attesation slot must be less than or equal to the parent of the slot of the block
* that contained the attestation.
*/
if a.slot > self.parent_block_slot {
return Err(AttestationValidationError::ParentSlotTooHigh);
}
/*
* The slot of this attestation must not be more than cycle_length + 1 distance
* from the parent_slot of block that contained it.
*/
if a.slot < self.parent_block_slot
.saturating_sub(u64::from(self.cycle_length).saturating_add(1)) {
return Err(AttestationValidationError::ParentSlotTooLow);
}
/*
* The attestation must indicate that its last justified slot is the same as the last justified
* slot known to us.
*/
if a.justified_slot > self.last_justified_slot {
return Err(AttestationValidationError::JustifiedSlotIncorrect);
}
/*
* There is no need to include more oblique parents hashes than there are blocks
* in a cycle.
*/
if a.oblique_parent_hashes.len() > usize::from(self.cycle_length) {
return Err(AttestationValidationError::TooManyObliqueHashes);
}
/*
* Retrieve the set of attestation indices for this slot and shard id.
*
* This is an array mapping the order that validators will appear in the bitfield to the
* canonincal index of a validator.
*/
let attestation_indices = self.attester_map.get(&(a.slot, a.shard_id))
.ok_or(AttestationValidationError::BadAttesterMap)?;
/*
* The bitfield must be no longer than the minimum required to represent each validator in the
* attestation indices for this slot and shard id.
*/
if a.attester_bitfield.num_bytes() !=
bytes_for_bits(attestation_indices.len())
{
return Err(AttestationValidationError::BadBitfieldLength);
}
/*
* If there are excess bits in the bitfield because the number of a validators in not a
* multiple of 8, reject this attestation record.
*
* Allow extra set bits would permit mutliple different byte layouts (and therefore hashes) to
* refer to the same AttesationRecord.
*/
if a.attester_bitfield.len() > attestation_indices.len() {
return Err(AttestationValidationError::InvalidBitfieldEndBits)
}
/*
* Generate the parent hashes for this attestation
*/
let parent_hashes = attestation_parent_hashes(
self.cycle_length,
self.block_slot,
a.slot,
&self.parent_hashes,
&a.oblique_parent_hashes)?;
/*
* The specified justified block hash supplied in the attestation must be in the chain at
* the given slot number.
*
* First, we find the latest parent hash from the parent_hashes array. Then, using the
* block store (database) we iterate back through the blocks until we find (or fail to
* find) the justified block hash referenced in the attestation record.
*/
let latest_parent_hash = parent_hashes.last()
.ok_or(AttestationValidationError::BadCurrentHashes)?;
match self.block_store.block_at_slot(&latest_parent_hash, a.justified_slot)? {
Some((ref hash, _)) if *hash == a.justified_block_hash.to_vec() => (),
_ => return Err(AttestationValidationError::InvalidJustifiedBlockHash)
};
/*
* Generate the message that this attestation aggregate signature must sign across.
*/
let signed_message = {
generate_signed_message(
a.slot,
&parent_hashes,
a.shard_id,
&a.shard_block_hash,
a.justified_slot)
};
let voted_hashset =
verify_aggregate_signature_for_indices(
&signed_message,
&a.aggregate_sig,
&attestation_indices,
&a.attester_bitfield,
&self.validator_store)?;
/*
* If the hashset of voters is None, the signature verification failed.
*/
match voted_hashset {
None => Err(AttestationValidationError::BadAggregateSignature),
Some(hashset) => Ok(hashset),
}
}
}
fn bytes_for_bits(bits: usize) -> usize {
(bits.saturating_sub(1) / 8) + 1
}
impl From<ParentHashesError> for AttestationValidationError {
fn from(e: ParentHashesError) -> Self {
match e {
ParentHashesError::BadCurrentHashes
=> AttestationValidationError::BadCurrentHashes,
ParentHashesError::BadObliqueHashes
=> AttestationValidationError::BadObliqueHashes,
ParentHashesError::SlotTooLow
=> AttestationValidationError::BlockSlotTooLow,
ParentHashesError::SlotTooHigh
=> AttestationValidationError::BlockSlotTooHigh,
ParentHashesError::IntWrapping
=> AttestationValidationError::IntWrapping
}
}
}
impl From<BlockAtSlotError> for AttestationValidationError {
fn from(e: BlockAtSlotError) -> Self {
match e {
BlockAtSlotError::DBError(s) => AttestationValidationError::DBError(s),
_ => AttestationValidationError::InvalidJustifiedBlockHash
}
}
}
impl From<DBError> for AttestationValidationError {
fn from(e: DBError) -> Self {
AttestationValidationError::DBError(e.message)
}
}
impl From<SignatureVerificationError> for AttestationValidationError {
fn from(e: SignatureVerificationError) -> Self {
match e {
SignatureVerificationError::BadValidatorIndex
=> AttestationValidationError::BadAttesterMap,
SignatureVerificationError::PublicKeyCorrupt
=> AttestationValidationError::PublicKeyCorrupt,
SignatureVerificationError::NoPublicKeyForValidator
=> AttestationValidationError::NoPublicKeyForValidator,
SignatureVerificationError::DBError(s)
=> AttestationValidationError::DBError(s),
}
}
}

View File

@ -0,0 +1,395 @@
extern crate rayon;
use self::rayon::prelude::*;
use std::sync::{
Arc,
RwLock,
};
use super::attestation_validation::{
AttestationValidationContext,
AttestationValidationError,
};
use super::types::{
AttestationRecord,
AttesterMap,
Block,
ProposerMap,
};
use super::ssz_helpers::attestation_ssz_splitter::{
split_one_attestation,
split_all_attestations,
AttestationSplitError,
};
use super::ssz_helpers::ssz_block::{
SszBlock,
SszBlockError,
};
use super::db::{
ClientDB,
DBError,
};
use super::db::stores::{
BlockStore,
PoWChainStore,
ValidatorStore,
};
use super::ssz::{
Decodable,
DecodeError,
};
use super::types::Hash256;
#[derive(Debug, PartialEq)]
pub enum BlockStatus {
NewBlock,
KnownBlock,
}
#[derive(Debug, PartialEq)]
pub enum SszBlockValidationError {
FutureSlot,
SlotAlreadyFinalized,
UnknownPoWChainRef,
UnknownParentHash,
BadAttestationSsz,
ParentSlotHigherThanBlockSlot,
AttestationValidationError(AttestationValidationError),
AttestationSignatureFailed,
ProposerAttestationHasObliqueHashes,
NoProposerSignature,
BadProposerMap,
RwLockPoisoned,
DBError(String),
}
/// The context against which a block should be validated.
pub struct BlockValidationContext<T>
where T: ClientDB + Sized
{
/// The slot as determined by the system time.
pub present_slot: u64,
/// The cycle_length as determined by the chain configuration.
pub cycle_length: u8,
/// The last justified slot as per the client's view of the canonical chain.
pub last_justified_slot: u64,
/// The last justified block hash as per the client's view of the canonical chain.
pub last_justified_block_hash: Hash256,
/// The last finalized slot as per the client's view of the canonical chain.
pub last_finalized_slot: u64,
/// A vec of the hashes of the blocks preceeding the present slot.
pub parent_hashes: Arc<Vec<Hash256>>,
/// A map of slots to a block proposer validation index.
pub proposer_map: Arc<ProposerMap>,
/// A map of (slot, shard_id) to the attestation set of validation indices.
pub attester_map: Arc<AttesterMap>,
/// The store containing block information.
pub block_store: Arc<BlockStore<T>>,
/// The store containing validator information.
pub validator_store: Arc<ValidatorStore<T>>,
/// The store containing information about the proof-of-work chain.
pub pow_store: Arc<PoWChainStore<T>>,
}
impl<T> BlockValidationContext<T>
where T: ClientDB
{
/// Validate some SszBlock against a block validation context. An SszBlock varies from a Block in
/// that is a read-only structure that reads directly from encoded SSZ.
///
/// The reason to validate an SzzBlock is to avoid decoding it in its entirety if there is
/// a suspicion that the block might be invalid. Such a suspicion should be applied to
/// all blocks coming from the network.
///
/// This function will determine if the block is new, already known or invalid (either
/// intrinsically or due to some application error.)
///
/// Note: this function does not implement randao_reveal checking as it is not in the
/// specification.
#[allow(dead_code)]
pub fn validate_ssz_block(&self, b: &SszBlock)
-> Result<(BlockStatus, Option<Block>), SszBlockValidationError>
where T: ClientDB + Sized
{
/*
* If this block is already known, return immediately and indicate the the block is
* known. Don't attempt to deserialize the block.
*/
let block_hash = &b.block_hash();
if self.block_store.block_exists(&block_hash)? {
return Ok((BlockStatus::KnownBlock, None));
}
/*
* If the block slot corresponds to a slot in the future, return immediately with an error.
*
* It is up to the calling fn to determine what should be done with "future" blocks (e.g.,
* cache or discard).
*/
let block_slot = b.slot_number();
if block_slot > self.present_slot {
return Err(SszBlockValidationError::FutureSlot);
}
/*
* If the block is unknown (assumed unknown because we checked the db earlier in this
* function) and it comes from a slot that is already finalized, drop the block.
*
* If a slot is finalized, there's no point in considering any other blocks for that slot.
*
* TODO: We can more strongly throw away blocks based on the `last_finalized_block` related
* to this `last_finalized_slot`. Namely, any block in a future slot must include the
* `last_finalized_block` in it's chain.
*/
if block_slot <= self.last_finalized_slot {
return Err(SszBlockValidationError::SlotAlreadyFinalized);
}
/*
* If the PoW chain hash is not known to us, drop it.
*
* We only accept blocks that reference a known PoW hash.
*
* Note: it is not clear what a "known" PoW chain ref is. Likely it means the block hash is
* "sufficienty deep in the canonical PoW chain". This should be clarified as the spec
* crystallizes.
*/
let pow_chain_ref = b.pow_chain_ref();
if !self.pow_store.block_hash_exists(b.pow_chain_ref())? {
return Err(SszBlockValidationError::UnknownPoWChainRef);
}
/*
* Store a slice of the serialized attestations from the block SSZ.
*/
let attestations_ssz = &b.attestations();
/*
* Get a slice of the first serialized attestation (the 0'th) and decode it into
* a full AttestationRecord object.
*
* The first attestation must be validated separately as it must contain a signature of the
* proposer of the previous block (this is checked later in this function).
*/
let (first_attestation_ssz, next_index) = split_one_attestation(
&attestations_ssz,
0)?;
let (first_attestation, _) = AttestationRecord::ssz_decode(
&first_attestation_ssz, 0)?;
/*
* The first attestation may not have oblique hashes.
*
* The presence of oblique hashes in the first attestation would indicate that the proposer
* of the previous block is attesting to some other block than the one they produced.
*/
if first_attestation.oblique_parent_hashes.len() > 0 {
return Err(SszBlockValidationError::ProposerAttestationHasObliqueHashes);
}
/*
* Read the parent hash from the block we are validating then attempt to load
* that parent block ssz from the database.
*
* If that parent doesn't exist in the database or is invalid, reject the block.
*
* Also, read the slot from the parent block for later use.
*/
let parent_hash = b.parent_hash();
let parent_block_slot = match self.block_store.get_serialized_block(&parent_hash)? {
None => return Err(SszBlockValidationError::UnknownParentHash),
Some(ssz) => {
let parent_block = SszBlock::from_slice(&ssz[..])?;
parent_block.slot_number()
}
};
/*
* The parent block slot must be less than the block slot.
*
* In other words, the parent must come before the child.
*/
if parent_block_slot >= block_slot {
return Err(SszBlockValidationError::ParentSlotHigherThanBlockSlot);
}
/*
* Generate the context in which attestations will be validated.
*/
let attestation_validation_context = Arc::new(AttestationValidationContext {
block_slot,
parent_block_slot,
cycle_length: self.cycle_length,
last_justified_slot: self.last_justified_slot,
parent_hashes: self.parent_hashes.clone(),
block_store: self.block_store.clone(),
validator_store: self.validator_store.clone(),
attester_map: self.attester_map.clone(),
});
/*
* Validate this first attestation.
*/
let attestation_voters = attestation_validation_context
.validate_attestation(&first_attestation)?;
/*
* Attempt to read load the parent block proposer from the proposer map. Return with an
* error if it fails.
*
* If the signature of proposer for the parent slot was not present in the first (0'th)
* attestation of this block, reject the block.
*/
let parent_block_proposer = self.proposer_map.get(&parent_block_slot)
.ok_or(SszBlockValidationError::BadProposerMap)?;
if !attestation_voters.contains(&parent_block_proposer) {
return Err(SszBlockValidationError::NoProposerSignature);
}
/*
* Split the remaining attestations into a vector of slices, each containing
* a single serialized attestation record.
*/
let other_attestations = split_all_attestations(attestations_ssz,
next_index)?;
/*
* Verify each other AttestationRecord.
*
* This uses the `rayon` library to do "sometimes" parallelization. Put simply,
* if there are some spare threads, the verification of attestation records will happen
* concurrently.
*
* There is a thread-safe `failure` variable which is set whenever an attestation fails
* validation. This is so all attestation validation is halted if a single bad attestation
* is found.
*/
let failure: RwLock<Option<SszBlockValidationError>> = RwLock::new(None);
let mut deserialized_attestations: Vec<AttestationRecord> = other_attestations
.par_iter()
.filter_map(|attestation_ssz| {
/*
* If some thread has set the `failure` variable to `Some(error)` the abandon
* attestation serialization and validation.
*/
if let Some(_) = *failure.read().unwrap() {
return None;
}
/*
* If there has not been a failure yet, attempt to serialize and validate the
* attestation.
*/
match AttestationRecord::ssz_decode(&attestation_ssz, 0) {
/*
* Deserialization failed, therefore the block is invalid.
*/
Err(e) => {
let mut failure = failure.write().unwrap();
*failure = Some(SszBlockValidationError::from(e));
None
}
/*
* Deserialization succeeded and the attestation should be validated.
*/
Ok((attestation, _)) => {
match attestation_validation_context.validate_attestation(&attestation) {
/*
* Attestation validation failed with some error.
*/
Err(e) => {
let mut failure = failure.write().unwrap();
*failure = Some(SszBlockValidationError::from(e));
None
}
/*
* Attestation validation succeded.
*/
Ok(_) => Some(attestation)
}
}
}
})
.collect();
match failure.into_inner() {
Err(_) => return Err(SszBlockValidationError::RwLockPoisoned),
Ok(failure) => {
match failure {
Some(error) => return Err(error),
_ => ()
}
}
}
/*
* Add the first attestation to the vec of deserialized attestations at
* index 0.
*/
deserialized_attestations.insert(0, first_attestation);
/*
* If we have reached this point, the block is a new valid block that is worthy of
* processing.
*/
let block = Block {
parent_hash: Hash256::from(parent_hash),
slot_number: block_slot,
randao_reveal: Hash256::from(b.randao_reveal()),
attestations: deserialized_attestations,
pow_chain_ref: Hash256::from(pow_chain_ref),
active_state_root: Hash256::from(b.act_state_root()),
crystallized_state_root: Hash256::from(b.cry_state_root()),
};
Ok((BlockStatus::NewBlock, Some(block)))
}
}
impl From<DBError> for SszBlockValidationError {
fn from(e: DBError) -> Self {
SszBlockValidationError::DBError(e.message)
}
}
impl From<AttestationSplitError> for SszBlockValidationError {
fn from(e: AttestationSplitError) -> Self {
match e {
AttestationSplitError::TooShort =>
SszBlockValidationError::BadAttestationSsz
}
}
}
impl From<SszBlockError> for SszBlockValidationError {
fn from(e: SszBlockError) -> Self {
match e {
SszBlockError::TooShort =>
SszBlockValidationError::DBError("Bad parent block in db.".to_string()),
SszBlockError::TooLong =>
SszBlockValidationError::DBError("Bad parent block in db.".to_string()),
}
}
}
impl From<DecodeError> for SszBlockValidationError {
fn from(e: DecodeError) -> Self {
match e {
DecodeError::TooShort =>
SszBlockValidationError::BadAttestationSsz,
DecodeError::TooLong =>
SszBlockValidationError::BadAttestationSsz,
}
}
}
impl From<AttestationValidationError> for SszBlockValidationError {
fn from(e: AttestationValidationError) -> Self {
SszBlockValidationError::AttestationValidationError(e)
}
}
/*
* Tests for block validation are contained in the root directory "tests" directory (AKA
* "integration tests directory").
*/

View File

@ -0,0 +1,12 @@
extern crate db;
extern crate bls;
extern crate hashing;
extern crate ssz;
extern crate ssz_helpers;
extern crate types;
pub mod attestation_validation;
mod attestation_parent_hashes;
pub mod block_validation;
mod message_generation;
mod signature_verification;

View File

@ -0,0 +1,70 @@
use super::ssz::SszStream;
use super::hashing::canonical_hash;
use super::types::Hash256;
/// Generates the message used to validate the signature provided with an AttestationRecord.
///
/// Ensures that the signer of the message has a view of the chain that is compatible with ours.
pub fn generate_signed_message(
slot: u64,
parent_hashes: &[Hash256],
shard_id: u16,
shard_block_hash: &Hash256,
justified_slot: u64)
-> Vec<u8>
{
/*
* Note: it's a little risky here to use SSZ, because the encoding is not necessarily SSZ
* (for example, SSZ might change whilst this doesn't).
*
* I have suggested switching this to ssz here:
* https://github.com/ethereum/eth2.0-specs/issues/5
*
* If this doesn't happen, it would be safer to not use SSZ at all.
*/
let mut ssz_stream = SszStream::new();
ssz_stream.append(&slot);
ssz_stream.append_vec(&parent_hashes.to_vec());
ssz_stream.append(&shard_id);
ssz_stream.append(shard_block_hash);
ssz_stream.append(&justified_slot);
let bytes = ssz_stream.drain();
canonical_hash(&bytes)
}
#[cfg(test)]
mod tests {
use super::*;
#[test]
fn test_generate_signed_message() {
let slot = 93;
let parent_hashes: Vec<Hash256> = (0..12)
.map(|i| Hash256::from(i as u64))
.collect();
let shard_id = 15;
let shard_block_hash = Hash256::from("shard_block_hash".as_bytes());
let justified_slot = 18;
let output = generate_signed_message(
slot,
&parent_hashes,
shard_id,
&shard_block_hash,
justified_slot);
/*
* Note: this is not some well-known test vector, it's simply the result of running
* this and printing the output.
*
* Once well-known test vectors are established, they should be placed here.
*/
let expected = vec![
149, 99, 94, 229, 72, 144, 233, 14, 164, 16, 143, 53, 94, 48,
118, 179, 33, 181, 172, 215, 2, 191, 176, 18, 188, 172, 137,
178, 236, 66, 74, 120
];
assert_eq!(output, expected);
}
}

View File

@ -0,0 +1,183 @@
use std::collections::HashSet;
use super::bls::{
AggregateSignature,
AggregatePublicKey,
};
use super::db::ClientDB;
use super::db::stores::{
ValidatorStore,
ValidatorStoreError,
};
use super::types::Bitfield;
#[derive(Debug, PartialEq)]
pub enum SignatureVerificationError {
BadValidatorIndex,
PublicKeyCorrupt,
NoPublicKeyForValidator,
DBError(String),
}
/// Verify an aggregate signature across the supplied message.
///
/// The public keys used for verification are collected by mapping
/// each true bitfield bit to canonical ValidatorRecord index through
/// the attestation_indicies map.
///
/// Each public key is loaded from the store on-demand.
pub fn verify_aggregate_signature_for_indices<T>(
message: &[u8],
agg_sig: &AggregateSignature,
attestation_indices: &[usize],
bitfield: &Bitfield,
validator_store: &ValidatorStore<T>)
-> Result<Option<HashSet<usize>>, SignatureVerificationError>
where T: ClientDB + Sized
{
let mut voters = HashSet::new();
let mut agg_pub_key = AggregatePublicKey::new();
for i in 0..attestation_indices.len() {
let voted = bitfield.get_bit(i);
if voted {
/*
* De-reference the attestation index into a canonical ValidatorRecord index.
*/
let validator = *attestation_indices.get(i)
.ok_or(SignatureVerificationError::BadValidatorIndex)?;
/*
* Load the validators public key from our store.
*/
let pub_key = validator_store
.get_public_key_by_index(validator)?
.ok_or(SignatureVerificationError::NoPublicKeyForValidator)?;
/*
* Add the validators public key to the aggregate public key.
*/
agg_pub_key.add(&pub_key);
/*
* Add validator to the set of voters for this attestation record.
*/
voters.insert(validator);
}
}
/*
* Verify the aggregate public key against the aggregate signature.
*
* This verification will only succeed if the exact set of public keys
* were added to the aggregate public key as those that signed the aggregate signature.
*/
if agg_sig.verify(&message, &agg_pub_key) {
Ok(Some(voters))
} else {
Ok(None)
}
}
impl From<ValidatorStoreError> for SignatureVerificationError {
fn from(error: ValidatorStoreError) -> Self {
match error {
ValidatorStoreError::DBError(s) =>
SignatureVerificationError::DBError(s),
ValidatorStoreError::DecodeError =>
SignatureVerificationError::PublicKeyCorrupt,
}
}
}
#[cfg(test)]
mod tests {
use super::*;
use super::super::bls::{
Keypair,
Signature,
};
use super::super::db::MemoryDB;
use std::sync::Arc;
/*
* Cases that still need testing:
*
* - No signatures.
* - Database failure.
* - Unknown validator index.
* - Extra validator on signature.
*/
#[test]
fn test_signature_verification() {
let message = "cats".as_bytes();
let signing_keypairs = vec![
Keypair::random(),
Keypair::random(),
Keypair::random(),
Keypair::random(),
Keypair::random(),
Keypair::random(),
];
let non_signing_keypairs = vec![
Keypair::random(),
Keypair::random(),
Keypair::random(),
Keypair::random(),
Keypair::random(),
Keypair::random(),
];
/*
* Signing keypairs first, then non-signing
*/
let mut all_keypairs = signing_keypairs.clone();
all_keypairs.append(&mut non_signing_keypairs.clone());
let attestation_indices: Vec<usize> = (0..all_keypairs.len())
.collect();
let mut bitfield = Bitfield::new();
for i in 0..signing_keypairs.len() {
bitfield.set_bit(i, true);
}
let db = Arc::new(MemoryDB::open());
let store = ValidatorStore::new(db);
for (i, keypair) in all_keypairs.iter().enumerate() {
store.put_public_key_by_index(i, &keypair.pk).unwrap();
}
let mut agg_sig = AggregateSignature::new();
for keypair in &signing_keypairs {
let sig = Signature::new(&message, &keypair.sk);
agg_sig.add(&sig);
}
/*
* Test using all valid parameters.
*/
let voters = verify_aggregate_signature_for_indices(
&message,
&agg_sig,
&attestation_indices,
&bitfield,
&store).unwrap();
let voters = voters.unwrap();
(0..signing_keypairs.len())
.for_each(|i| assert!(voters.contains(&i)));
(signing_keypairs.len()..non_signing_keypairs.len())
.for_each(|i| assert!(!voters.contains(&i)));
/*
* Add another validator to the bitfield, run validation will all other
* parameters the same and assert that it fails.
*/
bitfield.set_bit(signing_keypairs.len() + 1, true);
let voters = verify_aggregate_signature_for_indices(
&message,
&agg_sig,
&attestation_indices,
&bitfield,
&store).unwrap();
assert_eq!(voters, None);
}
}

View File

@ -0,0 +1,234 @@
use std::sync::Arc;
use super::db::{
MemoryDB,
};
use super::db::stores::{
ValidatorStore,
BlockStore,
};
use super::types::{
AttestationRecord,
AttesterMap,
Bitfield,
Block,
Hash256,
};
use super::validation::attestation_validation::{
AttestationValidationContext,
};
use super::bls::{
AggregateSignature,
Keypair,
SecretKey,
Signature,
};
use super::ssz::SszStream;
use super::hashing::{
canonical_hash,
};
pub struct TestStore {
pub db: Arc<MemoryDB>,
pub validator: Arc<ValidatorStore<MemoryDB>>,
pub block: Arc<BlockStore<MemoryDB>>,
}
impl TestStore {
pub fn new() -> Self {
let db = Arc::new(MemoryDB::open());
let validator = Arc::new(ValidatorStore::new(db.clone()));
let block = Arc::new(BlockStore::new(db.clone()));
Self {
db,
validator,
block,
}
}
}
pub struct TestRig {
pub attestation: AttestationRecord,
pub context: AttestationValidationContext<MemoryDB>,
pub stores: TestStore,
pub attester_count: usize,
}
fn generate_message_hash(slot: u64,
parent_hashes: &[Hash256],
shard_id: u16,
shard_block_hash: &Hash256,
justified_slot: u64)
-> Vec<u8>
{
let mut stream = SszStream::new();
stream.append(&slot);
stream.append_vec(&parent_hashes.to_vec());
stream.append(&shard_id);
stream.append(shard_block_hash);
stream.append(&justified_slot);
let bytes = stream.drain();
canonical_hash(&bytes)
}
pub fn generate_attestation(shard_id: u16,
shard_block_hash: &Hash256,
block_slot: u64,
attestation_slot: u64,
justified_slot: u64,
justified_block_hash: &Hash256,
cycle_length: u8,
parent_hashes: &[Hash256],
signing_keys: &[Option<SecretKey>],
block_store: &BlockStore<MemoryDB>)
-> AttestationRecord
{
let mut attester_bitfield = Bitfield::new();
let mut aggregate_sig = AggregateSignature::new();
let parent_hashes_slice = {
let distance: usize = (block_slot - attestation_slot) as usize;
let last: usize = parent_hashes.len() - distance;
let first: usize = last - usize::from(cycle_length);
&parent_hashes[first..last]
};
/*
* Create a justified block at the correct slot and store it in the db.
*/
create_block_at_slot(&block_store, &justified_block_hash, justified_slot);
/*
* Generate the message that will be signed across for this attr record.
*/
let attestation_message = generate_message_hash(
attestation_slot,
parent_hashes_slice,
shard_id,
shard_block_hash,
justified_slot);
for (i, secret_key) in signing_keys.iter().enumerate() {
/*
* If the signing key is Some, set the bitfield bit to true
* and sign the aggregate sig.
*/
if let Some(sk) = secret_key {
attester_bitfield.set_bit(i, true);
let sig = Signature::new(&attestation_message, sk);
aggregate_sig.add(&sig);
}
}
AttestationRecord {
slot: attestation_slot,
shard_id,
oblique_parent_hashes: vec![],
shard_block_hash: shard_block_hash.clone(),
attester_bitfield,
justified_slot,
justified_block_hash: justified_block_hash.clone(),
aggregate_sig,
}
}
/// Create a minimum viable block at some slot.
///
/// Allows the validation function to read the block and verify its slot.
pub fn create_block_at_slot(block_store: &BlockStore<MemoryDB>, hash: &Hash256, slot: u64) {
let mut justified_block = Block::zero();
justified_block.attestations.push(AttestationRecord::zero());
justified_block.slot_number = slot;
let mut s = SszStream::new();
s.append(&justified_block);
let justified_block_ssz = s.drain();
block_store.put_serialized_block(&hash.to_vec(), &justified_block_ssz).unwrap();
}
/// Inserts a justified_block_hash in a position that will be referenced by an attestation record.
pub fn insert_justified_block_hash(
parent_hashes: &mut Vec<Hash256>,
justified_block_hash: &Hash256,
block_slot: u64,
attestation_slot: u64)
{
let attestation_parent_hash_index = parent_hashes.len() - 1 -
(block_slot as usize - attestation_slot as usize);
parent_hashes[attestation_parent_hash_index] = justified_block_hash.clone();
}
pub fn setup_attestation_validation_test(shard_id: u16, attester_count: usize)
-> TestRig
{
let stores = TestStore::new();
let block_slot = 10000;
let cycle_length: u8 = 64;
let mut parent_hashes: Vec<Hash256> = (0..(cycle_length * 2))
.map(|i| Hash256::from(i as u64))
.collect();
let attestation_slot = block_slot - 1;
let parent_block_slot = attestation_slot;
let last_justified_slot = attestation_slot - 1;
let justified_block_hash = Hash256::from("justified_block".as_bytes());
let shard_block_hash = Hash256::from("shard_block".as_bytes());
/*
* Insert the required justified_block_hash into parent_hashes
*/
insert_justified_block_hash(
&mut parent_hashes,
&justified_block_hash,
block_slot,
attestation_slot);
let parent_hashes = Arc::new(parent_hashes);
let mut keypairs = vec![];
let mut signing_keys = vec![];
let mut attester_map = AttesterMap::new();
let mut attesters = vec![];
/*
* Generate a random keypair for each validator and clone it into the
* list of keypairs. Store it in the database.
*/
for i in 0..attester_count {
let keypair = Keypair::random();
keypairs.push(keypair.clone());
stores.validator.put_public_key_by_index(i, &keypair.pk).unwrap();
signing_keys.push(Some(keypair.sk.clone()));
attesters.push(i);
}
attester_map.insert((attestation_slot, shard_id), attesters);
let context: AttestationValidationContext<MemoryDB> = AttestationValidationContext {
block_slot,
parent_block_slot,
cycle_length,
last_justified_slot,
parent_hashes: parent_hashes.clone(),
block_store: stores.block.clone(),
validator_store: stores.validator.clone(),
attester_map: Arc::new(attester_map),
};
let attestation = generate_attestation(
shard_id,
&shard_block_hash,
block_slot,
attestation_slot,
last_justified_slot,
&justified_block_hash,
cycle_length,
&parent_hashes.clone(),
&signing_keys,
&stores.block);
TestRig {
attestation,
context,
stores,
attester_count,
}
}

View File

@ -0,0 +1,9 @@
pub mod helpers;
mod tests;
use super::bls;
use super::db;
use super::ssz;
use super::types;
use super::hashing;
use super::validation;

View File

@ -0,0 +1,217 @@
use std::sync::Arc;
use super::helpers::{
TestRig,
setup_attestation_validation_test,
create_block_at_slot,
};
use super::validation::attestation_validation::{
AttestationValidationError,
};
use super::types::AttesterMap;
use super::bls::{
AggregateSignature,
};
use super::types::{
Hash256,
};
fn generic_rig() -> TestRig {
let shard_id = 10;
let validator_count = 2;
setup_attestation_validation_test(shard_id, validator_count)
}
#[test]
fn test_attestation_validation_valid() {
let rig = generic_rig();
let result = rig.context.validate_attestation(&rig.attestation);
let voter_map = result.unwrap();
assert_eq!(voter_map.len(), 2);
}
#[test]
fn test_attestation_validation_invalid_parent_slot_too_high() {
let mut rig = generic_rig();
rig.context.parent_block_slot = rig.attestation.slot - 1;
let result = rig.context.validate_attestation(&rig.attestation);
assert_eq!(result, Err(AttestationValidationError::ParentSlotTooHigh));
}
#[test]
fn test_attestation_validation_invalid_parent_slot_too_low() {
let mut rig = generic_rig();
rig.attestation.slot = rig.context.parent_block_slot - u64::from(rig.context.cycle_length) - 2;
let result = rig.context.validate_attestation(&rig.attestation);
assert_eq!(result, Err(AttestationValidationError::ParentSlotTooLow));
}
#[test]
fn test_attestation_validation_invalid_block_slot_too_high() {
let mut rig = generic_rig();
rig.context.block_slot = rig.attestation.slot - 1;
let result = rig.context.validate_attestation(&rig.attestation);
assert_eq!(result, Err(AttestationValidationError::BlockSlotTooHigh));
}
#[test]
fn test_attestation_validation_invalid_block_slot_too_low() {
let mut rig = generic_rig();
rig.context.block_slot = rig.context.block_slot + u64::from(rig.context.cycle_length);
let result = rig.context.validate_attestation(&rig.attestation);
assert_eq!(result, Err(AttestationValidationError::BlockSlotTooLow));
}
#[test]
fn test_attestation_validation_invalid_justified_slot_incorrect() {
let mut rig = generic_rig();
let original = rig.attestation.justified_slot;
rig.attestation.justified_slot = original - 1;
// Ensures we don't get a bad justified block error instead.
create_block_at_slot(
&rig.stores.block,
&rig.attestation.justified_block_hash,
rig.attestation.justified_slot);
let result = rig.context.validate_attestation(&rig.attestation);
assert_eq!(result, Err(AttestationValidationError::BadAggregateSignature));
rig.attestation.justified_slot = original + 1;
// Ensures we don't get a bad justified block error instead.
create_block_at_slot(
&rig.stores.block,
&rig.attestation.justified_block_hash,
rig.attestation.justified_slot);
// Ensures we don't get an error that the last justified slot is ahead of the context justified
// slot.
rig.context.last_justified_slot = rig.attestation.justified_slot;
let result = rig.context.validate_attestation(&rig.attestation);
assert_eq!(result, Err(AttestationValidationError::BadAggregateSignature));
}
#[test]
fn test_attestation_validation_invalid_too_many_oblique() {
let mut rig = generic_rig();
let obliques: Vec<Hash256> = (0..(rig.context.cycle_length + 1))
.map(|i| Hash256::from((i * 2) as u64))
.collect();
rig.attestation.oblique_parent_hashes = obliques;
let result = rig.context.validate_attestation(&rig.attestation);
assert_eq!(result, Err(AttestationValidationError::TooManyObliqueHashes));
}
#[test]
fn test_attestation_validation_invalid_bad_attester_map() {
let mut rig = generic_rig();
rig.context.attester_map = Arc::new(AttesterMap::new());
let result = rig.context.validate_attestation(&rig.attestation);
assert_eq!(result, Err(AttestationValidationError::BadAttesterMap));
}
#[test]
fn test_attestation_validation_invalid_bad_bitfield_length() {
let mut rig = generic_rig();
/*
* Extend the bitfield by one byte
*
* This is a little hacky and makes assumptions about the internals
* of the bitfield.
*/
let one_byte_higher = rig.attester_count + 8;
rig.attestation.attester_bitfield.set_bit(one_byte_higher, true);
rig.attestation.attester_bitfield.set_bit(one_byte_higher, false);
let result = rig.context.validate_attestation(&rig.attestation);
assert_eq!(result, Err(AttestationValidationError::BadBitfieldLength));
}
#[test]
fn test_attestation_validation_invalid_invalid_bitfield_end_bit() {
let mut rig = generic_rig();
let one_bit_high = rig.attester_count + 1;
rig.attestation.attester_bitfield.set_bit(one_bit_high, true);
let result = rig.context.validate_attestation(&rig.attestation);
assert_eq!(result, Err(AttestationValidationError::InvalidBitfieldEndBits));
}
#[test]
fn test_attestation_validation_invalid_invalid_bitfield_end_bit_with_irreguar_bitfield_len() {
let mut rig = generic_rig();
/*
* This test ensure that if the number of attesters is "irregular" (with respect to the
* bitfield), and there is a invalid bit is set, validation will still fail.
*
* "Irregular" here means that number of validators + 1 is not a clean multiple of eight.
*
* This test exists to ensure that the application can distinguish between the highest set
* bit in a bitfield and the byte length of that bitfield
*/
let one_bit_high = rig.attester_count + 1;
assert!(one_bit_high % 8 != 0, "the test is ineffective in this case.");
rig.attestation.attester_bitfield.set_bit(one_bit_high, true);
let result = rig.context.validate_attestation(&rig.attestation);
assert_eq!(result, Err(AttestationValidationError::InvalidBitfieldEndBits));
}
#[test]
fn test_attestation_validation_invalid_unknown_justified_block_hash() {
let mut rig = generic_rig();
rig.attestation.justified_block_hash = Hash256::from("unknown block hash".as_bytes());
let result = rig.context.validate_attestation(&rig.attestation);
assert_eq!(result, Err(AttestationValidationError::InvalidJustifiedBlockHash));
}
#[test]
fn test_attestation_validation_invalid_unknown_justified_block_hash_wrong_slot() {
let rig = generic_rig();
/*
* justified_block_hash points to a block with a slot that is too high.
*/
create_block_at_slot(
&rig.stores.block,
&rig.attestation.justified_block_hash,
rig.attestation.justified_slot + 1);
let result = rig.context.validate_attestation(&rig.attestation);
assert_eq!(result, Err(AttestationValidationError::InvalidJustifiedBlockHash));
/*
* justified_block_hash points to a block with a slot that is too low.
*/
create_block_at_slot(
&rig.stores.block,
&rig.attestation.justified_block_hash,
rig.attestation.justified_slot - 1);
let result = rig.context.validate_attestation(&rig.attestation);
assert_eq!(result, Err(AttestationValidationError::InvalidJustifiedBlockHash));
}
#[test]
fn test_attestation_validation_invalid_empty_signature() {
let mut rig = generic_rig();
rig.attestation.aggregate_sig = AggregateSignature::new();
let result = rig.context.validate_attestation(&rig.attestation);
assert_eq!(result, Err(AttestationValidationError::BadAggregateSignature));
}

View File

@ -0,0 +1,243 @@
use std::sync::Arc;
use super::attestation_validation::helpers::{
generate_attestation,
insert_justified_block_hash,
};
use super::bls::{
Keypair,
};
use super::db::{
MemoryDB,
};
use super::db::stores::{
BlockStore,
PoWChainStore,
ValidatorStore,
};
use super::types::{
AttestationRecord,
AttesterMap,
Block,
Hash256,
ProposerMap,
};
use super::ssz_helpers::ssz_block::SszBlock;
use super::validation::block_validation::{
BlockValidationContext,
SszBlockValidationError,
BlockStatus,
};
use super::ssz::{
SszStream,
};
#[derive(Debug)]
pub struct BlockTestParams {
pub total_validators: usize,
pub cycle_length: u8,
pub shard_count: u16,
pub shards_per_slot: u16,
pub validators_per_shard: usize,
pub block_slot: u64,
pub attestations_justified_slot: u64,
pub parent_proposer_index: usize,
pub validation_context_slot: u64,
pub validation_context_justified_slot: u64,
pub validation_context_justified_block_hash: Hash256,
pub validation_context_finalized_slot: u64,
}
pub struct TestStore {
pub db: Arc<MemoryDB>,
pub block: Arc<BlockStore<MemoryDB>>,
pub pow_chain: Arc<PoWChainStore<MemoryDB>>,
pub validator: Arc<ValidatorStore<MemoryDB>>,
}
impl TestStore {
pub fn new() -> Self {
let db = Arc::new(MemoryDB::open());
let block = Arc::new(BlockStore::new(db.clone()));
let pow_chain = Arc::new(PoWChainStore::new(db.clone()));
let validator = Arc::new(ValidatorStore::new(db.clone()));
Self {
db,
block,
pow_chain,
validator,
}
}
}
type ParentHashes = Vec<Hash256>;
/// Setup for a block validation function, without actually executing the
/// block validation function.
pub fn setup_block_validation_scenario(params: &BlockTestParams)
-> (Block, ParentHashes, AttesterMap, ProposerMap, TestStore)
{
let stores = TestStore::new();
let cycle_length = params.cycle_length;
let shards_per_slot = params.shards_per_slot;
let validators_per_shard = params.validators_per_shard;
let block_slot = params.block_slot;
let attestations_justified_slot = params.attestations_justified_slot;
let mut parent_hashes: Vec<Hash256> = (0..(cycle_length * 2))
.map(|i| Hash256::from(i as u64))
.collect();
let parent_hash = Hash256::from("parent_hash".as_bytes());
let randao_reveal = Hash256::from("randao_reveal".as_bytes());
let justified_block_hash = Hash256::from("justified_hash".as_bytes());
let pow_chain_ref = Hash256::from("pow_chain".as_bytes());
let active_state_root = Hash256::from("active_state".as_bytes());
let crystallized_state_root = Hash256::from("cry_state".as_bytes());
let shard_block_hash = Hash256::from("shard_block_hash".as_bytes());
/*
* Store a valid PoW chain ref
*/
stores.pow_chain.put_block_hash(pow_chain_ref.as_ref()).unwrap();
/*
* Generate a minimum viable parent block and store it in the database.
*/
let mut parent_block = Block::zero();
let parent_attestation = AttestationRecord::zero();
parent_block.slot_number = block_slot - 1;
parent_block.attestations.push(parent_attestation);
let parent_block_ssz = serialize_block(&parent_block);
stores.block.put_serialized_block(parent_hash.as_ref(), &parent_block_ssz).unwrap();
let proposer_map = {
let mut proposer_map = ProposerMap::new();
proposer_map.insert(parent_block.slot_number, params.parent_proposer_index);
proposer_map
};
let (attester_map, attestations, _keypairs) = {
let mut i = 0;
let attestation_slot = block_slot - 1;
let mut attester_map = AttesterMap::new();
let mut attestations = vec![];
let mut keypairs = vec![];
/*
* Insert the required justified_block_hash into parent_hashes
*/
insert_justified_block_hash(
&mut parent_hashes,
&justified_block_hash,
block_slot,
attestation_slot);
/*
* For each shard in this slot, generate an attestation.
*/
for shard in 0..shards_per_slot {
let mut signing_keys = vec![];
let mut attesters = vec![];
/*
* Generate a random keypair for each validator and clone it into the
* list of keypairs. Store it in the database.
*/
for _ in 0..validators_per_shard {
let keypair = Keypair::random();
keypairs.push(keypair.clone());
stores.validator.put_public_key_by_index(i, &keypair.pk).unwrap();
signing_keys.push(Some(keypair.sk.clone()));
attesters.push(i);
i += 1;
}
attester_map.insert((attestation_slot, shard), attesters);
let attestation = generate_attestation(
shard,
&shard_block_hash,
block_slot,
attestation_slot,
attestations_justified_slot,
&justified_block_hash,
cycle_length,
&parent_hashes,
&signing_keys[..],
&stores.block);
attestations.push(attestation);
}
(attester_map, attestations, keypairs)
};
let block = Block {
parent_hash,
slot_number: block_slot,
randao_reveal,
attestations,
pow_chain_ref,
active_state_root,
crystallized_state_root,
};
(block,
parent_hashes,
attester_map,
proposer_map,
stores)
}
/// Helper function to take some Block and SSZ serialize it.
pub fn serialize_block(b: &Block) -> Vec<u8> {
let mut stream = SszStream::new();
stream.append(b);
stream.drain()
}
/// Setup and run a block validation scenario, given some parameters.
///
/// Returns the Result returned from the block validation function.
pub fn run_block_validation_scenario<F>(
params: &BlockTestParams,
mutator_func: F)
-> Result<(BlockStatus, Option<Block>), SszBlockValidationError>
where F: FnOnce(Block, AttesterMap, ProposerMap, TestStore)
-> (Block, AttesterMap, ProposerMap, TestStore)
{
let (block,
parent_hashes,
attester_map,
proposer_map,
stores) = setup_block_validation_scenario(&params);
let (block,
attester_map,
proposer_map,
stores) = mutator_func(block, attester_map, proposer_map, stores);
let ssz_bytes = serialize_block(&block);
let ssz_block = SszBlock::from_slice(&ssz_bytes[..])
.unwrap();
let context = BlockValidationContext {
present_slot: params.validation_context_slot,
cycle_length: params.cycle_length,
last_justified_slot: params.validation_context_justified_slot,
last_justified_block_hash: params.validation_context_justified_block_hash,
last_finalized_slot: params.validation_context_finalized_slot,
parent_hashes: Arc::new(parent_hashes),
proposer_map: Arc::new(proposer_map),
attester_map: Arc::new(attester_map),
block_store: stores.block.clone(),
validator_store: stores.validator.clone(),
pow_store: stores.pow_chain.clone()
};
let validation_status = context.validate_ssz_block(&ssz_block);
/*
* If validation returned a block, make sure it's the same block we supplied to it.
*
* I.e., there were no errors during the serialization -> deserialization process.
*/
if let Ok((_, Some(returned_block))) = &validation_status {
assert_eq!(*returned_block, block);
};
validation_status
}

View File

@ -0,0 +1,12 @@
mod helpers;
mod tests;
use super::bls;
use super::db;
use super::hashing;
use super::ssz;
use super::ssz_helpers;
use super::types;
use super::validation;
use super::attestation_validation;

View File

@ -0,0 +1,261 @@
use super::bls::{
AggregateSignature,
};
use super::helpers::{
BlockTestParams,
TestStore,
run_block_validation_scenario,
serialize_block,
};
use super::types::{
Block,
Hash256,
ProposerMap,
};
use super::ssz_helpers::ssz_block::SszBlock;
use super::validation::block_validation::{
SszBlockValidationError,
BlockStatus,
};
use super::validation::attestation_validation::{
AttestationValidationError,
};
use super::hashing::canonical_hash;
fn get_simple_params() -> BlockTestParams {
let validators_per_shard: usize = 5;
let cycle_length: u8 = 2;
let shard_count: u16 = 4;
let shards_per_slot: u16 = shard_count / u16::from(cycle_length);
let total_validators: usize = validators_per_shard * shard_count as usize;
let block_slot = u64::from(cycle_length) * 10000;
let attestations_justified_slot = block_slot - u64::from(cycle_length);
let parent_proposer_index = 0;
let validation_context_slot = block_slot;
let validation_context_justified_slot = attestations_justified_slot;
let validation_context_justified_block_hash = Hash256::from("justified_hash".as_bytes());
let validation_context_finalized_slot = 0;
BlockTestParams {
total_validators,
cycle_length,
shard_count,
shards_per_slot,
validators_per_shard,
parent_proposer_index,
block_slot,
attestations_justified_slot,
validation_context_slot,
validation_context_justified_slot,
validation_context_justified_block_hash,
validation_context_finalized_slot,
}
}
// TODO: test bad ssz serialization
#[test]
fn test_block_validation_valid() {
let params = get_simple_params();
let mutator = |block: Block, attester_map, proposer_map, stores| {
/*
* Do not mutate
*/
(block, attester_map, proposer_map, stores)
};
let status = run_block_validation_scenario(
&params,
mutator);
assert_eq!(status.unwrap().0, BlockStatus::NewBlock);
}
#[test]
fn test_block_validation_valid_known_block() {
let params = get_simple_params();
let mutator = |block: Block, attester_map, proposer_map, stores: TestStore| {
/*
* Pre-store the block in the database
*/
let block_ssz = serialize_block(&block);
let block_hash = canonical_hash(&block_ssz);
stores.block.put_serialized_block(&block_hash, &block_ssz).unwrap();
(block, attester_map, proposer_map, stores)
};
let status = run_block_validation_scenario(
&params,
mutator);
assert_eq!(status.unwrap(), (BlockStatus::KnownBlock, None));
}
#[test]
fn test_block_validation_parent_slot_too_high() {
let params = get_simple_params();
let mutator = |mut block: Block, attester_map, proposer_map, stores| {
block.slot_number = params.validation_context_justified_slot + 1;
(block, attester_map, proposer_map, stores)
};
let status = run_block_validation_scenario(
&params,
mutator);
assert_eq!(status, Err(SszBlockValidationError::ParentSlotHigherThanBlockSlot));
}
#[test]
fn test_block_validation_invalid_future_slot() {
let params = get_simple_params();
let mutator = |mut block: Block, attester_map, proposer_map, stores| {
block.slot_number = block.slot_number + 1;
(block, attester_map, proposer_map, stores)
};
let status = run_block_validation_scenario(
&params,
mutator);
assert_eq!(status, Err(SszBlockValidationError::FutureSlot));
}
#[test]
fn test_block_validation_invalid_slot_already_finalized() {
let mut params = get_simple_params();
params.validation_context_finalized_slot = params.block_slot;
params.validation_context_justified_slot = params.validation_context_finalized_slot +
u64::from(params.cycle_length);
let mutator = |block, attester_map, proposer_map, stores| {
/*
* Do not mutate
*/
(block, attester_map, proposer_map, stores)
};
let status = run_block_validation_scenario(
&params,
mutator);
assert_eq!(status, Err(SszBlockValidationError::SlotAlreadyFinalized));
}
#[test]
fn test_block_validation_invalid_unknown_pow_hash() {
let params = get_simple_params();
let mutator = |mut block: Block, attester_map, proposer_map, stores| {
block.pow_chain_ref = Hash256::from("unknown pow hash".as_bytes());
(block, attester_map, proposer_map, stores)
};
let status = run_block_validation_scenario(
&params,
mutator);
assert_eq!(status, Err(SszBlockValidationError::UnknownPoWChainRef));
}
#[test]
fn test_block_validation_invalid_unknown_parent_hash() {
let params = get_simple_params();
let mutator = |mut block: Block, attester_map, proposer_map, stores| {
block.parent_hash = Hash256::from("unknown parent block".as_bytes());
(block, attester_map, proposer_map, stores)
};
let status = run_block_validation_scenario(
&params,
mutator);
assert_eq!(status, Err(SszBlockValidationError::UnknownParentHash));
}
#[test]
fn test_block_validation_invalid_1st_attestation_signature() {
let params = get_simple_params();
let mutator = |mut block: Block, attester_map, proposer_map, stores| {
/*
* Set the second attestaion record to have an invalid signature.
*/
block.attestations[0].aggregate_sig = AggregateSignature::new();
(block, attester_map, proposer_map, stores)
};
let status = run_block_validation_scenario(
&params,
mutator);
assert_eq!(status, Err(SszBlockValidationError::AttestationValidationError(
AttestationValidationError::BadAggregateSignature)));
}
#[test]
fn test_block_validation_invalid_no_parent_proposer_signature() {
let params = get_simple_params();
let mutator = |block: Block, attester_map, mut proposer_map: ProposerMap, stores: TestStore| {
/*
* Set the proposer for this slot to be a validator that does not exist.
*/
let ssz = stores.block.get_serialized_block(&block.parent_hash.as_ref()).unwrap().unwrap();
let parent_block_slot = SszBlock::from_slice(&ssz[..]).unwrap().slot_number();
proposer_map.insert(parent_block_slot, params.total_validators + 1);
(block, attester_map, proposer_map, stores)
};
let status = run_block_validation_scenario(
&params,
mutator);
assert_eq!(status, Err(SszBlockValidationError::NoProposerSignature));
}
#[test]
fn test_block_validation_invalid_bad_proposer_map() {
let params = get_simple_params();
let mutator = |block, attester_map, _, stores| {
/*
* Initialize a new, empty proposer map
*/
let proposer_map = ProposerMap::new();
(block, attester_map, proposer_map, stores)
};
let status = run_block_validation_scenario(
&params,
mutator);
assert_eq!(status, Err(SszBlockValidationError::BadProposerMap));
}
#[test]
fn test_block_validation_invalid_2nd_attestation_signature() {
let params = get_simple_params();
let mutator = |mut block: Block, attester_map, proposer_map, stores| {
/*
* Set the second attestaion record to have an invalid signature.
*/
block.attestations[1].aggregate_sig = AggregateSignature::new();
(block, attester_map, proposer_map, stores)
};
let status = run_block_validation_scenario(
&params,
mutator);
assert_eq!(status, Err(SszBlockValidationError::AttestationValidationError(
AttestationValidationError::BadAggregateSignature)));
}

View File

@ -0,0 +1,20 @@
extern crate validation;
extern crate bls;
extern crate db;
extern crate hashing;
extern crate ssz;
extern crate ssz_helpers;
extern crate types;
#[cfg(test)]
mod attestation_validation;
#[cfg(test)]
mod block_validation;
/*
use lighthouse::bls;
use lighthouse::db;
use lighthouse::state;
use lighthouse::utils;
*/

View File

@ -4,7 +4,10 @@ version = "0.1.0"
authors = ["Paul Hauner <paul@paulhauner.com>"]
[dependencies]
blake2-rfc = "0.2.18"
bls = { path = "../../beacon_chain/utils/bls" }
bytes = "0.4.10"
rocksdb = "0.10.1"
blake2-rfc = "0.2.18"
ssz = { path = "../../beacon_chain/utils/ssz" }
ssz_helpers = { path = "../../beacon_chain/utils/ssz_helpers" }
types = { path = "../../beacon_chain/types" }

View File

@ -1,3 +1,9 @@
extern crate ssz_helpers;
use self::ssz_helpers::ssz_block::{
SszBlock,
SszBlockError,
};
use std::sync::Arc;
use super::{
ClientDB,
@ -5,6 +11,13 @@ use super::{
};
use super::BLOCKS_DB_COLUMN as DB_COLUMN;
#[derive(Clone, Debug, PartialEq)]
pub enum BlockAtSlotError {
UnknownBlock,
InvalidBlock,
DBError(String),
}
pub struct BlockStore<T>
where T: ClientDB
{
@ -42,17 +55,57 @@ impl<T: ClientDB> BlockStore<T> {
// TODO: implement logic for canonical chain
self.db.exists(DB_COLUMN, hash)
}
/// Retrieve the block at a slot given a "head_hash" and a slot.
///
/// A "head_hash" must be a block hash with a slot number greater than or equal to the desired
/// slot.
///
/// This function will read each block down the chain until it finds a block with the given
/// slot number. If the slot is skipped, the function will return None.
///
/// If a block is found, a tuple of (block_hash, serialized_block) is returned.
pub fn block_at_slot(&self, head_hash: &[u8], slot: u64)
-> Result<Option<(Vec<u8>, Vec<u8>)>, BlockAtSlotError>
{
match self.get_serialized_block(head_hash)? {
None => Err(BlockAtSlotError::UnknownBlock),
Some(ssz) => {
let block = SszBlock::from_slice(&ssz)
.map_err(|_| BlockAtSlotError::InvalidBlock)?;
match block.slot_number() {
s if s == slot => Ok(Some((head_hash.to_vec(), ssz.to_vec()))),
s if s < slot => Ok(None),
_ => self.block_at_slot(block.parent_hash(), slot)
}
}
}
}
}
impl From<DBError> for BlockAtSlotError {
fn from(e: DBError) -> Self {
BlockAtSlotError::DBError(e.message)
}
}
#[cfg(test)]
mod tests {
extern crate ssz;
extern crate types;
use self::types::block::Block;
use self::types::attestation_record::AttestationRecord;
use self::types::Hash256;
use self::ssz::SszStream;
use super::*;
use super::super::super::MemoryDB;
use std::thread;
use std::sync::Arc;
#[test]
fn test_block_store_on_disk_db() {
fn test_block_store_on_memory_db() {
let db = Arc::new(MemoryDB::open());
let bs = Arc::new(BlockStore::new(db.clone()));
@ -89,4 +142,74 @@ mod tests {
}
}
}
#[test]
fn test_block_at_slot() {
let db = Arc::new(MemoryDB::open());
let bs = Arc::new(BlockStore::new(db.clone()));
let blocks = (0..5).into_iter()
.map(|_| {
let mut block = Block::zero();
let ar = AttestationRecord::zero();
block.attestations.push(ar);
block
});
let hashes = [
Hash256::from("zero".as_bytes()),
Hash256::from("one".as_bytes()),
Hash256::from("two".as_bytes()),
Hash256::from("three".as_bytes()),
Hash256::from("four".as_bytes()),
];
let parent_hashes = [
Hash256::from("genesis".as_bytes()),
Hash256::from("zero".as_bytes()),
Hash256::from("one".as_bytes()),
Hash256::from("two".as_bytes()),
Hash256::from("three".as_bytes()),
];
let slots = [0, 1, 3, 4, 5];
for (i, mut block) in blocks.enumerate() {
block.parent_hash = parent_hashes[i];
block.slot_number = slots[i];
let mut s = SszStream::new();
s.append(&block);
let ssz = s.drain();
bs.put_serialized_block(&hashes[i].to_vec(), &ssz).unwrap();
}
let tuple = bs.block_at_slot(&hashes[4], 5).unwrap().unwrap();
let block = SszBlock::from_slice(&tuple.1).unwrap();
assert_eq!(block.slot_number(), 5);
assert_eq!(tuple.0, hashes[4].to_vec());
let tuple = bs.block_at_slot(&hashes[4], 4).unwrap().unwrap();
let block = SszBlock::from_slice(&tuple.1).unwrap();
assert_eq!(block.slot_number(), 4);
assert_eq!(tuple.0, hashes[3].to_vec());
let tuple = bs.block_at_slot(&hashes[4], 3).unwrap().unwrap();
let block = SszBlock::from_slice(&tuple.1).unwrap();
assert_eq!(block.slot_number(), 3);
assert_eq!(tuple.0, hashes[2].to_vec());
let tuple = bs.block_at_slot(&hashes[4], 0).unwrap().unwrap();
let block = SszBlock::from_slice(&tuple.1).unwrap();
assert_eq!(block.slot_number(), 0);
assert_eq!(tuple.0, hashes[0].to_vec());
let ssz = bs.block_at_slot(&hashes[4], 2).unwrap();
assert_eq!(ssz, None);
let ssz = bs.block_at_slot(&hashes[4], 6).unwrap();
assert_eq!(ssz, None);
let ssz = bs.block_at_slot(&Hash256::from("unknown".as_bytes()), 2);
assert_eq!(ssz, Err(BlockAtSlotError::UnknownBlock));
}
}

View File

@ -7,7 +7,10 @@ mod block_store;
mod pow_chain_store;
mod validator_store;
pub use self::block_store::BlockStore;
pub use self::block_store::{
BlockStore,
BlockAtSlotError,
};
pub use self::pow_chain_store::PoWChainStore;
pub use self::validator_store::{
ValidatorStore,