Merge pull request #187 from sigp/slot-newtype

[WIP] Add `Slot` and `Epoch` newtypes
This commit is contained in:
Age Manning 2019-02-07 14:24:16 +11:00 committed by GitHub
commit a0eb359738
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
47 changed files with 1006 additions and 320 deletions

View File

@ -12,7 +12,7 @@ use types::{
beacon_state::{BlockProcessingError, CommitteesError, SlotProcessingError},
readers::{BeaconBlockReader, BeaconStateReader},
AttestationData, BeaconBlock, BeaconBlockBody, BeaconState, ChainSpec, Eth1Data,
FreeAttestation, Hash256, PublicKey, Signature,
FreeAttestation, Hash256, PublicKey, Signature, Slot,
};
use crate::attestation_aggregator::{AttestationAggregator, Outcome as AggregationOutcome};
@ -196,10 +196,10 @@ where
/// It is important to note that this is _not_ the state corresponding to the canonical head
/// block, instead it is that state which may or may not have had additional per slot/epoch
/// processing applied to it.
pub fn advance_state(&self, slot: u64) -> Result<(), SlotProcessingError> {
pub fn advance_state(&self, slot: Slot) -> Result<(), SlotProcessingError> {
let state_slot = self.state.read().slot;
let head_block_root = self.head().beacon_block_root;
for _ in state_slot..slot {
for _ in state_slot.as_u64()..slot.as_u64() {
self.state
.write()
.per_slot_processing(head_block_root.clone(), &self.spec)?;
@ -246,9 +246,10 @@ where
/// This is distinct to `present_slot`, which simply reads the latest state. If a
/// call to `read_slot_clock` results in a higher slot than a call to `present_slot`,
/// `self.state` should undergo per slot processing.
pub fn read_slot_clock(&self) -> Option<u64> {
pub fn read_slot_clock(&self) -> Option<Slot> {
match self.slot_clock.present_slot() {
Ok(some_slot) => some_slot,
Ok(Some(some_slot)) => Some(some_slot),
Ok(None) => None,
_ => None,
}
}
@ -258,7 +259,7 @@ where
/// This is distinct to `read_slot_clock`, which reads from the actual system clock. If
/// `self.state` has not been transitioned it is possible for the system clock to be on a
/// different slot to what is returned from this call.
pub fn present_slot(&self) -> u64 {
pub fn present_slot(&self) -> Slot {
self.state.read().slot
}
@ -266,7 +267,7 @@ where
///
/// Information is read from the present `beacon_state` shuffling, so only information from the
/// present and prior epoch is available.
pub fn block_proposer(&self, slot: u64) -> Result<usize, CommitteesError> {
pub fn block_proposer(&self, slot: Slot) -> Result<usize, CommitteesError> {
let index = self
.state
.read()
@ -276,7 +277,7 @@ where
}
/// Returns the justified slot for the present state.
pub fn justified_slot(&self) -> u64 {
pub fn justified_slot(&self) -> Slot {
self.state.read().justified_slot
}
@ -287,7 +288,7 @@ where
pub fn validator_attestion_slot_and_shard(
&self,
validator_index: usize,
) -> Result<Option<(u64, u64)>, CommitteesError> {
) -> Result<Option<(Slot, u64)>, CommitteesError> {
if let Some((slot, shard, _committee)) = self
.state
.read()
@ -461,7 +462,7 @@ where
// Transition the parent state to the present slot.
let mut state = parent_state;
for _ in state.slot..present_slot {
for _ in state.slot.as_u64()..present_slot.as_u64() {
if let Err(e) = state.per_slot_processing(parent_block_root.clone(), &self.spec) {
return Ok(BlockProcessingOutcome::InvalidBlock(
InvalidBlock::SlotProcessingError(e),
@ -533,7 +534,7 @@ where
);
let parent_root = state
.get_block_root(state.slot.saturating_sub(1), &self.spec)?
.get_block_root(state.slot.saturating_sub(1_u64), &self.spec)?
.clone();
let mut block = BeaconBlock {

View File

@ -9,7 +9,7 @@ use std::sync::Arc;
use types::{
readers::{BeaconBlockReader, BeaconStateReader},
validator_registry::get_active_validator_indices,
Hash256,
Hash256, Slot,
};
#[derive(Debug, PartialEq)]
@ -118,7 +118,7 @@ fn get_vote_count<T: ClientDB>(
block_store: &Arc<BeaconBlockStore<T>>,
attestation_targets: &[Hash256],
block_root: &Hash256,
slot: u64,
slot: Slot,
) -> Result<u64, Error> {
let mut count = 0;
for target in attestation_targets {
@ -138,7 +138,7 @@ fn get_child_hashes_and_slots<T: ClientDB>(
block_store: &Arc<BeaconBlockStore<T>>,
root_hash: &Hash256,
leaf_hashes: &HashSet<Hash256>,
) -> Result<HashSet<(Hash256, u64)>, Error> {
) -> Result<HashSet<(Hash256, Slot)>, Error> {
let mut hash_set = HashSet::new();
for leaf_hash in leaf_hashes {

View File

@ -13,7 +13,7 @@ use std::fs::File;
use std::io::prelude::*;
use std::iter::FromIterator;
use std::sync::Arc;
use types::{BeaconBlock, ChainSpec, FreeAttestation, Keypair, Validator};
use types::{BeaconBlock, ChainSpec, FreeAttestation, Keypair, Slot, Validator};
/// The beacon chain harness simulates a single beacon node with `validator_count` validators connected
/// to it. Each validator is provided a borrow to the beacon chain, where it may read
@ -40,7 +40,7 @@ impl BeaconChainHarness {
let block_store = Arc::new(BeaconBlockStore::new(db.clone()));
let state_store = Arc::new(BeaconStateStore::new(db.clone()));
let slot_clock = TestingSlotClock::new(spec.genesis_slot);
let slot_clock = TestingSlotClock::new(spec.genesis_slot.as_u64());
// Remove the validators present in the spec (if any).
spec.initial_validators = Vec::with_capacity(validator_count);
@ -60,7 +60,7 @@ impl BeaconChainHarness {
.par_iter()
.map(|keypair| Validator {
pubkey: keypair.pk.clone(),
activation_slot: 0,
activation_slot: Slot::new(0),
..std::default::Default::default()
})
.collect();
@ -115,12 +115,12 @@ impl BeaconChainHarness {
/// This is the equivalent of advancing a system clock forward one `SLOT_DURATION`.
///
/// Returns the new slot.
pub fn increment_beacon_chain_slot(&mut self) -> u64 {
pub fn increment_beacon_chain_slot(&mut self) -> Slot {
let slot = self.beacon_chain.present_slot() + 1;
debug!("Incrementing BeaconChain slot to {}.", slot);
self.beacon_chain.slot_clock.set_slot(slot);
self.beacon_chain.slot_clock.set_slot(slot.as_u64());
self.beacon_chain.advance_state(slot).unwrap();
slot
}

View File

@ -11,7 +11,7 @@ use db::ClientDB;
use parking_lot::RwLock;
use slot_clock::SlotClock;
use std::sync::Arc;
use types::{AttestationData, BeaconBlock, FreeAttestation, PublicKey, Signature};
use types::{AttestationData, BeaconBlock, FreeAttestation, PublicKey, Signature, Slot};
// mod attester;
// mod producer;
@ -51,7 +51,7 @@ impl<T: ClientDB, U: SlotClock> DirectBeaconNode<T, U> {
impl<T: ClientDB, U: SlotClock> AttesterBeaconNode for DirectBeaconNode<T, U> {
fn produce_attestation_data(
&self,
_slot: u64,
_slot: Slot,
shard: u64,
) -> Result<Option<AttestationData>, NodeError> {
match self.beacon_chain.produce_attestation_data(shard) {
@ -87,7 +87,7 @@ impl<T: ClientDB, U: SlotClock> BeaconBlockNode for DirectBeaconNode<T, U> {
/// Requests a new `BeaconBlock from the `BeaconChain`.
fn produce_beacon_block(
&self,
slot: u64,
slot: Slot,
randao_reveal: &Signature,
) -> Result<Option<BeaconBlock>, BeaconBlockNodeError> {
let (block, _state) = self

View File

@ -8,7 +8,7 @@ use block_producer::{
use db::ClientDB;
use slot_clock::SlotClock;
use std::sync::Arc;
use types::PublicKey;
use types::{PublicKey, Slot};
/// Connects directly to a borrowed `BeaconChain` and reads attester/proposer duties directly from
/// it.
@ -27,7 +27,7 @@ impl<T: ClientDB, U: SlotClock> DirectDuties<T, U> {
}
impl<T: ClientDB, U: SlotClock> ProducerDutiesReader for DirectDuties<T, U> {
fn is_block_production_slot(&self, slot: u64) -> Result<bool, ProducerDutiesReaderError> {
fn is_block_production_slot(&self, slot: Slot) -> Result<bool, ProducerDutiesReaderError> {
let validator_index = self
.beacon_chain
.validator_index(&self.pubkey)
@ -49,7 +49,7 @@ impl<T: ClientDB, U: SlotClock> AttesterDutiesReader for DirectDuties<T, U> {
}
}
fn attestation_shard(&self, slot: u64) -> Result<Option<u64>, AttesterDutiesReaderError> {
fn attestation_shard(&self, slot: Slot) -> Result<Option<u64>, AttesterDutiesReaderError> {
if let Some(validator_index) = self.validator_index() {
match self
.beacon_chain

View File

@ -9,7 +9,7 @@ use block_producer::{BlockProducer, Error as BlockPollError};
use db::MemoryDB;
use slot_clock::TestingSlotClock;
use std::sync::Arc;
use types::{BeaconBlock, ChainSpec, FreeAttestation, Keypair};
use types::{BeaconBlock, ChainSpec, FreeAttestation, Keypair, Slot};
#[derive(Debug, PartialEq)]
pub enum BlockProduceError {
@ -59,7 +59,7 @@ impl ValidatorHarness {
beacon_chain: Arc<BeaconChain<MemoryDB, TestingSlotClock>>,
spec: Arc<ChainSpec>,
) -> Self {
let slot_clock = Arc::new(TestingSlotClock::new(spec.genesis_slot));
let slot_clock = Arc::new(TestingSlotClock::new(spec.genesis_slot.as_u64()));
let signer = Arc::new(LocalSigner::new(keypair.clone()));
let beacon_node = Arc::new(DirectBeaconNode::new(beacon_chain.clone()));
let epoch_map = Arc::new(DirectDuties::new(keypair.pk.clone(), beacon_chain.clone()));
@ -127,7 +127,7 @@ impl ValidatorHarness {
/// Set the validators slot clock to the specified slot.
///
/// The validators slot clock will always read this value until it is set to something else.
pub fn set_slot(&mut self, slot: u64) {
self.slot_clock.set_slot(slot)
pub fn set_slot(&mut self, slot: Slot) {
self.slot_clock.set_slot(slot.as_u64())
}
}

View File

@ -1,13 +1,13 @@
use env_logger::{Builder, Env};
use log::debug;
use test_harness::BeaconChainHarness;
use types::ChainSpec;
use types::{ChainSpec, Slot};
#[test]
#[ignore]
fn it_can_build_on_genesis_block() {
let mut spec = ChainSpec::foundation();
spec.genesis_slot = spec.epoch_length * 8;
spec.genesis_slot = Slot::new(spec.epoch_length * 8);
/*
spec.shard_count = spec.shard_count / 8;

View File

@ -2,7 +2,7 @@ use super::BLOCKS_DB_COLUMN as DB_COLUMN;
use super::{ClientDB, DBError};
use ssz::Decodable;
use std::sync::Arc;
use types::{readers::BeaconBlockReader, BeaconBlock, Hash256};
use types::{readers::BeaconBlockReader, BeaconBlock, Hash256, Slot};
#[derive(Clone, Debug, PartialEq)]
pub enum BeaconBlockAtSlotError {
@ -71,7 +71,7 @@ impl<T: ClientDB> BeaconBlockStore<T> {
pub fn block_at_slot(
&self,
head_hash: &Hash256,
slot: u64,
slot: Slot,
) -> Result<Option<(Hash256, impl BeaconBlockReader)>, BeaconBlockAtSlotError> {
let mut current_hash = *head_hash;
@ -119,12 +119,12 @@ mod tests {
let mut rng = XorShiftRng::from_seed([42; 16]);
let mut block = BeaconBlock::random_for_test(&mut rng);
block.slot = 10;
block.slot = Slot::from(10_u64);
let block_root = block.canonical_root();
bs.put(&block_root, &ssz_encode(&block)).unwrap();
let result = bs.block_at_slot(&block_root, 11).unwrap();
let result = bs.block_at_slot(&block_root, Slot::from(11_u64)).unwrap();
assert_eq!(result, None);
}
@ -138,7 +138,7 @@ mod tests {
db.put(DB_COLUMN, hash, ssz).unwrap();
assert_eq!(
store.block_at_slot(hash, 42),
store.block_at_slot(hash, Slot::from(42_u64)),
Err(BeaconBlockAtSlotError::DBError(
"Bad BeaconBlock SSZ.".into()
))
@ -156,7 +156,7 @@ mod tests {
db.put(DB_COLUMN, hash, ssz).unwrap();
assert_eq!(
store.block_at_slot(other_hash, 42),
store.block_at_slot(other_hash, Slot::from(42_u64)),
Err(BeaconBlockAtSlotError::UnknownBeaconBlock(*other_hash))
);
}
@ -221,7 +221,7 @@ mod tests {
Hash256::from(&[2; 32][..]),
Hash256::from(&[3; 32][..]),
];
let slots = [0, 1, 3, 4, 5];
let slots: Vec<Slot> = vec![0, 1, 3, 4, 5].iter().map(|x| Slot::new(*x)).collect();
// Generate a vec of random blocks and store them in the DB.
let block_count = 5;
@ -249,14 +249,14 @@ mod tests {
assert_eq!(reader.slot(), slots[slot_index]);
}
let ssz = bs.block_at_slot(&hashes[4], 2).unwrap();
let ssz = bs.block_at_slot(&hashes[4], Slot::new(2)).unwrap();
assert_eq!(ssz, None);
let ssz = bs.block_at_slot(&hashes[4], 6).unwrap();
let ssz = bs.block_at_slot(&hashes[4], Slot::new(6)).unwrap();
assert_eq!(ssz, None);
let bad_hash = &Hash256::from("unknown".as_bytes());
let ssz = bs.block_at_slot(bad_hash, 2);
let ssz = bs.block_at_slot(bad_hash, Slot::new(2));
assert_eq!(
ssz,
Err(BeaconBlockAtSlotError::UnknownBeaconBlock(*bad_hash))

View File

@ -3,7 +3,7 @@ mod traits;
use slot_clock::SlotClock;
use std::sync::Arc;
use types::{AttestationData, FreeAttestation, Signature};
use types::{AttestationData, FreeAttestation, Signature, Slot};
pub use self::traits::{
BeaconNode, BeaconNodeError, DutiesReader, DutiesReaderError, PublishOutcome, Signer,
@ -13,14 +13,14 @@ const PHASE_0_CUSTODY_BIT: bool = false;
#[derive(Debug, PartialEq)]
pub enum PollOutcome {
AttestationProduced(u64),
AttestationNotRequired(u64),
SlashableAttestationNotProduced(u64),
BeaconNodeUnableToProduceAttestation(u64),
ProducerDutiesUnknown(u64),
SlotAlreadyProcessed(u64),
SignerRejection(u64),
ValidatorIsUnknown(u64),
AttestationProduced(Slot),
AttestationNotRequired(Slot),
SlashableAttestationNotProduced(Slot),
BeaconNodeUnableToProduceAttestation(Slot),
ProducerDutiesUnknown(Slot),
SlotAlreadyProcessed(Slot),
SignerRejection(Slot),
ValidatorIsUnknown(Slot),
}
#[derive(Debug, PartialEq)]
@ -40,7 +40,7 @@ pub enum Error {
///
/// Relies upon an external service to keep the `EpochDutiesMap` updated.
pub struct Attester<T: SlotClock, U: BeaconNode, V: DutiesReader, W: Signer> {
pub last_processed_slot: Option<u64>,
pub last_processed_slot: Option<Slot>,
duties: Arc<V>,
slot_clock: Arc<T>,
beacon_node: Arc<U>,
@ -91,7 +91,7 @@ impl<T: SlotClock, U: BeaconNode, V: DutiesReader, W: Signer> Attester<T, U, V,
}
}
fn produce_attestation(&mut self, slot: u64, shard: u64) -> Result<PollOutcome, Error> {
fn produce_attestation(&mut self, slot: Slot, shard: u64) -> Result<PollOutcome, Error> {
let attestation_data = match self.beacon_node.produce_attestation_data(slot, shard)? {
Some(attestation_data) => attestation_data,
None => return Ok(PollOutcome::BeaconNodeUnableToProduceAttestation(slot)),
@ -122,7 +122,7 @@ impl<T: SlotClock, U: BeaconNode, V: DutiesReader, W: Signer> Attester<T, U, V,
Ok(PollOutcome::AttestationProduced(slot))
}
fn is_processed_slot(&self, slot: u64) -> bool {
fn is_processed_slot(&self, slot: Slot) -> bool {
match self.last_processed_slot {
Some(processed_slot) if slot <= processed_slot => true,
_ => false,
@ -193,7 +193,7 @@ mod tests {
let signer = Arc::new(LocalSigner::new(Keypair::random()));
let mut duties = EpochMap::new(spec.epoch_length);
let attest_slot = 100;
let attest_slot = Slot::new(100);
let attest_epoch = attest_slot / spec.epoch_length;
let attest_shard = 12;
duties.insert_attestation_shard(attest_slot, attest_shard);
@ -212,28 +212,28 @@ mod tests {
beacon_node.set_next_publish_result(Ok(PublishOutcome::ValidAttestation));
// One slot before attestation slot...
slot_clock.set_slot(attest_slot - 1);
slot_clock.set_slot(attest_slot.as_u64() - 1);
assert_eq!(
attester.poll(),
Ok(PollOutcome::AttestationNotRequired(attest_slot - 1))
);
// On the attest slot...
slot_clock.set_slot(attest_slot);
slot_clock.set_slot(attest_slot.as_u64());
assert_eq!(
attester.poll(),
Ok(PollOutcome::AttestationProduced(attest_slot))
);
// Trying the same attest slot again...
slot_clock.set_slot(attest_slot);
slot_clock.set_slot(attest_slot.as_u64());
assert_eq!(
attester.poll(),
Ok(PollOutcome::SlotAlreadyProcessed(attest_slot))
);
// One slot after the attest slot...
slot_clock.set_slot(attest_slot + 1);
slot_clock.set_slot(attest_slot.as_u64() + 1);
assert_eq!(
attester.poll(),
Ok(PollOutcome::AttestationNotRequired(attest_slot + 1))
@ -241,7 +241,7 @@ mod tests {
// In an epoch without known duties...
let slot = (attest_epoch + 1) * spec.epoch_length;
slot_clock.set_slot(slot);
slot_clock.set_slot(slot.into());
assert_eq!(
attester.poll(),
Ok(PollOutcome::ProducerDutiesUnknown(slot))

View File

@ -1,10 +1,11 @@
use crate::{DutiesReader, DutiesReaderError};
use std::collections::HashMap;
use types::{Epoch, Slot};
pub struct EpochMap {
epoch_length: u64,
validator_index: Option<u64>,
map: HashMap<u64, (u64, u64)>,
map: HashMap<Epoch, (Slot, u64)>,
}
impl EpochMap {
@ -16,9 +17,8 @@ impl EpochMap {
}
}
pub fn insert_attestation_shard(&mut self, slot: u64, shard: u64) {
let epoch = slot / self.epoch_length;
pub fn insert_attestation_shard(&mut self, slot: Slot, shard: u64) {
let epoch = slot.epoch(self.epoch_length);
self.map.insert(epoch, (slot, shard));
}
@ -28,8 +28,8 @@ impl EpochMap {
}
impl DutiesReader for EpochMap {
fn attestation_shard(&self, slot: u64) -> Result<Option<u64>, DutiesReaderError> {
let epoch = slot / self.epoch_length;
fn attestation_shard(&self, slot: Slot) -> Result<Option<u64>, DutiesReaderError> {
let epoch = slot.epoch(self.epoch_length);
match self.map.get(&epoch) {
Some((attest_slot, attest_shard)) if *attest_slot == slot => Ok(Some(*attest_shard)),

View File

@ -1,6 +1,6 @@
use crate::traits::{BeaconNode, BeaconNodeError, PublishOutcome};
use std::sync::RwLock;
use types::{AttestationData, FreeAttestation};
use types::{AttestationData, FreeAttestation, Slot};
type ProduceResult = Result<Option<AttestationData>, BeaconNodeError>;
type PublishResult = Result<PublishOutcome, BeaconNodeError>;
@ -8,7 +8,7 @@ type PublishResult = Result<PublishOutcome, BeaconNodeError>;
/// A test-only struct used to simulate a Beacon Node.
#[derive(Default)]
pub struct SimulatedBeaconNode {
pub produce_input: RwLock<Option<(u64, u64)>>,
pub produce_input: RwLock<Option<(Slot, u64)>>,
pub produce_result: RwLock<Option<ProduceResult>>,
pub publish_input: RwLock<Option<FreeAttestation>>,
@ -26,7 +26,7 @@ impl SimulatedBeaconNode {
}
impl BeaconNode for SimulatedBeaconNode {
fn produce_attestation_data(&self, slot: u64, shard: u64) -> ProduceResult {
fn produce_attestation_data(&self, slot: Slot, shard: u64) -> ProduceResult {
*self.produce_input.write().unwrap() = Some((slot, shard));
match *self.produce_result.read().unwrap() {
Some(ref r) => r.clone(),

View File

@ -1,4 +1,4 @@
use types::{AttestationData, FreeAttestation, Signature};
use types::{AttestationData, FreeAttestation, Signature, Slot};
#[derive(Debug, PartialEq, Clone)]
pub enum BeaconNodeError {
@ -16,7 +16,7 @@ pub enum PublishOutcome {
pub trait BeaconNode: Send + Sync {
fn produce_attestation_data(
&self,
slot: u64,
slot: Slot,
shard: u64,
) -> Result<Option<AttestationData>, BeaconNodeError>;
@ -37,7 +37,7 @@ pub enum DutiesReaderError {
/// Informs a validator of their duties (e.g., block production).
pub trait DutiesReader: Send + Sync {
/// Returns `Some(shard)` if this slot is an attestation slot. Otherwise, returns `None.`
fn attestation_shard(&self, slot: u64) -> Result<Option<u64>, DutiesReaderError>;
fn attestation_shard(&self, slot: Slot) -> Result<Option<u64>, DutiesReaderError>;
/// Returns `Some(shard)` if this slot is an attestation slot. Otherwise, returns `None.`
fn validator_index(&self) -> Option<u64>;

View File

@ -4,7 +4,7 @@ mod traits;
use slot_clock::SlotClock;
use ssz::ssz_encode;
use std::sync::Arc;
use types::{BeaconBlock, ChainSpec, PublicKey};
use types::{BeaconBlock, ChainSpec, PublicKey, Slot};
pub use self::traits::{
BeaconNode, BeaconNodeError, DutiesReader, DutiesReaderError, PublishOutcome, Signer,
@ -13,21 +13,21 @@ pub use self::traits::{
#[derive(Debug, PartialEq)]
pub enum PollOutcome {
/// A new block was produced.
BlockProduced(u64),
BlockProduced(Slot),
/// A block was not produced as it would have been slashable.
SlashableBlockNotProduced(u64),
SlashableBlockNotProduced(Slot),
/// The validator duties did not require a block to be produced.
BlockProductionNotRequired(u64),
BlockProductionNotRequired(Slot),
/// The duties for the present epoch were not found.
ProducerDutiesUnknown(u64),
ProducerDutiesUnknown(Slot),
/// The slot has already been processed, execution was skipped.
SlotAlreadyProcessed(u64),
SlotAlreadyProcessed(Slot),
/// The Beacon Node was unable to produce a block at that slot.
BeaconNodeUnableToProduceBlock(u64),
BeaconNodeUnableToProduceBlock(Slot),
/// The signer failed to sign the message.
SignerRejection(u64),
SignerRejection(Slot),
/// The public key for this validator is not an active validator.
ValidatorIsUnknown(u64),
ValidatorIsUnknown(Slot),
}
#[derive(Debug, PartialEq)]
@ -47,7 +47,7 @@ pub enum Error {
///
/// Relies upon an external service to keep the `EpochDutiesMap` updated.
pub struct BlockProducer<T: SlotClock, U: BeaconNode, V: DutiesReader, W: Signer> {
pub last_processed_slot: Option<u64>,
pub last_processed_slot: Option<Slot>,
pubkey: PublicKey,
spec: Arc<ChainSpec>,
epoch_map: Arc<V>,
@ -115,7 +115,7 @@ impl<T: SlotClock, U: BeaconNode, V: DutiesReader, W: Signer> BlockProducer<T, U
}
}
fn is_processed_slot(&self, slot: u64) -> bool {
fn is_processed_slot(&self, slot: Slot) -> bool {
match self.last_processed_slot {
Some(processed_slot) if processed_slot >= slot => true,
_ => false,
@ -132,7 +132,7 @@ impl<T: SlotClock, U: BeaconNode, V: DutiesReader, W: Signer> BlockProducer<T, U
///
/// The slash-protection code is not yet implemented. There is zero protection against
/// slashing.
fn produce_block(&mut self, slot: u64) -> Result<PollOutcome, Error> {
fn produce_block(&mut self, slot: Slot) -> Result<PollOutcome, Error> {
let randao_reveal = {
let producer_nonce = self.beacon_node.proposer_nonce(&self.pubkey)?;
@ -236,8 +236,8 @@ mod tests {
let signer = Arc::new(LocalSigner::new(Keypair::random()));
let mut epoch_map = EpochMap::new(spec.epoch_length);
let produce_slot = 100;
let produce_epoch = produce_slot / spec.epoch_length;
let produce_slot = Slot::new(100);
let produce_epoch = produce_slot.epoch(spec.epoch_length);
epoch_map.map.insert(produce_epoch, produce_slot);
let epoch_map = Arc::new(epoch_map);
let keypair = Keypair::random();
@ -257,39 +257,39 @@ mod tests {
beacon_node.set_next_nonce_result(Ok(0));
// One slot before production slot...
slot_clock.set_slot(produce_slot - 1);
slot_clock.set_slot(produce_slot.as_u64() - 1);
assert_eq!(
block_producer.poll(),
Ok(PollOutcome::BlockProductionNotRequired(produce_slot - 1))
);
// On the produce slot...
slot_clock.set_slot(produce_slot);
slot_clock.set_slot(produce_slot.as_u64());
assert_eq!(
block_producer.poll(),
Ok(PollOutcome::BlockProduced(produce_slot))
Ok(PollOutcome::BlockProduced(produce_slot.into()))
);
// Trying the same produce slot again...
slot_clock.set_slot(produce_slot);
slot_clock.set_slot(produce_slot.as_u64());
assert_eq!(
block_producer.poll(),
Ok(PollOutcome::SlotAlreadyProcessed(produce_slot))
);
// One slot after the produce slot...
slot_clock.set_slot(produce_slot + 1);
slot_clock.set_slot(produce_slot.as_u64() + 1);
assert_eq!(
block_producer.poll(),
Ok(PollOutcome::BlockProductionNotRequired(produce_slot + 1))
);
// In an epoch without known duties...
let slot = (produce_epoch + 1) * spec.epoch_length;
let slot = (produce_epoch.as_u64() + 1) * spec.epoch_length;
slot_clock.set_slot(slot);
assert_eq!(
block_producer.poll(),
Ok(PollOutcome::ProducerDutiesUnknown(slot))
Ok(PollOutcome::ProducerDutiesUnknown(Slot::new(slot)))
);
}
}

View File

@ -1,9 +1,10 @@
use crate::{DutiesReader, DutiesReaderError};
use std::collections::HashMap;
use types::{Epoch, Slot};
pub struct EpochMap {
epoch_length: u64,
pub map: HashMap<u64, u64>,
pub map: HashMap<Epoch, Slot>,
}
impl EpochMap {
@ -16,8 +17,8 @@ impl EpochMap {
}
impl DutiesReader for EpochMap {
fn is_block_production_slot(&self, slot: u64) -> Result<bool, DutiesReaderError> {
let epoch = slot / self.epoch_length;
fn is_block_production_slot(&self, slot: Slot) -> Result<bool, DutiesReaderError> {
let epoch = slot.epoch(self.epoch_length);
match self.map.get(&epoch) {
Some(s) if *s == slot => Ok(true),
Some(s) if *s != slot => Ok(false),

View File

@ -1,6 +1,6 @@
use crate::traits::{BeaconNode, BeaconNodeError, PublishOutcome};
use std::sync::RwLock;
use types::{BeaconBlock, PublicKey, Signature};
use types::{BeaconBlock, PublicKey, Signature, Slot};
type NonceResult = Result<u64, BeaconNodeError>;
type ProduceResult = Result<Option<BeaconBlock>, BeaconNodeError>;
@ -12,7 +12,7 @@ pub struct SimulatedBeaconNode {
pub nonce_input: RwLock<Option<PublicKey>>,
pub nonce_result: RwLock<Option<NonceResult>>,
pub produce_input: RwLock<Option<(u64, Signature)>>,
pub produce_input: RwLock<Option<(Slot, Signature)>>,
pub produce_result: RwLock<Option<ProduceResult>>,
pub publish_input: RwLock<Option<BeaconBlock>>,
@ -46,7 +46,7 @@ impl BeaconNode for SimulatedBeaconNode {
}
/// Returns the value specified by the `set_next_produce_result`.
fn produce_beacon_block(&self, slot: u64, randao_reveal: &Signature) -> ProduceResult {
fn produce_beacon_block(&self, slot: Slot, randao_reveal: &Signature) -> ProduceResult {
*self.produce_input.write().unwrap() = Some((slot, randao_reveal.clone()));
match *self.produce_result.read().unwrap() {
Some(ref r) => r.clone(),

View File

@ -1,4 +1,4 @@
use types::{BeaconBlock, PublicKey, Signature};
use types::{BeaconBlock, PublicKey, Signature, Slot};
#[derive(Debug, PartialEq, Clone)]
pub enum BeaconNodeError {
@ -22,7 +22,7 @@ pub trait BeaconNode: Send + Sync {
/// Returns Ok(None) if the Beacon Node is unable to produce at the given slot.
fn produce_beacon_block(
&self,
slot: u64,
slot: Slot,
randao_reveal: &Signature,
) -> Result<Option<BeaconBlock>, BeaconNodeError>;
@ -42,7 +42,7 @@ pub enum DutiesReaderError {
/// Informs a validator of their duties (e.g., block production).
pub trait DutiesReader: Send + Sync {
fn is_block_production_slot(&self, slot: u64) -> Result<bool, DutiesReaderError>;
fn is_block_production_slot(&self, slot: Slot) -> Result<bool, DutiesReaderError>;
}
/// Signs message using an internally-maintained private key.

View File

@ -6,7 +6,7 @@ use db::stores::BeaconBlockStore;
use db::{ClientDB, DBError};
use ssz::{Decodable, DecodeError};
use std::sync::Arc;
use types::{BeaconBlock, Hash256};
use types::{BeaconBlock, Hash256, Slot};
pub enum ForkChoiceError {
BadSszInDatabase,
@ -37,7 +37,7 @@ where
/*
* Loop through all the head blocks and find the highest slot.
*/
let highest_slot: Option<u64> = None;
let highest_slot: Option<Slot> = None;
for (_, block) in &head_blocks {
let slot = block.slot;

View File

@ -17,5 +17,6 @@ rand = "0.5.5"
serde = "1.0"
serde_derive = "1.0"
serde_json = "1.0"
slog = "^2.2.3"
ssz = { path = "../utils/ssz" }
vec_shuffle = { path = "../utils/vec_shuffle" }

View File

@ -1,5 +1,5 @@
use super::{AttestationDataAndCustodyBit, Hash256};
use crate::test_utils::TestRandom;
use crate::{AttestationDataAndCustodyBit, Hash256, Slot};
use rand::RngCore;
use serde_derive::Serialize;
use ssz::{hash, Decodable, DecodeError, Encodable, SszStream, TreeHash};
@ -17,13 +17,13 @@ pub const SSZ_ATTESTION_DATA_LENGTH: usize = {
#[derive(Debug, Clone, PartialEq, Default, Serialize, Hash)]
pub struct AttestationData {
pub slot: u64,
pub slot: Slot,
pub shard: u64,
pub beacon_block_root: Hash256,
pub epoch_boundary_root: Hash256,
pub shard_block_root: Hash256,
pub latest_crosslink_root: Hash256,
pub justified_slot: u64,
pub justified_slot: Slot,
pub justified_block_root: Hash256,
}
@ -32,13 +32,13 @@ impl Eq for AttestationData {}
impl AttestationData {
pub fn zero() -> Self {
Self {
slot: 0,
slot: Slot::from(0_u64),
shard: 0,
beacon_block_root: Hash256::zero(),
epoch_boundary_root: Hash256::zero(),
shard_block_root: Hash256::zero(),
latest_crosslink_root: Hash256::zero(),
justified_slot: 0,
justified_slot: Slot::from(0_u64),
justified_block_root: Hash256::zero(),
}
}
@ -71,14 +71,14 @@ impl Encodable for AttestationData {
impl Decodable for AttestationData {
fn ssz_decode(bytes: &[u8], i: usize) -> Result<(Self, usize), DecodeError> {
let (slot, i) = u64::ssz_decode(bytes, i)?;
let (shard, i) = u64::ssz_decode(bytes, i)?;
let (beacon_block_root, i) = Hash256::ssz_decode(bytes, i)?;
let (epoch_boundary_root, i) = Hash256::ssz_decode(bytes, i)?;
let (shard_block_root, i) = Hash256::ssz_decode(bytes, i)?;
let (latest_crosslink_root, i) = Hash256::ssz_decode(bytes, i)?;
let (justified_slot, i) = u64::ssz_decode(bytes, i)?;
let (justified_block_root, i) = Hash256::ssz_decode(bytes, i)?;
let (slot, i) = <_>::ssz_decode(bytes, i)?;
let (shard, i) = <_>::ssz_decode(bytes, i)?;
let (beacon_block_root, i) = <_>::ssz_decode(bytes, i)?;
let (epoch_boundary_root, i) = <_>::ssz_decode(bytes, i)?;
let (shard_block_root, i) = <_>::ssz_decode(bytes, i)?;
let (latest_crosslink_root, i) = <_>::ssz_decode(bytes, i)?;
let (justified_slot, i) = <_>::ssz_decode(bytes, i)?;
let (justified_block_root, i) = <_>::ssz_decode(bytes, i)?;
let attestation_data = AttestationData {
slot,

View File

@ -1,5 +1,5 @@
use super::{BeaconBlockBody, ChainSpec, Eth1Data, Hash256, ProposalSignedData};
use crate::test_utils::TestRandom;
use crate::{BeaconBlockBody, ChainSpec, Eth1Data, Hash256, ProposalSignedData, Slot};
use bls::Signature;
use rand::RngCore;
use serde_derive::Serialize;
@ -7,7 +7,7 @@ use ssz::{hash, Decodable, DecodeError, Encodable, SszStream, TreeHash};
#[derive(Debug, PartialEq, Clone, Serialize)]
pub struct BeaconBlock {
pub slot: u64,
pub slot: Slot,
pub parent_root: Hash256,
pub state_root: Hash256,
pub randao_reveal: Signature,

View File

@ -1,8 +1,8 @@
use crate::test_utils::TestRandom;
use crate::{
validator::StatusFlags, validator_registry::get_active_validator_indices, AggregatePublicKey,
Attestation, AttestationData, BeaconBlock, Bitfield, ChainSpec, Crosslink, Eth1Data,
Eth1DataVote, Exit, Fork, Hash256, PendingAttestation, PublicKey, Signature, Validator,
Attestation, AttestationData, BeaconBlock, Bitfield, ChainSpec, Crosslink, Epoch, Eth1Data,
Eth1DataVote, Exit, Fork, Hash256, PendingAttestation, PublicKey, Signature, Slot, Validator,
};
use bls::bls_verify_aggregate;
use honey_badger_split::SplitExt;
@ -28,7 +28,7 @@ const DOMAIN_ATTESTATION: u64 = 1;
pub enum Error {
InsufficientValidators,
BadBlockSignature,
InvalidEpoch(u64, Range<u64>),
InvalidEpoch(Slot, Range<Epoch>),
CommitteesError(CommitteesError),
}
@ -81,7 +81,7 @@ pub enum WinningRootError {
#[derive(Debug, PartialEq)]
pub enum CommitteesError {
InvalidEpoch(u64, Range<u64>),
InvalidEpoch,
InsufficientNumberOfValidators,
}
@ -151,14 +151,14 @@ type CustodyChallenge = usize;
#[derive(Debug, PartialEq, Clone, Default, Serialize)]
pub struct BeaconState {
// Misc
pub slot: u64,
pub slot: Slot,
pub genesis_time: u64,
pub fork_data: Fork,
// Validator registry
pub validator_registry: Vec<Validator>,
pub validator_balances: Vec<u64>,
pub validator_registry_update_slot: u64,
pub validator_registry_update_slot: Slot,
pub validator_registry_exit_count: u64,
pub validator_registry_delta_chain_tip: Hash256,
@ -167,8 +167,8 @@ pub struct BeaconState {
pub latest_vdf_outputs: Vec<Hash256>,
pub previous_epoch_start_shard: u64,
pub current_epoch_start_shard: u64,
pub previous_epoch_calculation_slot: u64,
pub current_epoch_calculation_slot: u64,
pub previous_epoch_calculation_slot: Slot,
pub current_epoch_calculation_slot: Slot,
pub previous_epoch_seed: Hash256,
pub current_epoch_seed: Hash256,
@ -176,10 +176,10 @@ pub struct BeaconState {
pub custody_challenges: Vec<CustodyChallenge>,
// Finality
pub previous_justified_slot: u64,
pub justified_slot: u64,
pub previous_justified_slot: Slot,
pub justified_slot: Slot,
pub justification_bitfield: u64,
pub finalized_slot: u64,
pub finalized_slot: Slot,
// Recent state
pub latest_crosslinks: Vec<Crosslink>,
@ -198,20 +198,20 @@ impl BeaconState {
Hash256::from(&self.hash_tree_root()[..])
}
pub fn current_epoch(&self, spec: &ChainSpec) -> u64 {
self.slot / spec.epoch_length
pub fn current_epoch(&self, spec: &ChainSpec) -> Epoch {
self.slot.epoch(spec.epoch_length)
}
pub fn previous_epoch(&self, spec: &ChainSpec) -> u64 {
self.current_epoch(spec).saturating_sub(1)
pub fn previous_epoch(&self, spec: &ChainSpec) -> Epoch {
self.current_epoch(spec).saturating_sub(1_u64)
}
pub fn current_epoch_start_slot(&self, spec: &ChainSpec) -> u64 {
self.current_epoch(spec) * spec.epoch_length
pub fn current_epoch_start_slot(&self, spec: &ChainSpec) -> Slot {
self.current_epoch(spec).start_slot(spec.epoch_length)
}
pub fn previous_epoch_start_slot(&self, spec: &ChainSpec) -> u64 {
self.previous_epoch(spec) * spec.epoch_length
pub fn previous_epoch_start_slot(&self, spec: &ChainSpec) -> Slot {
self.previous_epoch(spec).start_slot(spec.epoch_length)
}
/// Returns the number of committees per slot.
@ -231,23 +231,6 @@ impl BeaconState {
)
}
/// Returns the start slot and end slot of the current epoch containing `self.slot`.
pub fn get_current_epoch_boundaries(&self, epoch_length: u64) -> Range<u64> {
let slot_in_epoch = self.slot % epoch_length;
let start = self.slot - slot_in_epoch;
let end = self.slot + (epoch_length - slot_in_epoch);
start..end
}
/// Returns the start slot and end slot of the current epoch containing `self.slot`.
pub fn get_previous_epoch_boundaries(&self, spec: &ChainSpec) -> Range<u64> {
let current_epoch = self.slot / spec.epoch_length;
let previous_epoch = current_epoch.saturating_sub(1);
let start = previous_epoch * spec.epoch_length;
let end = start + spec.epoch_length;
start..end
}
fn get_previous_epoch_committee_count_per_slot(&self, spec: &ChainSpec) -> u64 {
let previous_active_validators = get_active_validator_indices(
&self.validator_registry,
@ -266,24 +249,24 @@ impl BeaconState {
pub fn get_crosslink_committees_at_slot(
&self,
slot: u64,
slot: Slot,
spec: &ChainSpec,
) -> Result<Vec<(Vec<usize>, u64)>, CommitteesError> {
let epoch = slot / spec.epoch_length;
let current_epoch = self.slot / spec.epoch_length;
let previous_epoch = if current_epoch == spec.genesis_slot {
let epoch = slot.epoch(spec.epoch_length);
let current_epoch = self.current_epoch(spec);
let previous_epoch = if current_epoch == spec.genesis_slot.epoch(spec.epoch_length) {
current_epoch
} else {
current_epoch.saturating_sub(1)
current_epoch.saturating_sub(1_u64)
};
let next_epoch = current_epoch + 1;
ensure!(
(previous_epoch <= epoch) & (epoch < next_epoch),
CommitteesError::InvalidEpoch(slot, previous_epoch..current_epoch)
CommitteesError::InvalidEpoch
);
let offset = slot % spec.epoch_length;
let offset = slot.as_u64() % spec.epoch_length;
let (committees_per_slot, shuffling, slot_start_shard) = if epoch < current_epoch {
let committees_per_slot = self.get_previous_epoch_committee_count_per_slot(spec);
@ -332,11 +315,11 @@ impl BeaconState {
let block_proposer = self.get_beacon_proposer_index(self.slot, spec)?;
self.validator_registry[block_proposer].proposer_slots += 1;
self.latest_randao_mixes[(self.slot % spec.latest_randao_mixes_length) as usize] =
self.latest_randao_mixes[((self.slot - 1) % spec.latest_randao_mixes_length) as usize];
self.latest_randao_mixes[(self.slot % spec.latest_randao_mixes_length).as_usize()] = self
.latest_randao_mixes[((self.slot - 1) % spec.latest_randao_mixes_length).as_usize()];
// Block roots.
self.latest_block_roots[((self.slot - 1) % spec.latest_block_roots_length) as usize] =
self.latest_block_roots[((self.slot - 1) % spec.latest_block_roots_length).as_usize()] =
previous_block_root;
if self.slot % spec.latest_block_roots_length == 0 {
@ -350,9 +333,9 @@ impl BeaconState {
&self,
validator_index: usize,
spec: &ChainSpec,
) -> Result<Option<(u64, u64, u64)>, CommitteesError> {
) -> Result<Option<(Slot, u64, u64)>, CommitteesError> {
let mut result = None;
for slot in self.get_current_epoch_boundaries(spec.epoch_length) {
for slot in self.current_epoch(spec).slot_iter(spec.epoch_length) {
for (committee, shard) in self.get_crosslink_committees_at_slot(slot, spec)? {
if let Some(committee_index) = committee.iter().position(|&i| i == validator_index)
{
@ -426,13 +409,14 @@ impl BeaconState {
// TODO: check this is correct.
let new_mix = {
let mut mix = self.latest_randao_mixes
[(self.slot % spec.latest_randao_mixes_length) as usize]
.to_vec();
[(self.slot % spec.latest_randao_mixes_length).as_usize()]
.to_vec();
mix.append(&mut ssz_encode(&block.randao_reveal));
Hash256::from(&hash(&mix)[..])
};
self.latest_randao_mixes[(self.slot % spec.latest_randao_mixes_length) as usize] = new_mix;
self.latest_randao_mixes[(self.slot % spec.latest_randao_mixes_length).as_usize()] =
new_mix;
/*
* Eth1 data
@ -593,7 +577,7 @@ impl BeaconState {
Ok(())
}
pub fn get_shuffling(&self, seed: Hash256, slot: u64, spec: &ChainSpec) -> Vec<Vec<usize>> {
pub fn get_shuffling(&self, seed: Hash256, slot: Slot, spec: &ChainSpec) -> Vec<Vec<usize>> {
let slot = slot - (slot % spec.epoch_length);
let active_validator_indices = get_active_validator_indices(&self.validator_registry, slot);
@ -602,7 +586,7 @@ impl BeaconState {
self.get_committee_count_per_slot(active_validator_indices.len(), spec);
// TODO: check that Hash256 matches 'int_to_bytes32'.
let seed = seed ^ Hash256::from(slot);
let seed = seed ^ Hash256::from(slot.as_u64());
let shuffled_active_validator_indices =
shuffle(&seed, active_validator_indices).expect("Max validator count exceed!");
@ -616,7 +600,7 @@ impl BeaconState {
/// If the state does not contain an index for a beacon proposer at the requested `slot`, then `None` is returned.
pub fn get_beacon_proposer_index(
&self,
slot: u64,
slot: Slot,
spec: &ChainSpec,
) -> Result<usize, CommitteesError> {
let committees = self.get_crosslink_committees_at_slot(slot, spec)?;
@ -624,7 +608,7 @@ impl BeaconState {
.first()
.ok_or(CommitteesError::InsufficientNumberOfValidators)
.and_then(|(first_committee, _)| {
let index = (slot as usize)
let index = (slot.as_usize())
.checked_rem(first_committee.len())
.ok_or(CommitteesError::InsufficientNumberOfValidators)?;
// NOTE: next index will not panic as we have already returned if this is the case
@ -653,7 +637,10 @@ impl BeaconState {
let current_epoch_attestations: Vec<&PendingAttestation> = self
.latest_attestations
.par_iter()
.filter(|a| a.data.slot / spec.epoch_length == self.current_epoch(spec))
.filter(|a| {
(a.data.slot / spec.epoch_length).epoch(spec.epoch_length)
== self.current_epoch(spec)
})
.collect();
debug!(
@ -708,7 +695,8 @@ impl BeaconState {
.par_iter()
.filter(|a| {
//TODO: ensure these saturating subs are correct.
a.data.slot / spec.epoch_length == self.previous_epoch(spec)
(a.data.slot / spec.epoch_length).epoch(spec.epoch_length)
== self.previous_epoch(spec)
})
.collect();
@ -864,7 +852,7 @@ impl BeaconState {
HashMap::new();
// for slot in self.slot.saturating_sub(2 * spec.epoch_length)..self.slot {
for slot in self.get_previous_epoch_boundaries(spec) {
for slot in self.previous_epoch(spec).slot_iter(spec.epoch_length) {
let crosslink_committees_at_slot = self.get_crosslink_committees_at_slot(slot, spec)?;
for (crosslink_committee, shard) in crosslink_committees_at_slot {
@ -909,7 +897,7 @@ impl BeaconState {
* Justification and finalization
*/
let epochs_since_finality =
self.slot.saturating_sub(self.finalized_slot) / spec.epoch_length;
(self.slot.saturating_sub(self.finalized_slot) / spec.epoch_length).as_u64();
// TODO: fix this extra map
let previous_epoch_justified_attester_indices_hashset: HashSet<usize> =
@ -1028,7 +1016,7 @@ impl BeaconState {
/*
* Crosslinks
*/
for slot in self.get_previous_epoch_boundaries(spec) {
for slot in self.previous_epoch(spec).slot_iter(spec.epoch_length) {
let crosslink_committees_at_slot = self.get_crosslink_committees_at_slot(slot, spec)?;
for (_crosslink_committee, shard) in crosslink_committees_at_slot {
@ -1097,8 +1085,7 @@ impl BeaconState {
+ self.get_current_epoch_committee_count_per_slot(spec) as u64 * spec.epoch_length)
% spec.shard_count;
self.current_epoch_seed = self.get_randao_mix(
self.current_epoch_calculation_slot
.saturating_sub(spec.seed_lookahead),
self.current_epoch_calculation_slot - spec.seed_lookahead,
spec,
);
} else {
@ -1107,8 +1094,7 @@ impl BeaconState {
if epochs_since_last_registry_change.is_power_of_two() {
self.current_epoch_calculation_slot = self.slot;
self.current_epoch_seed = self.get_randao_mix(
self.current_epoch_calculation_slot
.saturating_sub(spec.seed_lookahead),
self.current_epoch_calculation_slot - spec.seed_lookahead,
spec,
);
}
@ -1117,13 +1103,16 @@ impl BeaconState {
self.process_penalties_and_exits(spec);
let e = self.slot / spec.epoch_length;
self.latest_penalized_balances[((e + 1) % spec.latest_penalized_exit_length) as usize] =
self.latest_penalized_balances[(e % spec.latest_penalized_exit_length) as usize];
self.latest_penalized_balances[((e + 1) % spec.latest_penalized_exit_length).as_usize()] =
self.latest_penalized_balances[(e % spec.latest_penalized_exit_length).as_usize()];
self.latest_attestations = self
.latest_attestations
.iter()
.filter(|a| a.data.slot / spec.epoch_length >= self.current_epoch(spec))
.filter(|a| {
(a.data.slot / spec.epoch_length).epoch(spec.epoch_length)
>= self.current_epoch(spec)
})
.cloned()
.collect();
@ -1146,8 +1135,8 @@ impl BeaconState {
{
let e = (self.slot / spec.epoch_length) % spec.latest_penalized_exit_length;
let total_at_start = self.latest_penalized_balances
[((e + 1) % spec.latest_penalized_exit_length) as usize];
let total_at_end = self.latest_penalized_balances[e as usize];
[((e + 1) % spec.latest_penalized_exit_length).as_usize()];
let total_at_end = self.latest_penalized_balances[e.as_usize()];
let total_penalities = total_at_end.saturating_sub(total_at_start);
let penalty = self.get_effective_balance(index, spec)
* std::cmp::min(total_penalities * 3, total_balance)
@ -1187,10 +1176,10 @@ impl BeaconState {
self.validator_registry[index].status_flags = Some(StatusFlags::Withdrawable);
}
fn get_randao_mix(&mut self, slot: u64, spec: &ChainSpec) -> Hash256 {
fn get_randao_mix(&mut self, slot: Slot, spec: &ChainSpec) -> Hash256 {
assert!(self.slot < slot + spec.latest_randao_mixes_length);
assert!(slot <= self.slot);
self.latest_randao_mixes[(slot & spec.latest_randao_mixes_length) as usize]
self.latest_randao_mixes[(slot % spec.latest_randao_mixes_length).as_usize()]
}
fn update_validator_registry(&mut self, spec: &ChainSpec) {
@ -1260,7 +1249,7 @@ impl BeaconState {
}
}
fn entry_exit_effect_slot(&self, slot: u64, spec: &ChainSpec) -> u64 {
fn entry_exit_effect_slot(&self, slot: Slot, spec: &ChainSpec) -> Slot {
(slot - slot % spec.epoch_length) + spec.epoch_length + spec.entry_exit_delay
}
@ -1288,9 +1277,7 @@ impl BeaconState {
) -> Result<u64, InclusionError> {
let attestation =
self.earliest_included_attestation(attestations, validator_index, spec)?;
Ok(attestation
.slot_included
.saturating_sub(attestation.data.slot))
Ok((attestation.slot_included - attestation.data.slot).as_u64())
}
fn inclusion_slot(
@ -1298,7 +1285,7 @@ impl BeaconState {
attestations: &[&PendingAttestation],
validator_index: usize,
spec: &ChainSpec,
) -> Result<u64, InclusionError> {
) -> Result<Slot, InclusionError> {
let attestation =
self.earliest_included_attestation(attestations, validator_index, spec)?;
Ok(attestation.slot_included)
@ -1350,10 +1337,10 @@ impl BeaconState {
std::cmp::min(self.validator_balances[validator_index], spec.max_deposit)
}
pub fn get_block_root(&self, slot: u64, spec: &ChainSpec) -> Option<&Hash256> {
pub fn get_block_root(&self, slot: Slot, spec: &ChainSpec) -> Option<&Hash256> {
if self.slot <= slot + spec.latest_block_roots_length && slot <= self.slot {
self.latest_block_roots
.get((slot % spec.latest_block_roots_length) as usize)
.get((slot % spec.latest_block_roots_length).as_usize())
} else {
None
}
@ -1589,7 +1576,7 @@ fn penalize_validator(_state: &BeaconState, _proposer_index: usize) {
// TODO: stubbed out.
}
fn get_domain(_fork: &Fork, _slot: u64, _domain_type: u64) -> u64 {
fn get_domain(_fork: &Fork, _slot: Slot, _domain_type: u64) -> u64 {
// TODO: stubbed out.
0
}

View File

@ -1,12 +1,12 @@
use super::Hash256;
use crate::test_utils::TestRandom;
use crate::{Hash256, Slot};
use rand::RngCore;
use serde_derive::Serialize;
use ssz::{hash, Decodable, DecodeError, Encodable, SszStream, TreeHash};
#[derive(Clone, Debug, PartialEq, Serialize)]
pub struct Crosslink {
pub slot: u64,
pub slot: Slot,
pub shard_block_root: Hash256,
}
@ -14,7 +14,7 @@ impl Crosslink {
/// Generates a new instance where `dynasty` and `hash` are both zero.
pub fn zero() -> Self {
Self {
slot: 0,
slot: Slot::from(0_u64),
shard_block_root: Hash256::zero(),
}
}

View File

@ -1,4 +1,4 @@
use crate::test_utils::TestRandom;
use crate::{test_utils::TestRandom, Slot};
use bls::Signature;
use rand::RngCore;
use serde_derive::Serialize;
@ -6,7 +6,7 @@ use ssz::{hash, Decodable, DecodeError, Encodable, SszStream, TreeHash};
#[derive(Debug, PartialEq, Clone, Serialize)]
pub struct Exit {
pub slot: u64,
pub slot: Slot,
pub validator_index: u32,
pub signature: Signature,
}

View File

@ -1,4 +1,4 @@
use crate::test_utils::TestRandom;
use crate::{test_utils::TestRandom, Slot};
use rand::RngCore;
use serde_derive::Serialize;
use ssz::{hash, Decodable, DecodeError, Encodable, SszStream, TreeHash};
@ -7,7 +7,7 @@ use ssz::{hash, Decodable, DecodeError, Encodable, SszStream, TreeHash};
pub struct Fork {
pub pre_fork_version: u64,
pub post_fork_version: u64,
pub fork_slot: u64,
pub fork_slot: Slot,
}
impl Encodable for Fork {

View File

@ -19,17 +19,17 @@ pub mod free_attestation;
pub mod pending_attestation;
pub mod proposal_signed_data;
pub mod proposer_slashing;
pub mod readers;
pub mod shard_committee;
pub mod shard_reassignment_record;
pub mod slashable_vote_data;
pub mod slot_epoch;
pub mod spec;
pub mod special_record;
pub mod validator;
pub mod validator_registry;
pub mod validator_registry_delta_block;
pub mod readers;
use ethereum_types::{H160, H256, U256};
use std::collections::HashMap;
@ -54,6 +54,7 @@ pub use crate::proposal_signed_data::ProposalSignedData;
pub use crate::proposer_slashing::ProposerSlashing;
pub use crate::shard_committee::ShardCommittee;
pub use crate::slashable_vote_data::SlashableVoteData;
pub use crate::slot_epoch::{Epoch, Slot};
pub use crate::spec::ChainSpec;
pub use crate::special_record::{SpecialRecord, SpecialRecordKind};
pub use crate::validator::{StatusFlags as ValidatorStatusFlags, Validator};

View File

@ -1,5 +1,5 @@
use super::{AttestationData, Bitfield};
use crate::test_utils::TestRandom;
use crate::{AttestationData, Bitfield, Slot};
use rand::RngCore;
use serde_derive::Serialize;
use ssz::{hash, Decodable, DecodeError, Encodable, SszStream, TreeHash};
@ -9,7 +9,7 @@ pub struct PendingAttestation {
pub data: AttestationData,
pub aggregation_bitfield: Bitfield,
pub custody_bitfield: Bitfield,
pub slot_included: u64,
pub slot_included: Slot,
}
impl Encodable for PendingAttestation {

View File

@ -1,12 +1,12 @@
use super::Hash256;
use crate::test_utils::TestRandom;
use crate::{Hash256, Slot};
use rand::RngCore;
use serde_derive::Serialize;
use ssz::{hash, Decodable, DecodeError, Encodable, SszStream, TreeHash};
#[derive(Debug, PartialEq, Clone, Default, Serialize)]
pub struct ProposalSignedData {
pub slot: u64,
pub slot: Slot,
pub shard: u64,
pub block_root: Hash256,
}

View File

@ -1,5 +1,4 @@
use super::state_reader::BeaconStateReader;
use crate::{BeaconBlock, Hash256};
use crate::{BeaconBlock, Hash256, Slot};
use std::fmt::Debug;
/// The `BeaconBlockReader` provides interfaces for reading a subset of fields of a `BeaconBlock`.
@ -11,7 +10,7 @@ use std::fmt::Debug;
/// Note: presently, direct SSZ reading has not been implemented so this trait is being used for
/// "future proofing".
pub trait BeaconBlockReader: Debug + PartialEq {
fn slot(&self) -> u64;
fn slot(&self) -> Slot;
fn parent_root(&self) -> Hash256;
fn state_root(&self) -> Hash256;
fn canonical_root(&self) -> Hash256;
@ -19,7 +18,7 @@ pub trait BeaconBlockReader: Debug + PartialEq {
}
impl BeaconBlockReader for BeaconBlock {
fn slot(&self) -> u64 {
fn slot(&self) -> Slot {
self.slot
}

View File

@ -1,4 +1,4 @@
use crate::{BeaconState, Hash256};
use crate::{BeaconState, Hash256, Slot};
use std::fmt::Debug;
/// The `BeaconStateReader` provides interfaces for reading a subset of fields of a `BeaconState`.
@ -10,13 +10,13 @@ use std::fmt::Debug;
/// Note: presently, direct SSZ reading has not been implemented so this trait is being used for
/// "future proofing".
pub trait BeaconStateReader: Debug + PartialEq {
fn slot(&self) -> u64;
fn slot(&self) -> Slot;
fn canonical_root(&self) -> Hash256;
fn into_beacon_state(self) -> Option<BeaconState>;
}
impl BeaconStateReader for BeaconState {
fn slot(&self) -> u64 {
fn slot(&self) -> Slot {
self.slot
}

View File

@ -1,4 +1,4 @@
use crate::test_utils::TestRandom;
use crate::{test_utils::TestRandom, Slot};
use rand::RngCore;
use serde_derive::Serialize;
use ssz::{hash, Decodable, DecodeError, Encodable, SszStream, TreeHash};
@ -7,7 +7,7 @@ use ssz::{hash, Decodable, DecodeError, Encodable, SszStream, TreeHash};
pub struct ShardReassignmentRecord {
pub validator_index: u64,
pub shard: u64,
pub slot: u64,
pub slot: Slot,
}
impl Encodable for ShardReassignmentRecord {

View File

@ -0,0 +1,677 @@
/// The `Slot` and `Epoch` types are defined as newtypes over u64 to enforce type-safety between
/// the two types.
///
/// `Slot` and `Epoch` have implementations which permit conversion, comparison and math operations
/// between each and `u64`, however specifically not between each other.
///
/// All math operations on `Slot` and `Epoch` are saturating, they never wrap.
///
/// It would be easy to define `PartialOrd` and other traits generically across all types which
/// implement `Into<u64>`, however this would allow operations between `Slots` and `Epochs` which
/// may lead to programming errors which are not detected by the compiler.
use crate::test_utils::TestRandom;
use rand::RngCore;
use serde_derive::Serialize;
use slog;
use ssz::{hash, Decodable, DecodeError, Encodable, SszStream, TreeHash};
use std::cmp::{Ord, Ordering};
use std::fmt;
use std::iter::Iterator;
use std::ops::{Add, AddAssign, Div, DivAssign, Mul, MulAssign, Rem, Sub, SubAssign};
macro_rules! impl_from_into_u64 {
($main: ident) => {
impl From<u64> for $main {
fn from(n: u64) -> $main {
$main(n)
}
}
impl Into<u64> for $main {
fn into(self) -> u64 {
self.0
}
}
impl $main {
pub fn as_u64(&self) -> u64 {
self.0
}
}
};
}
macro_rules! impl_from_into_usize {
($main: ident) => {
impl From<usize> for $main {
fn from(n: usize) -> $main {
$main(n as u64)
}
}
impl Into<usize> for $main {
fn into(self) -> usize {
self.0 as usize
}
}
impl $main {
pub fn as_usize(&self) -> usize {
self.0 as usize
}
}
};
}
macro_rules! impl_math_between {
($main: ident, $other: ident) => {
impl PartialOrd<$other> for $main {
/// Utilizes `partial_cmp` on the underlying `u64`.
fn partial_cmp(&self, other: &$other) -> Option<Ordering> {
Some(self.0.cmp(&(*other).into()))
}
}
impl PartialEq<$other> for $main {
fn eq(&self, other: &$other) -> bool {
let other: u64 = (*other).into();
self.0 == other
}
}
impl Add<$other> for $main {
type Output = $main;
fn add(self, other: $other) -> $main {
$main::from(self.0.saturating_add(other.into()))
}
}
impl AddAssign<$other> for $main {
fn add_assign(&mut self, other: $other) {
self.0 = self.0.saturating_add(other.into());
}
}
impl Sub<$other> for $main {
type Output = $main;
fn sub(self, other: $other) -> $main {
$main::from(self.0.saturating_sub(other.into()))
}
}
impl SubAssign<$other> for $main {
fn sub_assign(&mut self, other: $other) {
self.0 = self.0.saturating_sub(other.into());
}
}
impl Mul<$other> for $main {
type Output = $main;
fn mul(self, rhs: $other) -> $main {
let rhs: u64 = rhs.into();
$main::from(self.0.saturating_mul(rhs))
}
}
impl MulAssign<$other> for $main {
fn mul_assign(&mut self, rhs: $other) {
let rhs: u64 = rhs.into();
self.0 = self.0.saturating_mul(rhs)
}
}
impl Div<$other> for $main {
type Output = $main;
fn div(self, rhs: $other) -> $main {
let rhs: u64 = rhs.into();
if rhs == 0 {
panic!("Cannot divide by zero-valued Slot/Epoch")
}
$main::from(self.0 / rhs)
}
}
impl DivAssign<$other> for $main {
fn div_assign(&mut self, rhs: $other) {
let rhs: u64 = rhs.into();
if rhs == 0 {
panic!("Cannot divide by zero-valued Slot/Epoch")
}
self.0 = self.0 / rhs
}
}
impl Rem<$other> for $main {
type Output = $main;
fn rem(self, modulus: $other) -> $main {
let modulus: u64 = modulus.into();
$main::from(self.0 % modulus)
}
}
};
}
macro_rules! impl_math {
($type: ident) => {
impl $type {
pub fn saturating_sub<T: Into<$type>>(&self, other: T) -> $type {
*self - other.into()
}
pub fn checked_div<T: Into<$type>>(&self, rhs: T) -> Option<$type> {
let rhs: $type = rhs.into();
if rhs == 0 {
None
} else {
Some(*self / rhs)
}
}
pub fn is_power_of_two(&self) -> bool {
self.0.is_power_of_two()
}
}
impl Ord for $type {
fn cmp(&self, other: &$type) -> Ordering {
let other: u64 = (*other).into();
self.0.cmp(&other)
}
}
};
}
macro_rules! impl_display {
($type: ident) => {
impl fmt::Display for $type {
fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result {
write!(f, "{}", self.0)
}
}
impl slog::Value for $type {
fn serialize(
&self,
record: &slog::Record,
key: slog::Key,
serializer: &mut slog::Serializer,
) -> slog::Result {
self.0.serialize(record, key, serializer)
}
}
};
}
macro_rules! impl_ssz {
($type: ident) => {
impl Encodable for $type {
fn ssz_append(&self, s: &mut SszStream) {
s.append(&self.0);
}
}
impl Decodable for $type {
fn ssz_decode(bytes: &[u8], i: usize) -> Result<(Self, usize), DecodeError> {
let (value, i) = <_>::ssz_decode(bytes, i)?;
Ok(($type(value), i))
}
}
impl TreeHash for $type {
fn hash_tree_root(&self) -> Vec<u8> {
let mut result: Vec<u8> = vec![];
result.append(&mut self.0.hash_tree_root());
hash(&result)
}
}
impl<T: RngCore> TestRandom<T> for $type {
fn random_for_test(rng: &mut T) -> Self {
$type::from(u64::random_for_test(rng))
}
}
};
}
macro_rules! impl_common {
($type: ident) => {
impl_from_into_u64!($type);
impl_from_into_usize!($type);
impl_math_between!($type, $type);
impl_math_between!($type, u64);
impl_math!($type);
impl_display!($type);
impl_ssz!($type);
};
}
#[derive(Eq, Debug, Clone, Copy, Default, Serialize, Hash)]
pub struct Slot(u64);
#[derive(Eq, Debug, Clone, Copy, Default, Serialize, Hash)]
pub struct Epoch(u64);
impl_common!(Slot);
impl_common!(Epoch);
impl Slot {
pub fn new(slot: u64) -> Slot {
Slot(slot)
}
pub fn epoch(&self, epoch_length: u64) -> Epoch {
Epoch::from(self.0 / epoch_length)
}
pub fn max_value() -> Slot {
Slot(u64::max_value())
}
}
impl Epoch {
pub fn new(slot: u64) -> Epoch {
Epoch(slot)
}
pub fn start_slot(&self, epoch_length: u64) -> Slot {
Slot::from(self.0.saturating_mul(epoch_length))
}
pub fn end_slot(&self, epoch_length: u64) -> Slot {
Slot::from(
self.0
.saturating_add(1)
.saturating_mul(epoch_length)
.saturating_sub(1),
)
}
pub fn slot_iter(&self, epoch_length: u64) -> SlotIter {
SlotIter {
current: self.start_slot(epoch_length),
epoch: self,
epoch_length,
}
}
}
pub struct SlotIter<'a> {
current: Slot,
epoch: &'a Epoch,
epoch_length: u64,
}
impl<'a> Iterator for SlotIter<'a> {
type Item = Slot;
fn next(&mut self) -> Option<Slot> {
if self.current == self.epoch.end_slot(self.epoch_length) {
None
} else {
let previous = self.current;
self.current += 1;
Some(previous)
}
}
}
#[cfg(test)]
mod tests {
use super::*;
macro_rules! new_tests {
($type: ident) => {
#[test]
fn new() {
assert_eq!($type(0), $type::new(0));
assert_eq!($type(3), $type::new(3));
assert_eq!($type(u64::max_value()), $type::new(u64::max_value()));
}
};
}
macro_rules! from_into_tests {
($type: ident, $other: ident) => {
#[test]
fn into() {
let x: $other = $type(0).into();
assert_eq!(x, 0);
let x: $other = $type(3).into();
assert_eq!(x, 3);
let x: $other = $type(u64::max_value()).into();
// Note: this will fail on 32 bit systems. This is expected as we don't have a proper
// 32-bit system strategy in place.
assert_eq!(x, $other::max_value());
}
#[test]
fn from() {
assert_eq!($type(0), $type::from(0_u64));
assert_eq!($type(3), $type::from(3_u64));
assert_eq!($type(u64::max_value()), $type::from($other::max_value()));
}
};
}
macro_rules! math_between_tests {
($type: ident, $other: ident) => {
#[test]
fn partial_ord() {
let assert_partial_ord = |a: u64, partial_ord: Ordering, b: u64| {
let other: $other = $type(b).into();
assert_eq!($type(a).partial_cmp(&other), Some(partial_ord));
};
assert_partial_ord(1, Ordering::Less, 2);
assert_partial_ord(2, Ordering::Greater, 1);
assert_partial_ord(0, Ordering::Less, u64::max_value());
assert_partial_ord(u64::max_value(), Ordering::Greater, 0);
}
#[test]
fn partial_eq() {
let assert_partial_eq = |a: u64, b: u64, is_equal: bool| {
let other: $other = $type(b).into();
assert_eq!($type(a).eq(&other), is_equal);
};
assert_partial_eq(0, 0, true);
assert_partial_eq(0, 1, false);
assert_partial_eq(1, 0, false);
assert_partial_eq(1, 1, true);
assert_partial_eq(u64::max_value(), u64::max_value(), true);
assert_partial_eq(0, u64::max_value(), false);
assert_partial_eq(u64::max_value(), 0, false);
}
#[test]
fn add_and_add_assign() {
let assert_add = |a: u64, b: u64, result: u64| {
let other: $other = $type(b).into();
assert_eq!($type(a) + other, $type(result));
let mut add_assigned = $type(a);
add_assigned += other;
assert_eq!(add_assigned, $type(result));
};
assert_add(0, 1, 1);
assert_add(1, 0, 1);
assert_add(1, 2, 3);
assert_add(2, 1, 3);
assert_add(7, 7, 14);
// Addition should be saturating.
assert_add(u64::max_value(), 1, u64::max_value());
assert_add(u64::max_value(), u64::max_value(), u64::max_value());
}
#[test]
fn sub_and_sub_assign() {
let assert_sub = |a: u64, b: u64, result: u64| {
let other: $other = $type(b).into();
assert_eq!($type(a) - other, $type(result));
let mut sub_assigned = $type(a);
sub_assigned -= other;
assert_eq!(sub_assigned, $type(result));
};
assert_sub(1, 0, 1);
assert_sub(2, 1, 1);
assert_sub(14, 7, 7);
assert_sub(u64::max_value(), 1, u64::max_value() - 1);
assert_sub(u64::max_value(), u64::max_value(), 0);
// Subtraction should be saturating
assert_sub(0, 1, 0);
assert_sub(1, 2, 0);
}
#[test]
fn mul_and_mul_assign() {
let assert_mul = |a: u64, b: u64, result: u64| {
let other: $other = $type(b).into();
assert_eq!($type(a) * other, $type(result));
let mut mul_assigned = $type(a);
mul_assigned *= other;
assert_eq!(mul_assigned, $type(result));
};
assert_mul(2, 2, 4);
assert_mul(1, 2, 2);
assert_mul(0, 2, 0);
// Multiplication should be saturating.
assert_mul(u64::max_value(), 2, u64::max_value());
}
#[test]
fn div_and_div_assign() {
let assert_div = |a: u64, b: u64, result: u64| {
let other: $other = $type(b).into();
assert_eq!($type(a) / other, $type(result));
let mut div_assigned = $type(a);
div_assigned /= other;
assert_eq!(div_assigned, $type(result));
};
assert_div(0, 2, 0);
assert_div(2, 2, 1);
assert_div(100, 50, 2);
assert_div(128, 2, 64);
assert_div(u64::max_value(), 2, 2_u64.pow(63) - 1);
}
#[test]
#[should_panic]
fn div_panics_with_divide_by_zero() {
let other: $other = $type(0).into();
let _ = $type(2) / other;
}
#[test]
#[should_panic]
fn div_assign_panics_with_divide_by_zero() {
let other: $other = $type(0).into();
let mut assigned = $type(2);
assigned /= other;
}
#[test]
fn rem() {
let assert_rem = |a: u64, b: u64, result: u64| {
let other: $other = $type(b).into();
assert_eq!($type(a) % other, $type(result));
};
assert_rem(3, 2, 1);
assert_rem(40, 2, 0);
assert_rem(10, 100, 10);
assert_rem(302042, 3293, 2379);
}
};
}
macro_rules! math_tests {
($type: ident) => {
#[test]
fn saturating_sub() {
let assert_saturating_sub = |a: u64, b: u64, result: u64| {
assert_eq!($type(a).saturating_sub($type(b)), $type(result));
};
assert_saturating_sub(1, 0, 1);
assert_saturating_sub(2, 1, 1);
assert_saturating_sub(14, 7, 7);
assert_saturating_sub(u64::max_value(), 1, u64::max_value() - 1);
assert_saturating_sub(u64::max_value(), u64::max_value(), 0);
// Subtraction should be saturating
assert_saturating_sub(0, 1, 0);
assert_saturating_sub(1, 2, 0);
}
#[test]
fn checked_div() {
let assert_checked_div = |a: u64, b: u64, result: Option<u64>| {
let division_result_as_u64 = match $type(a).checked_div($type(b)) {
None => None,
Some(val) => Some(val.as_u64()),
};
assert_eq!(division_result_as_u64, result);
};
assert_checked_div(0, 2, Some(0));
assert_checked_div(2, 2, Some(1));
assert_checked_div(100, 50, Some(2));
assert_checked_div(128, 2, Some(64));
assert_checked_div(u64::max_value(), 2, Some(2_u64.pow(63) - 1));
assert_checked_div(2, 0, None);
assert_checked_div(0, 0, None);
assert_checked_div(u64::max_value(), 0, None);
}
#[test]
fn is_power_of_two() {
let assert_is_power_of_two = |a: u64, result: bool| {
assert_eq!(
$type(a).is_power_of_two(),
result,
"{}.is_power_of_two() != {}",
a,
result
);
};
assert_is_power_of_two(0, false);
assert_is_power_of_two(1, true);
assert_is_power_of_two(2, true);
assert_is_power_of_two(3, false);
assert_is_power_of_two(4, true);
assert_is_power_of_two(2_u64.pow(4), true);
assert_is_power_of_two(u64::max_value(), false);
}
#[test]
fn ord() {
let assert_ord = |a: u64, ord: Ordering, b: u64| {
assert_eq!($type(a).cmp(&$type(b)), ord);
};
assert_ord(1, Ordering::Less, 2);
assert_ord(2, Ordering::Greater, 1);
assert_ord(0, Ordering::Less, u64::max_value());
assert_ord(u64::max_value(), Ordering::Greater, 0);
}
};
}
macro_rules! ssz_tests {
($type: ident) => {
#[test]
pub fn test_ssz_round_trip() {
let mut rng = XorShiftRng::from_seed([42; 16]);
let original = $type::random_for_test(&mut rng);
let bytes = ssz_encode(&original);
let (decoded, _) = $type::ssz_decode(&bytes, 0).unwrap();
assert_eq!(original, decoded);
}
#[test]
pub fn test_hash_tree_root() {
let mut rng = XorShiftRng::from_seed([42; 16]);
let original = $type::random_for_test(&mut rng);
let result = original.hash_tree_root();
assert_eq!(result.len(), 32);
// TODO: Add further tests
// https://github.com/sigp/lighthouse/issues/170
}
};
}
macro_rules! all_tests {
($type: ident) => {
new_tests!($type);
math_between_tests!($type, $type);
math_tests!($type);
ssz_tests!($type);
mod u64_tests {
use super::*;
from_into_tests!($type, u64);
math_between_tests!($type, u64);
#[test]
pub fn as_64() {
let x = $type(0).as_u64();
assert_eq!(x, 0);
let x = $type(3).as_u64();
assert_eq!(x, 3);
let x = $type(u64::max_value()).as_u64();
assert_eq!(x, u64::max_value());
}
}
mod usize_tests {
use super::*;
from_into_tests!($type, usize);
#[test]
pub fn as_usize() {
let x = $type(0).as_usize();
assert_eq!(x, 0);
let x = $type(3).as_usize();
assert_eq!(x, 3);
let x = $type(u64::max_value()).as_usize();
assert_eq!(x, usize::max_value());
}
}
};
}
#[cfg(test)]
mod slot_tests {
use super::*;
use crate::test_utils::{SeedableRng, TestRandom, XorShiftRng};
use ssz::ssz_encode;
all_tests!(Slot);
}
#[cfg(test)]
mod epoch_tests {
use super::*;
use crate::test_utils::{SeedableRng, TestRandom, XorShiftRng};
use ssz::ssz_encode;
all_tests!(Epoch);
}
}

View File

@ -1,7 +1,7 @@
use super::ChainSpec;
use bls::{Keypair, PublicKey, SecretKey, Signature};
use crate::{Address, Eth1Data, Hash256, Validator};
use crate::{Address, Eth1Data, Hash256, Slot, Validator};
/// The size of a validators deposit in GWei.
pub const DEPOSIT_GWEI: u64 = 32_000_000_000;
@ -37,9 +37,9 @@ impl ChainSpec {
* Initial Values
*/
genesis_fork_version: 0,
genesis_slot: 0,
genesis_slot: Slot::from(0_u64),
genesis_start_shard: 0,
far_future_slot: u64::max_value(),
far_future_slot: Slot::from(u64::max_value()),
zero_hash: Hash256::zero(),
empty_signature: Signature::empty_signature(),
bls_withdrawal_prefix_byte: 0x00,
@ -110,14 +110,14 @@ fn initial_validators_for_testing() -> Vec<Validator> {
pubkey: keypair.pk.clone(),
withdrawal_credentials: Hash256::zero(),
proposer_slots: 0,
activation_slot: u64::max_value(),
exit_slot: u64::max_value(),
withdrawal_slot: u64::max_value(),
penalized_slot: u64::max_value(),
activation_slot: Slot::max_value(),
exit_slot: Slot::max_value(),
withdrawal_slot: Slot::max_value(),
penalized_slot: Slot::max_value(),
exit_count: 0,
status_flags: None,
latest_custody_reseed_slot: 0,
penultimate_custody_reseed_slot: 0,
latest_custody_reseed_slot: Slot::from(0_u64),
penultimate_custody_reseed_slot: Slot::from(0_u64),
};
initial_validators.push(validator);
}

View File

@ -1,6 +1,6 @@
mod foundation;
use crate::{Address, Eth1Data, Hash256, Validator};
use crate::{Address, Eth1Data, Hash256, Slot, Validator};
use bls::Signature;
#[derive(PartialEq, Debug, Clone)]
@ -29,9 +29,9 @@ pub struct ChainSpec {
* Initial Values
*/
pub genesis_fork_version: u64,
pub genesis_slot: u64,
pub genesis_slot: Slot,
pub genesis_start_shard: u64,
pub far_future_slot: u64,
pub far_future_slot: Slot,
pub zero_hash: Hash256,
pub empty_signature: Signature,
pub bls_withdrawal_prefix_byte: u8,

View File

@ -1,5 +1,4 @@
use super::Hash256;
use crate::{test_utils::TestRandom, PublicKey};
use crate::{test_utils::TestRandom, Hash256, PublicKey, Slot};
use rand::RngCore;
use serde_derive::Serialize;
use ssz::{hash, Decodable, DecodeError, Encodable, SszStream, TreeHash};
@ -48,19 +47,19 @@ pub struct Validator {
pub pubkey: PublicKey,
pub withdrawal_credentials: Hash256,
pub proposer_slots: u64,
pub activation_slot: u64,
pub exit_slot: u64,
pub withdrawal_slot: u64,
pub penalized_slot: u64,
pub activation_slot: Slot,
pub exit_slot: Slot,
pub withdrawal_slot: Slot,
pub penalized_slot: Slot,
pub exit_count: u64,
pub status_flags: Option<StatusFlags>,
pub latest_custody_reseed_slot: u64,
pub penultimate_custody_reseed_slot: u64,
pub latest_custody_reseed_slot: Slot,
pub penultimate_custody_reseed_slot: Slot,
}
impl Validator {
/// This predicate indicates if the validator represented by this record is considered "active" at `slot`.
pub fn is_active_at(&self, slot: u64) -> bool {
pub fn is_active_at(&self, slot: Slot) -> bool {
self.activation_slot <= slot && slot < self.exit_slot
}
}
@ -72,14 +71,14 @@ impl Default for Validator {
pubkey: PublicKey::default(),
withdrawal_credentials: Hash256::default(),
proposer_slots: 0,
activation_slot: std::u64::MAX,
exit_slot: std::u64::MAX,
withdrawal_slot: std::u64::MAX,
penalized_slot: std::u64::MAX,
activation_slot: Slot::from(std::u64::MAX),
exit_slot: Slot::from(std::u64::MAX),
withdrawal_slot: Slot::from(std::u64::MAX),
penalized_slot: Slot::from(std::u64::MAX),
exit_count: 0,
status_flags: None,
latest_custody_reseed_slot: 0, // NOTE: is `GENESIS_SLOT`
penultimate_custody_reseed_slot: 0, // NOTE: is `GENESIS_SLOT`
latest_custody_reseed_slot: Slot::from(0_u64), // NOTE: is `GENESIS_SLOT`
penultimate_custody_reseed_slot: Slot::from(0_u64), // NOTE: is `GENESIS_SLOT`
}
}
}
@ -203,10 +202,11 @@ mod tests {
let activation_slot = u64::random_for_test(&mut rng);
let exit_slot = activation_slot + 234;
validator.activation_slot = activation_slot;
validator.exit_slot = exit_slot;
validator.activation_slot = Slot::from(activation_slot);
validator.exit_slot = Slot::from(exit_slot);
for slot in (activation_slot - 100)..(exit_slot + 100) {
let slot = Slot::from(slot);
if slot < activation_slot {
assert!(!validator.is_active_at(slot));
} else if slot >= exit_slot {

View File

@ -1,9 +1,10 @@
/// Contains logic to manipulate a `&[Validator]`.
/// For now, we avoid defining a newtype and just have flat functions here.
use super::validator::*;
use crate::Slot;
/// Given an indexed sequence of `validators`, return the indices corresponding to validators that are active at `slot`.
pub fn get_active_validator_indices(validators: &[Validator], slot: u64) -> Vec<usize> {
pub fn get_active_validator_indices(validators: &[Validator], slot: Slot) -> Vec<usize> {
validators
.iter()
.enumerate()
@ -27,7 +28,7 @@ mod tests {
let mut rng = XorShiftRng::from_seed([42; 16]);
let validators = vec![];
let some_slot = u64::random_for_test(&mut rng);
let some_slot = Slot::random_for_test(&mut rng);
let indices = get_active_validator_indices(&validators, some_slot);
assert_eq!(indices, vec![]);
}
@ -41,7 +42,7 @@ mod tests {
validators.push(Validator::default())
}
let some_slot = u64::random_for_test(&mut rng);
let some_slot = Slot::random_for_test(&mut rng);
let indices = get_active_validator_indices(&validators, some_slot);
assert_eq!(indices, vec![]);
}
@ -50,7 +51,7 @@ mod tests {
fn can_get_all_active_validator_indices() {
let mut rng = XorShiftRng::from_seed([42; 16]);
let count_validators = 10;
let some_slot = u64::random_for_test(&mut rng);
let some_slot = Slot::random_for_test(&mut rng);
let mut validators = (0..count_validators)
.into_iter()
@ -60,8 +61,8 @@ mod tests {
let activation_offset = u64::random_for_test(&mut rng);
let exit_offset = u64::random_for_test(&mut rng);
validator.activation_slot = some_slot.checked_sub(activation_offset).unwrap_or(0);
validator.exit_slot = some_slot.checked_add(exit_offset).unwrap_or(std::u64::MAX);
validator.activation_slot = some_slot - activation_offset;
validator.exit_slot = some_slot + exit_offset;
validator
})
@ -81,13 +82,13 @@ mod tests {
fn set_validators_to_default_entry_exit(validators: &mut [Validator]) {
for validator in validators.iter_mut() {
validator.activation_slot = std::u64::MAX;
validator.exit_slot = std::u64::MAX;
validator.activation_slot = Slot::max_value();
validator.exit_slot = Slot::max_value();
}
}
// sets all `validators` to be active as of some slot prior to `slot`. returns the activation slot.
fn set_validators_to_activated(validators: &mut [Validator], slot: u64) -> u64 {
fn set_validators_to_activated(validators: &mut [Validator], slot: Slot) -> Slot {
let activation_slot = slot - 10;
for validator in validators.iter_mut() {
validator.activation_slot = activation_slot;
@ -96,7 +97,7 @@ mod tests {
}
// sets all `validators` to be exited as of some slot before `slot`.
fn set_validators_to_exited(validators: &mut [Validator], slot: u64, activation_slot: u64) {
fn set_validators_to_exited(validators: &mut [Validator], slot: Slot, activation_slot: Slot) {
assert!(activation_slot < slot);
let mut exit_slot = activation_slot + 10;
while exit_slot >= slot {
@ -114,18 +115,18 @@ mod tests {
let mut rng = XorShiftRng::from_seed([42; 16]);
const COUNT_PARTITIONS: usize = 3;
const COUNT_VALIDATORS: usize = 3 * COUNT_PARTITIONS;
let some_slot: u64 = u64::random_for_test(&mut rng);
let some_slot: Slot = Slot::random_for_test(&mut rng);
let mut validators = (0..COUNT_VALIDATORS)
.into_iter()
.map(|_| {
let mut validator = Validator::default();
let activation_offset = u64::random_for_test(&mut rng);
let exit_offset = u64::random_for_test(&mut rng);
let activation_offset = Slot::random_for_test(&mut rng);
let exit_offset = Slot::random_for_test(&mut rng);
validator.activation_slot = some_slot.checked_sub(activation_offset).unwrap_or(0);
validator.exit_slot = some_slot.checked_add(exit_offset).unwrap_or(std::u64::MAX);
validator.activation_slot = some_slot - activation_offset;
validator.exit_slot = some_slot + exit_offset;
validator
})

View File

@ -1,5 +1,4 @@
use super::Hash256;
use crate::test_utils::TestRandom;
use crate::{test_utils::TestRandom, Hash256, Slot};
use bls::PublicKey;
use rand::RngCore;
use serde_derive::Serialize;
@ -11,7 +10,7 @@ pub struct ValidatorRegistryDeltaBlock {
pub latest_registry_delta_root: Hash256,
pub validator_index: u32,
pub pubkey: PublicKey,
pub slot: u64,
pub slot: Slot,
pub flag: u64,
}
@ -22,7 +21,7 @@ impl Default for ValidatorRegistryDeltaBlock {
latest_registry_delta_root: Hash256::zero(),
validator_index: std::u32::MAX,
pubkey: PublicKey::default(),
slot: std::u64::MAX,
slot: Slot::from(std::u64::MAX),
flag: std::u64::MAX,
}
}

View File

@ -5,3 +5,4 @@ authors = ["Paul Hauner <paul@paulhauner.com>"]
edition = "2018"
[dependencies]
types = { path = "../../types" }

View File

@ -3,9 +3,10 @@ mod testing_slot_clock;
pub use crate::system_time_slot_clock::{Error as SystemTimeSlotClockError, SystemTimeSlotClock};
pub use crate::testing_slot_clock::{Error as TestingSlotClockError, TestingSlotClock};
pub use types::Slot;
pub trait SlotClock: Send + Sync {
type Error;
fn present_slot(&self) -> Result<Option<u64>, Self::Error>;
fn present_slot(&self) -> Result<Option<Slot>, Self::Error>;
}

View File

@ -1,5 +1,6 @@
use super::SlotClock;
use std::time::{Duration, SystemTime};
use types::Slot;
pub use std::time::SystemTimeError;
@ -38,7 +39,7 @@ impl SystemTimeSlotClock {
impl SlotClock for SystemTimeSlotClock {
type Error = Error;
fn present_slot(&self) -> Result<Option<u64>, Error> {
fn present_slot(&self) -> Result<Option<Slot>, Error> {
let syslot_time = SystemTime::now();
let duration_since_epoch = syslot_time.duration_since(SystemTime::UNIX_EPOCH)?;
let duration_since_genesis =
@ -56,8 +57,10 @@ impl From<SystemTimeError> for Error {
}
}
fn slot_from_duration(slot_duration_seconds: u64, duration: Duration) -> Option<u64> {
duration.as_secs().checked_div(slot_duration_seconds)
fn slot_from_duration(slot_duration_seconds: u64, duration: Duration) -> Option<Slot> {
Some(Slot::new(
duration.as_secs().checked_div(slot_duration_seconds)?,
))
}
#[cfg(test)]
@ -81,19 +84,19 @@ mod tests {
genesis_seconds: genesis,
slot_duration_seconds: slot_time,
};
assert_eq!(clock.present_slot().unwrap(), Some(89));
assert_eq!(clock.present_slot().unwrap(), Some(Slot::new(89)));
let clock = SystemTimeSlotClock {
genesis_seconds: since_epoch.as_secs(),
slot_duration_seconds: slot_time,
};
assert_eq!(clock.present_slot().unwrap(), Some(0));
assert_eq!(clock.present_slot().unwrap(), Some(Slot::new(0)));
let clock = SystemTimeSlotClock {
genesis_seconds: since_epoch.as_secs() - slot_time * 42 - 5,
slot_duration_seconds: slot_time,
};
assert_eq!(clock.present_slot().unwrap(), Some(42));
assert_eq!(clock.present_slot().unwrap(), Some(Slot::new(42)));
}
#[test]
@ -102,23 +105,23 @@ mod tests {
assert_eq!(
slot_from_duration(slot_time, Duration::from_secs(0)),
Some(0)
Some(Slot::new(0))
);
assert_eq!(
slot_from_duration(slot_time, Duration::from_secs(10)),
Some(0)
Some(Slot::new(0))
);
assert_eq!(
slot_from_duration(slot_time, Duration::from_secs(100)),
Some(1)
Some(Slot::new(1))
);
assert_eq!(
slot_from_duration(slot_time, Duration::from_secs(101)),
Some(1)
Some(Slot::new(1))
);
assert_eq!(
slot_from_duration(slot_time, Duration::from_secs(1000)),
Some(10)
Some(Slot::new(10))
);
}

View File

@ -1,5 +1,6 @@
use super::SlotClock;
use std::sync::RwLock;
use types::Slot;
#[derive(Debug, PartialEq)]
pub enum Error {}
@ -27,9 +28,9 @@ impl TestingSlotClock {
impl SlotClock for TestingSlotClock {
type Error = Error;
fn present_slot(&self) -> Result<Option<u64>, Error> {
fn present_slot(&self) -> Result<Option<Slot>, Error> {
let slot = *self.slot.read().expect("TestingSlotClock poisoned.");
Ok(Some(slot))
Ok(Some(Slot::new(slot)))
}
}
@ -40,8 +41,8 @@ mod tests {
#[test]
fn test_slot_now() {
let clock = TestingSlotClock::new(10);
assert_eq!(clock.present_slot(), Ok(Some(10)));
assert_eq!(clock.present_slot(), Ok(Some(Slot::new(10))));
clock.set_slot(123);
assert_eq!(clock.present_slot(), Ok(Some(123)));
assert_eq!(clock.present_slot(), Ok(Some(Slot::new(123))));
}
}

View File

@ -1,5 +1,5 @@
use bls::verify_proof_of_possession;
use types::{BeaconState, ChainSpec, Deposit, Validator};
use types::{BeaconState, ChainSpec, Deposit, Slot, Validator};
#[derive(Debug, PartialEq, Clone)]
pub enum ValidatorInductionError {
@ -48,8 +48,8 @@ pub fn process_deposit(
penalized_slot: spec.far_future_slot,
exit_count: 0,
status_flags: None,
latest_custody_reseed_slot: 0,
penultimate_custody_reseed_slot: 0,
latest_custody_reseed_slot: Slot::new(0),
penultimate_custody_reseed_slot: Slot::new(0),
};
let _index = state.validator_registry.len();

View File

@ -5,7 +5,7 @@ use protos::services::{
use protos::services_grpc::BeaconBlockServiceClient;
use ssz::{ssz_encode, Decodable};
use std::sync::Arc;
use types::{BeaconBlock, BeaconBlockBody, Eth1Data, Hash256, PublicKey, Signature};
use types::{BeaconBlock, BeaconBlockBody, Eth1Data, Hash256, PublicKey, Signature, Slot};
/// A newtype designed to wrap the gRPC-generated service so the `BeaconNode` trait may be
/// implemented upon it.
@ -32,11 +32,11 @@ impl BeaconNode for BeaconBlockGrpcClient {
/// BN is unable to find a parent block.
fn produce_beacon_block(
&self,
slot: u64,
slot: Slot,
randao_reveal: &Signature,
) -> Result<Option<BeaconBlock>, BeaconNodeError> {
let mut req = ProduceBeaconBlockRequest::new();
req.set_slot(slot);
req.set_slot(slot.as_u64());
let reply = self
.client
@ -54,7 +54,7 @@ impl BeaconNode for BeaconBlockGrpcClient {
// TODO: this conversion is incomplete; fix it.
Ok(Some(BeaconBlock {
slot: block.get_slot(),
slot: Slot::new(block.get_slot()),
parent_root: Hash256::zero(),
state_root: Hash256::zero(),
randao_reveal,
@ -88,7 +88,7 @@ impl BeaconNode for BeaconBlockGrpcClient {
// TODO: this conversion is incomplete; fix it.
let mut grpc_block = GrpcBeaconBlock::new();
grpc_block.set_slot(block.slot);
grpc_block.set_slot(block.slot.as_u64());
grpc_block.set_block_root(vec![0]);
grpc_block.set_randao_reveal(ssz_encode(&block.randao_reveal));
grpc_block.set_signature(ssz_encode(&block.signature));

View File

@ -1,6 +1,7 @@
use block_producer::{DutiesReader, DutiesReaderError};
use std::collections::HashMap;
use std::sync::RwLock;
use types::{Epoch, Slot};
/// The information required for a validator to propose and attest during some epoch.
///
@ -10,14 +11,14 @@ use std::sync::RwLock;
#[derive(Debug, PartialEq, Clone, Copy, Default)]
pub struct EpochDuties {
pub validator_index: u64,
pub block_production_slot: Option<u64>,
pub block_production_slot: Option<Slot>,
// Future shard info
}
impl EpochDuties {
/// Returns `true` if the supplied `slot` is a slot in which the validator should produce a
/// block.
pub fn is_block_production_slot(&self, slot: u64) -> bool {
pub fn is_block_production_slot(&self, slot: Slot) -> bool {
match self.block_production_slot {
Some(s) if s == slot => true,
_ => false,
@ -32,7 +33,7 @@ pub enum EpochDutiesMapError {
/// Maps an `epoch` to some `EpochDuties` for a single validator.
pub struct EpochDutiesMap {
pub epoch_length: u64,
pub map: RwLock<HashMap<u64, EpochDuties>>,
pub map: RwLock<HashMap<Epoch, EpochDuties>>,
}
impl EpochDutiesMap {
@ -43,7 +44,7 @@ impl EpochDutiesMap {
}
}
pub fn get(&self, epoch: u64) -> Result<Option<EpochDuties>, EpochDutiesMapError> {
pub fn get(&self, epoch: Epoch) -> Result<Option<EpochDuties>, EpochDutiesMapError> {
let map = self.map.read().map_err(|_| EpochDutiesMapError::Poisoned)?;
match map.get(&epoch) {
Some(duties) => Ok(Some(duties.clone())),
@ -53,7 +54,7 @@ impl EpochDutiesMap {
pub fn insert(
&self,
epoch: u64,
epoch: Epoch,
epoch_duties: EpochDuties,
) -> Result<Option<EpochDuties>, EpochDutiesMapError> {
let mut map = self
@ -65,10 +66,8 @@ impl EpochDutiesMap {
}
impl DutiesReader for EpochDutiesMap {
fn is_block_production_slot(&self, slot: u64) -> Result<bool, DutiesReaderError> {
let epoch = slot
.checked_div(self.epoch_length)
.ok_or_else(|| DutiesReaderError::EpochLengthIsZero)?;
fn is_block_production_slot(&self, slot: Slot) -> Result<bool, DutiesReaderError> {
let epoch = slot.epoch(self.epoch_length);
let map = self.map.read().map_err(|_| DutiesReaderError::Poisoned)?;
let duties = map

View File

@ -3,7 +3,7 @@ use super::EpochDuties;
use protos::services::{ProposeBlockSlotRequest, PublicKey as IndexRequest};
use protos::services_grpc::ValidatorServiceClient;
use ssz::ssz_encode;
use types::PublicKey;
use types::{Epoch, PublicKey, Slot};
impl BeaconNode for ValidatorServiceClient {
/// Request the shuffling from the Beacon Node (BN).
@ -14,7 +14,7 @@ impl BeaconNode for ValidatorServiceClient {
/// Note: presently only block production information is returned.
fn request_shuffling(
&self,
epoch: u64,
epoch: Epoch,
public_key: &PublicKey,
) -> Result<Option<EpochDuties>, BeaconNodeError> {
// Lookup the validator index for the supplied public key.
@ -29,7 +29,7 @@ impl BeaconNode for ValidatorServiceClient {
let mut req = ProposeBlockSlotRequest::new();
req.set_validator_index(validator_index);
req.set_epoch(epoch);
req.set_epoch(epoch.as_u64());
let reply = self
.propose_block_slot(&req)
@ -41,6 +41,11 @@ impl BeaconNode for ValidatorServiceClient {
None
};
let block_production_slot = match block_production_slot {
Some(slot) => Some(Slot::new(slot)),
None => None,
};
Ok(Some(EpochDuties {
validator_index,
block_production_slot,

View File

@ -12,20 +12,20 @@ use self::traits::{BeaconNode, BeaconNodeError};
use bls::PublicKey;
use slot_clock::SlotClock;
use std::sync::Arc;
use types::ChainSpec;
use types::{ChainSpec, Epoch, Slot};
#[derive(Debug, PartialEq, Clone, Copy)]
pub enum PollOutcome {
/// The `EpochDuties` were not updated during this poll.
NoChange(u64),
NoChange(Epoch),
/// The `EpochDuties` for the `epoch` were previously unknown, but obtained in the poll.
NewDuties(u64, EpochDuties),
NewDuties(Epoch, EpochDuties),
/// New `EpochDuties` were obtained, different to those which were previously known. This is
/// likely to be the result of chain re-organisation.
DutiesChanged(u64, EpochDuties),
DutiesChanged(Epoch, EpochDuties),
/// The Beacon Node was unable to return the duties as the validator is unknown, or the
/// shuffling for the epoch is unknown.
UnknownValidatorOrEpoch(u64),
UnknownValidatorOrEpoch(Epoch),
}
#[derive(Debug, PartialEq)]
@ -62,9 +62,7 @@ impl<T: SlotClock, U: BeaconNode> DutiesManager<T, U> {
.map_err(|_| Error::SlotClockError)?
.ok_or(Error::SlotUnknowable)?;
let epoch = slot
.checked_div(self.spec.epoch_length)
.ok_or(Error::EpochLengthIsZero)?;
let epoch = slot.epoch(self.spec.epoch_length);
if let Some(duties) = self.beacon_node.request_shuffling(epoch, &self.pubkey)? {
// If these duties were known, check to see if they're updates or identical.
@ -130,25 +128,34 @@ mod tests {
// Configure response from the BeaconNode.
let duties = EpochDuties {
validator_index: 0,
block_production_slot: Some(10),
block_production_slot: Some(Slot::new(10)),
};
beacon_node.set_next_shuffling_result(Ok(Some(duties)));
// Get the duties for the first time...
assert_eq!(manager.poll(), Ok(PollOutcome::NewDuties(0, duties)));
assert_eq!(
manager.poll(),
Ok(PollOutcome::NewDuties(Epoch::new(0), duties))
);
// Get the same duties again...
assert_eq!(manager.poll(), Ok(PollOutcome::NoChange(0)));
assert_eq!(manager.poll(), Ok(PollOutcome::NoChange(Epoch::new(0))));
// Return new duties.
let duties = EpochDuties {
validator_index: 0,
block_production_slot: Some(11),
block_production_slot: Some(Slot::new(11)),
};
beacon_node.set_next_shuffling_result(Ok(Some(duties)));
assert_eq!(manager.poll(), Ok(PollOutcome::DutiesChanged(0, duties)));
assert_eq!(
manager.poll(),
Ok(PollOutcome::DutiesChanged(Epoch::new(0), duties))
);
// Return no duties.
beacon_node.set_next_shuffling_result(Ok(None));
assert_eq!(manager.poll(), Ok(PollOutcome::UnknownValidatorOrEpoch(0)));
assert_eq!(
manager.poll(),
Ok(PollOutcome::UnknownValidatorOrEpoch(Epoch::new(0)))
);
}
}

View File

@ -2,13 +2,14 @@ use super::traits::{BeaconNode, BeaconNodeError};
use super::EpochDuties;
use bls::PublicKey;
use std::sync::RwLock;
use types::Epoch;
type ShufflingResult = Result<Option<EpochDuties>, BeaconNodeError>;
/// A test-only struct used to simulate a Beacon Node.
#[derive(Default)]
pub struct TestBeaconNode {
pub request_shuffling_input: RwLock<Option<(u64, PublicKey)>>,
pub request_shuffling_input: RwLock<Option<(Epoch, PublicKey)>>,
pub request_shuffling_result: RwLock<Option<ShufflingResult>>,
}
@ -21,7 +22,7 @@ impl TestBeaconNode {
impl BeaconNode for TestBeaconNode {
/// Returns the value specified by the `set_next_shuffling_result`.
fn request_shuffling(&self, epoch: u64, public_key: &PublicKey) -> ShufflingResult {
fn request_shuffling(&self, epoch: Epoch, public_key: &PublicKey) -> ShufflingResult {
*self.request_shuffling_input.write().unwrap() = Some((epoch, public_key.clone()));
match *self.request_shuffling_result.read().unwrap() {
Some(ref r) => r.clone(),

View File

@ -1,5 +1,6 @@
use super::EpochDuties;
use bls::PublicKey;
use types::Epoch;
#[derive(Debug, PartialEq, Clone)]
pub enum BeaconNodeError {
@ -13,7 +14,7 @@ pub trait BeaconNode: Send + Sync {
/// Returns Ok(None) if the public key is unknown, or the shuffling for that epoch is unknown.
fn request_shuffling(
&self,
epoch: u64,
epoch: Epoch,
public_key: &PublicKey,
) -> Result<Option<EpochDuties>, BeaconNodeError>;
}