Add block validation code

This was created in another branch along with a whole bunch of other
commits building out databases and other common elements.

This full history of these works is available on master, however it
should not be necessary.
This commit is contained in:
Paul Hauner 2018-09-26 13:00:25 +10:00
parent c8ff539686
commit 9dcec214af
No known key found for this signature in database
GPG Key ID: 303E4494BB28068C
10 changed files with 883 additions and 1 deletions

View File

@ -1,10 +1,13 @@
use super::bls;
use super::common;
use super::db;
use super::ssz;
use super::utils;
mod structs;
mod ssz_splitter;
mod validation;
pub use self::structs::{
AttestationRecord,
@ -15,3 +18,7 @@ pub use self::ssz_splitter::{
split_one_attestation,
AttestationSplitError,
};
pub use self::validation::{
validate_attestation,
AttestationValidationError,
};

View File

@ -0,0 +1,203 @@
use std::collections::HashSet;
use std::sync::Arc;
use super::structs::AttestationRecord;
use super::AttesterMap;
use super::attestation_parent_hashes::{
attestation_parent_hashes,
ParentHashesError,
};
use super::db::{
ClientDB,
DBError
};
use super::db::stores::{
BlockStore,
ValidatorStore,
};
use super::utils::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 {
SlotTooHigh,
SlotTooLow,
JustifiedSlotTooHigh,
UnknownJustifiedBlock,
TooManyObliqueHashes,
BadCurrentHashes,
BadObliqueHashes,
BadAttesterMap,
IntWrapping,
PublicKeyCorrupt,
NoPublicKeyForValidator,
BadBitfieldLength,
InvalidBitfield,
InvalidBitfieldEndBits,
NoSignatures,
NonZeroTrailingBits,
AggregateSignatureFail,
DBError(String),
}
fn bytes_for_bits(bits: usize) -> usize {
(bits.saturating_sub(1) / 8) + 1
}
fn any_of_last_n_bits_are_set(byte: u8, n: usize) -> bool {
let shift = 8_u16.saturating_sub(n as u16);
((!0 << shift) & u16::from(byte)) > 0
}
pub fn validate_attestation<T>(a: &AttestationRecord,
block_slot: u64,
cycle_length: u8,
known_last_justified_slot: u64,
known_parent_hashes: &Arc<Vec<Hash256>>,
block_store: &Arc<BlockStore<T>>,
validator_store: &Arc<ValidatorStore<T>>,
attester_map: &Arc<AttesterMap>)
-> Result<Option<HashSet<usize>>, AttestationValidationError>
where T: ClientDB + Sized
{
/*
* The attesation slot must not be higher than the block that contained it.
*/
if a.slot > block_slot {
return Err(AttestationValidationError::SlotTooHigh);
}
/*
* The slot of this attestation must not be more than cycle_length + 1 distance
* from the block that contained it.
*
* The below code stays overflow-safe as long as cycle length is a < 64 bit integer.
*/
if a.slot < block_slot.saturating_sub(u64::from(cycle_length) + 1) {
return Err(AttestationValidationError::SlotTooLow);
}
/*
* The attestation must indicate that its last justified slot is the same as the last justified
* slot known to us.
*/
if a.justified_slot > known_last_justified_slot {
return Err(AttestationValidationError::JustifiedSlotTooHigh);
}
/*
* 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(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 = 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 indicies 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.
*/
let last_byte =
a.attester_bitfield.get_byte(a.attester_bitfield.num_bytes() - 1)
.ok_or(AttestationValidationError::InvalidBitfield)?;
if any_of_last_n_bits_are_set(*last_byte, a.attester_bitfield.len() % 8) {
return Err(AttestationValidationError::InvalidBitfieldEndBits)
}
/*
* The specified justified block hash must be known to us
*/
if !block_store.block_exists(&a.justified_block_hash)? {
return Err(AttestationValidationError::UnknownJustifiedBlock)
}
let signed_message = {
let parent_hashes = attestation_parent_hashes(
cycle_length,
block_slot,
a.slot,
&known_parent_hashes,
&a.oblique_parent_hashes)?;
generate_signed_message(
a.slot,
&parent_hashes,
a.shard_id,
&a.shard_block_hash,
a.justified_slot)
};
let voted_hashmap =
verify_aggregate_signature_for_indices(
&signed_message,
&a.aggregate_sig,
&attestation_indices,
&a.attester_bitfield,
&validator_store)?;
Ok(voted_hashmap)
}
impl From<ParentHashesError> for AttestationValidationError {
fn from(e: ParentHashesError) -> Self {
match e {
ParentHashesError::BadCurrentHashes
=> AttestationValidationError::BadCurrentHashes,
ParentHashesError::BadObliqueHashes
=> AttestationValidationError::BadObliqueHashes,
ParentHashesError::SlotTooLow
=> AttestationValidationError::SlotTooLow,
ParentHashesError::SlotTooHigh
=> AttestationValidationError::SlotTooHigh,
ParentHashesError::IntWrapping
=> AttestationValidationError::IntWrapping
}
}
}
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,32 @@
use super::ssz::SszStream;
use super::utils::hash::canonical_hash;
use super::utils::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)
}

View File

@ -0,0 +1,16 @@
use super::common::maps::AttesterMap;
use super::db;
use super::bls;
use super::structs;
use super::ssz;
use super::common::attestation_parent_hashes;
use super::utils;
mod attestation_validation;
mod signature_verification;
mod message_generation;
pub use self::attestation_validation::{
validate_attestation,
AttestationValidationError,
};

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::utils::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 to the 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

@ -1,11 +1,14 @@
extern crate blake2_rfc;
use super::attestation_record;
use super::common;
use super::db;
use super::ssz;
use super::utils;
use super::attestation_record;
mod structs;
mod ssz_block;
mod validation;
pub use self::structs::Block;
pub use self::ssz_block::SszBlock;

View File

@ -0,0 +1,19 @@
mod validate_ssz_block;
#[cfg(test)]
mod tests;
use super::attestation_record;
use super::SszBlock;
use super::db;
use super::ssz;
use super::utils;
use super::common::maps::{
AttesterMap,
ProposerMap,
};
pub use self::validate_ssz_block::{
validate_ssz_block,
SszBlockValidationError,
BlockStatus,
};

View File

@ -0,0 +1,191 @@
extern crate ssz;
use self::ssz::{
SszStream,
};
use std::sync::Arc;
use super::{
validate_ssz_block,
BlockStatus,
AttesterMap,
ProposerMap,
};
use super::db::stores::{
BlockStore,
PoWChainStore,
ValidatorStore,
};
use super::db::{
MemoryDB,
};
use super::utils::hash::canonical_hash;
use super::utils::types::{
Hash256,
Bitfield,
};
use super::SszBlock;
use super::super::Block;
use super::super::attestation_record::AttestationRecord;
use super::super::super::bls::{
Keypair,
Signature,
AggregateSignature,
};
struct TestStore {
db: Arc<MemoryDB>,
block: Arc<BlockStore<MemoryDB>>,
pow_chain: Arc<PoWChainStore<MemoryDB>>,
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,
}
}
}
#[test]
fn test_block_validation() {
let stores = TestStore::new();
let cycle_length: u8 = 2;
let shard_count: u16 = 2;
let present_slot = u64::from(cycle_length) * 10000;
let justified_slot = present_slot - u64::from(cycle_length);
let justified_block_hash = Hash256::from("justified_hash".as_bytes());
let shard_block_hash = Hash256::from("shard_hash".as_bytes());
let parent_hashes: Vec<Hash256> = (0..(cycle_length * 2))
.map(|i| Hash256::from(i as u64))
.collect();
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());
stores.pow_chain.put_block_hash(pow_chain_ref.as_ref()).unwrap();
stores.block.put_block(justified_block_hash.as_ref(), &vec![42]).unwrap();
let validators_per_shard = 10;
let block_slot = present_slot;
let validator_index: usize = 0;
let proposer_map = {
let mut proposer_map = ProposerMap::new();
proposer_map.insert(present_slot, validator_index);
proposer_map
};
let (attester_map, attestations, _keypairs) = {
let mut i = 0;
let mut attester_map = AttesterMap::new();
let mut attestations = vec![];
let mut keypairs = vec![];
for shard in 0..shard_count {
let mut attesters = vec![];
let mut attester_bitfield = Bitfield::new();
let mut aggregate_sig = AggregateSignature::new();
let attestation_slot = block_slot - 1;
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]
};
let attestation_message = {
let mut stream = SszStream::new();
stream.append(&attestation_slot);
stream.append_vec(&parent_hashes_slice.to_vec());
stream.append(&shard);
stream.append(&shard_block_hash);
stream.append(&justified_slot);
let bytes = stream.drain();
canonical_hash(&bytes)
};
for attestation_index in 0..validators_per_shard {
/*
* Add the attester to the attestation indices for this shard.
*/
attesters.push(i);
/*
* Set the voters bit on the bitfield to true.
*/
attester_bitfield.set_bit(attestation_index, true);
/*
* Generate a random keypair for this validatior and clone it into the
* list of keypairs.
*/
let keypair = Keypair::random();
keypairs.push(keypair.clone());
/*
* Store the validators public key in the database.
*/
stores.validator.put_public_key_by_index(i, &keypair.pk).unwrap();
/*
* Generate a new signature and aggregate it on the rolling signature.
*/
let sig = Signature::new(&attestation_message, &keypair.sk);
aggregate_sig.add(&sig);
/*
* Increment the validator counter to monotonically assign validators.
*/
i += 1;
}
attester_map.insert((attestation_slot, shard), attesters);
let attestation = AttestationRecord {
slot: attestation_slot,
shard_id: shard,
oblique_parent_hashes: vec![],
shard_block_hash,
attester_bitfield,
justified_slot,
justified_block_hash,
aggregate_sig,
};
attestations.push(attestation);
}
(attester_map, attestations, keypairs)
};
let block = Block {
parent_hash: Hash256::from("parent".as_bytes()),
slot_number: block_slot,
randao_reveal: Hash256::from("randao".as_bytes()),
attestations,
pow_chain_ref,
active_state_root,
crystallized_state_root,
};
let mut stream = SszStream::new();
stream.append(&block);
let serialized_block = stream.drain();
let ssz_block = SszBlock::from_slice(&serialized_block[..]).unwrap();
let status = validate_ssz_block(
&ssz_block,
present_slot,
cycle_length,
justified_slot,
&Arc::new(parent_hashes),
&Arc::new(proposer_map),
&Arc::new(attester_map),
&stores.block.clone(),
&stores.validator.clone(),
&stores.pow_chain.clone()).unwrap();
assert_eq!(status, BlockStatus::NewBlock);
}

View File

@ -0,0 +1,227 @@
use std::sync::Arc;
use super::attestation_record::{
validate_attestation,
AttestationValidationError,
};
use super::attestation_record::{
AttestationRecord,
split_one_attestation,
split_all_attestations,
AttestationSplitError,
};
use super::{
AttesterMap,
ProposerMap,
};
use super::SszBlock;
use super::db::{
ClientDB,
DBError,
};
use super::db::stores::{
BlockStore,
PoWChainStore,
ValidatorStore,
};
use super::ssz::{
Decodable,
DecodeError,
};
use super::utils::types::Hash256;
#[derive(Debug, PartialEq)]
pub enum BlockStatus {
NewBlock,
KnownBlock,
}
#[derive(Debug, PartialEq)]
pub enum SszBlockValidationError {
FutureSlot,
UnknownPoWChainRef,
BadAttestationSsz,
AttestationValidationError(AttestationValidationError),
AttestationSignatureFailed,
FirstAttestationSignatureFailed,
NoProposerSignature,
BadProposerMap,
DatabaseError(String),
}
/// Validate some SszBlock. 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.
///
/// Of course, this function will only be more efficient if a block is already serialized.
/// Serializing a complete block and then validating with this function will be less
/// efficient than just validating the original block.
///
/// This function will determine if the block is new, already known or invalid (either
/// intrinsically or due to some application error.)
#[allow(dead_code)]
pub fn validate_ssz_block<T>(b: &SszBlock,
expected_slot: u64,
cycle_length: u8,
last_justified_slot: u64,
parent_hashes: &Arc<Vec<Hash256>>,
proposer_map: &Arc<ProposerMap>,
attester_map: &Arc<AttesterMap>,
block_store: &Arc<BlockStore<T>>,
validator_store: &Arc<ValidatorStore<T>>,
pow_store: &Arc<PoWChainStore<T>>)
-> Result<BlockStatus, SszBlockValidationError>
where T: ClientDB + Sized
{
/*
* If this block is already known, return immediately.
*/
if block_store.block_exists(&b.block_hash())? {
return Ok(BlockStatus::KnownBlock);
}
/*
* Copy the block slot (will be used multiple times)
*/
let block_slot = b.slot_number();
/*
* If the block slot corresponds to a slot in the future (according to the local time),
* drop it.
*/
if block_slot > expected_slot {
return Err(SszBlockValidationError::FutureSlot);
}
/*
* 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 "sufficienty deep in the canonical PoW chain".
*/
if !pow_store.block_hash_exists(b.pow_chain_ref())? {
return Err(SszBlockValidationError::UnknownPoWChainRef);
}
/*
* Store a reference to the serialized attestations from the block.
*/
let attestations_ssz = &b.attestations();
/*
* Get a slice of the first serialized attestation (the 0th) and decode it into
* a full AttestationRecord object.
*/
let (first_attestation_ssz, next_index) = split_one_attestation(
&attestations_ssz,
0)?;
let (first_attestation, _) = AttestationRecord::ssz_decode(
&first_attestation_ssz, 0)?;
/*
* Validate this first attestation.
*
* It is a requirement that the block proposer for this slot
* must have signed the 0th attestation record.
*/
let attestation_voters = validate_attestation(
&first_attestation,
block_slot,
cycle_length,
last_justified_slot,
&parent_hashes,
&block_store,
&validator_store,
&attester_map)?;
/*
* If the set of voters is None, the attestation was invalid.
*/
let attestation_voters = attestation_voters
.ok_or(SszBlockValidationError::
FirstAttestationSignatureFailed)?;
/*
* Read the proposer from the map of slot -> validator index.
*/
let proposer = proposer_map.get(&block_slot)
.ok_or(SszBlockValidationError::BadProposerMap)?;
/*
* If the proposer for this slot was not a voter, reject the block.
*/
if !attestation_voters.contains(&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.
*
* TODO: make this parallelized.
*/
for attestation in other_attestations {
let (a, _) = AttestationRecord::ssz_decode(&attestation, 0)?;
let attestation_voters = validate_attestation(
&a,
block_slot,
cycle_length,
last_justified_slot,
&parent_hashes,
&block_store,
&validator_store,
&attester_map)?;
if attestation_voters.is_none() {
return Err(SszBlockValidationError::
AttestationSignatureFailed);
}
}
/*
* If we have reached this point, the block is a new valid block that is worthy of
* processing.
*/
Ok(BlockStatus::NewBlock)
}
impl From<DBError> for SszBlockValidationError {
fn from(e: DBError) -> Self {
SszBlockValidationError::DatabaseError(e.message)
}
}
impl From<AttestationSplitError> for SszBlockValidationError {
fn from(e: AttestationSplitError) -> Self {
match e {
AttestationSplitError::TooShort =>
SszBlockValidationError::BadAttestationSsz
}
}
}
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)
}
}

View File

@ -16,4 +16,5 @@ pub mod shard_and_committee;
pub mod validator_record;
use super::bls;
use super::db;
use super::utils;