Update to spec v0.9.1 (#597)

* Update to spec v0.9.0

* Update to v0.9.1

* Bump spec tags for v0.9.1

* Formatting, fix CI failures

* Resolve accidental KeyPair merge conflict

* Document new BeaconState functions

* Fix incorrect cache drops in `advance_caches`

* Update fork choice for v0.9.1

* Clean up some FIXMEs

* Fix a few docs/logs
This commit is contained in:
Michael Sproul 2019-11-21 11:47:30 +11:00 committed by GitHub
parent b7a0feb725
commit 24e941d175
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
105 changed files with 1211 additions and 2940 deletions

View File

@ -4,7 +4,7 @@ default:
image: 'sigp/lighthouse:latest'
cache:
paths:
- tests/ef_tests/*-v0.8.3.tar.gz
- tests/ef_tests/*-v0.9.1.tar.gz
stages:
- test
@ -47,7 +47,7 @@ test-ef-fake-crypto:
GIT_SUBMODULE_STRATEGY: normal
script:
- make make-ef-tests
- cargo test --manifest-path tests/ef_tests/Cargo.toml --release --features ef_tests fake_crypto
- cargo test --manifest-path tests/ef_tests/Cargo.toml --release --features ef_tests,fake_crypto
documentation:
stage: document

View File

@ -16,7 +16,7 @@ use ssz::Encode;
use state_processing::per_block_processing::{
errors::{
AttestationValidationError, AttesterSlashingValidationError, DepositValidationError,
ExitValidationError, ProposerSlashingValidationError, TransferValidationError,
ExitValidationError, ProposerSlashingValidationError,
},
verify_attestation_for_state, VerifySignatures,
};
@ -218,6 +218,44 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
ReverseBlockRootIterator::new((head.beacon_block_root, head.beacon_block.slot), iter)
}
/// Traverse backwards from `block_root` to find the block roots of its ancestors.
///
/// ## Notes
///
/// `slot` always decreases by `1`.
/// - Skipped slots contain the root of the closest prior
/// non-skipped slot (identical to the way they are stored in `state.block_roots`) .
/// - Iterator returns `(Hash256, Slot)`.
/// - The provided `block_root` is included as the first item in the iterator.
pub fn rev_iter_block_roots_from(
&self,
block_root: Hash256,
) -> Result<ReverseBlockRootIterator<T::EthSpec, T::Store>, Error> {
let block = self
.get_block(&block_root)?
.ok_or_else(|| Error::MissingBeaconBlock(block_root))?;
let state = self
.get_state(&block.state_root)?
.ok_or_else(|| Error::MissingBeaconState(block.state_root))?;
let iter = BlockRootsIterator::owned(self.store.clone(), state);
Ok(ReverseBlockRootIterator::new(
(block_root, block.slot),
iter,
))
}
/// Traverse backwards from `block_root` to find the root of the ancestor block at `slot`.
pub fn get_ancestor_block_root(
&self,
block_root: Hash256,
slot: Slot,
) -> Result<Option<Hash256>, Error> {
Ok(self
.rev_iter_block_roots_from(block_root)?
.find(|(_, ancestor_slot)| *ancestor_slot == slot)
.map(|(ancestor_block_root, _)| ancestor_block_root))
}
/// Iterates across all `(state_root, slot)` pairs from the head of the chain (inclusive) to
/// the earliest reachable ancestor (may or may not be genesis).
///
@ -248,6 +286,18 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
Ok(self.store.get(block_root)?)
}
/// Returns the state at the given root, if any.
///
/// ## Errors
///
/// May return a database error.
pub fn get_state(
&self,
state_root: &Hash256,
) -> Result<Option<BeaconState<T::EthSpec>>, Error> {
Ok(self.store.get(state_root)?)
}
/// Returns a `Checkpoint` representing the head block and state. Contains the "best block";
/// the head of the canonical `BeaconChain`.
///
@ -385,15 +435,15 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
}
state
.get_beacon_proposer_index(slot, RelativeEpoch::Current, &self.spec)
.get_beacon_proposer_index(slot, &self.spec)
.map_err(Into::into)
}
/// Returns the attestation slot and shard for a given validator index.
/// Returns the attestation slot and committee index for a given validator index.
///
/// Information is read from the current state, so only information from the present and prior
/// epoch is available.
pub fn validator_attestation_slot_and_shard(
pub fn validator_attestation_slot_and_index(
&self,
validator_index: usize,
epoch: Epoch,
@ -420,25 +470,25 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
if let Some(attestation_duty) =
state.get_attestation_duties(validator_index, RelativeEpoch::Current)?
{
Ok(Some((attestation_duty.slot, attestation_duty.shard)))
Ok(Some((attestation_duty.slot, attestation_duty.index)))
} else {
Ok(None)
}
}
/// Produce an `AttestationData` that is valid for the given `slot` `shard`.
/// Produce an `AttestationData` that is valid for the given `slot`, `index`.
///
/// Always attests to the canonical chain.
pub fn produce_attestation_data(
&self,
shard: u64,
slot: Slot,
index: CommitteeIndex,
) -> Result<AttestationData, Error> {
let state = self.state_at_slot(slot)?;
let head = self.head();
self.produce_attestation_data_for_block(
shard,
index,
head.beacon_block_root,
head.beacon_block.slot,
&state,
@ -451,7 +501,7 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
/// function should be used as it attests to the canonical chain.
pub fn produce_attestation_data_for_block(
&self,
shard: u64,
index: CommitteeIndex,
head_block_root: Hash256,
head_block_slot: Slot,
state: &BeaconState<T::EthSpec>,
@ -492,18 +542,6 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
root: target_root,
};
let parent_crosslink = state.get_current_crosslink(shard)?;
let crosslink = Crosslink {
shard,
parent_root: Hash256::from_slice(&parent_crosslink.tree_hash_root()),
start_epoch: parent_crosslink.end_epoch,
end_epoch: std::cmp::min(
target.epoch,
parent_crosslink.end_epoch + self.spec.max_epochs_per_crosslink,
),
data_root: Hash256::zero(),
};
// Collect some metrics.
metrics::inc_counter(&metrics::ATTESTATION_PRODUCTION_SUCCESSES);
metrics::stop_timer(timer);
@ -512,15 +550,16 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
self.log,
"Produced beacon attestation data";
"beacon_block_root" => format!("{}", head_block_root),
"shard" => shard,
"slot" => state.slot
"slot" => state.slot,
"index" => index
);
Ok(AttestationData {
slot: state.slot,
index,
beacon_block_root: head_block_root,
source: state.current_justified_checkpoint.clone(),
target,
crosslink,
})
}
@ -549,7 +588,7 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
self.log,
"Beacon attestation imported";
"target_epoch" => attestation.data.target.epoch,
"shard" => attestation.data.crosslink.shard,
"index" => attestation.data.index,
);
let _ = self
.event_handler
@ -668,16 +707,14 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
state.build_committee_cache(RelativeEpoch::Current, &self.spec)?;
let attestation_slot = state.get_attestation_data_slot(&attestation.data)?;
// Reject any attestation where the `state` loaded from `data.beacon_block_root`
// has a higher slot than the attestation.
//
// Permitting this would allow for attesters to vote on _future_ slots.
if state.slot > attestation_slot {
if state.slot > attestation.data.slot {
Ok(AttestationProcessingOutcome::AttestsToFutureState {
state: state.slot,
attestation: attestation_slot,
attestation: attestation.data.slot,
})
} else {
self.process_attestation_for_state_and_block(
@ -776,6 +813,12 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
Ok(AttestationProcessingOutcome::Invalid(e))
} else {
// If the attestation is from the current or previous epoch, supply it to the fork
// choice. This is FMD GHOST.
let current_epoch = self.epoch()?;
if attestation.data.target.epoch == current_epoch
|| attestation.data.target.epoch == current_epoch - 1
{
// Provide the attestation to fork choice, updating the validator latest messages but
// _without_ finding and updating the head.
if let Err(e) = self
@ -791,6 +834,7 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
);
return Err(e.into());
}
}
// Provide the valid attestation to op pool, which may choose to retain the
// attestation for inclusion in a future block.
@ -829,22 +873,6 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
}
}
/// Accept some transfer and queue it for inclusion in an appropriate block.
pub fn process_transfer(&self, transfer: Transfer) -> Result<(), TransferValidationError> {
match self.wall_clock_state() {
Ok(state) => self.op_pool.insert_transfer(transfer, &state, &self.spec),
Err(e) => {
error!(
&self.log,
"Unable to process transfer";
"error" => format!("{:?}", e),
"reason" => "no state"
);
Ok(())
}
}
}
/// Accept some proposer slashing and queue it for inclusion in an appropriate block.
pub fn process_proposer_slashing(
&self,
@ -905,7 +933,7 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
self.log,
"Beacon block imported";
"block_root" => format!("{:?}", block_root),
"block_slot" => format!("{:?}", block_root),
"block_slot" => format!("{:?}", block.slot.as_u64()),
);
let _ = self.event_handler.register(EventKind::BeaconBlockImported {
block_root: *block_root,
@ -1115,7 +1143,10 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
metrics::start_timer(&metrics::BLOCK_PROCESSING_FORK_CHOICE_REGISTER);
// Register the new block with the fork choice service.
if let Err(e) = self.fork_choice.process_block(&state, &block, block_root) {
if let Err(e) = self
.fork_choice
.process_block(self, &state, &block, block_root)
{
error!(
self.log,
"Add block to fork choice failed";
@ -1230,7 +1261,6 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
.deposits_for_block_inclusion(&state, &self.spec)?
.into(),
voluntary_exits: self.op_pool.get_voluntary_exits(&state, &self.spec).into(),
transfers: self.op_pool.get_transfers(&state, &self.spec).into(),
},
};

View File

@ -231,7 +231,12 @@ where
.genesis_block_root
.ok_or_else(|| "fork_choice_backend requires a genesis_block_root")?;
self.fork_choice = Some(ForkChoice::new(store, backend, genesis_block_root));
self.fork_choice = Some(ForkChoice::new(
store,
backend,
genesis_block_root,
self.spec.genesis_slot,
));
Ok(self)
}

View File

@ -18,7 +18,6 @@ macro_rules! easy_from_to {
#[derive(Debug, PartialEq)]
pub enum BeaconChainError {
InsufficientValidators,
BadRecentBlockRoots,
UnableToReadSlot,
RevertedFinalizedEpoch {
previous_epoch: Epoch,

View File

@ -487,7 +487,7 @@ mod test {
let keypair = generate_deterministic_keypair(i as usize);
let mut builder =
TestingDepositBuilder::new(keypair.pk.clone(), spec.max_effective_balance);
builder.sign(&DepositTestTask::Valid, &keypair, spec);
builder.sign(DepositTestTask::Valid, &keypair, spec);
let deposit_data = builder.build().data;
DepositLog {

View File

@ -1,10 +1,11 @@
use crate::{metrics, BeaconChain, BeaconChainTypes};
use crate::{errors::BeaconChainError, metrics, BeaconChain, BeaconChainTypes};
use lmd_ghost::LmdGhost;
use state_processing::common::get_attesting_indices;
use parking_lot::RwLock;
use state_processing::{common::get_attesting_indices, per_slot_processing};
use std::sync::Arc;
use store::{Error as StoreError, Store};
use types::{
Attestation, BeaconBlock, BeaconState, BeaconStateError, Epoch, EthSpec, Hash256, Slot,
Attestation, BeaconBlock, BeaconState, BeaconStateError, Checkpoint, EthSpec, Hash256, Slot,
};
type Result<T> = std::result::Result<T, Error>;
@ -16,6 +17,7 @@ pub enum Error {
BackendError(String),
BeaconStateError(BeaconStateError),
StoreError(StoreError),
BeaconChainError(Box<BeaconChainError>),
}
pub struct ForkChoice<T: BeaconChainTypes> {
@ -26,6 +28,10 @@ pub struct ForkChoice<T: BeaconChainTypes> {
/// Does not necessarily need to be the _actual_ genesis, it suffices to be the finalized root
/// whenever the struct was instantiated.
genesis_block_root: Hash256,
/// The fork choice rule's current view of the justified checkpoint.
justified_checkpoint: RwLock<Checkpoint>,
/// The best justified checkpoint we've seen, which may be ahead of `justified_checkpoint`.
best_justified_checkpoint: RwLock<Checkpoint>,
}
impl<T: BeaconChainTypes> ForkChoice<T> {
@ -33,39 +39,91 @@ impl<T: BeaconChainTypes> ForkChoice<T> {
///
/// "Genesis" does not necessarily need to be the absolute genesis, it can be some finalized
/// block.
pub fn new(store: Arc<T::Store>, backend: T::LmdGhost, genesis_block_root: Hash256) -> Self {
pub fn new(
store: Arc<T::Store>,
backend: T::LmdGhost,
genesis_block_root: Hash256,
genesis_slot: Slot,
) -> Self {
let justified_checkpoint = Checkpoint {
epoch: genesis_slot.epoch(T::EthSpec::slots_per_epoch()),
root: genesis_block_root,
};
Self {
store: store.clone(),
backend,
genesis_block_root,
justified_checkpoint: RwLock::new(justified_checkpoint.clone()),
best_justified_checkpoint: RwLock::new(justified_checkpoint),
}
}
/// Determine whether the fork choice's view of the justified checkpoint should be updated.
///
/// To prevent the bouncing attack, an update is allowed only in these conditions:
///
/// * We're in the first SAFE_SLOTS_TO_UPDATE_JUSTIFIED slots of the epoch, or
/// * The new justified checkpoint is a descendant of the current justified checkpoint
fn should_update_justified_checkpoint(
&self,
chain: &BeaconChain<T>,
new_justified_checkpoint: &Checkpoint,
) -> Result<bool> {
if Self::compute_slots_since_epoch_start(chain.slot()?)
< chain.spec.safe_slots_to_update_justified
{
return Ok(true);
}
let justified_checkpoint = self.justified_checkpoint.read().clone();
let current_justified_block = chain
.get_block(&justified_checkpoint.root)?
.ok_or_else(|| Error::MissingBlock(justified_checkpoint.root))?;
let new_justified_block = chain
.get_block(&new_justified_checkpoint.root)?
.ok_or_else(|| Error::MissingBlock(new_justified_checkpoint.root))?;
let slots_per_epoch = T::EthSpec::slots_per_epoch();
Ok(
new_justified_block.slot > justified_checkpoint.epoch.start_slot(slots_per_epoch)
&& chain.get_ancestor_block_root(
new_justified_checkpoint.root,
current_justified_block.slot,
)? == Some(justified_checkpoint.root),
)
}
/// Calculate how far `slot` lies from the start of its epoch.
fn compute_slots_since_epoch_start(slot: Slot) -> u64 {
let slots_per_epoch = T::EthSpec::slots_per_epoch();
(slot - slot.epoch(slots_per_epoch).start_slot(slots_per_epoch)).as_u64()
}
/// Run the fork choice rule to determine the head.
pub fn find_head(&self, chain: &BeaconChain<T>) -> Result<Hash256> {
let timer = metrics::start_timer(&metrics::FORK_CHOICE_FIND_HEAD_TIMES);
let start_slot = |epoch: Epoch| epoch.start_slot(T::EthSpec::slots_per_epoch());
// From the specification:
//
// Let justified_head be the descendant of finalized_head with the highest epoch that has
// been justified for at least 1 epoch ... If no such descendant exists,
// set justified_head to finalized_head.
let (start_state, start_block_root, start_block_slot) = {
let state = &chain.head().beacon_state;
// Check if we should update our view of the justified checkpoint.
// Doing this check here should be quasi-equivalent to the update in the `on_tick`
// function of the spec, so long as `find_head` is called at least once during the first
// SAFE_SLOTS_TO_UPDATE_JUSTIFIED slots.
let best_justified_checkpoint = self.best_justified_checkpoint.read();
if self.should_update_justified_checkpoint(chain, &best_justified_checkpoint)? {
*self.justified_checkpoint.write() = best_justified_checkpoint.clone();
}
let (block_root, block_slot) =
if state.current_epoch() + 1 > state.current_justified_checkpoint.epoch {
(
state.current_justified_checkpoint.root,
start_slot(state.current_justified_checkpoint.epoch),
)
} else {
(
state.finalized_checkpoint.root,
start_slot(state.finalized_checkpoint.epoch),
)
};
let current_justified_checkpoint = self.justified_checkpoint.read().clone();
let (block_root, block_justified_slot) = (
current_justified_checkpoint.root,
current_justified_checkpoint
.epoch
.start_slot(T::EthSpec::slots_per_epoch()),
);
let block = chain
.store
@ -79,12 +137,17 @@ impl<T: BeaconChainTypes> ForkChoice<T> {
block_root
};
let state = chain
.store
.get::<BeaconState<T::EthSpec>>(&block.state_root)?
let mut state = chain
.get_state(&block.state_root)?
.ok_or_else(|| Error::MissingState(block.state_root))?;
(state, block_root, block_slot)
// Fast-forward the state to the start slot of the epoch where it was justified.
for _ in block.slot.as_u64()..block_justified_slot.as_u64() {
per_slot_processing(&mut state, &chain.spec)
.map_err(|e| BeaconChainError::SlotProcessingError(e))?
}
(state, block_root, block_justified_slot)
};
// A function that returns the weight for some validator index.
@ -107,10 +170,11 @@ impl<T: BeaconChainTypes> ForkChoice<T> {
/// Process all attestations in the given `block`.
///
/// Assumes the block (and therefore it's attestations) are valid. It is a logic error to
/// Assumes the block (and therefore its attestations) are valid. It is a logic error to
/// provide an invalid block.
pub fn process_block(
&self,
chain: &BeaconChain<T>,
state: &BeaconState<T::EthSpec>,
block: &BeaconBlock<T::EthSpec>,
block_root: Hash256,
@ -133,6 +197,16 @@ impl<T: BeaconChainTypes> ForkChoice<T> {
}
}
// Check if we should update our view of the justified checkpoint
if state.current_justified_checkpoint.epoch > self.justified_checkpoint.read().epoch {
*self.best_justified_checkpoint.write() = state.current_justified_checkpoint.clone();
if self
.should_update_justified_checkpoint(chain, &state.current_justified_checkpoint)?
{
*self.justified_checkpoint.write() = state.current_justified_checkpoint.clone();
}
}
// This does not apply a vote to the block, it just makes fork choice aware of the block so
// it can still be identified as the head even if it doesn't have any votes.
//
@ -224,6 +298,12 @@ impl From<BeaconStateError> for Error {
}
}
impl From<BeaconChainError> for Error {
fn from(e: BeaconChainError) -> Error {
Error::BeaconChainError(Box::new(e))
}
}
impl From<StoreError> for Error {
fn from(e: StoreError) -> Error {
Error::StoreError(e)

View File

@ -28,7 +28,7 @@ pub use parking_lot;
pub use slot_clock;
pub use state_processing::per_block_processing::errors::{
AttestationValidationError, AttesterSlashingValidationError, DepositValidationError,
ExitValidationError, ProposerSlashingValidationError, TransferValidationError,
ExitValidationError, ProposerSlashingValidationError,
};
pub use store;
pub use types;

View File

@ -172,8 +172,6 @@ lazy_static! {
try_create_int_gauge("beacon_head_state_finalized_root", "Finalized root at the head of the chain");
pub static ref HEAD_STATE_FINALIZED_EPOCH: Result<IntGauge> =
try_create_int_gauge("beacon_head_state_finalized_epoch", "Finalized epoch at the head of the chain");
pub static ref HEAD_STATE_SHARDS: Result<IntGauge> =
try_create_int_gauge("beacon_head_state_shard_total", "Count of shards in the beacon chain");
pub static ref HEAD_STATE_TOTAL_VALIDATORS: Result<IntGauge> =
try_create_int_gauge("beacon_head_state_total_validators_total", "Count of validators at the head of the chain");
pub static ref HEAD_STATE_ACTIVE_VALIDATORS: Result<IntGauge> =
@ -226,7 +224,6 @@ fn scrape_head_state<T: BeaconChainTypes>(state: &BeaconState<T::EthSpec>, state
&HEAD_STATE_FINALIZED_EPOCH,
state.finalized_checkpoint.epoch,
);
set_gauge_by_usize(&HEAD_STATE_SHARDS, state.previous_crosslinks.len());
set_gauge_by_usize(&HEAD_STATE_TOTAL_VALIDATORS, state.validators.len());
set_gauge_by_u64(&HEAD_STATE_VALIDATOR_BALANCES, state.balances.iter().sum());
set_gauge_by_usize(

View File

@ -15,9 +15,8 @@ use std::time::Duration;
use store::MemoryStore;
use tree_hash::{SignedRoot, TreeHash};
use types::{
AggregateSignature, Attestation, AttestationDataAndCustodyBit, BeaconBlock, BeaconState,
BitList, ChainSpec, Domain, EthSpec, Hash256, Keypair, RelativeEpoch, SecretKey, Signature,
Slot,
AggregateSignature, Attestation, BeaconBlock, BeaconState, BitList, ChainSpec, Domain, EthSpec,
Hash256, Keypair, SecretKey, Signature, Slot,
};
pub use crate::persisted_beacon_chain::{PersistedBeaconChain, BEACON_CHAIN_DB_KEY};
@ -203,7 +202,7 @@ impl<E: EthSpec> BeaconChainHarness<HarnessType<E>> {
.block_proposer(slot)
.expect("should get block proposer from chain"),
_ => state
.get_beacon_proposer_index(slot, RelativeEpoch::Current, &self.spec)
.get_beacon_proposer_index(slot, &self.spec)
.expect("should get block proposer from state"),
};
@ -280,13 +279,13 @@ impl<E: EthSpec> BeaconChainHarness<HarnessType<E>> {
let mut attestations = vec![];
state
.get_crosslink_committees_at_slot(state.slot)
.get_beacon_committees_at_slot(state.slot)
.expect("should get committees")
.iter()
.for_each(|cc| {
let committee_size = cc.committee.len();
.for_each(|bc| {
let committee_size = bc.committee.len();
let mut local_attestations: Vec<Attestation<E>> = cc
let mut local_attestations: Vec<Attestation<E>> = bc
.committee
.par_iter()
.enumerate()
@ -297,7 +296,7 @@ impl<E: EthSpec> BeaconChainHarness<HarnessType<E>> {
let data = self
.chain
.produce_attestation_data_for_block(
cc.shard,
bc.index,
head_block_root,
head_block_slot,
state,
@ -309,18 +308,15 @@ impl<E: EthSpec> BeaconChainHarness<HarnessType<E>> {
aggregation_bits
.set(i, true)
.expect("should be able to set aggregation bits");
let custody_bits = BitList::with_capacity(committee_size)
.expect("should make custody bits");
let signature = {
let message = AttestationDataAndCustodyBit {
data: data.clone(),
custody_bit: false,
}
.tree_hash_root();
let message = data.tree_hash_root();
let domain =
spec.get_domain(data.target.epoch, Domain::Attestation, fork);
let domain = spec.get_domain(
data.target.epoch,
Domain::BeaconAttester,
fork,
);
let mut agg_sig = AggregateSignature::new();
agg_sig.add(&Signature::new(
@ -335,7 +331,6 @@ impl<E: EthSpec> BeaconChainHarness<HarnessType<E>> {
let attestation = Attestation {
aggregation_bits,
data,
custody_bits,
signature,
};

View File

@ -17,7 +17,7 @@ use std::sync::Arc;
use tokio;
use tokio::sync::mpsc;
use types::beacon_state::EthSpec;
use types::{Attestation, BeaconBlock, BitList, Epoch, RelativeEpoch, Shard, Slot};
use types::{Attestation, BeaconBlock, BitList, CommitteeIndex, Epoch, RelativeEpoch, Slot};
#[derive(Debug, Serialize, Deserialize)]
pub struct ValidatorDuty {
@ -25,8 +25,8 @@ pub struct ValidatorDuty {
pub validator_pubkey: String,
/// The slot at which the validator must attest.
pub attestation_slot: Option<Slot>,
/// The shard in which the validator must attest.
pub attestation_shard: Option<Shard>,
/// The index of the committee within `slot` of which the validator is a member.
pub attestation_committee_index: Option<CommitteeIndex>,
/// The slot in which a validator must propose a block, or `null` if block production is not required.
pub block_proposal_slot: Option<Slot>,
}
@ -36,7 +36,7 @@ impl ValidatorDuty {
ValidatorDuty {
validator_pubkey: "".to_string(),
attestation_slot: None,
attestation_shard: None,
attestation_committee_index: None,
block_proposal_slot: None,
}
}
@ -90,7 +90,7 @@ pub fn get_validator_duties<T: BeaconChainTypes + 'static>(req: Request<Body>) -
.slot_iter(T::EthSpec::slots_per_epoch())
.map(|slot| {
head_state
.get_beacon_proposer_index(slot, relative_epoch, &beacon_chain.spec)
.get_beacon_proposer_index(slot, &beacon_chain.spec)
.map_err(|e| {
ApiError::ServerError(format!(
"Unable to get proposer index for validator: {:?}",
@ -125,7 +125,7 @@ pub fn get_validator_duties<T: BeaconChainTypes + 'static>(req: Request<Body>) -
match head_state.get_attestation_duties(val_index, relative_epoch) {
Ok(Some(d)) => {
duty.attestation_slot = Some(d.slot);
duty.attestation_shard = Some(d.shard);
duty.attestation_committee_index = Some(d.index);
}
Ok(None) => {}
Err(e) => {
@ -311,7 +311,7 @@ pub fn get_new_attestation<T: BeaconChainTypes + 'static>(req: Request<Body>) ->
let mut aggregation_bits = BitList::with_capacity(val_duty.committee_len)
.expect("An empty BitList should always be created, or we have bigger problems.");
aggregation_bits
.set(val_duty.committee_index, poc_bit)
.set(val_duty.committee_position, poc_bit)
.map_err(|e| {
ApiError::ServerError(format!(
"Unable to set aggregation bits for the attestation: {:?}",
@ -334,21 +334,19 @@ pub fn get_new_attestation<T: BeaconChainTypes + 'static>(req: Request<Body>) ->
return Err(ApiError::BadRequest(format!("Attestation data can only be requested for the current slot ({:?}), not your requested slot ({:?})", current_slot, requested_slot)));
}
let shard = query
.first_of(&["shard"])
let index = query
.first_of(&["index"])
.map(|(_key, value)| value)?
.parse::<u64>()
.map_err(|e| ApiError::BadRequest(format!("Shard is not a valid u64 value: {:?}", e)))?;
.map_err(|e| ApiError::BadRequest(format!("Index is not a valid u64 value: {:?}", e)))?;
let attestation_data = beacon_chain
.produce_attestation_data(shard, current_slot.into())
.produce_attestation_data(current_slot.into(), index)
.map_err(|e| ApiError::ServerError(format!("Could not produce an attestation: {:?}", e)))?;
let attestation: Attestation<T::EthSpec> = Attestation {
aggregation_bits,
data: attestation_data,
custody_bits: BitList::with_capacity(val_duty.committee_len)
.expect("Should be able to create an empty BitList for the custody bits."),
signature: AggregateSignature::new(),
};

View File

@ -48,11 +48,12 @@ impl<T: BeaconChainTypes> AttestationService for AttestationServiceInstance<T> {
);
// Then get the AttestationData from the beacon chain
// NOTE(v0.9): shard is incorrectly named, all this should be deleted
let shard = req.get_shard();
let slot_requested = req.get_slot();
let attestation_data = match self
.chain
.produce_attestation_data(shard, Slot::from(slot_requested))
.produce_attestation_data(Slot::from(slot_requested), shard)
{
Ok(v) => v,
Err(e) => {
@ -115,7 +116,7 @@ impl<T: BeaconChainTypes> AttestationService for AttestationServiceInstance<T> {
self.log,
"Valid attestation from RPC";
"target_epoch" => attestation.data.target.epoch,
"shard" => attestation.data.crosslink.shard,
"index" => attestation.data.index,
);
// valid attestation, propagate to the network

View File

@ -66,9 +66,7 @@ impl<T: BeaconChainTypes> ValidatorService for ValidatorServiceInstance<T> {
let validator_proposers: Result<Vec<usize>, _> = epoch
.slot_iter(T::EthSpec::slots_per_epoch())
.map(|slot| {
state.get_beacon_proposer_index(slot, RelativeEpoch::Current, &self.chain.spec)
})
.map(|slot| state.get_beacon_proposer_index(slot, &self.chain.spec))
.collect();
let validator_proposers = match validator_proposers {
Ok(v) => v,
@ -170,9 +168,9 @@ impl<T: BeaconChainTypes> ValidatorService for ValidatorServiceInstance<T> {
duty.set_none(false)
}
duty.set_committee_index(attestation_duties.committee_index as u64);
duty.set_committee_index(attestation_duties.committee_position as u64);
duty.set_attestation_slot(attestation_duties.slot.as_u64());
duty.set_attestation_shard(attestation_duties.shard);
duty.set_attestation_shard(attestation_duties.index);
duty.set_committee_len(attestation_duties.committee_len as u64);
active_validator.set_duty(duty);

View File

@ -35,7 +35,7 @@ $ lighthouse bn testnet -f recent 8
In a new terminal window, start the validator client with:
```bash
$ lighthouse bn testnet -b insecure 0 8
$ lighthouse vc testnet -b insecure 0 8
```
> Notes:

View File

@ -8,6 +8,7 @@ edition = "2018"
parking_lot = "0.9.0"
store = { path = "../../beacon_node/store" }
types = { path = "../types" }
itertools = "0.8.1"
[dev-dependencies]
criterion = "0.3.0"

View File

@ -49,7 +49,7 @@ pub trait LmdGhost<S: Store, E: EthSpec>: Send + Sync {
/// Runs an integrity verification function on fork choice algorithm.
///
/// Returns `Ok(())` if the underlying fork choice has maintained it's integrity,
/// Returns `Ok(())` if the underlying fork choice has maintained its integrity,
/// `Err(description)` otherwise.
fn verify_integrity(&self) -> Result<()>;
}

View File

@ -4,6 +4,7 @@
//!
//! This implementation is incomplete and has known bugs. Do not use in production.
use super::{LmdGhost, Result as SuperResult};
use itertools::Itertools;
use parking_lot::RwLock;
use std::collections::HashMap;
use std::fmt;
@ -20,6 +21,7 @@ pub enum Error {
MissingBlock(Hash256),
MissingState(Hash256),
MissingChild(Hash256),
MissingSuccessor(Hash256, Hash256),
NotInTree(Hash256),
NoCommonAncestor((Hash256, Hash256)),
StoreError(StoreError),
@ -177,8 +179,8 @@ where
if current_hash != subtree_hash {
let children = self.get_node(current_hash)?.children.clone();
for child_hash in children {
self.retain_subtree(child_hash, subtree_hash)?;
for child in children {
self.retain_subtree(child.hash, subtree_hash)?;
}
self.nodes.remove(&current_hash);
@ -239,7 +241,7 @@ where
let _root_weight = self.update_weight(start_block_root, weight_fn)?;
let start_node = self.get_node(start_block_root)?;
let head_node = self.find_head_from(start_node)?;
let head_node = self.find_head_from(start_node, start_block_slot)?;
Ok(head_node.block_hash)
}
@ -251,31 +253,32 @@ where
}
}
fn find_head_from<'a>(&'a self, start_node: &'a Node) -> Result<&'a Node> {
if start_node.does_not_have_children() {
Ok(start_node)
} else {
// Corresponds to the loop in `get_head` in the spec.
fn find_head_from<'a>(
&'a self,
start_node: &'a Node,
justified_slot: Slot,
) -> Result<&'a Node> {
let children = start_node
.children
.iter()
.map(|hash| self.get_node(*hash))
// This check is primarily for the first iteration, where we must ensure that
// we only consider votes that were made after the last justified checkpoint.
.filter(|c| c.successor_slot > justified_slot)
.map(|c| self.get_node(c.hash))
.collect::<Result<Vec<&Node>>>()?;
// TODO: check if `max_by` is `O(n^2)`.
if children.is_empty() {
Ok(start_node)
} else {
let best_child = children
.iter()
.max_by(|a, b| {
if a.weight != b.weight {
a.weight.cmp(&b.weight)
} else {
a.block_hash.cmp(&b.block_hash)
}
})
.max_by_key(|child| (child.weight, child.block_hash))
// There can only be no maximum if there are no children. This code path is guarded
// against that condition.
.expect("There must be a maximally weighted node.");
self.find_head_from(best_child)
self.find_head_from(best_child, justified_slot)
}
}
@ -288,8 +291,8 @@ where
let mut weight = 0;
for &child in &node.children {
weight += self.update_weight(child, weight_fn)?;
for child in &node.children {
weight += self.update_weight(child.hash, weight_fn)?;
}
for &voter in &node.voters {
@ -323,13 +326,13 @@ where
//
// Load the child of the node and set it's parent to be the parent of this
// node (viz., graft the node's child to the node's parent)
let child = self.get_mut_node(node.children[0])?;
let child = self.get_mut_node(node.children[0].hash)?;
child.parent_hash = node.parent_hash;
// Graft the parent of this node to it's child.
if let Some(parent_hash) = node.parent_hash {
let parent = self.get_mut_node(parent_hash)?;
parent.replace_child(node.block_hash, node.children[0])?;
parent.replace_child_hash(node.block_hash, node.children[0].hash)?;
}
self.nodes.remove(&vote.hash);
@ -376,15 +379,16 @@ where
let node = node.clone();
if let Some(parent_hash) = node.parent_hash {
if (node.children.len() == 1) && !node.has_votes() {
let child_hash = node.children[0];
if node.children.len() == 1 && !node.has_votes() {
let child = &node.children[0];
// Graft the single descendant `node` to the `parent` of node.
self.get_mut_node(child_hash)?.parent_hash = Some(parent_hash);
self.get_mut_node(child.hash)?.parent_hash = Some(parent_hash);
// Detach `node` from `parent`, replacing it with `child`.
// Preserve the parent's direct descendant slot.
self.get_mut_node(parent_hash)?
.replace_child(hash, child_hash)?;
.replace_child_hash(hash, child.hash)?;
true
} else {
@ -442,6 +446,40 @@ where
Ok(())
}
/// Find the direct successor block of `ancestor` if `descendant` is a descendant.
fn find_ancestor_successor_opt(
&self,
ancestor: Hash256,
descendant: Hash256,
) -> Result<Option<Hash256>> {
Ok(std::iter::once(descendant)
.chain(
self.iter_ancestors(descendant)?
.take_while(|(_, slot)| *slot >= self.root_slot())
.map(|(block_hash, _)| block_hash),
)
.tuple_windows()
.find_map(|(successor, block_hash)| {
if block_hash == ancestor {
Some(successor)
} else {
None
}
}))
}
/// Same as `find_ancestor_successor_opt` but will return an error instead of an option.
fn find_ancestor_successor(&self, ancestor: Hash256, descendant: Hash256) -> Result<Hash256> {
self.find_ancestor_successor_opt(ancestor, descendant)?
.ok_or_else(|| Error::MissingSuccessor(ancestor, descendant))
}
/// Look up the successor of the given `ancestor`, returning the slot of that block.
fn find_ancestor_successor_slot(&self, ancestor: Hash256, descendant: Hash256) -> Result<Slot> {
let successor_hash = self.find_ancestor_successor(ancestor, descendant)?;
Ok(self.get_block(successor_hash)?.slot)
}
/// Add `node` to the reduced tree, returning an error if `node` is not rooted in the tree.
fn add_node(&mut self, mut node: Node) -> Result<()> {
// Find the highest (by slot) ancestor of the given node in the reduced tree.
@ -460,7 +498,9 @@ where
// `node` to it.
// 3. Graft `node` to an existing node.
if !prev_in_tree.children.is_empty() {
for &child_hash in &prev_in_tree.children {
for child_link in &prev_in_tree.children {
let child_hash = child_link.hash;
// 1. Graft the new node between two existing nodes.
//
// If `node` is a descendant of `prev_in_tree` but an ancestor of a child connected to
@ -468,19 +508,20 @@ where
//
// This means that `node` can be grafted between `prev_in_tree` and the child that is a
// descendant of both `node` and `prev_in_tree`.
if self
.iter_ancestors(child_hash)?
.take_while(|(_, slot)| *slot >= self.root_slot())
.any(|(ancestor, _slot)| ancestor == node.block_hash)
if let Some(successor) =
self.find_ancestor_successor_opt(node.block_hash, child_hash)?
{
let child = self.get_mut_node(child_hash)?;
// Graft `child` to `node`.
child.parent_hash = Some(node.block_hash);
// Graft `node` to `child`.
node.children.push(child_hash);
node.children.push(ChildLink {
hash: child_hash,
successor_slot: self.get_block(successor)?.slot,
});
// Detach `child` from `prev_in_tree`, replacing it with `node`.
prev_in_tree.replace_child(child_hash, node.block_hash)?;
prev_in_tree.replace_child_hash(child_hash, node.block_hash)?;
// Graft `node` to `prev_in_tree`.
node.parent_hash = Some(prev_in_tree.block_hash);
@ -495,7 +536,8 @@ where
// any of the children of `prev_in_tree`, we know that `node` is on a different fork to
// all of the children of `prev_in_tree`.
if node.parent_hash.is_none() {
for &child_hash in &prev_in_tree.children {
for child_link in &prev_in_tree.children {
let child_hash = child_link.hash;
// Find the highest (by slot) common ancestor between `node` and `child`.
//
// The common ancestor is the last block before `node` and `child` forked.
@ -506,24 +548,37 @@ where
// must add this new block into the tree (because it is a decision node
// between two forks).
if ancestor_hash != prev_in_tree.block_hash {
let child = self.get_mut_node(child_hash)?;
// Create a new `common_ancestor` node which represents the `ancestor_hash`
// block, has `prev_in_tree` as the parent and has both `node` and `child`
// as children.
let common_ancestor = Node {
block_hash: ancestor_hash,
parent_hash: Some(prev_in_tree.block_hash),
children: vec![node.block_hash, child_hash],
children: vec![
ChildLink {
hash: node.block_hash,
successor_slot: self.find_ancestor_successor_slot(
ancestor_hash,
node.block_hash,
)?,
},
ChildLink {
hash: child_hash,
successor_slot: self
.find_ancestor_successor_slot(ancestor_hash, child_hash)?,
},
],
..Node::default()
};
let child = self.get_mut_node(child_hash)?;
// Graft `child` and `node` to `common_ancestor`.
child.parent_hash = Some(common_ancestor.block_hash);
node.parent_hash = Some(common_ancestor.block_hash);
// Detach `child` from `prev_in_tree`, replacing it with `common_ancestor`.
prev_in_tree.replace_child(child_hash, common_ancestor.block_hash)?;
prev_in_tree.replace_child_hash(child_hash, common_ancestor.block_hash)?;
// Store the new `common_ancestor` node.
self.nodes
@ -540,7 +595,11 @@ where
//
// Graft `node` to `prev_in_tree` and `prev_in_tree` to `node`
node.parent_hash = Some(prev_in_tree.block_hash);
prev_in_tree.children.push(node.block_hash);
prev_in_tree.children.push(ChildLink {
hash: node.block_hash,
successor_slot: self
.find_ancestor_successor_slot(prev_in_tree.block_hash, node.block_hash)?,
});
}
// Update `prev_in_tree`. A mutable reference was not maintained to satisfy the borrow
@ -655,7 +714,17 @@ where
node.children
.iter()
.map(|child| verify_node_exists(*child, "child_must_exist".to_string()))
.map(|child| {
verify_node_exists(child.hash, "child_must_exist".to_string())?;
if self.find_ancestor_successor_slot(node.block_hash, child.hash)?
== child.successor_slot
{
Ok(())
} else {
Err("successor slot on child link is incorrect".to_string())
}
})
.collect::<std::result::Result<(), String>>()?;
verify_node_exists(node.block_hash, "block hash must exist".to_string())?;
@ -698,25 +767,35 @@ where
#[derive(Default, Clone, Debug)]
pub struct Node {
/// Hash of the parent node in the reduced tree (not necessarily parent block).
pub parent_hash: Option<Hash256>,
pub children: Vec<Hash256>,
pub children: Vec<ChildLink>,
pub weight: u64,
pub block_hash: Hash256,
pub voters: Vec<usize>,
}
impl Node {
pub fn does_not_have_children(&self) -> bool {
self.children.is_empty()
#[derive(Default, Clone, Debug)]
pub struct ChildLink {
/// Hash of the child block (may not be a direct descendant).
pub hash: Hash256,
/// Slot of the block which is a direct descendant on the chain leading to `hash`.
///
/// Node <--- Successor <--- ... <--- Child
pub successor_slot: Slot,
}
pub fn replace_child(&mut self, old: Hash256, new: Hash256) -> Result<()> {
impl Node {
/// Replace a child with a new child, whilst preserving the successor slot.
///
/// The new child should have the same ancestor successor block as the old one.
pub fn replace_child_hash(&mut self, old: Hash256, new: Hash256) -> Result<()> {
let i = self
.children
.iter()
.position(|&c| c == old)
.position(|c| c.hash == old)
.ok_or_else(|| Error::MissingChild(old))?;
self.children[i] = new;
self.children[i].hash = new;
Ok(())
}
@ -725,7 +804,7 @@ impl Node {
let i = self
.children
.iter()
.position(|&c| c == child)
.position(|c| c.hash == child)
.ok_or_else(|| Error::MissingChild(child))?;
self.children.remove(i);

View File

@ -45,7 +45,9 @@ struct ForkedHarness {
pub genesis_block: BeaconBlock<TestEthSpec>,
pub honest_head: RootAndSlot,
pub faulty_head: RootAndSlot,
/// Honest roots in reverse order (slot high to low)
pub honest_roots: Vec<RootAndSlot>,
/// Faulty roots in reverse order (slot high to low)
pub faulty_roots: Vec<RootAndSlot>,
}
@ -225,7 +227,7 @@ fn single_voter_persistent_instance_reverse_order() {
"New tree should have integrity"
);
for (root, slot) in harness.honest_roots.iter().rev() {
for (root, slot) in &harness.honest_roots {
lmd.process_attestation(0, *root, *slot)
.expect("fork choice should accept attestations to honest roots in reverse");
@ -237,11 +239,15 @@ fn single_voter_persistent_instance_reverse_order() {
}
// The honest head should be selected.
let (head_root, head_slot) = harness.honest_roots.first().unwrap();
let (finalized_root, _) = harness.honest_roots.last().unwrap();
let (head_root, _) = harness.honest_roots.first().unwrap();
let (finalized_root, finalized_slot) = harness.honest_roots.last().unwrap();
assert_eq!(
lmd.find_head(*head_slot, *finalized_root, ForkedHarness::weight_function),
lmd.find_head(
*finalized_slot,
*finalized_root,
ForkedHarness::weight_function
),
Ok(*head_root),
"Honest head should be selected"
);
@ -253,7 +259,7 @@ fn single_voter_persistent_instance_reverse_order() {
fn single_voter_many_instance_honest_blocks_voting_forwards() {
let harness = &FORKED_HARNESS;
for (root, slot) in &harness.honest_roots {
for (root, slot) in harness.honest_roots.iter().rev() {
let lmd = harness.new_fork_choice();
lmd.process_attestation(0, *root, *slot)
.expect("fork choice should accept attestations to honest roots");
@ -272,7 +278,7 @@ fn single_voter_many_instance_honest_blocks_voting_in_reverse() {
let harness = &FORKED_HARNESS;
// Same as above, but in reverse order (votes on the highest honest block first).
for (root, slot) in harness.honest_roots.iter().rev() {
for (root, slot) in &harness.honest_roots {
let lmd = harness.new_fork_choice();
lmd.process_attestation(0, *root, *slot)
.expect("fork choice should accept attestations to honest roots in reverse");
@ -291,7 +297,7 @@ fn single_voter_many_instance_honest_blocks_voting_in_reverse() {
fn single_voter_many_instance_faulty_blocks_voting_forwards() {
let harness = &FORKED_HARNESS;
for (root, slot) in &harness.faulty_roots {
for (root, slot) in harness.faulty_roots.iter().rev() {
let lmd = harness.new_fork_choice();
lmd.process_attestation(0, *root, *slot)
.expect("fork choice should accept attestations to faulty roots");
@ -309,7 +315,7 @@ fn single_voter_many_instance_faulty_blocks_voting_forwards() {
fn single_voter_many_instance_faulty_blocks_voting_in_reverse() {
let harness = &FORKED_HARNESS;
for (root, slot) in harness.faulty_roots.iter().rev() {
for (root, slot) in &harness.faulty_roots {
let lmd = harness.new_fork_choice();
lmd.process_attestation(0, *root, *slot)
.expect("fork choice should accept attestations to faulty roots in reverse");
@ -322,6 +328,44 @@ fn single_voter_many_instance_faulty_blocks_voting_in_reverse() {
}
}
/// Ensure that votes with slots before the justified slot are not counted.
#[test]
fn discard_votes_before_justified_slot() {
let harness = &FORKED_HARNESS;
let lmd = harness.new_fork_choice();
let (genesis_root, genesis_slot) = *harness.honest_roots.last().unwrap();
// Add attestations from all validators for all honest blocks.
for (root, slot) in harness.honest_roots.iter().rev() {
for i in 0..VALIDATOR_COUNT {
lmd.process_attestation(i, *root, *slot)
.expect("should accept attestations in increasing order");
}
// Head starting from 0 checkpoint (genesis) should be current root
assert_eq!(
lmd.find_head(genesis_slot, genesis_root, ForkedHarness::weight_function),
Ok(*root),
"Honest head should be selected"
);
// Head from one slot after genesis should still be genesis, because the successor
// block of the genesis block has slot `genesis_slot + 1` which isn't greater than
// the slot we're starting from. This is a very artifical test, but one that's easy to
// describe.
assert_eq!(
lmd.find_head(
genesis_slot + 1,
genesis_root,
ForkedHarness::weight_function
),
Ok(genesis_root)
);
}
}
/// Ensures that the finalized root can be set to all values in `roots`.
fn test_update_finalized_root(roots: &[(Hash256, Slot)]) {
let harness = &FORKED_HARNESS;

View File

@ -6,7 +6,6 @@ edition = "2018"
[dependencies]
int_to_bytes = { path = "../utils/int_to_bytes" }
itertools = "0.8.1"
parking_lot = "0.9.0"
types = { path = "../types" }
state_processing = { path = "../state_processing" }

View File

@ -35,16 +35,14 @@ impl<'a, T: EthSpec> MaxCover for AttMaxCover<'a, T> {
/// Sneaky: we keep all the attestations together in one bucket, even though
/// their aggregation bitfields refer to different committees. In order to avoid
/// confusing committees when updating covering sets, we update only those attestations
/// whose shard and epoch match the attestation being included in the solution, by the logic
/// that a shard and epoch uniquely identify a committee.
/// whose slot and index match the attestation being included in the solution, by the logic
/// that a slot and index uniquely identify a committee.
fn update_covering_set(
&mut self,
best_att: &Attestation<T>,
covered_validators: &BitList<T::MaxValidatorsPerCommittee>,
) {
if self.att.data.crosslink.shard == best_att.data.crosslink.shard
&& self.att.data.target.epoch == best_att.data.target.epoch
{
if self.att.data.slot == best_att.data.slot && self.att.data.index == best_att.data.index {
self.fresh_validators.difference_inplace(covered_validators);
}
}
@ -80,11 +78,12 @@ pub fn earliest_attestation_validators<T: EthSpec>(
state_attestations
.iter()
// In a single epoch, an attester should only be attesting for one shard.
// In a single epoch, an attester should only be attesting for one slot and index.
// TODO: we avoid including slashable attestations in the state here,
// but maybe we should do something else with them (like construct slashings).
.filter(|existing_attestation| {
existing_attestation.data.crosslink.shard == attestation.data.crosslink.shard
existing_attestation.data.slot == attestation.data.slot
&& existing_attestation.data.index == attestation.data.index
})
.for_each(|existing_attestation| {
// Remove the validators who have signed the existing attestation (they are not new)

View File

@ -29,7 +29,7 @@ impl AttestationId {
state: &BeaconState<T>,
spec: &ChainSpec,
) -> Vec<u8> {
int_to_bytes8(spec.get_domain(epoch, Domain::Attestation, &state.fork))
int_to_bytes8(spec.get_domain(epoch, Domain::BeaconAttester, &state.fork))
}
pub fn domain_bytes_match(&self, domain_bytes: &[u8]) -> bool {

View File

@ -7,24 +7,22 @@ pub use persistence::PersistedOperationPool;
use attestation::{earliest_attestation_validators, AttMaxCover};
use attestation_id::AttestationId;
use itertools::Itertools;
use max_cover::maximum_cover;
use parking_lot::RwLock;
use state_processing::per_block_processing::errors::{
AttestationValidationError, AttesterSlashingValidationError, DepositValidationError,
ExitValidationError, ProposerSlashingValidationError, TransferValidationError,
ExitValidationError, ProposerSlashingValidationError,
};
use state_processing::per_block_processing::{
get_slashable_indices_modular, verify_attestation_for_block_inclusion,
verify_attester_slashing, verify_exit, verify_exit_time_independent_only,
verify_proposer_slashing, verify_transfer, verify_transfer_time_independent_only,
VerifySignatures,
verify_proposer_slashing, VerifySignatures,
};
use std::collections::{btree_map::Entry, hash_map, BTreeMap, HashMap, HashSet};
use std::marker::PhantomData;
use types::{
typenum::Unsigned, Attestation, AttesterSlashing, BeaconState, ChainSpec, Deposit, EthSpec,
ProposerSlashing, Transfer, Validator, VoluntaryExit,
ProposerSlashing, Validator, VoluntaryExit,
};
#[derive(Default, Debug)]
@ -43,8 +41,6 @@ pub struct OperationPool<T: EthSpec + Default> {
proposer_slashings: RwLock<HashMap<u64, ProposerSlashing>>,
/// Map from exiting validator to their exit data.
voluntary_exits: RwLock<HashMap<u64, VoluntaryExit>>,
/// Set of transfers.
transfers: RwLock<HashSet<Transfer>>,
_phantom: PhantomData<T>,
}
@ -375,44 +371,6 @@ impl<T: EthSpec> OperationPool<T> {
);
}
/// Insert a transfer into the pool, checking it for validity in the process.
pub fn insert_transfer(
&self,
transfer: Transfer,
state: &BeaconState<T>,
spec: &ChainSpec,
) -> Result<(), TransferValidationError> {
// The signature of the transfer isn't hashed, but because we check
// it before we insert into the HashSet, we can't end up with duplicate
// transactions.
verify_transfer_time_independent_only(state, &transfer, VerifySignatures::True, spec)?;
self.transfers.write().insert(transfer);
Ok(())
}
/// Get a list of transfers for inclusion in a block.
// TODO: improve the economic optimality of this function by accounting for
// dependencies between transfers in the same block e.g. A pays B, B pays C
pub fn get_transfers(&self, state: &BeaconState<T>, spec: &ChainSpec) -> Vec<Transfer> {
self.transfers
.read()
.iter()
.filter(|transfer| {
verify_transfer(state, transfer, VerifySignatures::False, spec).is_ok()
})
.sorted_by_key(|transfer| std::cmp::Reverse(transfer.fee))
.take(T::MaxTransfers::to_usize())
.cloned()
.collect()
}
/// Prune the set of transfers by removing all those whose slot has already passed.
pub fn prune_transfers(&self, finalized_state: &BeaconState<T>) {
self.transfers
.write()
.retain(|transfer| transfer.slot > finalized_state.slot)
}
/// Prune all types of transactions given the latest finalized state.
pub fn prune_all(&self, finalized_state: &BeaconState<T>, spec: &ChainSpec) {
self.prune_attestations(finalized_state);
@ -420,7 +378,6 @@ impl<T: EthSpec> OperationPool<T> {
self.prune_proposer_slashings(finalized_state);
self.prune_attester_slashings(finalized_state, spec);
self.prune_voluntary_exits(finalized_state);
self.prune_transfers(finalized_state);
}
}
@ -467,7 +424,6 @@ impl<T: EthSpec + Default> PartialEq for OperationPool<T> {
&& *self.attester_slashings.read() == *other.attester_slashings.read()
&& *self.proposer_slashings.read() == *other.proposer_slashings.read()
&& *self.voluntary_exits.read() == *other.voluntary_exits.read()
&& *self.transfers.read() == *other.transfers.read()
}
}
@ -611,7 +567,7 @@ mod tests {
/// Signed by all validators in `committee[signing_range]` and `committee[extra_signer]`.
fn signed_attestation<R: std::slice::SliceIndex<[usize], Output = [usize]>, E: EthSpec>(
committee: &[usize],
shard: u64,
index: u64,
keypairs: &[Keypair],
signing_range: R,
slot: Slot,
@ -620,32 +576,30 @@ mod tests {
extra_signer: Option<usize>,
) -> Attestation<E> {
let mut builder = TestingAttestationBuilder::new(
&AttestationTestTask::Valid,
AttestationTestTask::Valid,
state,
committee,
slot,
shard,
index,
spec,
);
let signers = &committee[signing_range];
let committee_keys = signers.iter().map(|&i| &keypairs[i].sk).collect::<Vec<_>>();
builder.sign(
&AttestationTestTask::Valid,
AttestationTestTask::Valid,
signers,
&committee_keys,
&state.fork,
spec,
false,
);
extra_signer.map(|c_idx| {
let validator_index = committee[c_idx];
builder.sign(
&AttestationTestTask::Valid,
AttestationTestTask::Valid,
&[validator_index],
&[&keypairs[validator_index].sk],
&state.fork,
spec,
false,
)
});
builder.build()
@ -677,16 +631,16 @@ mod tests {
attestation_test_state::<MainnetEthSpec>(1);
let slot = state.slot - 1;
let committees = state
.get_crosslink_committees_at_slot(slot)
.get_beacon_committees_at_slot(slot)
.unwrap()
.into_iter()
.map(CrosslinkCommittee::into_owned)
.map(BeaconCommittee::into_owned)
.collect::<Vec<_>>();
for cc in committees {
for bc in committees {
let att1 = signed_attestation(
&cc.committee,
cc.shard,
&bc.committee,
bc.index,
keypairs,
..2,
slot,
@ -695,8 +649,8 @@ mod tests {
None,
);
let att2 = signed_attestation(
&cc.committee,
cc.shard,
&bc.committee,
bc.index,
keypairs,
..,
slot,
@ -720,7 +674,7 @@ mod tests {
.unwrap();
assert_eq!(
cc.committee.len() - 2,
bc.committee.len() - 2,
earliest_attestation_validators(&att2, state).num_set_bits()
);
}
@ -736,10 +690,10 @@ mod tests {
let slot = state.slot - 1;
let committees = state
.get_crosslink_committees_at_slot(slot)
.get_beacon_committees_at_slot(slot)
.unwrap()
.into_iter()
.map(CrosslinkCommittee::into_owned)
.map(BeaconCommittee::into_owned)
.collect::<Vec<_>>();
assert_eq!(
@ -748,12 +702,12 @@ mod tests {
"we expect just one committee with this many validators"
);
for cc in &committees {
for bc in &committees {
let step_size = 2;
for i in (0..cc.committee.len()).step_by(step_size) {
for i in (0..bc.committee.len()).step_by(step_size) {
let att = signed_attestation(
&cc.committee,
cc.shard,
&bc.committee,
bc.index,
keypairs,
i..i + step_size,
slot,
@ -805,16 +759,16 @@ mod tests {
let slot = state.slot - 1;
let committees = state
.get_crosslink_committees_at_slot(slot)
.get_beacon_committees_at_slot(slot)
.unwrap()
.into_iter()
.map(CrosslinkCommittee::into_owned)
.map(BeaconCommittee::into_owned)
.collect::<Vec<_>>();
for cc in &committees {
for bc in &committees {
let att = signed_attestation(
&cc.committee,
cc.shard,
&bc.committee,
bc.index,
keypairs,
..,
slot,
@ -842,20 +796,20 @@ mod tests {
let slot = state.slot - 1;
let committees = state
.get_crosslink_committees_at_slot(slot)
.get_beacon_committees_at_slot(slot)
.unwrap()
.into_iter()
.map(CrosslinkCommittee::into_owned)
.map(BeaconCommittee::into_owned)
.collect::<Vec<_>>();
let step_size = 2;
for cc in &committees {
for bc in &committees {
// Create attestations that overlap on `step_size` validators, like:
// {0,1,2,3}, {2,3,4,5}, {4,5,6,7}, ...
for i in (0..cc.committee.len() - step_size).step_by(step_size) {
for i in (0..bc.committee.len() - step_size).step_by(step_size) {
let att = signed_attestation(
&cc.committee,
cc.shard,
&bc.committee,
bc.index,
keypairs,
i..i + 2 * step_size,
slot,
@ -890,20 +844,20 @@ mod tests {
let slot = state.slot - 1;
let committees = state
.get_crosslink_committees_at_slot(slot)
.get_beacon_committees_at_slot(slot)
.unwrap()
.into_iter()
.map(CrosslinkCommittee::into_owned)
.map(BeaconCommittee::into_owned)
.collect::<Vec<_>>();
let max_attestations = <MainnetEthSpec as EthSpec>::MaxAttestations::to_usize();
let target_committee_size = spec.target_committee_size as usize;
let insert_attestations = |cc: &OwnedCrosslinkCommittee, step_size| {
let insert_attestations = |bc: &OwnedBeaconCommittee, step_size| {
for i in (0..target_committee_size).step_by(step_size) {
let att = signed_attestation(
&cc.committee,
cc.shard,
&bc.committee,
bc.index,
keypairs,
i..i + step_size,
slot,

View File

@ -21,8 +21,6 @@ pub struct PersistedOperationPool<T: EthSpec> {
proposer_slashings: Vec<ProposerSlashing>,
/// Voluntary exits.
voluntary_exits: Vec<VoluntaryExit>,
/// Transfers.
transfers: Vec<Transfer>,
}
impl<T: EthSpec> PersistedOperationPool<T> {
@ -63,15 +61,12 @@ impl<T: EthSpec> PersistedOperationPool<T> {
.map(|(_, exit)| exit.clone())
.collect();
let transfers = operation_pool.transfers.read().iter().cloned().collect();
Self {
attestations,
deposits,
attester_slashings,
proposer_slashings,
voluntary_exits,
transfers,
}
}
@ -102,7 +97,6 @@ impl<T: EthSpec> PersistedOperationPool<T> {
.map(|exit| (exit.validator_index, exit))
.collect(),
);
let transfers = RwLock::new(self.transfers.into_iter().collect());
OperationPool {
attestations,
@ -110,7 +104,6 @@ impl<T: EthSpec> PersistedOperationPool<T> {
attester_slashings,
proposer_slashings,
voluntary_exits,
transfers,
_phantom: Default::default(),
}
}

View File

@ -3,19 +3,13 @@ use types::*;
/// Returns validator indices which participated in the attestation, sorted by increasing index.
///
/// Spec v0.8.1
/// Spec v0.9.1
pub fn get_attesting_indices<T: EthSpec>(
state: &BeaconState<T>,
attestation_data: &AttestationData,
bitlist: &BitList<T::MaxValidatorsPerCommittee>,
) -> Result<BTreeSet<usize>, BeaconStateError> {
let target_relative_epoch =
RelativeEpoch::from_epoch(state.current_epoch(), attestation_data.target.epoch)?;
let committee = state.get_crosslink_committee_for_shard(
attestation_data.crosslink.shard,
target_relative_epoch,
)?;
let committee = state.get_beacon_committee(attestation_data.slot, attestation_data.index)?;
if bitlist.len() != committee.committee.len() {
return Err(BeaconStateError::InvalidBitfield);

View File

@ -1,41 +0,0 @@
use tree_hash::TreeHash;
use types::*;
/// Return the compact committee root at `relative_epoch`.
///
/// Spec v0.8.3
pub fn get_compact_committees_root<T: EthSpec>(
state: &BeaconState<T>,
relative_epoch: RelativeEpoch,
spec: &ChainSpec,
) -> Result<Hash256, BeaconStateError> {
let mut committees =
FixedVector::<_, T::ShardCount>::from_elem(CompactCommittee::<T>::default());
let start_shard = state.get_epoch_start_shard(relative_epoch)?;
for committee_number in 0..state.get_committee_count(relative_epoch)? {
let shard = (start_shard + committee_number) % T::ShardCount::to_u64();
for &index in state
.get_crosslink_committee_for_shard(shard, relative_epoch)?
.committee
{
let validator = state
.validators
.get(index)
.ok_or(BeaconStateError::UnknownValidator)?;
committees[shard as usize]
.pubkeys
.push(validator.pubkey.clone())?;
let compact_balance = validator.effective_balance / spec.effective_balance_increment;
// `index` (top 6 bytes) + `slashed` (16th bit) + `compact_balance` (bottom 15 bits)
let compact_validator: u64 =
((index as u64) << 16) + (u64::from(validator.slashed) << 15) + compact_balance;
committees[shard as usize]
.compact_validators
.push(compact_validator)?;
}
}
Ok(Hash256::from_slice(&committees.tree_hash_root()))
}

View File

@ -6,139 +6,19 @@ type Result<T> = std::result::Result<T, BlockOperationError<Invalid>>;
/// Convert `attestation` to (almost) indexed-verifiable form.
///
/// Spec v0.8.0
/// Spec v0.9.1
pub fn get_indexed_attestation<T: EthSpec>(
state: &BeaconState<T>,
attestation: &Attestation<T>,
) -> Result<IndexedAttestation<T>> {
// Note: we rely on both calls to `get_attesting_indices` to check the bitfield lengths
// against the committee length
let attesting_indices =
get_attesting_indices(state, &attestation.data, &attestation.aggregation_bits)?;
let custody_bit_1_indices =
get_attesting_indices(state, &attestation.data, &attestation.custody_bits)?;
verify!(
custody_bit_1_indices.is_subset(&attesting_indices),
Invalid::CustodyBitfieldNotSubset
);
let custody_bit_0_indices = &attesting_indices - &custody_bit_1_indices;
Ok(IndexedAttestation {
custody_bit_0_indices: VariableList::new(
custody_bit_0_indices
.into_iter()
.map(|x| x as u64)
.collect(),
)?,
custody_bit_1_indices: VariableList::new(
custody_bit_1_indices
.into_iter()
.map(|x| x as u64)
.collect(),
attesting_indices: VariableList::new(
attesting_indices.into_iter().map(|x| x as u64).collect(),
)?,
data: attestation.data.clone(),
signature: attestation.signature.clone(),
})
}
#[cfg(test)]
mod test {
use super::*;
use itertools::{Either, Itertools};
use types::test_utils::*;
#[test]
fn custody_bitfield_indexing() {
let validator_count = 128;
let spec = MinimalEthSpec::default_spec();
let state_builder =
TestingBeaconStateBuilder::<MinimalEthSpec>::from_default_keypairs_file_if_exists(
validator_count,
&spec,
);
let (mut state, keypairs) = state_builder.build();
state.build_all_caches(&spec).unwrap();
state.slot += 1;
let shard = 0;
let cc = state
.get_crosslink_committee_for_shard(shard, RelativeEpoch::Current)
.unwrap();
// Make a third of the validators sign with custody bit 0, a third with custody bit 1
// and a third not sign at all.
assert!(
cc.committee.len() >= 4,
"need at least 4 validators per committee for this test to work"
);
let (mut bit_0_indices, mut bit_1_indices): (Vec<_>, Vec<_>) = cc
.committee
.iter()
.enumerate()
.filter(|(i, _)| i % 3 != 0)
.partition_map(|(i, index)| {
if i % 3 == 1 {
Either::Left(*index)
} else {
Either::Right(*index)
}
});
assert!(!bit_0_indices.is_empty());
assert!(!bit_1_indices.is_empty());
let bit_0_keys = bit_0_indices
.iter()
.map(|validator_index| &keypairs[*validator_index].sk)
.collect::<Vec<_>>();
let bit_1_keys = bit_1_indices
.iter()
.map(|validator_index| &keypairs[*validator_index].sk)
.collect::<Vec<_>>();
let mut attestation_builder = TestingAttestationBuilder::new(
&AttestationTestTask::Valid,
&state,
&cc.committee,
cc.slot,
shard,
&spec,
);
attestation_builder
.sign(
&AttestationTestTask::Valid,
&bit_0_indices,
&bit_0_keys,
&state.fork,
&spec,
false,
)
.sign(
&AttestationTestTask::Valid,
&bit_1_indices,
&bit_1_keys,
&state.fork,
&spec,
true,
);
let attestation = attestation_builder.build();
let indexed_attestation = get_indexed_attestation(&state, &attestation).unwrap();
bit_0_indices.sort();
bit_1_indices.sort();
assert!(indexed_attestation
.custody_bit_0_indices
.iter()
.copied()
.eq(bit_0_indices.iter().map(|idx| *idx as u64)));
assert!(indexed_attestation
.custody_bit_1_indices
.iter()
.copied()
.eq(bit_1_indices.iter().map(|idx| *idx as u64)));
}
}

View File

@ -3,7 +3,7 @@ use types::{BeaconStateError as Error, *};
/// Initiate the exit of the validator of the given `index`.
///
/// Spec v0.8.1
/// Spec v0.9.1
pub fn initiate_validator_exit<T: EthSpec>(
state: &mut BeaconState<T>,
index: usize,

View File

@ -1,11 +1,9 @@
mod get_attesting_indices;
mod get_compact_committees_root;
mod get_indexed_attestation;
mod initiate_validator_exit;
mod slash_validator;
pub use get_attesting_indices::get_attesting_indices;
pub use get_compact_committees_root::get_compact_committees_root;
pub use get_indexed_attestation::get_indexed_attestation;
pub use initiate_validator_exit::initiate_validator_exit;
pub use slash_validator::slash_validator;

View File

@ -4,7 +4,7 @@ use types::{BeaconStateError as Error, *};
/// Slash the validator with index ``index``.
///
/// Spec v0.8.0
/// Spec v0.9.1
pub fn slash_validator<T: EthSpec>(
state: &mut BeaconState<T>,
slashed_index: usize,
@ -35,8 +35,7 @@ pub fn slash_validator<T: EthSpec>(
);
// Apply proposer and whistleblower rewards
let proposer_index =
state.get_beacon_proposer_index(state.slot, RelativeEpoch::Current, spec)?;
let proposer_index = state.get_beacon_proposer_index(state.slot, spec)?;
let whistleblower_index = opt_whistleblower_index.unwrap_or(proposer_index);
let whistleblower_reward = validator_effective_balance / spec.whistleblower_reward_quotient;
let proposer_reward = whistleblower_reward / spec.proposer_reward_quotient;

View File

@ -1,12 +1,11 @@
use super::per_block_processing::{errors::BlockProcessingError, process_deposit};
use crate::common::get_compact_committees_root;
use tree_hash::TreeHash;
use types::typenum::U4294967296;
use types::*;
/// Initialize a `BeaconState` from genesis data.
///
/// Spec v0.8.0
/// Spec v0.9.1
// TODO: this is quite inefficient and we probably want to rethink how we do this
pub fn initialize_beacon_state_from_eth1<T: EthSpec>(
eth1_block_hash: Hash256,
@ -24,6 +23,9 @@ pub fn initialize_beacon_state_from_eth1<T: EthSpec>(
};
let mut state = BeaconState::new(genesis_time, eth1_data, spec);
// Seed RANDAO with Eth1 entropy
state.fill_randao_mixes_with(eth1_block_hash);
// Process deposits
let leaves: Vec<_> = deposits
.iter()
@ -40,21 +42,12 @@ pub fn initialize_beacon_state_from_eth1<T: EthSpec>(
// Now that we have our validators, initialize the caches (including the committees)
state.build_all_caches(spec)?;
// Populate active_index_roots and compact_committees_roots
let indices_list = VariableList::<usize, T::ValidatorRegistryLimit>::from(
state.get_active_validator_indices(T::genesis_epoch()),
);
let active_index_root = Hash256::from_slice(&indices_list.tree_hash_root());
let committee_root = get_compact_committees_root(&state, RelativeEpoch::Current, spec)?;
state.fill_active_index_roots_with(active_index_root);
state.fill_compact_committees_roots_with(committee_root);
Ok(state)
}
/// Determine whether a candidate genesis state is suitable for starting the chain.
///
/// Spec v0.8.1
/// Spec v0.9.1
pub fn is_valid_genesis_state<T: EthSpec>(state: &BeaconState<T>, spec: &ChainSpec) -> bool {
state.genesis_time >= spec.min_genesis_time
&& state.get_active_validator_indices(T::genesis_epoch()).len() as u64

View File

@ -2,9 +2,7 @@ use crate::common::{initiate_validator_exit, slash_validator};
use errors::{BlockOperationError, BlockProcessingError, HeaderInvalid, IntoWithIndex};
use rayon::prelude::*;
use signature_sets::{block_proposal_signature_set, randao_signature_set};
use std::collections::HashSet;
use std::convert::TryInto;
use std::iter::FromIterator;
use tree_hash::SignedRoot;
use types::*;
@ -21,9 +19,6 @@ pub use verify_deposit::{
get_existing_validator_index, verify_deposit_merkle_proof, verify_deposit_signature,
};
pub use verify_exit::{verify_exit, verify_exit_time_independent_only};
pub use verify_transfer::{
execute_transfer, verify_transfer, verify_transfer_time_independent_only,
};
pub mod block_processing_builder;
mod block_signature_verifier;
@ -36,7 +31,6 @@ mod verify_attester_slashing;
mod verify_deposit;
mod verify_exit;
mod verify_proposer_slashing;
mod verify_transfer;
/// The strategy to be used when validating the block's signatures.
#[derive(PartialEq, Clone, Copy)]
@ -74,7 +68,7 @@ impl VerifySignatures {
/// signature. If it is `None` the signed root is calculated here. This parameter only exists to
/// avoid re-calculating the root when it is already known.
///
/// Spec v0.8.0
/// Spec v0.9.1
pub fn per_block_processing<T: EthSpec>(
mut state: &mut BeaconState<T>,
block: &BeaconBlock<T>,
@ -128,14 +122,13 @@ pub fn per_block_processing<T: EthSpec>(
verify_signatures,
spec,
)?;
process_transfers(&mut state, &block.body.transfers, verify_signatures, spec)?;
Ok(())
}
/// Processes the block header.
///
/// Spec v0.8.0
/// Spec v0.9.1
pub fn process_block_header<T: EthSpec>(
state: &mut BeaconState<T>,
block: &BeaconBlock<T>,
@ -158,7 +151,7 @@ pub fn process_block_header<T: EthSpec>(
state.latest_block_header = block.temporary_block_header();
// Verify proposer is not slashed
let proposer_idx = state.get_beacon_proposer_index(block.slot, RelativeEpoch::Current, spec)?;
let proposer_idx = state.get_beacon_proposer_index(block.slot, spec)?;
let proposer = &state.validators[proposer_idx];
verify!(
!proposer.slashed,
@ -174,7 +167,7 @@ pub fn process_block_header<T: EthSpec>(
/// Verifies the signature of a block.
///
/// Spec v0.8.0
/// Spec v0.9.1
pub fn verify_block_signature<T: EthSpec>(
state: &BeaconState<T>,
block: &BeaconBlock<T>,
@ -192,7 +185,7 @@ pub fn verify_block_signature<T: EthSpec>(
/// Verifies the `randao_reveal` against the block's proposer pubkey and updates
/// `state.latest_randao_mixes`.
///
/// Spec v0.8.0
/// Spec v0.9.1
pub fn process_randao<T: EthSpec>(
state: &mut BeaconState<T>,
block: &BeaconBlock<T>,
@ -215,7 +208,7 @@ pub fn process_randao<T: EthSpec>(
/// Update the `state.eth1_data_votes` based upon the `eth1_data` provided.
///
/// Spec v0.8.0
/// Spec v0.9.1
pub fn process_eth1_data<T: EthSpec>(
state: &mut BeaconState<T>,
eth1_data: &Eth1Data,
@ -240,7 +233,7 @@ pub fn process_eth1_data<T: EthSpec>(
/// Returns `Ok(())` if the validation and state updates completed successfully, otherwise returns
/// an `Err` describing the invalid object or cause of failure.
///
/// Spec v0.8.0
/// Spec v0.9.1
pub fn process_proposer_slashings<T: EthSpec>(
state: &mut BeaconState<T>,
proposer_slashings: &[ProposerSlashing],
@ -269,7 +262,7 @@ pub fn process_proposer_slashings<T: EthSpec>(
/// Returns `Ok(())` if the validation and state updates completed successfully, otherwise returns
/// an `Err` describing the invalid object or cause of failure.
///
/// Spec v0.8.0
/// Spec v0.9.1
pub fn process_attester_slashings<T: EthSpec>(
state: &mut BeaconState<T>,
attester_slashings: &[AttesterSlashing<T>],
@ -323,7 +316,7 @@ pub fn process_attester_slashings<T: EthSpec>(
/// Returns `Ok(())` if the validation and state updates completed successfully, otherwise returns
/// an `Err` describing the invalid object or cause of failure.
///
/// Spec v0.8.0
/// Spec v0.9.1
pub fn process_attestations<T: EthSpec>(
state: &mut BeaconState<T>,
attestations: &[Attestation<T>],
@ -343,14 +336,12 @@ pub fn process_attestations<T: EthSpec>(
})?;
// Update the state in series.
let proposer_index =
state.get_beacon_proposer_index(state.slot, RelativeEpoch::Current, spec)? as u64;
let proposer_index = state.get_beacon_proposer_index(state.slot, spec)? as u64;
for attestation in attestations {
let attestation_slot = state.get_attestation_data_slot(&attestation.data)?;
let pending_attestation = PendingAttestation {
aggregation_bits: attestation.aggregation_bits.clone(),
data: attestation.data.clone(),
inclusion_delay: (state.slot - attestation_slot).as_u64(),
inclusion_delay: (state.slot - attestation.data.slot).as_u64(),
proposer_index,
};
@ -371,7 +362,7 @@ pub fn process_attestations<T: EthSpec>(
/// Returns `Ok(())` if the validation and state updates completed successfully, otherwise returns
/// an `Err` describing the invalid object or cause of failure.
///
/// Spec v0.8.0
/// Spec v0.9.1
pub fn process_deposits<T: EthSpec>(
state: &mut BeaconState<T>,
deposits: &[Deposit],
@ -408,7 +399,7 @@ pub fn process_deposits<T: EthSpec>(
/// Process a single deposit, optionally verifying its merkle proof.
///
/// Spec v0.8.1
/// Spec v0.9.1
pub fn process_deposit<T: EthSpec>(
state: &mut BeaconState<T>,
deposit: &Deposit,
@ -474,7 +465,7 @@ pub fn process_deposit<T: EthSpec>(
/// Returns `Ok(())` if the validation and state updates completed successfully, otherwise returns
/// an `Err` describing the invalid object or cause of failure.
///
/// Spec v0.8.0
/// Spec v0.9.1
pub fn process_exits<T: EthSpec>(
state: &mut BeaconState<T>,
voluntary_exits: &[VoluntaryExit],
@ -496,39 +487,3 @@ pub fn process_exits<T: EthSpec>(
Ok(())
}
/// Validates each `Transfer` and updates the state, short-circuiting on an invalid object.
///
/// Returns `Ok(())` if the validation and state updates completed successfully, otherwise returns
/// an `Err` describing the invalid object or cause of failure.
///
/// Spec v0.8.0
pub fn process_transfers<T: EthSpec>(
state: &mut BeaconState<T>,
transfers: &[Transfer],
verify_signatures: VerifySignatures,
spec: &ChainSpec,
) -> Result<(), BlockProcessingError> {
let expected_transfers = HashSet::<_>::from_iter(transfers).len();
// Verify that there are no duplicate transfers
block_verify!(
transfers.len() == expected_transfers,
BlockProcessingError::DuplicateTransfers {
duplicates: transfers.len().saturating_sub(expected_transfers)
}
);
transfers
.par_iter()
.enumerate()
.try_for_each(|(i, transfer)| {
verify_transfer(&state, transfer, verify_signatures, spec)
.map_err(|e| e.into_with_index(i))
})?;
for (i, transfer) in transfers.iter().enumerate() {
execute_transfer(state, transfer, spec).map_err(|e| e.into_with_index(i))?;
}
Ok(())
}

View File

@ -55,9 +55,7 @@ impl<T: EthSpec> BlockProcessingBuilder<T> {
)),
}
let proposer_index = state
.get_beacon_proposer_index(state.slot, RelativeEpoch::Current, spec)
.unwrap();
let proposer_index = state.get_beacon_proposer_index(state.slot, spec).unwrap();
let keypair = &keypairs[proposer_index];
match randao_sk {
@ -99,9 +97,7 @@ impl<T: EthSpec> BlockProcessingBuilder<T> {
)),
}
let proposer_index = state
.get_beacon_proposer_index(state.slot, RelativeEpoch::Current, spec)
.unwrap();
let proposer_index = state.get_beacon_proposer_index(state.slot, spec).unwrap();
let keypair = &keypairs[proposer_index];
match randao_sk {
@ -112,7 +108,7 @@ impl<T: EthSpec> BlockProcessingBuilder<T> {
ExitTestTask::AlreadyInitiated => {
for _ in 0..2 {
self.block_builder.insert_exit(
&test_task,
test_task,
&mut state,
(0 as usize).try_into().unwrap(),
&keypairs[0].sk,
@ -123,7 +119,7 @@ impl<T: EthSpec> BlockProcessingBuilder<T> {
_ => {
for (i, keypair) in keypairs.iter().take(num_exits).enumerate() {
self.block_builder.insert_exit(
&test_task,
test_task,
&mut state,
(i as usize).try_into().unwrap(),
&keypair.sk,
@ -140,7 +136,7 @@ impl<T: EthSpec> BlockProcessingBuilder<T> {
pub fn build_with_n_attestations(
mut self,
test_task: &AttestationTestTask,
test_task: AttestationTestTask,
num_attestations: u64,
randao_sk: Option<SecretKey>,
previous_block_root: Option<Hash256>,
@ -158,9 +154,7 @@ impl<T: EthSpec> BlockProcessingBuilder<T> {
)),
}
let proposer_index = state
.get_beacon_proposer_index(state.slot, RelativeEpoch::Current, spec)
.unwrap();
let proposer_index = state.get_beacon_proposer_index(state.slot, spec).unwrap();
let keypair = &keypairs[proposer_index];
match randao_sk {
@ -185,7 +179,7 @@ impl<T: EthSpec> BlockProcessingBuilder<T> {
pub fn build_with_attester_slashing(
mut self,
test_task: &AttesterSlashingTestTask,
test_task: AttesterSlashingTestTask,
num_attester_slashings: u64,
randao_sk: Option<SecretKey>,
previous_block_root: Option<Hash256>,
@ -203,9 +197,7 @@ impl<T: EthSpec> BlockProcessingBuilder<T> {
)),
}
let proposer_index = state
.get_beacon_proposer_index(state.slot, RelativeEpoch::Current, spec)
.unwrap();
let proposer_index = state.get_beacon_proposer_index(state.slot, spec).unwrap();
let keypair = &keypairs[proposer_index];
match randao_sk {
@ -236,7 +228,7 @@ impl<T: EthSpec> BlockProcessingBuilder<T> {
pub fn build_with_proposer_slashing(
mut self,
test_task: &ProposerSlashingTestTask,
test_task: ProposerSlashingTestTask,
num_proposer_slashings: u64,
randao_sk: Option<SecretKey>,
previous_block_root: Option<Hash256>,
@ -254,9 +246,7 @@ impl<T: EthSpec> BlockProcessingBuilder<T> {
)),
}
let proposer_index = state
.get_beacon_proposer_index(state.slot, RelativeEpoch::Current, spec)
.unwrap();
let proposer_index = state.get_beacon_proposer_index(state.slot, spec).unwrap();
let keypair = &keypairs[proposer_index];
match randao_sk {
@ -298,9 +288,7 @@ impl<T: EthSpec> BlockProcessingBuilder<T> {
)),
}
let proposer_index = state
.get_beacon_proposer_index(state.slot, RelativeEpoch::Current, spec)
.unwrap();
let proposer_index = state.get_beacon_proposer_index(state.slot, spec).unwrap();
let keypair = &keypairs[proposer_index];
match randao_sk {

View File

@ -86,7 +86,6 @@ impl<'a, T: EthSpec> BlockSignatureVerifier<'a, T> {
* Deposits are not included because they can legally have invalid signatures.
*/
verifier.include_exits()?;
verifier.include_transfers()?;
verifier.verify()
}
@ -209,19 +208,4 @@ impl<'a, T: EthSpec> BlockSignatureVerifier<'a, T> {
Ok(())
}
/// Includes all signatures in `self.block.body.transfers` for verification.
fn include_transfers(&mut self) -> Result<()> {
let mut sets = self
.block
.body
.transfers
.iter()
.map(|transfer| transfer_signature_set(&self.state, transfer, &self.spec))
.collect::<SignatureSetResult<_>>()?;
self.sets.append(&mut sets);
Ok(())
}
}

View File

@ -16,9 +16,6 @@ pub enum BlockProcessingError {
expected: usize,
found: usize,
},
DuplicateTransfers {
duplicates: usize,
},
HeaderInvalid {
reason: HeaderInvalid,
},
@ -46,10 +43,6 @@ pub enum BlockProcessingError {
index: usize,
reason: ExitInvalid,
},
TransferInvalid {
index: usize,
reason: TransferInvalid,
},
BeaconStateError(BeaconStateError),
SignatureSetError(SignatureSetError),
SszTypesError(ssz_types::Error),
@ -119,8 +112,7 @@ impl_into_block_processing_error_with_index!(
IndexedAttestationInvalid,
AttestationInvalid,
DepositInvalid,
ExitInvalid,
TransferInvalid
ExitInvalid
);
pub type HeaderValidationError = BlockOperationError<HeaderInvalid>;
@ -129,7 +121,6 @@ pub type ProposerSlashingValidationError = BlockOperationError<ProposerSlashingI
pub type AttestationValidationError = BlockOperationError<AttestationInvalid>;
pub type DepositValidationError = BlockOperationError<DepositInvalid>;
pub type ExitValidationError = BlockOperationError<ExitInvalid>;
pub type TransferValidationError = BlockOperationError<TransferInvalid>;
#[derive(Debug, PartialEq)]
pub enum BlockOperationError<T> {
@ -174,10 +165,10 @@ pub enum HeaderInvalid {
pub enum ProposerSlashingInvalid {
/// The proposer index is not a known validator.
ProposerUnknown(u64),
/// The two proposal have different epochs.
/// The two proposal have different slots.
///
/// (proposal_1_slot, proposal_2_slot)
ProposalEpochMismatch(Slot, Slot),
ProposalSlotMismatch(Slot, Slot),
/// The proposals are identical and therefore not slashable.
ProposalsIdentical,
/// The specified proposer cannot be slashed because they are already slashed, or not active.
@ -207,8 +198,8 @@ pub enum AttesterSlashingInvalid {
/// Describes why an object is invalid.
#[derive(Debug, PartialEq)]
pub enum AttestationInvalid {
/// Shard exceeds SHARD_COUNT.
BadShard,
/// Commmittee index exceeds number of committees in that slot.
BadCommitteeIndex,
/// Attestation included before the inclusion delay.
IncludedTooEarly {
state: Slot,
@ -229,36 +220,18 @@ pub enum AttestationInvalid {
attestation: Checkpoint,
is_current: bool,
},
/// Attestation crosslink root does not match the state crosslink root for the attestations
/// slot.
BadParentCrosslinkHash,
/// Attestation crosslink start epoch does not match the end epoch of the state crosslink.
BadParentCrosslinkStartEpoch,
/// Attestation crosslink end epoch does not match the expected value.
BadParentCrosslinkEndEpoch,
/// The custody bitfield has some bits set `true`. This is not allowed in phase 0.
CustodyBitfieldHasSetBits,
/// There are no set bits on the attestation -- an attestation must be signed by at least one
/// validator.
AggregationBitfieldIsEmpty,
/// The custody bitfield length is not the smallest possible size to represent the committee.
BadCustodyBitfieldLength {
committee_len: usize,
bitfield_len: usize,
},
/// The aggregation bitfield length is not the smallest possible size to represent the committee.
BadAggregationBitfieldLength {
committee_len: usize,
bitfield_len: usize,
},
/// The bits set in the custody bitfield are not a subset of those set in the aggregation bits.
CustodyBitfieldNotSubset,
/// There was no known committee in this `epoch` for the given shard and slot.
NoCommitteeForShard { shard: u64, slot: Slot },
/// The validator index was unknown.
UnknownValidator(u64),
/// The attestation signature verification failed.
BadSignature,
/// The shard block root was not set to zero. This is a phase 0 requirement.
ShardBlockRootNotZero,
/// The indexed attestation created from this attestation was found to be invalid.
BadIndexedAttestation(IndexedAttestationInvalid),
}
@ -280,14 +253,6 @@ impl From<BlockOperationError<IndexedAttestationInvalid>>
#[derive(Debug, PartialEq)]
pub enum IndexedAttestationInvalid {
/// The custody bit 0 validators intersect with the bit 1 validators.
CustodyBitValidatorsIntersect,
/// The custody bitfield has some bits set `true`. This is not allowed in phase 0.
CustodyBitfieldHasSetBits,
/// The custody bitfield violated a type-level bound.
CustodyBitfieldBoundsError(ssz_types::Error),
/// No validator indices were specified.
NoValidatorIndices,
/// The number of indices exceeds the global maximum.
///
/// (max_indices, indices_given)
@ -339,56 +304,3 @@ pub enum ExitInvalid {
/// been invalid or an internal error occurred.
SignatureSetError(SignatureSetError),
}
#[derive(Debug, PartialEq)]
pub enum TransferInvalid {
/// The validator indicated by `transfer.from` is unknown.
FromValidatorUnknown(u64),
/// The validator indicated by `transfer.to` is unknown.
ToValidatorUnknown(u64),
/// The balance of `transfer.from` is insufficient.
///
/// (required, available)
FromBalanceInsufficient(u64, u64),
/// Adding `transfer.fee` to `transfer.amount` causes an overflow.
///
/// (transfer_fee, transfer_amount)
FeeOverflow(u64, u64),
/// This transfer would result in the `transfer.from` account to have `0 < balance <
/// min_deposit_amount`
///
/// (resulting_amount, min_deposit_amount)
SenderDust(u64, u64),
/// This transfer would result in the `transfer.to` account to have `0 < balance <
/// min_deposit_amount`
///
/// (resulting_amount, min_deposit_amount)
RecipientDust(u64, u64),
/// The state slot does not match `transfer.slot`.
///
/// (state_slot, transfer_slot)
StateSlotMismatch(Slot, Slot),
/// The `transfer.slot` is in the past relative to the state slot.
///
///
/// (state_slot, transfer_slot)
TransferSlotInPast(Slot, Slot),
/// The `transfer.from` validator has been activated and is not withdrawable.
///
/// (from_validator)
FromValidatorIneligibleForTransfer(u64),
/// The validators withdrawal credentials do not match `transfer.pubkey`.
///
/// (state_credentials, transfer_pubkey_credentials)
WithdrawalCredentialsMismatch(Hash256, Hash256),
/// The deposit was not signed by `deposit.pubkey`.
BadSignature,
/// Overflow when adding to `transfer.to` balance.
///
/// (to_balance, transfer_amount)
ToBalanceOverflow(u64, u64),
/// Overflow when adding to beacon proposer balance.
///
/// (proposer_balance, transfer_fee)
ProposerBalanceOverflow(u64, u64),
}

View File

@ -1,8 +1,6 @@
use super::errors::{BlockOperationError, IndexedAttestationInvalid as Invalid};
use super::signature_sets::indexed_attestation_signature_set;
use crate::VerifySignatures;
use std::collections::HashSet;
use std::iter::FromIterator;
use types::*;
type Result<T> = std::result::Result<T, BlockOperationError<Invalid>>;
@ -13,38 +11,26 @@ fn error(reason: Invalid) -> BlockOperationError<Invalid> {
/// Verify an `IndexedAttestation`.
///
/// Spec v0.8.0
/// Spec v0.9.1
pub fn is_valid_indexed_attestation<T: EthSpec>(
state: &BeaconState<T>,
indexed_attestation: &IndexedAttestation<T>,
verify_signatures: VerifySignatures,
spec: &ChainSpec,
) -> Result<()> {
let bit_0_indices = &indexed_attestation.custody_bit_0_indices;
let bit_1_indices = &indexed_attestation.custody_bit_1_indices;
// Verify no index has custody bit equal to 1 [to be removed in phase 1]
verify!(bit_1_indices.is_empty(), Invalid::CustodyBitfieldHasSetBits);
let indices = &indexed_attestation.attesting_indices;
// Verify max number of indices
let total_indices = bit_0_indices.len() + bit_1_indices.len();
verify!(
total_indices <= T::MaxValidatorsPerCommittee::to_usize(),
Invalid::MaxIndicesExceed(T::MaxValidatorsPerCommittee::to_usize(), total_indices)
indices.len() <= T::MaxValidatorsPerCommittee::to_usize(),
Invalid::MaxIndicesExceed(T::MaxValidatorsPerCommittee::to_usize(), indices.len())
);
// Verify index sets are disjoint
let custody_bit_intersection: HashSet<&u64> =
&HashSet::from_iter(bit_0_indices.iter()) & &HashSet::from_iter(bit_1_indices.iter());
verify!(
custody_bit_intersection.is_empty(),
Invalid::CustodyBitValidatorsIntersect
);
// Check that both vectors of indices are sorted
// Check that indices are sorted
let check_sorted = |list: &[u64]| -> Result<()> {
list.windows(2).enumerate().try_for_each(|(i, pair)| {
if pair[0] >= pair[1] {
// The spec allows duplicates, so use strict comparison (>).
if pair[0] > pair[1] {
Err(error(Invalid::BadValidatorIndicesOrdering(i)))
} else {
Ok(())
@ -52,8 +38,7 @@ pub fn is_valid_indexed_attestation<T: EthSpec>(
})?;
Ok(())
};
check_sorted(&bit_0_indices)?;
check_sorted(&bit_1_indices)?;
check_sorted(indices)?;
if verify_signatures.is_true() {
verify!(

View File

@ -2,14 +2,13 @@
//! validated individually, or alongside in others in a potentially cheaper bulk operation.
//!
//! This module exposes one function to extract each type of `SignatureSet` from a `BeaconBlock`.
use bls::SignatureSet;
use bls::{SignatureSet, SignedMessage};
use std::convert::TryInto;
use tree_hash::{SignedRoot, TreeHash};
use types::{
AggregateSignature, AttestationDataAndCustodyBit, AttesterSlashing, BeaconBlock,
BeaconBlockHeader, BeaconState, BeaconStateError, ChainSpec, DepositData, Domain, EthSpec,
Hash256, IndexedAttestation, ProposerSlashing, PublicKey, RelativeEpoch, Signature, Transfer,
VoluntaryExit,
AggregateSignature, AttesterSlashing, BeaconBlock, BeaconBlockHeader, BeaconState,
BeaconStateError, ChainSpec, DepositData, Domain, EthSpec, Hash256, IndexedAttestation,
ProposerSlashing, PublicKey, Signature, VoluntaryExit,
};
pub type Result<T> = std::result::Result<T, Error>;
@ -42,8 +41,7 @@ pub fn block_proposal_signature_set<'a, T: EthSpec>(
block_signed_root: Option<Hash256>,
spec: &'a ChainSpec,
) -> Result<SignatureSet<'a>> {
let proposer_index =
state.get_beacon_proposer_index(block.slot, RelativeEpoch::Current, spec)?;
let proposer_index = state.get_beacon_proposer_index(block.slot, spec)?;
let block_proposer = &state
.validators
.get(proposer_index)
@ -75,8 +73,7 @@ pub fn randao_signature_set<'a, T: EthSpec>(
block: &'a BeaconBlock<T>,
spec: &'a ChainSpec,
) -> Result<SignatureSet<'a>> {
let block_proposer = &state.validators
[state.get_beacon_proposer_index(block.slot, RelativeEpoch::Current, spec)?];
let block_proposer = &state.validators[state.get_beacon_proposer_index(block.slot, spec)?];
let domain = spec.get_domain(
block.slot.epoch(T::slots_per_epoch()),
@ -141,31 +138,20 @@ pub fn indexed_attestation_signature_set<'a, 'b, T: EthSpec>(
indexed_attestation: &'b IndexedAttestation<T>,
spec: &'a ChainSpec,
) -> Result<SignatureSet<'a>> {
let message_0 = AttestationDataAndCustodyBit {
data: indexed_attestation.data.clone(),
custody_bit: false,
}
.tree_hash_root();
let message_1 = AttestationDataAndCustodyBit {
data: indexed_attestation.data.clone(),
custody_bit: true,
}
.tree_hash_root();
let message = indexed_attestation.data.tree_hash_root();
let signed_message = SignedMessage::new(
get_pubkeys(state, &indexed_attestation.attesting_indices)?,
message,
);
let domain = spec.get_domain(
indexed_attestation.data.target.epoch,
Domain::Attestation,
Domain::BeaconAttester,
&state.fork,
);
Ok(SignatureSet::dual(
signature,
message_0,
get_pubkeys(state, &indexed_attestation.custody_bit_0_indices)?,
message_1,
get_pubkeys(state, &indexed_attestation.custody_bit_1_indices)?,
domain,
))
Ok(SignatureSet::new(signature, vec![signed_message], domain))
}
/// Returns the signature set for the given `attester_slashing` and corresponding `pubkeys`.
@ -244,28 +230,6 @@ pub fn exit_signature_set<'a, T: EthSpec>(
))
}
/// Returns a signature set that is valid if the `Transfer` was signed by `transfer.pubkey`.
pub fn transfer_signature_set<'a, T: EthSpec>(
state: &'a BeaconState<T>,
transfer: &'a Transfer,
spec: &'a ChainSpec,
) -> Result<SignatureSet<'a>> {
let domain = spec.get_domain(
transfer.slot.epoch(T::slots_per_epoch()),
Domain::Transfer,
&state.fork,
);
let message = transfer.signed_root();
Ok(SignatureSet::single(
&transfer.signature,
&transfer.pubkey,
message,
domain,
))
}
/// Maps validator indices to public keys.
fn get_pubkeys<'a, 'b, T, I>(
state: &'a BeaconState<T>,

View File

@ -393,6 +393,7 @@ fn invalid_exit_already_exited() {
);
}
/* FIXME: needs updating for v0.9
#[test]
fn invalid_exit_not_active() {
use std::cmp::max;
@ -421,6 +422,7 @@ fn invalid_exit_not_active() {
})
);
}
*/
#[test]
fn invalid_exit_already_initiated() {
@ -546,7 +548,7 @@ fn valid_attestations() {
let builder = get_builder(&spec, SLOT_OFFSET, VALIDATOR_COUNT);
let test_task = AttestationTestTask::Valid;
let (block, mut state) =
builder.build_with_n_attestations(&test_task, NUM_ATTESTATIONS, None, None, &spec);
builder.build_with_n_attestations(test_task, NUM_ATTESTATIONS, None, None, &spec);
let result = per_block_processing(
&mut state,
@ -560,91 +562,14 @@ fn valid_attestations() {
assert_eq!(result, Ok(()));
}
#[test]
fn invalid_attestation_parent_crosslink_start_epoch() {
let spec = MainnetEthSpec::default_spec();
let builder = get_builder(&spec, SLOT_OFFSET, VALIDATOR_COUNT);
let test_task = AttestationTestTask::BadParentCrosslinkStartEpoch;
let (block, mut state) =
builder.build_with_n_attestations(&test_task, NUM_ATTESTATIONS, None, None, &spec);
let result = per_block_processing(
&mut state,
&block,
None,
BlockSignatureStrategy::VerifyIndividual,
&spec,
);
// Expecting BadParentCrosslinkEndEpoch because we manually set an invalid crosslink start epoch
assert_eq!(
result,
Err(BlockProcessingError::AttestationInvalid {
index: 0,
reason: AttestationInvalid::BadParentCrosslinkStartEpoch
})
);
}
#[test]
fn invalid_attestation_parent_crosslink_end_epoch() {
let spec = MainnetEthSpec::default_spec();
let builder = get_builder(&spec, SLOT_OFFSET, VALIDATOR_COUNT);
let test_task = AttestationTestTask::BadParentCrosslinkEndEpoch;
let (block, mut state) =
builder.build_with_n_attestations(&test_task, NUM_ATTESTATIONS, None, None, &spec);
let result = per_block_processing(
&mut state,
&block,
None,
BlockSignatureStrategy::VerifyIndividual,
&spec,
);
// Expecting BadParentCrosslinkEndEpoch because we manually set an invalid crosslink end epoch
assert_eq!(
result,
Err(BlockProcessingError::AttestationInvalid {
index: 0,
reason: AttestationInvalid::BadParentCrosslinkEndEpoch
})
);
}
#[test]
fn invalid_attestation_parent_crosslink_hash() {
let spec = MainnetEthSpec::default_spec();
let builder = get_builder(&spec, SLOT_OFFSET, VALIDATOR_COUNT);
let test_task = AttestationTestTask::BadParentCrosslinkHash;
let (block, mut state) =
builder.build_with_n_attestations(&test_task, NUM_ATTESTATIONS, None, None, &spec);
let result = per_block_processing(
&mut state,
&block,
None,
BlockSignatureStrategy::VerifyIndividual,
&spec,
);
// Expecting BadParentCrosslinkHash because we manually set an invalid crosslink parent_root
assert_eq!(
result,
Err(BlockProcessingError::AttestationInvalid {
index: 0,
reason: AttestationInvalid::BadParentCrosslinkHash
})
);
}
/* FIXME: needs updating for v0.9
#[test]
fn invalid_attestation_no_committee_for_shard() {
let spec = MainnetEthSpec::default_spec();
let builder = get_builder(&spec, SLOT_OFFSET, VALIDATOR_COUNT);
let test_task = AttestationTestTask::NoCommiteeForShard;
let (block, mut state) =
builder.build_with_n_attestations(&test_task, NUM_ATTESTATIONS, None, None, &spec);
builder.build_with_n_attestations(test_task, NUM_ATTESTATIONS, None, None, &spec);
let result = per_block_processing(
&mut state,
@ -658,10 +583,14 @@ fn invalid_attestation_no_committee_for_shard() {
assert_eq!(
result,
Err(BlockProcessingError::BeaconStateError(
BeaconStateError::NoCommitteeForShard
BeaconStateError::NoCommittee {
slot: Slot::new(0),
index: 0
}
))
);
}
*/
#[test]
fn invalid_attestation_wrong_justified_checkpoint() {
@ -669,7 +598,7 @@ fn invalid_attestation_wrong_justified_checkpoint() {
let builder = get_builder(&spec, SLOT_OFFSET, VALIDATOR_COUNT);
let test_task = AttestationTestTask::WrongJustifiedCheckpoint;
let (block, mut state) =
builder.build_with_n_attestations(&test_task, NUM_ATTESTATIONS, None, None, &spec);
builder.build_with_n_attestations(test_task, NUM_ATTESTATIONS, None, None, &spec);
let result = per_block_processing(
&mut state,
@ -706,7 +635,7 @@ fn invalid_attestation_bad_target_too_low() {
let builder = get_builder(&spec, SLOT_OFFSET, VALIDATOR_COUNT);
let test_task = AttestationTestTask::BadTargetTooLow;
let (block, mut state) =
builder.build_with_n_attestations(&test_task, NUM_ATTESTATIONS, None, None, &spec);
builder.build_with_n_attestations(test_task, NUM_ATTESTATIONS, None, None, &spec);
let result = per_block_processing(
&mut state,
@ -716,16 +645,14 @@ fn invalid_attestation_bad_target_too_low() {
&spec,
);
// Expecting EpochTooLow because we manually set the
// Expecting BadTargetEpoch because we manually set the
// target field of the AttestationData object to be invalid
assert_eq!(
result,
Err(BlockProcessingError::BeaconStateError(
BeaconStateError::RelativeEpochError(RelativeEpochError::EpochTooLow {
base: state.current_epoch(),
other: Epoch::from(0 as u64),
Err(BlockProcessingError::AttestationInvalid {
index: 0,
reason: AttestationInvalid::BadTargetEpoch
})
))
);
}
@ -735,7 +662,7 @@ fn invalid_attestation_bad_target_too_high() {
let builder = get_builder(&spec, SLOT_OFFSET, VALIDATOR_COUNT);
let test_task = AttestationTestTask::BadTargetTooHigh;
let (block, mut state) =
builder.build_with_n_attestations(&test_task, NUM_ATTESTATIONS, None, None, &spec);
builder.build_with_n_attestations(test_task, NUM_ATTESTATIONS, None, None, &spec);
let result = per_block_processing(
&mut state,
@ -745,43 +672,13 @@ fn invalid_attestation_bad_target_too_high() {
&spec,
);
// Expecting EpochTooHigh because we manually set the
// Expecting BadTargetEpoch because we manually set the
// target field of the AttestationData object to be invalid
assert_eq!(
result,
Err(BlockProcessingError::BeaconStateError(
BeaconStateError::RelativeEpochError(RelativeEpochError::EpochTooHigh {
base: state.current_epoch(),
other: Epoch::from(10 as u64),
})
))
);
}
#[test]
fn invalid_attestation_bad_crosslink_data_root() {
let spec = MainnetEthSpec::default_spec();
let builder = get_builder(&spec, SLOT_OFFSET, VALIDATOR_COUNT);
let test_task = AttestationTestTask::BadParentCrosslinkDataRoot;
let (block, mut state) =
builder.build_with_n_attestations(&test_task, NUM_ATTESTATIONS, None, None, &spec);
let result = per_block_processing(
&mut state,
&block,
None,
BlockSignatureStrategy::VerifyIndividual,
&spec,
);
// Expecting ShardBlockRootNotZero because we manually set the
// data_root of the cross link to be non zero
assert_eq!(
result,
Err(BlockProcessingError::AttestationInvalid {
index: 0,
reason: AttestationInvalid::ShardBlockRootNotZero,
reason: AttestationInvalid::BadTargetEpoch
})
);
}
@ -792,7 +689,7 @@ fn invalid_attestation_bad_indexed_attestation_bad_signature() {
let builder = get_builder(&spec, SLOT_OFFSET, 33); // minmium number of validators required for this test
let test_task = AttestationTestTask::BadIndexedAttestationBadSignature;
let (block, mut state) =
builder.build_with_n_attestations(&test_task, NUM_ATTESTATIONS, None, None, &spec);
builder.build_with_n_attestations(test_task, NUM_ATTESTATIONS, None, None, &spec);
let result = per_block_processing(
&mut state,
@ -814,90 +711,13 @@ fn invalid_attestation_bad_indexed_attestation_bad_signature() {
);
}
#[test]
fn invalid_attestation_custody_bitfield_not_subset() {
let spec = MainnetEthSpec::default_spec();
let builder = get_builder(&spec, SLOT_OFFSET, 33); // minmium number of validators required for this test
let test_task = AttestationTestTask::CustodyBitfieldNotSubset;
let (block, mut state) =
builder.build_with_n_attestations(&test_task, NUM_ATTESTATIONS, None, None, &spec);
let result = per_block_processing(
&mut state,
&block,
None,
BlockSignatureStrategy::VerifyIndividual,
&spec,
);
// Expecting CustodyBitfieldNotSubset because we set custody_bit to true without setting the aggregation bits.
assert_eq!(
result,
Err(BlockProcessingError::AttestationInvalid {
index: 0,
reason: AttestationInvalid::CustodyBitfieldNotSubset
})
);
}
#[test]
fn invalid_attestation_custody_bitfield_has_set_bits() {
let spec = MainnetEthSpec::default_spec();
let builder = get_builder(&spec, SLOT_OFFSET, 33); // minmium number of validators required for this test
let test_task = AttestationTestTask::CustodyBitfieldHasSetBits;
let (block, mut state) =
builder.build_with_n_attestations(&test_task, NUM_ATTESTATIONS, None, None, &spec);
let result = per_block_processing(
&mut state,
&block,
None,
BlockSignatureStrategy::VerifyIndividual,
&spec,
);
// Expecting CustodyBitfieldHasSetBits because we set custody bits even though the custody_bit boolean is set to false
assert_eq!(
result,
Err(BlockProcessingError::AttestationInvalid {
index: 0,
reason: AttestationInvalid::BadIndexedAttestation(
IndexedAttestationInvalid::CustodyBitfieldHasSetBits
)
})
);
}
#[test]
fn invalid_attestation_bad_custody_bitfield_len() {
let spec = MainnetEthSpec::default_spec();
let builder = get_builder(&spec, SLOT_OFFSET, VALIDATOR_COUNT);
let test_task = AttestationTestTask::BadCustodyBitfieldLen;
let (block, mut state) =
builder.build_with_n_attestations(&test_task, NUM_ATTESTATIONS, None, None, &spec);
let result = per_block_processing(
&mut state,
&block,
None,
BlockSignatureStrategy::VerifyIndividual,
&spec,
);
// Expecting InvalidBitfield because the size of the custody_bitfield is bigger than the commitee size.
assert_eq!(
result,
Err(BlockProcessingError::BeaconStateError(
BeaconStateError::InvalidBitfield
))
);
}
#[test]
fn invalid_attestation_bad_aggregation_bitfield_len() {
let spec = MainnetEthSpec::default_spec();
let builder = get_builder(&spec, SLOT_OFFSET, VALIDATOR_COUNT);
let test_task = AttestationTestTask::BadAggregationBitfieldLen;
let (block, mut state) =
builder.build_with_n_attestations(&test_task, NUM_ATTESTATIONS, None, None, &spec);
builder.build_with_n_attestations(test_task, NUM_ATTESTATIONS, None, None, &spec);
let result = per_block_processing(
&mut state,
@ -922,7 +742,7 @@ fn invalid_attestation_bad_signature() {
let builder = get_builder(&spec, SLOT_OFFSET, 97); // minimal number of required validators for this test
let test_task = AttestationTestTask::BadSignature;
let (block, mut state) =
builder.build_with_n_attestations(&test_task, NUM_ATTESTATIONS, None, None, &spec);
builder.build_with_n_attestations(test_task, NUM_ATTESTATIONS, None, None, &spec);
let result = per_block_processing(
&mut state,
&block,
@ -949,7 +769,7 @@ fn invalid_attestation_included_too_early() {
let builder = get_builder(&spec, SLOT_OFFSET, VALIDATOR_COUNT);
let test_task = AttestationTestTask::IncludedTooEarly;
let (block, mut state) =
builder.build_with_n_attestations(&test_task, NUM_ATTESTATIONS, None, None, &spec);
builder.build_with_n_attestations(test_task, NUM_ATTESTATIONS, None, None, &spec);
let result = per_block_processing(
&mut state,
@ -965,9 +785,9 @@ fn invalid_attestation_included_too_early() {
Err(BlockProcessingError::AttestationInvalid {
index: 0,
reason: AttestationInvalid::IncludedTooEarly {
state: Slot::from(319 as u64),
state: state.slot,
delay: spec.min_attestation_inclusion_delay,
attestation: Slot::from(319 as u64)
attestation: block.body.attestations[0].data.slot,
}
})
);
@ -976,11 +796,11 @@ fn invalid_attestation_included_too_early() {
#[test]
fn invalid_attestation_included_too_late() {
let spec = MainnetEthSpec::default_spec();
// note to maintainer: might need to increase validator count if we get NoCommitteeForShard
// note to maintainer: might need to increase validator count if we get NoCommittee
let builder = get_builder(&spec, SLOT_OFFSET, VALIDATOR_COUNT);
let test_task = AttestationTestTask::IncludedTooLate;
let (block, mut state) =
builder.build_with_n_attestations(&test_task, NUM_ATTESTATIONS, None, None, &spec);
builder.build_with_n_attestations(test_task, NUM_ATTESTATIONS, None, None, &spec);
let result = per_block_processing(
&mut state,
@ -990,18 +810,13 @@ fn invalid_attestation_included_too_late() {
&spec,
);
// Expecting IncludedTooLate because the shard included in the crosslink is bigger than expected
assert!(
result
== Err(BlockProcessingError::BeaconStateError(
BeaconStateError::NoCommitteeForShard
))
|| result
== Err(BlockProcessingError::AttestationInvalid {
assert_eq!(
result,
Err(BlockProcessingError::AttestationInvalid {
index: 0,
reason: AttestationInvalid::IncludedTooLate {
state: state.slot,
attestation: Slot::from(254 as u64),
attestation: block.body.attestations[0].data.slot,
}
})
);
@ -1010,11 +825,11 @@ fn invalid_attestation_included_too_late() {
#[test]
fn invalid_attestation_bad_target_epoch() {
let spec = MainnetEthSpec::default_spec();
// note to maintainer: might need to increase validator count if we get NoCommitteeForShard
// note to maintainer: might need to increase validator count if we get NoCommittee
let builder = get_builder(&spec, SLOT_OFFSET, VALIDATOR_COUNT);
let test_task = AttestationTestTask::BadTargetEpoch;
let (block, mut state) =
builder.build_with_n_attestations(&test_task, NUM_ATTESTATIONS, None, None, &spec);
builder.build_with_n_attestations(test_task, NUM_ATTESTATIONS, None, None, &spec);
let result = per_block_processing(
&mut state,
@ -1028,7 +843,10 @@ fn invalid_attestation_bad_target_epoch() {
assert!(
result
== Err(BlockProcessingError::BeaconStateError(
BeaconStateError::NoCommitteeForShard
BeaconStateError::NoCommittee {
slot: Slot::new(0),
index: 0
}
))
|| result
== Err(BlockProcessingError::AttestationInvalid {
@ -1038,13 +856,14 @@ fn invalid_attestation_bad_target_epoch() {
);
}
/* FIXME: needs updating for v0.9
#[test]
fn invalid_attestation_bad_shard() {
let spec = MainnetEthSpec::default_spec();
let builder = get_builder(&spec, SLOT_OFFSET, VALIDATOR_COUNT);
let test_task = AttestationTestTask::BadShard;
let (block, mut state) =
builder.build_with_n_attestations(&test_task, NUM_ATTESTATIONS, None, None, &spec);
builder.build_with_n_attestations(test_task, NUM_ATTESTATIONS, None, None, &spec);
let result = per_block_processing(
&mut state,
@ -1054,7 +873,7 @@ fn invalid_attestation_bad_shard() {
&spec,
);
// Expecting BadShard or NoCommitteeForShard because the shard number is higher than ShardCount
// Expecting BadShard or NoCommittee because the shard number is higher than ShardCount
assert!(
result
== Err(BlockProcessingError::AttestationInvalid {
@ -1063,10 +882,14 @@ fn invalid_attestation_bad_shard() {
})
|| result
== Err(BlockProcessingError::BeaconStateError(
BeaconStateError::NoCommitteeForShard
BeaconStateError::NoCommittee {
slot: Slot::new(0),
index: 0
}
))
);
}
*/
#[test]
fn valid_insert_attester_slashing() {
@ -1075,7 +898,7 @@ fn valid_insert_attester_slashing() {
let test_task = AttesterSlashingTestTask::Valid;
let num_attester_slashings = 1;
let (block, mut state) =
builder.build_with_attester_slashing(&test_task, num_attester_slashings, None, None, &spec);
builder.build_with_attester_slashing(test_task, num_attester_slashings, None, None, &spec);
let result = per_block_processing(
&mut state,
@ -1096,7 +919,7 @@ fn invalid_attester_slashing_not_slashable() {
let test_task = AttesterSlashingTestTask::NotSlashable;
let num_attester_slashings = 1;
let (block, mut state) =
builder.build_with_attester_slashing(&test_task, num_attester_slashings, None, None, &spec);
builder.build_with_attester_slashing(test_task, num_attester_slashings, None, None, &spec);
let result = per_block_processing(
&mut state,
&block,
@ -1122,7 +945,7 @@ fn invalid_attester_slashing_1_invalid() {
let test_task = AttesterSlashingTestTask::IndexedAttestation1Invalid;
let num_attester_slashings = 1;
let (block, mut state) =
builder.build_with_attester_slashing(&test_task, num_attester_slashings, None, None, &spec);
builder.build_with_attester_slashing(test_task, num_attester_slashings, None, None, &spec);
let result = per_block_processing(
&mut state,
@ -1132,21 +955,11 @@ fn invalid_attester_slashing_1_invalid() {
&spec,
);
// Expecting IndexedAttestation1Invalid or IndexedAttestationInvalid because Attestation1 has CustodyBitfield bits set.
assert!(
result
== Err(BlockProcessingError::IndexedAttestationInvalid {
assert_eq!(
result,
Err(BlockProcessingError::IndexedAttestationInvalid {
index: 0,
reason: IndexedAttestationInvalid::CustodyBitfieldHasSetBits
})
|| result
== Err(BlockProcessingError::AttesterSlashingInvalid {
index: 0,
reason: AttesterSlashingInvalid::IndexedAttestation1Invalid(
BlockOperationError::Invalid(
IndexedAttestationInvalid::CustodyBitfieldHasSetBits
)
)
reason: IndexedAttestationInvalid::BadValidatorIndicesOrdering(0)
})
);
}
@ -1158,7 +971,7 @@ fn invalid_attester_slashing_2_invalid() {
let test_task = AttesterSlashingTestTask::IndexedAttestation2Invalid;
let num_attester_slashings = 1;
let (block, mut state) =
builder.build_with_attester_slashing(&test_task, num_attester_slashings, None, None, &spec);
builder.build_with_attester_slashing(test_task, num_attester_slashings, None, None, &spec);
let result = per_block_processing(
&mut state,
@ -1168,21 +981,11 @@ fn invalid_attester_slashing_2_invalid() {
&spec,
);
// Expecting IndexedAttestation2Invalid or IndexedAttestationInvalid because Attestation2 has CustodyBitfield bits set.
assert!(
result
== Err(BlockProcessingError::IndexedAttestationInvalid {
assert_eq!(
result,
Err(BlockProcessingError::IndexedAttestationInvalid {
index: 1,
reason: IndexedAttestationInvalid::CustodyBitfieldHasSetBits
})
|| result
== Err(BlockProcessingError::AttesterSlashingInvalid {
index: 1,
reason: AttesterSlashingInvalid::IndexedAttestation2Invalid(
BlockOperationError::Invalid(
IndexedAttestationInvalid::CustodyBitfieldHasSetBits
)
)
reason: IndexedAttestationInvalid::BadValidatorIndicesOrdering(0)
})
);
}
@ -1192,7 +995,7 @@ fn valid_insert_proposer_slashing() {
let spec = MainnetEthSpec::default_spec();
let builder = get_builder(&spec, SLOT_OFFSET, VALIDATOR_COUNT);
let test_task = ProposerSlashingTestTask::Valid;
let (block, mut state) = builder.build_with_proposer_slashing(&test_task, 1, None, None, &spec);
let (block, mut state) = builder.build_with_proposer_slashing(test_task, 1, None, None, &spec);
let result = per_block_processing(
&mut state,
@ -1211,7 +1014,7 @@ fn invalid_proposer_slashing_proposals_identical() {
let spec = MainnetEthSpec::default_spec();
let builder = get_builder(&spec, SLOT_OFFSET, VALIDATOR_COUNT);
let test_task = ProposerSlashingTestTask::ProposalsIdentical;
let (block, mut state) = builder.build_with_proposer_slashing(&test_task, 1, None, None, &spec);
let (block, mut state) = builder.build_with_proposer_slashing(test_task, 1, None, None, &spec);
let result = per_block_processing(
&mut state,
@ -1235,7 +1038,7 @@ fn invalid_proposer_slashing_proposer_unknown() {
let spec = MainnetEthSpec::default_spec();
let builder = get_builder(&spec, SLOT_OFFSET, VALIDATOR_COUNT);
let test_task = ProposerSlashingTestTask::ProposerUnknown;
let (block, mut state) = builder.build_with_proposer_slashing(&test_task, 1, None, None, &spec);
let (block, mut state) = builder.build_with_proposer_slashing(test_task, 1, None, None, &spec);
let result = per_block_processing(
&mut state,
@ -1260,7 +1063,7 @@ fn invalid_proposer_slashing_not_slashable() {
let spec = MainnetEthSpec::default_spec();
let builder = get_builder(&spec, SLOT_OFFSET, VALIDATOR_COUNT);
let test_task = ProposerSlashingTestTask::ProposerNotSlashable;
let (block, mut state) = builder.build_with_proposer_slashing(&test_task, 1, None, None, &spec);
let (block, mut state) = builder.build_with_proposer_slashing(test_task, 1, None, None, &spec);
state.validators[0].slashed = true;
let result = per_block_processing(
@ -1286,7 +1089,7 @@ fn invalid_bad_proposal_1_signature() {
let spec = MainnetEthSpec::default_spec();
let builder = get_builder(&spec, SLOT_OFFSET, VALIDATOR_COUNT);
let test_task = ProposerSlashingTestTask::BadProposal1Signature;
let (block, mut state) = builder.build_with_proposer_slashing(&test_task, 1, None, None, &spec);
let (block, mut state) = builder.build_with_proposer_slashing(test_task, 1, None, None, &spec);
let result = per_block_processing(
&mut state,
@ -1311,7 +1114,7 @@ fn invalid_bad_proposal_2_signature() {
let spec = MainnetEthSpec::default_spec();
let builder = get_builder(&spec, SLOT_OFFSET, VALIDATOR_COUNT);
let test_task = ProposerSlashingTestTask::BadProposal2Signature;
let (block, mut state) = builder.build_with_proposer_slashing(&test_task, 1, None, None, &spec);
let (block, mut state) = builder.build_with_proposer_slashing(test_task, 1, None, None, &spec);
let result = per_block_processing(
&mut state,
@ -1336,7 +1139,7 @@ fn invalid_proposer_slashing_proposal_epoch_mismatch() {
let spec = MainnetEthSpec::default_spec();
let builder = get_builder(&spec, SLOT_OFFSET, VALIDATOR_COUNT);
let test_task = ProposerSlashingTestTask::ProposalEpochMismatch;
let (block, mut state) = builder.build_with_proposer_slashing(&test_task, 1, None, None, &spec);
let (block, mut state) = builder.build_with_proposer_slashing(test_task, 1, None, None, &spec);
let result = per_block_processing(
&mut state,
@ -1351,7 +1154,7 @@ fn invalid_proposer_slashing_proposal_epoch_mismatch() {
result,
Err(BlockProcessingError::ProposerSlashingInvalid {
index: 0,
reason: ProposerSlashingInvalid::ProposalEpochMismatch(
reason: ProposerSlashingInvalid::ProposalSlotMismatch(
Slot::from(0 as u64),
Slot::from(128 as u64)
)
@ -1363,7 +1166,7 @@ fn get_builder(
spec: &ChainSpec,
slot_offset: u64,
num_validators: usize,
) -> (BlockProcessingBuilder<MainnetEthSpec>) {
) -> BlockProcessingBuilder<MainnetEthSpec> {
let mut builder = BlockProcessingBuilder::new(num_validators, &spec);
// Set the state and block to be in the last slot of the `slot_offset`th epoch.
@ -1371,6 +1174,5 @@ fn get_builder(
(MainnetEthSpec::genesis_epoch() + slot_offset).end_slot(MainnetEthSpec::slots_per_epoch());
builder.set_slot(last_slot_of_epoch);
builder.build_caches(&spec);
(builder)
builder
}

View File

@ -2,7 +2,6 @@ use super::errors::{AttestationInvalid as Invalid, BlockOperationError};
use super::VerifySignatures;
use crate::common::get_indexed_attestation;
use crate::per_block_processing::is_valid_indexed_attestation;
use tree_hash::TreeHash;
use types::*;
type Result<T> = std::result::Result<T, BlockOperationError<Invalid>>;
@ -16,7 +15,7 @@ fn error(reason: Invalid) -> BlockOperationError<Invalid> {
///
/// Optionally verifies the aggregate signature, depending on `verify_signatures`.
///
/// Spec v0.8.0
/// Spec v0.9.1
pub fn verify_attestation_for_block_inclusion<T: EthSpec>(
state: &BeaconState<T>,
attestation: &Attestation<T>,
@ -25,22 +24,19 @@ pub fn verify_attestation_for_block_inclusion<T: EthSpec>(
) -> Result<()> {
let data = &attestation.data;
// Check attestation slot.
let attestation_slot = state.get_attestation_data_slot(&data)?;
verify!(
attestation_slot + spec.min_attestation_inclusion_delay <= state.slot,
data.slot + spec.min_attestation_inclusion_delay <= state.slot,
Invalid::IncludedTooEarly {
state: state.slot,
delay: spec.min_attestation_inclusion_delay,
attestation: attestation_slot
attestation: data.slot,
}
);
verify!(
state.slot <= attestation_slot + T::slots_per_epoch(),
state.slot <= data.slot + T::slots_per_epoch(),
Invalid::IncludedTooLate {
state: state.slot,
attestation: attestation_slot
attestation: data.slot,
}
);
@ -53,7 +49,7 @@ pub fn verify_attestation_for_block_inclusion<T: EthSpec>(
/// Returns a descriptive `Err` if the attestation is malformed or does not accurately reflect the
/// prior blocks in `state`.
///
/// Spec v0.8.0
/// Spec v0.9.1
pub fn verify_attestation_for_state<T: EthSpec>(
state: &BeaconState<T>,
attestation: &Attestation<T>,
@ -63,35 +59,12 @@ pub fn verify_attestation_for_state<T: EthSpec>(
let data = &attestation.data;
verify!(
data.crosslink.shard < T::ShardCount::to_u64(),
Invalid::BadShard
data.index < state.get_committee_count_at_slot(data.slot)?,
Invalid::BadCommitteeIndex
);
// Verify the Casper FFG vote and crosslink data.
let parent_crosslink = verify_casper_ffg_vote(attestation, state)?;
verify!(
data.crosslink.parent_root == Hash256::from_slice(&parent_crosslink.tree_hash_root()),
Invalid::BadParentCrosslinkHash
);
verify!(
data.crosslink.start_epoch == parent_crosslink.end_epoch,
Invalid::BadParentCrosslinkStartEpoch
);
verify!(
data.crosslink.end_epoch
== std::cmp::min(
data.target.epoch,
parent_crosslink.end_epoch + spec.max_epochs_per_crosslink
),
Invalid::BadParentCrosslinkEndEpoch
);
// Crosslink data root is zero (to be removed in phase 1).
verify!(
attestation.data.crosslink.data_root == Hash256::zero(),
Invalid::ShardBlockRootNotZero
);
// Verify the Casper FFG vote.
verify_casper_ffg_vote(attestation, state)?;
// Check signature and bitfields
let indexed_attestation = get_indexed_attestation(state, attestation)?;
@ -102,13 +75,11 @@ pub fn verify_attestation_for_state<T: EthSpec>(
/// Check target epoch and source checkpoint.
///
/// Return the parent crosslink for further checks.
///
/// Spec v0.8.0
fn verify_casper_ffg_vote<'a, T: EthSpec>(
/// Spec v0.9.1
fn verify_casper_ffg_vote<T: EthSpec>(
attestation: &Attestation<T>,
state: &'a BeaconState<T>,
) -> Result<&'a Crosslink> {
state: &BeaconState<T>,
) -> Result<()> {
let data = &attestation.data;
if data.target.epoch == state.current_epoch() {
verify!(
@ -119,7 +90,7 @@ fn verify_casper_ffg_vote<'a, T: EthSpec>(
is_current: true,
}
);
Ok(state.get_current_crosslink(data.crosslink.shard)?)
Ok(())
} else if data.target.epoch == state.previous_epoch() {
verify!(
data.source == state.previous_justified_checkpoint,
@ -129,7 +100,7 @@ fn verify_casper_ffg_vote<'a, T: EthSpec>(
is_current: false,
}
);
Ok(state.get_previous_crosslink(data.crosslink.shard)?)
Ok(())
} else {
Err(error(Invalid::BadTargetEpoch))
}

View File

@ -15,7 +15,7 @@ fn error(reason: Invalid) -> BlockOperationError<Invalid> {
///
/// Returns `Ok(())` if the `AttesterSlashing` is valid, otherwise indicates the reason for invalidity.
///
/// Spec v0.8.1
/// Spec v0.9.1
pub fn verify_attester_slashing<T: EthSpec>(
state: &BeaconState<T>,
attester_slashing: &AttesterSlashing<T>,
@ -47,7 +47,7 @@ pub fn verify_attester_slashing<T: EthSpec>(
///
/// Returns Ok(indices) if `indices.len() > 0`.
///
/// Spec v0.8.1
/// Spec v0.9.1
pub fn get_slashable_indices<T: EthSpec>(
state: &BeaconState<T>,
attester_slashing: &AttesterSlashing<T>,
@ -71,15 +71,13 @@ where
let attestation_2 = &attester_slashing.attestation_2;
let attesting_indices_1 = attestation_1
.custody_bit_0_indices
.attesting_indices
.iter()
.chain(&attestation_1.custody_bit_1_indices)
.cloned()
.collect::<BTreeSet<_>>();
let attesting_indices_2 = attestation_2
.custody_bit_0_indices
.attesting_indices
.iter()
.chain(&attestation_2.custody_bit_1_indices)
.cloned()
.collect::<BTreeSet<_>>();

View File

@ -14,7 +14,7 @@ fn error(reason: DepositInvalid) -> BlockOperationError<DepositInvalid> {
/// Verify `Deposit.pubkey` signed `Deposit.signature`.
///
/// Spec v0.8.0
/// Spec v0.9.1
pub fn verify_deposit_signature(deposit_data: &DepositData, spec: &ChainSpec) -> Result<()> {
let deposit_signature_message = deposit_pubkey_signature_message(&deposit_data)
.ok_or_else(|| error(DepositInvalid::BadBlsBytes))?;
@ -46,7 +46,7 @@ pub fn get_existing_validator_index<T: EthSpec>(
/// The deposit index is provided as a parameter so we can check proofs
/// before they're due to be processed, and in parallel.
///
/// Spec v0.8.0
/// Spec v0.9.1
pub fn verify_deposit_merkle_proof<T: EthSpec>(
state: &BeaconState<T>,
deposit: &Deposit,

View File

@ -13,7 +13,7 @@ fn error(reason: ExitInvalid) -> BlockOperationError<ExitInvalid> {
///
/// Returns `Ok(())` if the `Exit` is valid, otherwise indicates the reason for invalidity.
///
/// Spec v0.8.0
/// Spec v0.9.1
pub fn verify_exit<T: EthSpec>(
state: &BeaconState<T>,
exit: &VoluntaryExit,
@ -25,7 +25,7 @@ pub fn verify_exit<T: EthSpec>(
/// Like `verify_exit` but doesn't run checks which may become true in future states.
///
/// Spec v0.8.0
/// Spec v0.9.1
pub fn verify_exit_time_independent_only<T: EthSpec>(
state: &BeaconState<T>,
exit: &VoluntaryExit,
@ -37,7 +37,7 @@ pub fn verify_exit_time_independent_only<T: EthSpec>(
/// Parametric version of `verify_exit` that skips some checks if `time_independent_only` is true.
///
/// Spec v0.8.0
/// Spec v0.9.1
fn verify_exit_parametric<T: EthSpec>(
state: &BeaconState<T>,
exit: &VoluntaryExit,

View File

@ -14,7 +14,7 @@ fn error(reason: Invalid) -> BlockOperationError<Invalid> {
///
/// Returns `Ok(())` if the `ProposerSlashing` is valid, otherwise indicates the reason for invalidity.
///
/// Spec v0.8.0
/// Spec v0.9.1
pub fn verify_proposer_slashing<T: EthSpec>(
proposer_slashing: &ProposerSlashing,
state: &BeaconState<T>,
@ -26,11 +26,10 @@ pub fn verify_proposer_slashing<T: EthSpec>(
.get(proposer_slashing.proposer_index as usize)
.ok_or_else(|| error(Invalid::ProposerUnknown(proposer_slashing.proposer_index)))?;
// Verify that the epoch is the same
// Verify slots match
verify!(
proposer_slashing.header_1.slot.epoch(T::slots_per_epoch())
== proposer_slashing.header_2.slot.epoch(T::slots_per_epoch()),
Invalid::ProposalEpochMismatch(
proposer_slashing.header_1.slot == proposer_slashing.header_2.slot,
Invalid::ProposalSlotMismatch(
proposer_slashing.header_1.slot,
proposer_slashing.header_2.slot
)

View File

@ -1,208 +0,0 @@
use super::errors::{BlockOperationError, TransferInvalid as Invalid};
use crate::per_block_processing::signature_sets::transfer_signature_set;
use crate::per_block_processing::VerifySignatures;
use bls::get_withdrawal_credentials;
use types::*;
type Result<T> = std::result::Result<T, BlockOperationError<Invalid>>;
fn error(reason: Invalid) -> BlockOperationError<Invalid> {
BlockOperationError::invalid(reason)
}
/// Indicates if a `Transfer` is valid to be included in a block in the current epoch of the given
/// state.
///
/// Returns `Ok(())` if the `Transfer` is valid, otherwise indicates the reason for invalidity.
///
/// Spec v0.8.0
pub fn verify_transfer<T: EthSpec>(
state: &BeaconState<T>,
transfer: &Transfer,
verify_signatures: VerifySignatures,
spec: &ChainSpec,
) -> Result<()> {
verify_transfer_parametric(state, transfer, verify_signatures, spec, false)
}
/// Like `verify_transfer` but doesn't run checks which may become true in future states.
///
/// Spec v0.8.0
pub fn verify_transfer_time_independent_only<T: EthSpec>(
state: &BeaconState<T>,
transfer: &Transfer,
verify_signatures: VerifySignatures,
spec: &ChainSpec,
) -> Result<()> {
verify_transfer_parametric(state, transfer, verify_signatures, spec, true)
}
/// Parametric version of `verify_transfer` that allows some checks to be skipped.
///
/// When `time_independent_only == true`, time-specific parameters are ignored, including:
///
/// - Balance considerations (e.g., adequate balance, not dust, etc).
/// - `transfer.slot` does not have to exactly match `state.slot`, it just needs to be in the
/// present or future.
/// - Validator transfer eligibility (e.g., is withdrawable)
///
/// Spec v0.8.0
fn verify_transfer_parametric<T: EthSpec>(
state: &BeaconState<T>,
transfer: &Transfer,
verify_signatures: VerifySignatures,
spec: &ChainSpec,
time_independent_only: bool,
) -> Result<()> {
let sender_balance = *state
.balances
.get(transfer.sender as usize)
.ok_or_else(|| error(Invalid::FromValidatorUnknown(transfer.sender)))?;
let recipient_balance = *state
.balances
.get(transfer.recipient as usize)
.ok_or_else(|| error(Invalid::FromValidatorUnknown(transfer.recipient)))?;
// Safely determine `amount + fee`.
let total_amount = transfer
.amount
.checked_add(transfer.fee)
.ok_or_else(|| error(Invalid::FeeOverflow(transfer.amount, transfer.fee)))?;
// Verify the sender has adequate balance.
verify!(
time_independent_only || sender_balance >= total_amount,
Invalid::FromBalanceInsufficient(total_amount, sender_balance)
);
// Verify sender balance will not be "dust" (i.e., greater than zero but less than the minimum deposit
// amount).
verify!(
time_independent_only
|| (sender_balance == total_amount)
|| (sender_balance >= (total_amount + spec.min_deposit_amount)),
Invalid::SenderDust(sender_balance - total_amount, spec.min_deposit_amount)
);
// Verify the recipient balance will not be dust.
verify!(
time_independent_only || ((recipient_balance + transfer.amount) >= spec.min_deposit_amount),
Invalid::RecipientDust(sender_balance - total_amount, spec.min_deposit_amount)
);
// If loosely enforcing `transfer.slot`, ensure the slot is not in the past. Otherwise, ensure
// the transfer slot equals the state slot.
if time_independent_only {
verify!(
state.slot <= transfer.slot,
Invalid::TransferSlotInPast(state.slot, transfer.slot)
);
} else {
verify!(
state.slot == transfer.slot,
Invalid::StateSlotMismatch(state.slot, transfer.slot)
);
}
// Load the sender `Validator` record from the state.
let sender_validator = state
.validators
.get(transfer.sender as usize)
.ok_or_else(|| error(Invalid::FromValidatorUnknown(transfer.sender)))?;
// Ensure one of the following is met:
//
// - Time dependent checks are being ignored.
// - The sender has never been eligible for activation.
// - The sender is withdrawable at the state's epoch.
// - The transfer will not reduce the sender below the max effective balance.
verify!(
time_independent_only
|| sender_validator.activation_eligibility_epoch == spec.far_future_epoch
|| sender_validator.is_withdrawable_at(state.current_epoch())
|| total_amount + spec.max_effective_balance <= sender_balance,
Invalid::FromValidatorIneligibleForTransfer(transfer.sender)
);
// Ensure the withdrawal credentials generated from the sender's pubkey match those stored in
// the validator registry.
//
// This ensures the validator can only perform a transfer when they are in control of the
// withdrawal address.
let transfer_withdrawal_credentials = Hash256::from_slice(
&get_withdrawal_credentials(&transfer.pubkey, spec.bls_withdrawal_prefix_byte)[..],
);
verify!(
sender_validator.withdrawal_credentials == transfer_withdrawal_credentials,
Invalid::WithdrawalCredentialsMismatch(
sender_validator.withdrawal_credentials,
transfer_withdrawal_credentials
)
);
if verify_signatures.is_true() {
verify!(
transfer_signature_set(state, transfer, spec)?.is_valid(),
Invalid::BadSignature
);
}
Ok(())
}
/// Executes a transfer on the state.
///
/// Does not check that the transfer is valid, however checks for overflow in all actions.
///
/// Spec v0.8.0
pub fn execute_transfer<T: EthSpec>(
state: &mut BeaconState<T>,
transfer: &Transfer,
spec: &ChainSpec,
) -> Result<()> {
let sender_balance = *state
.balances
.get(transfer.sender as usize)
.ok_or_else(|| error(Invalid::FromValidatorUnknown(transfer.sender)))?;
let recipient_balance = *state
.balances
.get(transfer.recipient as usize)
.ok_or_else(|| error(Invalid::ToValidatorUnknown(transfer.recipient)))?;
let proposer_index =
state.get_beacon_proposer_index(state.slot, RelativeEpoch::Current, spec)?;
let proposer_balance = state.balances[proposer_index];
let total_amount = transfer
.amount
.checked_add(transfer.fee)
.ok_or_else(|| error(Invalid::FeeOverflow(transfer.amount, transfer.fee)))?;
state.balances[transfer.sender as usize] =
sender_balance.checked_sub(total_amount).ok_or_else(|| {
error(Invalid::FromBalanceInsufficient(
total_amount,
sender_balance,
))
})?;
state.balances[transfer.recipient as usize] = recipient_balance
.checked_add(transfer.amount)
.ok_or_else(|| {
error(Invalid::ToBalanceOverflow(
recipient_balance,
transfer.amount,
))
})?;
state.balances[proposer_index] =
proposer_balance.checked_add(transfer.fee).ok_or_else(|| {
error(Invalid::ProposerBalanceOverflow(
proposer_balance,
transfer.fee,
))
})?;
Ok(())
}

View File

@ -1,10 +1,7 @@
use crate::common::get_compact_committees_root;
use errors::EpochProcessingError as Error;
use std::collections::HashMap;
use tree_hash::TreeHash;
use types::*;
use validator_statuses::{TotalBalances, ValidatorStatuses};
use winning_root::{winning_root, WinningRoot};
pub mod apply_rewards;
pub mod errors;
@ -12,23 +9,17 @@ pub mod process_slashings;
pub mod registry_updates;
pub mod tests;
pub mod validator_statuses;
pub mod winning_root;
pub use apply_rewards::process_rewards_and_penalties;
pub use process_slashings::process_slashings;
pub use registry_updates::process_registry_updates;
/// Maps a shard to a winning root.
///
/// It is generated during crosslink processing and later used to reward/penalize validators.
pub type WinningRootHashSet = HashMap<u64, WinningRoot>;
/// Performs per-epoch processing on some BeaconState.
///
/// Mutates the given `BeaconState`, returning early if an error is encountered. If an error is
/// returned, a state might be "half-processed" and therefore in an invalid state.
///
/// Spec v0.8.0
/// Spec v0.9.1
pub fn per_epoch_processing<T: EthSpec>(
state: &mut BeaconState<T>,
spec: &ChainSpec,
@ -47,9 +38,6 @@ pub fn per_epoch_processing<T: EthSpec>(
// Justification and finalization.
process_justification_and_finalization(state, &validator_statuses.total_balances)?;
// Crosslinks.
process_crosslinks(state, spec)?;
// Rewards and Penalties.
process_rewards_and_penalties(state, &mut validator_statuses, spec)?;
@ -78,7 +66,7 @@ pub fn per_epoch_processing<T: EthSpec>(
/// - `finalized_epoch`
/// - `finalized_root`
///
/// Spec v0.8.0
/// Spec v0.9.1
#[allow(clippy::if_same_then_else)] // For readability and consistency with spec.
pub fn process_justification_and_finalization<T: EthSpec>(
state: &mut BeaconState<T>,
@ -144,47 +132,9 @@ pub fn process_justification_and_finalization<T: EthSpec>(
Ok(())
}
/// Updates the following fields on the `BeaconState`:
///
/// - `previous_crosslinks`
/// - `current_crosslinks`
///
/// Also returns a `WinningRootHashSet` for later use during epoch processing.
///
/// Spec v0.8.0
pub fn process_crosslinks<T: EthSpec>(
state: &mut BeaconState<T>,
spec: &ChainSpec,
) -> Result<(), Error> {
state.previous_crosslinks = state.current_crosslinks.clone();
for &relative_epoch in &[RelativeEpoch::Previous, RelativeEpoch::Current] {
let epoch = relative_epoch.into_epoch(state.current_epoch());
for offset in 0..state.get_committee_count(relative_epoch)? {
let shard =
(state.get_epoch_start_shard(relative_epoch)? + offset) % T::ShardCount::to_u64();
let crosslink_committee =
state.get_crosslink_committee_for_shard(shard, relative_epoch)?;
let winning_root = winning_root(state, shard, epoch, spec)?;
if let Some(winning_root) = winning_root {
let total_committee_balance =
state.get_total_balance(&crosslink_committee.committee, spec)?;
if 3 * winning_root.total_attesting_balance >= 2 * total_committee_balance {
state.current_crosslinks[shard as usize] = winning_root.crosslink.clone();
}
}
}
}
Ok(())
}
/// Finish up an epoch update.
///
/// Spec v0.8.0
/// Spec v0.9.1
pub fn process_final_updates<T: EthSpec>(
state: &mut BeaconState<T>,
spec: &ChainSpec,
@ -211,23 +161,6 @@ pub fn process_final_updates<T: EthSpec>(
}
}
// Set active index root
let index_epoch = next_epoch + spec.activation_exit_delay;
let indices_list = VariableList::<usize, T::ValidatorRegistryLimit>::from(
state.get_active_validator_indices(index_epoch),
);
state.set_active_index_root(
index_epoch,
Hash256::from_slice(&indices_list.tree_hash_root()),
spec,
)?;
// Set committees root
state.set_compact_committee_root(
next_epoch,
get_compact_committees_root(state, RelativeEpoch::Next, spec)?,
)?;
// Reset slashings
state.set_slashings(next_epoch, 0)?;
@ -242,9 +175,6 @@ pub fn process_final_updates<T: EthSpec>(
.push(Hash256::from_slice(&historical_batch.tree_hash_root()))?;
}
// Update start shard.
state.start_shard = state.get_epoch_start_shard(RelativeEpoch::Next)?;
// Rotate current/previous epoch attestations
state.previous_epoch_attestations =
std::mem::replace(&mut state.current_epoch_attestations, VariableList::empty());

View File

@ -32,7 +32,7 @@ impl std::ops::AddAssign for Delta {
/// Apply attester and proposer rewards.
///
/// Spec v0.8.0
/// Spec v0.9.1
pub fn process_rewards_and_penalties<T: EthSpec>(
state: &mut BeaconState<T>,
validator_statuses: &mut ValidatorStatuses,
@ -53,11 +53,6 @@ pub fn process_rewards_and_penalties<T: EthSpec>(
get_attestation_deltas(&mut deltas, state, &validator_statuses, spec)?;
// Update statuses with the information from winning roots.
validator_statuses.process_winning_roots(state, spec)?;
get_crosslink_deltas(&mut deltas, state, &validator_statuses, spec)?;
get_proposer_deltas(&mut deltas, state, validator_statuses, spec)?;
// Apply the deltas, over-flowing but not under-flowing (saturating at 0 instead).
@ -71,7 +66,7 @@ pub fn process_rewards_and_penalties<T: EthSpec>(
/// For each attesting validator, reward the proposer who was first to include their attestation.
///
/// Spec v0.8.0
/// Spec v0.9.1
fn get_proposer_deltas<T: EthSpec>(
deltas: &mut Vec<Delta>,
state: &BeaconState<T>,
@ -79,10 +74,10 @@ fn get_proposer_deltas<T: EthSpec>(
spec: &ChainSpec,
) -> Result<(), Error> {
for (index, validator) in validator_statuses.statuses.iter().enumerate() {
if validator.is_previous_epoch_attester {
if validator.is_previous_epoch_attester && !validator.is_slashed {
let inclusion = validator
.inclusion_info
.expect("It is a logic error for an attester not to have an inclusion distance.");
.expect("It is a logic error for an attester not to have an inclusion delay.");
let base_reward = get_base_reward(
state,
@ -104,7 +99,7 @@ fn get_proposer_deltas<T: EthSpec>(
/// Apply rewards for participation in attestations during the previous epoch.
///
/// Spec v0.8.0
/// Spec v0.9.1
fn get_attestation_deltas<T: EthSpec>(
deltas: &mut Vec<Delta>,
state: &BeaconState<T>,
@ -137,7 +132,7 @@ fn get_attestation_deltas<T: EthSpec>(
/// Determine the delta for a single validator, sans proposer rewards.
///
/// Spec v0.8.0
/// Spec v0.9.1
fn get_attestation_delta<T: EthSpec>(
validator: &ValidatorStatus,
total_balances: &TotalBalances,
@ -171,13 +166,8 @@ fn get_attestation_delta<T: EthSpec>(
let max_attester_reward = base_reward - proposer_reward;
let inclusion = validator
.inclusion_info
.expect("It is a logic error for an attester not to have an inclusion distance.");
delta.reward(
max_attester_reward
* (T::SlotsPerEpoch::to_u64() + spec.min_attestation_inclusion_delay
- inclusion.distance)
/ T::SlotsPerEpoch::to_u64(),
);
.expect("It is a logic error for an attester not to have an inclusion delay.");
delta.reward(max_attester_reward / inclusion.delay);
} else {
delta.penalize(base_reward);
}
@ -222,43 +212,9 @@ fn get_attestation_delta<T: EthSpec>(
delta
}
/// Calculate the deltas based upon the winning roots for attestations during the previous epoch.
///
/// Spec v0.8.0
fn get_crosslink_deltas<T: EthSpec>(
deltas: &mut Vec<Delta>,
state: &BeaconState<T>,
validator_statuses: &ValidatorStatuses,
spec: &ChainSpec,
) -> Result<(), Error> {
for (index, validator) in validator_statuses.statuses.iter().enumerate() {
let mut delta = Delta::default();
let base_reward = get_base_reward(
state,
index,
validator_statuses.total_balances.current_epoch,
spec,
)?;
if let Some(ref winning_root) = validator.winning_root_info {
delta.reward(
base_reward * winning_root.total_attesting_balance
/ winning_root.total_committee_balance,
);
} else {
delta.penalize(base_reward);
}
deltas[index] += delta;
}
Ok(())
}
/// Returns the base reward for some validator.
///
/// Spec v0.8.0
/// Spec v0.9.1
fn get_base_reward<T: EthSpec>(
state: &BeaconState<T>,
index: usize,

View File

@ -2,7 +2,7 @@ use types::{BeaconStateError as Error, *};
/// Process slashings.
///
/// Spec v0.8.0
/// Spec v0.9.1
pub fn process_slashings<T: EthSpec>(
state: &mut BeaconState<T>,
total_balance: u64,

View File

@ -5,7 +5,7 @@ use types::*;
/// Performs a validator registry update, if required.
///
/// Spec v0.8.0
/// Spec v0.9.1
pub fn process_registry_updates<T: EthSpec>(
state: &mut BeaconState<T>,
spec: &ChainSpec,

View File

@ -1,4 +1,3 @@
use super::{winning_root::winning_root, WinningRootHashSet};
use crate::common::get_attesting_indices;
use types::*;
@ -12,34 +11,21 @@ macro_rules! set_self_if_other_is_true {
};
}
/// The information required to reward some validator for their participation in a "winning"
/// crosslink root.
#[derive(Default, Clone)]
pub struct WinningRootInfo {
/// The total balance of the crosslink committee.
pub total_committee_balance: u64,
/// The total balance of the crosslink committee that attested for the "winning" root.
pub total_attesting_balance: u64,
}
/// The information required to reward a block producer for including an attestation in a block.
#[derive(Clone, Copy)]
pub struct InclusionInfo {
/// The earliest slot a validator had an attestation included in the previous epoch.
pub slot: Slot,
/// The distance between the attestation slot and the slot that attestation was included in a
/// block.
pub distance: u64,
pub delay: u64,
/// The index of the proposer at the slot where the attestation was included.
pub proposer_index: usize,
}
impl Default for InclusionInfo {
/// Defaults to `slot` and `distance` at their maximum values and `proposer_index` at zero.
/// Defaults to `delay` at its maximum value and `proposer_index` at zero.
fn default() -> Self {
Self {
slot: Slot::max_value(),
distance: u64::max_value(),
delay: u64::max_value(),
proposer_index: 0,
}
}
@ -49,9 +35,8 @@ impl InclusionInfo {
/// Tests if some `other` `InclusionInfo` has a lower inclusion slot than `self`. If so,
/// replaces `self` with `other`.
pub fn update(&mut self, other: &Self) {
if other.slot < self.slot {
self.slot = other.slot;
self.distance = other.distance;
if other.delay < self.delay {
self.delay = other.delay;
self.proposer_index = other.proposer_index;
}
}
@ -88,9 +73,6 @@ pub struct ValidatorStatus {
/// Information used to reward the block producer of this validators earliest-included
/// attestation.
pub inclusion_info: Option<InclusionInfo>,
/// Information used to reward/penalize the validator if they voted in the super-majority for
/// some shard block.
pub winning_root_info: Option<WinningRootInfo>,
}
impl ValidatorStatus {
@ -162,7 +144,7 @@ impl ValidatorStatuses {
/// - Active validators
/// - Total balances for the current and previous epochs.
///
/// Spec v0.8.1
/// Spec v0.9.1
pub fn new<T: EthSpec>(
state: &BeaconState<T>,
spec: &ChainSpec,
@ -202,7 +184,7 @@ impl ValidatorStatuses {
/// Process some attestations from the given `state` updating the `statuses` and
/// `total_balances` fields.
///
/// Spec v0.8.1
/// Spec v0.9.1
pub fn process_attestations<T: EthSpec>(
&mut self,
state: &BeaconState<T>,
@ -228,19 +210,11 @@ impl ValidatorStatuses {
} else if a.data.target.epoch == state.previous_epoch() {
status.is_previous_epoch_attester = true;
// The inclusion slot and distance are only required for previous epoch attesters.
let attestation_slot = state.get_attestation_data_slot(&a.data)?;
let inclusion_slot = attestation_slot + a.inclusion_delay;
let relative_epoch =
RelativeEpoch::from_slot(state.slot, inclusion_slot, T::slots_per_epoch())?;
// The inclusion delay and proposer index are only required for previous epoch
// attesters.
status.inclusion_info = Some(InclusionInfo {
slot: inclusion_slot,
distance: a.inclusion_delay,
proposer_index: state.get_beacon_proposer_index(
inclusion_slot,
relative_epoch,
spec,
)?,
delay: a.inclusion_delay,
proposer_index: a.proposer_index as usize,
});
if target_matches_epoch_start_block(a, state, state.previous_epoch())? {
@ -284,66 +258,12 @@ impl ValidatorStatuses {
Ok(())
}
/// Update the `statuses` for each validator based upon whether or not they attested to the
/// "winning" shard block root for the previous epoch.
///
/// Spec v0.8.1
pub fn process_winning_roots<T: EthSpec>(
&mut self,
state: &BeaconState<T>,
spec: &ChainSpec,
) -> Result<(), BeaconStateError> {
// We must re-calculate the winning roots here because it is possible that they have
// changed since the first time they were calculated.
//
// This is because we altered the state during the first time we calculated the winning
// roots.
let winning_root_for_shards = {
let mut winning_root_for_shards = WinningRootHashSet::new();
let relative_epoch = RelativeEpoch::Previous;
let epoch = relative_epoch.into_epoch(state.current_epoch());
for offset in 0..state.get_committee_count(relative_epoch)? {
let shard = (state.get_epoch_start_shard(relative_epoch)? + offset)
% T::ShardCount::to_u64();
if let Some(winning_root) = winning_root(state, shard, epoch, spec)? {
winning_root_for_shards.insert(shard, winning_root);
}
}
winning_root_for_shards
};
// Loop through each slot in the previous epoch.
for slot in state.previous_epoch().slot_iter(T::slots_per_epoch()) {
let crosslink_committees_at_slot = state.get_crosslink_committees_at_slot(slot)?;
// Loop through each committee in the slot.
for c in crosslink_committees_at_slot {
// If there was some winning crosslink root for the committee's shard.
if let Some(winning_root) = winning_root_for_shards.get(&c.shard) {
let total_committee_balance = state.get_total_balance(&c.committee, spec)?;
for &validator_index in &winning_root.attesting_validator_indices {
// Take note of the balance information for the winning root, it will be
// used later to calculate rewards for that validator.
self.statuses[validator_index].winning_root_info = Some(WinningRootInfo {
total_committee_balance,
total_attesting_balance: winning_root.total_attesting_balance,
})
}
}
}
}
Ok(())
}
}
/// Returns `true` if the attestation's FFG target is equal to the hash of the `state`'s first
/// beacon block in the given `epoch`.
///
/// Spec v0.8.1
/// Spec v0.9.1
fn target_matches_epoch_start_block<T: EthSpec>(
a: &PendingAttestation<T>,
state: &BeaconState<T>,
@ -358,13 +278,12 @@ fn target_matches_epoch_start_block<T: EthSpec>(
/// Returns `true` if a `PendingAttestation` and `BeaconState` share the same beacon block hash for
/// the current slot of the `PendingAttestation`.
///
/// Spec v0.8.1
/// Spec v0.9.1
fn has_common_beacon_block_root<T: EthSpec>(
a: &PendingAttestation<T>,
state: &BeaconState<T>,
) -> Result<bool, BeaconStateError> {
let attestation_slot = state.get_attestation_data_slot(&a.data)?;
let state_block_root = *state.get_block_root(attestation_slot)?;
let state_block_root = *state.get_block_root(a.data.slot)?;
Ok(a.data.beacon_block_root == state_block_root)
}

View File

@ -1,130 +0,0 @@
use crate::common::get_attesting_indices;
use std::collections::{HashMap, HashSet};
use tree_hash::TreeHash;
use types::*;
#[derive(Clone, Debug)]
pub struct WinningRoot {
pub crosslink: Crosslink,
pub attesting_validator_indices: Vec<usize>,
pub total_attesting_balance: u64,
}
impl WinningRoot {
/// Returns `true` if `self` is a "better" candidate than `other`.
///
/// A winning root is "better" than another if it has a higher `total_attesting_balance`. Ties
/// are broken by favouring the higher `crosslink_data_root` value.
///
/// Spec v0.8.0
pub fn is_better_than(&self, other: &Self) -> bool {
(self.total_attesting_balance, self.crosslink.data_root)
> (other.total_attesting_balance, other.crosslink.data_root)
}
}
/// Returns the crosslink `data_root` with the highest total attesting balance for the given shard.
/// Breaks ties by favouring the smaller crosslink `data_root` hash.
///
/// The `WinningRoot` object also contains additional fields that are useful in later stages of
/// per-epoch processing.
///
/// Spec v0.8.0
pub fn winning_root<T: EthSpec>(
state: &BeaconState<T>,
shard: u64,
epoch: Epoch,
spec: &ChainSpec,
) -> Result<Option<WinningRoot>, BeaconStateError> {
let attestations: Vec<&_> = state
.get_matching_source_attestations(epoch)?
.iter()
.filter(|a| a.data.crosslink.shard == shard)
.collect();
// Build a map from crosslinks to attestations that support that crosslink.
let mut candidate_crosslink_map = HashMap::new();
let current_shard_crosslink_root = state.get_current_crosslink(shard)?.tree_hash_root();
for a in attestations {
if a.data.crosslink.parent_root.as_bytes() == &current_shard_crosslink_root[..]
|| a.data.crosslink.tree_hash_root() == current_shard_crosslink_root
{
let supporting_attestations = candidate_crosslink_map
.entry(&a.data.crosslink)
.or_insert_with(Vec::new);
supporting_attestations.push(a);
}
}
// Find the maximum crosslink.
let mut winning_root = None;
for (crosslink, attestations) in candidate_crosslink_map {
let attesting_validator_indices =
get_unslashed_attesting_indices_unsorted(state, &attestations)?;
let total_attesting_balance =
state.get_total_balance(&attesting_validator_indices, spec)?;
let candidate = WinningRoot {
crosslink: crosslink.clone(),
attesting_validator_indices,
total_attesting_balance,
};
if let Some(ref winner) = winning_root {
if candidate.is_better_than(&winner) {
winning_root = Some(candidate);
}
} else {
winning_root = Some(candidate);
}
}
Ok(winning_root)
}
pub fn get_unslashed_attesting_indices_unsorted<T: EthSpec>(
state: &BeaconState<T>,
attestations: &[&PendingAttestation<T>],
) -> Result<Vec<usize>, BeaconStateError> {
let mut output = HashSet::new();
for a in attestations {
output.extend(get_attesting_indices(state, &a.data, &a.aggregation_bits)?);
}
Ok(output
.into_iter()
.filter(|index| state.validators.get(*index).map_or(false, |v| !v.slashed))
.collect())
}
#[cfg(test)]
mod tests {
use super::*;
#[test]
fn is_better_than() {
let worse = WinningRoot {
crosslink: Crosslink {
shard: 0,
start_epoch: Epoch::new(0),
end_epoch: Epoch::new(1),
parent_root: Hash256::from_slice(&[0; 32]),
data_root: Hash256::from_slice(&[1; 32]),
},
attesting_validator_indices: vec![],
total_attesting_balance: 42,
};
let mut better = worse.clone();
better.crosslink.data_root = Hash256::from_slice(&[2; 32]);
assert!(better.is_better_than(&worse));
let better = WinningRoot {
total_attesting_balance: worse.total_attesting_balance + 1,
..worse.clone()
};
assert!(better.is_better_than(&worse));
}
}

View File

@ -9,7 +9,7 @@ pub enum Error {
/// Advances a state forward by one slot, performing per-epoch processing if required.
///
/// Spec v0.8.0
/// Spec v0.9.1
pub fn per_slot_processing<T: EthSpec>(
state: &mut BeaconState<T>,
spec: &ChainSpec,

View File

@ -15,7 +15,6 @@ pub struct BlockBuilder<T: EthSpec> {
pub num_attestations: usize,
pub num_deposits: usize,
pub num_exits: usize,
pub num_transfers: usize,
}
impl<T: EthSpec> BlockBuilder<T> {
@ -33,7 +32,6 @@ impl<T: EthSpec> BlockBuilder<T> {
num_attestations: 0,
num_deposits: 0,
num_exits: 0,
num_transfers: 0,
}
}
@ -43,7 +41,6 @@ impl<T: EthSpec> BlockBuilder<T> {
self.num_attestations = T::MaxAttestations::to_usize();
self.num_deposits = T::MaxDeposits::to_usize();
self.num_exits = T::MaxVoluntaryExits::to_usize();
self.num_transfers = T::MaxTransfers::to_usize();
}
pub fn set_slot(&mut self, slot: Slot) {
@ -61,9 +58,7 @@ impl<T: EthSpec> BlockBuilder<T> {
builder.set_slot(state.slot);
let proposer_index = state
.get_beacon_proposer_index(state.slot, RelativeEpoch::Current, spec)
.unwrap();
let proposer_index = state.get_beacon_proposer_index(state.slot, spec).unwrap();
let proposer_keypair = &keypairs[proposer_index];
@ -80,7 +75,7 @@ impl<T: EthSpec> BlockBuilder<T> {
let validator_index = validators_iter.next().expect("Insufficient validators.");
builder.insert_proposer_slashing(
&ProposerSlashingTestTask::Valid,
ProposerSlashingTestTask::Valid,
validator_index,
&keypairs[validator_index as usize].sk,
&state.fork,
@ -107,7 +102,7 @@ impl<T: EthSpec> BlockBuilder<T> {
}
builder.insert_attester_slashing(
&AttesterSlashingTestTask::Valid,
AttesterSlashingTestTask::Valid,
&attesters,
&secret_keys,
&state.fork,
@ -123,7 +118,7 @@ impl<T: EthSpec> BlockBuilder<T> {
let all_secret_keys: Vec<&SecretKey> = keypairs.iter().map(|keypair| &keypair.sk).collect();
builder
.insert_attestations(
&AttestationTestTask::Valid,
AttestationTestTask::Valid,
&state,
&all_secret_keys,
self.num_attestations as usize,
@ -151,7 +146,7 @@ impl<T: EthSpec> BlockBuilder<T> {
let validator_index = validators_iter.next().expect("Insufficient validators.");
builder.insert_exit(
&ExitTestTask::Valid,
ExitTestTask::Valid,
&mut state,
validator_index,
&keypairs[validator_index as usize].sk,
@ -163,24 +158,6 @@ impl<T: EthSpec> BlockBuilder<T> {
builder.block.body.voluntary_exits.len()
);
// Insert the maximum possible number of `Transfer` objects.
for _ in 0..self.num_transfers {
let validator_index = validators_iter.next().expect("Insufficient validators.");
// Manually set the validator to be withdrawn.
state.validators[validator_index as usize].withdrawable_epoch = state.previous_epoch();
builder.insert_transfer(
&state,
validator_index,
validator_index,
1,
keypairs[validator_index as usize].clone(),
spec,
);
}
info!("Inserted {} transfers.", builder.block.body.transfers.len());
// Set the eth1 data to be different from the state.
self.block_builder.block.body.eth1_data.block_hash = Hash256::from_slice(&[42; 32]);

View File

@ -205,6 +205,4 @@ mod signatures_minimal {
spec,
);
}
// Cannot test transfers because their length is zero.
}

View File

@ -9,7 +9,7 @@ use tree_hash_derive::{SignedRoot, TreeHash};
/// Details an attestation that can be slashable.
///
/// Spec v0.8.0
/// Spec v0.9.1
#[derive(
Debug,
Clone,
@ -26,7 +26,6 @@ use tree_hash_derive::{SignedRoot, TreeHash};
pub struct Attestation<T: EthSpec> {
pub aggregation_bits: BitList<T::MaxValidatorsPerCommittee>,
pub data: AttestationData,
pub custody_bits: BitList<T::MaxValidatorsPerCommittee>,
#[signed_root(skip_hashing)]
pub signature: AggregateSignature,
}
@ -47,7 +46,6 @@ impl<T: EthSpec> Attestation<T> {
debug_assert!(self.signers_disjoint_from(other));
self.aggregation_bits = self.aggregation_bits.union(&other.aggregation_bits);
self.custody_bits = self.custody_bits.union(&other.custody_bits);
self.signature.add_aggregate(&other.signature);
}
}

View File

@ -1,5 +1,5 @@
use crate::test_utils::TestRandom;
use crate::{Checkpoint, Crosslink, Hash256};
use crate::{Checkpoint, Hash256, Slot};
use serde_derive::{Deserialize, Serialize};
use ssz_derive::{Decode, Encode};
@ -8,20 +8,20 @@ use tree_hash_derive::TreeHash;
/// The data upon which an attestation is based.
///
/// Spec v0.8.0
/// Spec v0.9.1
#[derive(
Debug, Clone, PartialEq, Eq, Serialize, Deserialize, Hash, Encode, Decode, TreeHash, TestRandom,
)]
pub struct AttestationData {
pub slot: Slot,
pub index: u64,
// LMD GHOST vote
pub beacon_block_root: Hash256,
// FFG Vote
pub source: Checkpoint,
pub target: Checkpoint,
// Crosslink Vote
pub crosslink: Crosslink,
}
#[cfg(test)]

View File

@ -1,22 +0,0 @@
use super::AttestationData;
use crate::test_utils::TestRandom;
use serde_derive::{Deserialize, Serialize};
use ssz_derive::{Decode, Encode};
use test_random_derive::TestRandom;
use tree_hash_derive::TreeHash;
/// Used for pairing an attestation with a proof-of-custody.
///
/// Spec v0.8.1
#[derive(Debug, Clone, PartialEq, Serialize, Deserialize, Encode, Decode, TreeHash, TestRandom)]
pub struct AttestationDataAndCustodyBit {
pub data: AttestationData,
pub custody_bit: bool,
}
#[cfg(test)]
mod test {
use super::*;
ssz_tests!(AttestationDataAndCustodyBit);
}

View File

@ -3,8 +3,12 @@ use serde_derive::{Deserialize, Serialize};
#[derive(Debug, PartialEq, Clone, Copy, Default, Serialize, Deserialize)]
pub struct AttestationDuty {
/// The slot during which the attester must attest.
pub slot: Slot,
pub shard: Shard,
pub committee_index: usize,
/// The index of this committee within the committees in `slot`.
pub index: CommitteeIndex,
/// The position of the attester within the committee.
pub committee_position: usize,
/// The total number of attesters in the committee.
pub committee_len: usize,
}

View File

@ -7,7 +7,7 @@ use tree_hash_derive::TreeHash;
/// Two conflicting attestations.
///
/// Spec v0.8.0
/// Spec v0.9.1
#[derive(Debug, PartialEq, Clone, Serialize, Deserialize, Encode, Decode, TreeHash, TestRandom)]
#[serde(bound = "T: EthSpec")]
pub struct AttesterSlashing<T: EthSpec> {

View File

@ -10,7 +10,7 @@ use tree_hash_derive::{SignedRoot, TreeHash};
/// A block of the `BeaconChain`.
///
/// Spec v0.8.0
/// Spec v0.9.1
#[derive(
Debug,
PartialEq,
@ -36,7 +36,7 @@ pub struct BeaconBlock<T: EthSpec> {
impl<T: EthSpec> BeaconBlock<T> {
/// Returns an empty block to be used during genesis.
///
/// Spec v0.8.1
/// Spec v0.9.1
pub fn empty(spec: &ChainSpec) -> Self {
BeaconBlock {
slot: spec.genesis_slot,
@ -55,7 +55,6 @@ impl<T: EthSpec> BeaconBlock<T> {
attestations: VariableList::empty(),
deposits: VariableList::empty(),
voluntary_exits: VariableList::empty(),
transfers: VariableList::empty(),
},
signature: Signature::empty_signature(),
}
@ -68,7 +67,7 @@ impl<T: EthSpec> BeaconBlock<T> {
/// Returns the `signed_root` of the block.
///
/// Spec v0.8.1
/// Spec v0.9.1
pub fn canonical_root(&self) -> Hash256 {
Hash256::from_slice(&self.signed_root()[..])
}
@ -80,7 +79,7 @@ impl<T: EthSpec> BeaconBlock<T> {
///
/// Note: performs a full tree-hash of `self.body`.
///
/// Spec v0.8.1
/// Spec v0.9.1
pub fn block_header(&self) -> BeaconBlockHeader {
BeaconBlockHeader {
slot: self.slot,
@ -93,7 +92,7 @@ impl<T: EthSpec> BeaconBlock<T> {
/// Returns a "temporary" header, where the `state_root` is `Hash256::zero()`.
///
/// Spec v0.8.0
/// Spec v0.9.1
pub fn temporary_block_header(&self) -> BeaconBlockHeader {
BeaconBlockHeader {
state_root: Hash256::zero(),

View File

@ -10,7 +10,7 @@ use tree_hash_derive::TreeHash;
/// The body of a `BeaconChain` block, containing operations.
///
/// Spec v0.8.0
/// Spec v0.9.1
#[derive(Debug, PartialEq, Clone, Serialize, Deserialize, Encode, Decode, TreeHash, TestRandom)]
#[serde(bound = "T: EthSpec")]
pub struct BeaconBlockBody<T: EthSpec> {
@ -26,7 +26,6 @@ pub struct BeaconBlockBody<T: EthSpec> {
pub attestations: VariableList<Attestation<T>, T::MaxAttestations>,
pub deposits: VariableList<Deposit, T::MaxDeposits>,
pub voluntary_exits: VariableList<VoluntaryExit, T::MaxVoluntaryExits>,
pub transfers: VariableList<Transfer, T::MaxTransfers>,
}
#[cfg(test)]

View File

@ -10,7 +10,7 @@ use tree_hash_derive::{SignedRoot, TreeHash};
/// A header of a `BeaconBlock`.
///
/// Spec v0.8.1
/// Spec v0.9.1
#[derive(
Debug,
PartialEq,
@ -35,14 +35,14 @@ pub struct BeaconBlockHeader {
impl BeaconBlockHeader {
/// Returns the `tree_hash_root` of the header.
///
/// Spec v0.8.1
/// Spec v0.9.1
pub fn canonical_root(&self) -> Hash256 {
Hash256::from_slice(&self.signed_root()[..])
}
/// Given a `body`, consumes `self` and returns a complete `BeaconBlock`.
///
/// Spec v0.8.0
/// Spec v0.9.1
pub fn into_block<T: EthSpec>(self, body: BeaconBlockBody<T>) -> BeaconBlock<T> {
BeaconBlock {
slot: self.slot,

View File

@ -1,25 +1,25 @@
use crate::*;
#[derive(Default, Clone, Debug, PartialEq)]
pub struct CrosslinkCommittee<'a> {
pub struct BeaconCommittee<'a> {
pub slot: Slot,
pub shard: Shard,
pub index: CommitteeIndex,
pub committee: &'a [usize],
}
impl<'a> CrosslinkCommittee<'a> {
pub fn into_owned(self) -> OwnedCrosslinkCommittee {
OwnedCrosslinkCommittee {
impl<'a> BeaconCommittee<'a> {
pub fn into_owned(self) -> OwnedBeaconCommittee {
OwnedBeaconCommittee {
slot: self.slot,
shard: self.shard,
index: self.index,
committee: self.committee.to_vec(),
}
}
}
#[derive(Default, Clone, Debug, PartialEq)]
pub struct OwnedCrosslinkCommittee {
pub struct OwnedBeaconCommittee {
pub slot: Slot,
pub shard: Shard,
pub index: CommitteeIndex,
pub committee: Vec<usize>,
}

View File

@ -5,12 +5,13 @@ use crate::*;
use cached_tree_hash::{CachedTreeHash, MultiTreeHashCache, TreeHashCache};
use compare_fields_derive::CompareFields;
use eth2_hashing::hash;
use int_to_bytes::{int_to_bytes32, int_to_bytes8};
use int_to_bytes::{int_to_bytes4, int_to_bytes8};
use pubkey_cache::PubkeyCache;
use serde_derive::{Deserialize, Serialize};
use ssz::ssz_encode;
use ssz_derive::{Decode, Encode};
use ssz_types::{typenum::Unsigned, BitVector, FixedVector};
use swap_or_not_shuffle::compute_shuffled_index;
use test_random_derive::TestRandom;
use tree_hash::TreeHash;
use tree_hash_derive::{CachedTreeHash, TreeHash};
@ -31,7 +32,6 @@ const MAX_RANDOM_BYTE: u64 = (1 << 8) - 1;
pub enum Error {
EpochOutOfBounds,
SlotOutOfBounds,
ShardOutOfBounds,
UnknownValidator,
UnableToDetermineProducer,
InvalidBitfield,
@ -45,8 +45,10 @@ pub enum Error {
InsufficientAttestations,
InsufficientCommittees,
InsufficientStateRoots,
NoCommitteeForShard,
NoCommitteeForSlot,
NoCommittee {
slot: Slot,
index: CommitteeIndex,
},
ZeroSlotsPerEpoch,
PubkeyCacheInconsistent,
PubkeyCacheIncomplete {
@ -56,7 +58,7 @@ pub enum Error {
PreviousCommitteeCacheUninitialized,
CurrentCommitteeCacheUninitialized,
RelativeEpochError(RelativeEpochError),
CommitteeCacheUninitialized(RelativeEpoch),
CommitteeCacheUninitialized(Option<RelativeEpoch>),
SszTypesError(ssz_types::Error),
CachedTreeHashError(cached_tree_hash::Error),
}
@ -86,8 +88,6 @@ pub struct BeaconTreeHashCache {
validators: MultiTreeHashCache,
balances: TreeHashCache,
randao_mixes: TreeHashCache,
active_index_roots: TreeHashCache,
compact_committees_roots: TreeHashCache,
slashings: TreeHashCache,
}
@ -99,7 +99,7 @@ impl BeaconTreeHashCache {
/// The state of the `BeaconChain` at some slot.
///
/// Spec v0.8.0
/// Spec v0.9.1
#[derive(
Debug,
PartialEq,
@ -148,16 +148,9 @@ where
#[cached_tree_hash(balances)]
pub balances: VariableList<u64, T::ValidatorRegistryLimit>,
// Shuffling
pub start_shard: u64,
// Randomness
#[cached_tree_hash(randao_mixes)]
pub randao_mixes: FixedVector<Hash256, T::EpochsPerHistoricalVector>,
#[compare_fields(as_slice)]
#[cached_tree_hash(active_index_roots)]
pub active_index_roots: FixedVector<Hash256, T::EpochsPerHistoricalVector>,
#[compare_fields(as_slice)]
#[cached_tree_hash(compact_committees_roots)]
pub compact_committees_roots: FixedVector<Hash256, T::EpochsPerHistoricalVector>,
// Slashings
#[cached_tree_hash(slashings)]
@ -167,10 +160,6 @@ where
pub previous_epoch_attestations: VariableList<PendingAttestation<T>, T::MaxPendingAttestations>,
pub current_epoch_attestations: VariableList<PendingAttestation<T>, T::MaxPendingAttestations>,
// Crosslinks
pub previous_crosslinks: FixedVector<Crosslink, T::ShardCount>,
pub current_crosslinks: FixedVector<Crosslink, T::ShardCount>,
// Finality
#[test_random(default)]
pub justification_bits: BitVector<T::JustificationBitsLength>,
@ -210,7 +199,7 @@ impl<T: EthSpec> BeaconState<T> {
///
/// Not a complete genesis state, see `initialize_beacon_state_from_eth1`.
///
/// Spec v0.8.0
/// Spec v0.9.1
pub fn new(genesis_time: u64, eth1_data: Eth1Data, spec: &ChainSpec) -> Self {
BeaconState {
// Versioning
@ -233,11 +222,8 @@ impl<T: EthSpec> BeaconState<T> {
validators: VariableList::empty(), // Set later.
balances: VariableList::empty(), // Set later.
// Shuffling
start_shard: 0,
// Randomness
randao_mixes: FixedVector::from_elem(Hash256::zero()),
active_index_roots: FixedVector::from_elem(Hash256::zero()),
compact_committees_roots: FixedVector::from_elem(Hash256::zero()),
// Slashings
slashings: FixedVector::from_elem(0),
@ -246,10 +232,6 @@ impl<T: EthSpec> BeaconState<T> {
previous_epoch_attestations: VariableList::empty(),
current_epoch_attestations: VariableList::empty(),
// Crosslinks
previous_crosslinks: FixedVector::from_elem(Crosslink::default()),
current_crosslinks: FixedVector::from_elem(Crosslink::default()),
// Finality
justification_bits: BitVector::new(),
previous_justified_checkpoint: Checkpoint::default(),
@ -270,7 +252,7 @@ impl<T: EthSpec> BeaconState<T> {
/// Returns the `tree_hash_root` of the state.
///
/// Spec v0.8.1
/// Spec v0.9.1
pub fn canonical_root(&self) -> Hash256 {
Hash256::from_slice(&self.tree_hash_root()[..])
}
@ -299,7 +281,7 @@ impl<T: EthSpec> BeaconState<T> {
/// The epoch corresponding to `self.slot`.
///
/// Spec v0.8.1
/// Spec v0.9.1
pub fn current_epoch(&self) -> Epoch {
self.slot.epoch(T::slots_per_epoch())
}
@ -308,7 +290,7 @@ impl<T: EthSpec> BeaconState<T> {
///
/// If the current epoch is the genesis epoch, the genesis_epoch is returned.
///
/// Spec v0.8.1
/// Spec v0.9.1
pub fn previous_epoch(&self) -> Epoch {
let current_epoch = self.current_epoch();
if current_epoch > T::genesis_epoch() {
@ -320,43 +302,29 @@ impl<T: EthSpec> BeaconState<T> {
/// The epoch following `self.current_epoch()`.
///
/// Spec v0.8.1
/// Spec v0.9.1
pub fn next_epoch(&self) -> Epoch {
self.current_epoch() + 1
}
pub fn get_committee_count(&self, relative_epoch: RelativeEpoch) -> Result<u64, Error> {
let cache = self.cache(relative_epoch)?;
/// Compute the number of committees at `slot`.
///
/// Makes use of the committee cache and will fail if no cache exists for the slot's epoch.
///
/// Spec v0.9.1
pub fn get_committee_count_at_slot(&self, slot: Slot) -> Result<u64, Error> {
let cache = self.committee_cache_at_slot(slot)?;
Ok(cache.committees_per_slot() as u64)
}
/// Compute the number of committees in an entire epoch.
///
/// Spec v0.9.1
pub fn get_epoch_committee_count(&self, relative_epoch: RelativeEpoch) -> Result<u64, Error> {
let cache = self.committee_cache(relative_epoch)?;
Ok(cache.epoch_committee_count() as u64)
}
pub fn get_epoch_start_shard(&self, relative_epoch: RelativeEpoch) -> Result<u64, Error> {
let cache = self.cache(relative_epoch)?;
Ok(cache.epoch_start_shard())
}
/// Get the slot of an attestation.
///
/// Note: Utilizes the cache and will fail if the appropriate cache is not initialized.
///
/// Spec v0.8.0
pub fn get_attestation_data_slot(
&self,
attestation_data: &AttestationData,
) -> Result<Slot, Error> {
let target_relative_epoch =
RelativeEpoch::from_epoch(self.current_epoch(), attestation_data.target.epoch)?;
let cc = self.get_crosslink_committee_for_shard(
attestation_data.crosslink.shard,
target_relative_epoch,
)?;
Ok(cc.slot)
}
/// Return the cached active validator indices at some epoch.
///
/// Note: the indices are shuffled (i.e., not in ascending order).
@ -366,7 +334,7 @@ impl<T: EthSpec> BeaconState<T> {
&self,
relative_epoch: RelativeEpoch,
) -> Result<&[usize], Error> {
let cache = self.cache(relative_epoch)?;
let cache = self.committee_cache(relative_epoch)?;
Ok(&cache.active_validator_indices())
}
@ -375,7 +343,7 @@ impl<T: EthSpec> BeaconState<T> {
///
/// Does not utilize the cache, performs a full iteration over the validator registry.
///
/// Spec v0.8.1
/// Spec v0.9.1
pub fn get_active_validator_indices(&self, epoch: Epoch) -> Vec<usize> {
get_active_validator_indices(&self.validators, epoch)
}
@ -386,91 +354,106 @@ impl<T: EthSpec> BeaconState<T> {
///
/// Returns an error if that epoch is not cached, or the cache is not initialized.
pub fn get_shuffling(&self, relative_epoch: RelativeEpoch) -> Result<&[usize], Error> {
let cache = self.cache(relative_epoch)?;
let cache = self.committee_cache(relative_epoch)?;
Ok(cache.shuffling())
}
/// Returns the crosslink committees for some slot.
/// Get the Beacon committee at the given slot and index.
///
/// Note: Utilizes the cache and will fail if the appropriate cache is not initialized.
/// Utilises the committee cache.
///
/// Spec v0.8.1
pub fn get_crosslink_committees_at_slot(
/// Spec v0.9.1
pub fn get_beacon_committee(
&self,
slot: Slot,
) -> Result<Vec<CrosslinkCommittee>, Error> {
let relative_epoch = RelativeEpoch::from_slot(self.slot, slot, T::slots_per_epoch())?;
let cache = self.cache(relative_epoch)?;
index: CommitteeIndex,
) -> Result<BeaconCommittee, Error> {
let epoch = slot.epoch(T::slots_per_epoch());
let relative_epoch = RelativeEpoch::from_epoch(self.current_epoch(), epoch)?;
let cache = self.committee_cache(relative_epoch)?;
cache
.get_crosslink_committees_for_slot(slot)
.ok_or_else(|| Error::NoCommitteeForSlot)
.get_beacon_committee(slot, index)
.ok_or(Error::NoCommittee { slot, index })
}
/// Returns the crosslink committees for some shard in some cached epoch.
/// Get all of the Beacon committees at a given slot.
///
/// Note: Utilizes the cache and will fail if the appropriate cache is not initialized.
/// Utilises the committee cache.
///
/// Spec v0.8.1
pub fn get_crosslink_committee_for_shard(
&self,
shard: u64,
relative_epoch: RelativeEpoch,
) -> Result<CrosslinkCommittee, Error> {
let cache = self.cache(relative_epoch)?;
let committee = cache
.get_crosslink_committee_for_shard(shard)
.ok_or_else(|| Error::NoCommitteeForShard)?;
Ok(committee)
/// Spec v0.9.1
pub fn get_beacon_committees_at_slot(&self, slot: Slot) -> Result<Vec<BeaconCommittee>, Error> {
let cache = self.committee_cache_at_slot(slot)?;
cache.get_beacon_committees_at_slot(slot)
}
/// Returns the beacon proposer index for the `slot` in the given `relative_epoch`.
/// Compute the proposer (not necessarily for the Beacon chain) from a list of indices.
///
/// Spec v0.8.1
/// Spec v0.9.1
// NOTE: be sure to test this bad boy.
pub fn get_beacon_proposer_index(
pub fn compute_proposer_index(
&self,
slot: Slot,
relative_epoch: RelativeEpoch,
indices: &[usize],
seed: &[u8],
spec: &ChainSpec,
) -> Result<usize, Error> {
let cache = self.cache(relative_epoch)?;
let epoch = relative_epoch.into_epoch(self.current_epoch());
let first_committee = cache
.first_committee_at_slot(slot)
.ok_or_else(|| Error::SlotOutOfBounds)?;
let seed = self.get_seed(epoch, spec)?;
if first_committee.is_empty() {
if indices.is_empty() {
return Err(Error::InsufficientValidators);
}
let mut i = 0;
Ok(loop {
let candidate_index = first_committee[(epoch.as_usize() + i) % first_committee.len()];
loop {
let candidate_index = indices[compute_shuffled_index(
i % indices.len(),
indices.len(),
seed,
spec.shuffle_round_count,
)
.ok_or(Error::UnableToShuffle)?];
let random_byte = {
let mut preimage = seed.as_bytes().to_vec();
let mut preimage = seed.to_vec();
preimage.append(&mut int_to_bytes8((i / 32) as u64));
let hash = hash(&preimage);
hash[i % 32]
};
let effective_balance = self.validators[candidate_index].effective_balance;
if (effective_balance * MAX_RANDOM_BYTE)
>= (spec.max_effective_balance * u64::from(random_byte))
if effective_balance * MAX_RANDOM_BYTE
>= spec.max_effective_balance * u64::from(random_byte)
{
break candidate_index;
return Ok(candidate_index);
}
i += 1;
})
}
}
/// Returns the beacon proposer index for the `slot` in the given `relative_epoch`.
///
/// Spec v0.9.1
pub fn get_beacon_proposer_index(&self, slot: Slot, spec: &ChainSpec) -> Result<usize, Error> {
let epoch = slot.epoch(T::slots_per_epoch());
let seed = self.get_beacon_proposer_seed(slot, spec)?;
let indices = self.get_active_validator_indices(epoch);
self.compute_proposer_index(&indices, &seed, spec)
}
/// Compute the seed to use for the beacon proposer selection at the given `slot`.
///
/// Spec v0.9.1
fn get_beacon_proposer_seed(&self, slot: Slot, spec: &ChainSpec) -> Result<Vec<u8>, Error> {
let epoch = slot.epoch(T::slots_per_epoch());
let mut preimage = self
.get_seed(epoch, Domain::BeaconProposer, spec)?
.as_bytes()
.to_vec();
preimage.append(&mut int_to_bytes8(slot.as_u64()));
Ok(hash(&preimage))
}
/// Safely obtains the index for latest block roots, given some `slot`.
///
/// Spec v0.8.1
/// Spec v0.9.1
fn get_latest_block_roots_index(&self, slot: Slot) -> Result<usize, Error> {
if (slot < self.slot) && (self.slot <= slot + self.block_roots.len() as u64) {
Ok(slot.as_usize() % self.block_roots.len())
@ -481,7 +464,7 @@ impl<T: EthSpec> BeaconState<T> {
/// Return the block root at a recent `slot`.
///
/// Spec v0.8.1
/// Spec v0.9.1
pub fn get_block_root(&self, slot: Slot) -> Result<&Hash256, BeaconStateError> {
let i = self.get_latest_block_roots_index(slot)?;
Ok(&self.block_roots[i])
@ -489,7 +472,7 @@ impl<T: EthSpec> BeaconState<T> {
/// Return the block root at a recent `epoch`.
///
/// Spec v0.8.1
/// Spec v0.9.1
// NOTE: the spec calls this get_block_root
pub fn get_block_root_at_epoch(&self, epoch: Epoch) -> Result<&Hash256, BeaconStateError> {
self.get_block_root(epoch.start_slot(T::slots_per_epoch()))
@ -497,7 +480,7 @@ impl<T: EthSpec> BeaconState<T> {
/// Sets the block root for some given slot.
///
/// Spec v0.8.1
/// Spec v0.9.1
pub fn set_block_root(
&mut self,
slot: Slot,
@ -508,9 +491,14 @@ impl<T: EthSpec> BeaconState<T> {
Ok(())
}
/// Fill `randao_mixes` with
pub fn fill_randao_mixes_with(&mut self, index_root: Hash256) {
self.randao_mixes = FixedVector::from_elem(index_root);
}
/// Safely obtains the index for `randao_mixes`
///
/// Spec v0.8.1
/// Spec v0.9.1
fn get_randao_mix_index(
&self,
epoch: Epoch,
@ -532,7 +520,7 @@ impl<T: EthSpec> BeaconState<T> {
///
/// See `Self::get_randao_mix`.
///
/// Spec v0.8.0
/// Spec v0.9.1
pub fn update_randao_mix(&mut self, epoch: Epoch, signature: &Signature) -> Result<(), Error> {
let i = epoch.as_usize() % T::EpochsPerHistoricalVector::to_usize();
@ -545,7 +533,7 @@ impl<T: EthSpec> BeaconState<T> {
/// Return the randao mix at a recent ``epoch``.
///
/// Spec v0.8.1
/// Spec v0.9.1
pub fn get_randao_mix(&self, epoch: Epoch) -> Result<&Hash256, Error> {
let i = self.get_randao_mix_index(epoch, AllowNextEpoch::False)?;
Ok(&self.randao_mixes[i])
@ -553,115 +541,16 @@ impl<T: EthSpec> BeaconState<T> {
/// Set the randao mix at a recent ``epoch``.
///
/// Spec v0.8.1
/// Spec v0.9.1
pub fn set_randao_mix(&mut self, epoch: Epoch, mix: Hash256) -> Result<(), Error> {
let i = self.get_randao_mix_index(epoch, AllowNextEpoch::True)?;
self.randao_mixes[i] = mix;
Ok(())
}
/// Safely obtains the index for `active_index_roots`, given some `epoch`.
///
/// If `allow_next_epoch` is `True`, then we allow an _extra_ one epoch of lookahead.
///
/// Spec v0.8.1
fn get_active_index_root_index(
&self,
epoch: Epoch,
spec: &ChainSpec,
allow_next_epoch: AllowNextEpoch,
) -> Result<usize, Error> {
let current_epoch = self.current_epoch();
let lookahead = spec.activation_exit_delay;
let lookback = self.active_index_roots.len() as u64 - lookahead;
let epoch_upper_bound = allow_next_epoch.upper_bound_of(current_epoch) + lookahead;
if current_epoch < epoch + lookback && epoch <= epoch_upper_bound {
Ok(epoch.as_usize() % self.active_index_roots.len())
} else {
Err(Error::EpochOutOfBounds)
}
}
/// Return the `active_index_root` at a recent `epoch`.
///
/// Spec v0.8.1
pub fn get_active_index_root(&self, epoch: Epoch, spec: &ChainSpec) -> Result<Hash256, Error> {
let i = self.get_active_index_root_index(epoch, spec, AllowNextEpoch::False)?;
Ok(self.active_index_roots[i])
}
/// Set the `active_index_root` at a recent `epoch`.
///
/// Spec v0.8.1
pub fn set_active_index_root(
&mut self,
epoch: Epoch,
index_root: Hash256,
spec: &ChainSpec,
) -> Result<(), Error> {
let i = self.get_active_index_root_index(epoch, spec, AllowNextEpoch::True)?;
self.active_index_roots[i] = index_root;
Ok(())
}
/// Replace `active_index_roots` with clones of `index_root`.
///
/// Spec v0.8.0
pub fn fill_active_index_roots_with(&mut self, index_root: Hash256) {
self.active_index_roots = FixedVector::from_elem(index_root);
}
/// Safely obtains the index for `compact_committees_roots`, given some `epoch`.
///
/// Spec v0.8.1
fn get_compact_committee_root_index(
&self,
epoch: Epoch,
allow_next_epoch: AllowNextEpoch,
) -> Result<usize, Error> {
let current_epoch = self.current_epoch();
let len = T::EpochsPerHistoricalVector::to_u64();
if current_epoch < epoch + len && epoch <= allow_next_epoch.upper_bound_of(current_epoch) {
Ok(epoch.as_usize() % len as usize)
} else {
Err(Error::EpochOutOfBounds)
}
}
/// Return the `compact_committee_root` at a recent `epoch`.
///
/// Spec v0.8.1
pub fn get_compact_committee_root(&self, epoch: Epoch) -> Result<Hash256, Error> {
let i = self.get_compact_committee_root_index(epoch, AllowNextEpoch::False)?;
Ok(self.compact_committees_roots[i])
}
/// Set the `compact_committee_root` at a recent `epoch`.
///
/// Spec v0.8.1
pub fn set_compact_committee_root(
&mut self,
epoch: Epoch,
index_root: Hash256,
) -> Result<(), Error> {
let i = self.get_compact_committee_root_index(epoch, AllowNextEpoch::True)?;
self.compact_committees_roots[i] = index_root;
Ok(())
}
/// Replace `compact_committees_roots` with clones of `committee_root`.
///
/// Spec v0.8.0
pub fn fill_compact_committees_roots_with(&mut self, committee_root: Hash256) {
self.compact_committees_roots = FixedVector::from_elem(committee_root);
}
/// Safely obtains the index for latest state roots, given some `slot`.
///
/// Spec v0.8.1
/// Spec v0.9.1
fn get_latest_state_roots_index(&self, slot: Slot) -> Result<usize, Error> {
if (slot < self.slot) && (self.slot <= slot + Slot::from(self.state_roots.len())) {
Ok(slot.as_usize() % self.state_roots.len())
@ -672,7 +561,7 @@ impl<T: EthSpec> BeaconState<T> {
/// Gets the state root for some slot.
///
/// Spec v0.8.1
/// Spec v0.9.1
pub fn get_state_root(&self, slot: Slot) -> Result<&Hash256, Error> {
let i = self.get_latest_state_roots_index(slot)?;
Ok(&self.state_roots[i])
@ -680,7 +569,7 @@ impl<T: EthSpec> BeaconState<T> {
/// Gets the oldest (earliest slot) state root.
///
/// Spec v0.8.1
/// Spec v0.9.1
pub fn get_oldest_state_root(&self) -> Result<&Hash256, Error> {
let i =
self.get_latest_state_roots_index(self.slot - Slot::from(self.state_roots.len()))?;
@ -689,7 +578,7 @@ impl<T: EthSpec> BeaconState<T> {
/// Sets the latest state root for slot.
///
/// Spec v0.8.1
/// Spec v0.9.1
pub fn set_state_root(&mut self, slot: Slot, state_root: Hash256) -> Result<(), Error> {
let i = self.get_latest_state_roots_index(slot)?;
self.state_roots[i] = state_root;
@ -698,7 +587,7 @@ impl<T: EthSpec> BeaconState<T> {
/// Safely obtain the index for `slashings`, given some `epoch`.
///
/// Spec v0.8.1
/// Spec v0.9.1
fn get_slashings_index(
&self,
epoch: Epoch,
@ -718,14 +607,14 @@ impl<T: EthSpec> BeaconState<T> {
/// Get a reference to the entire `slashings` vector.
///
/// Spec v0.8.0
/// Spec v0.9.1
pub fn get_all_slashings(&self) -> &[u64] {
&self.slashings
}
/// Get the total slashed balances for some epoch.
///
/// Spec v0.8.1
/// Spec v0.9.1
pub fn get_slashings(&self, epoch: Epoch) -> Result<u64, Error> {
let i = self.get_slashings_index(epoch, AllowNextEpoch::False)?;
Ok(self.slashings[i])
@ -733,7 +622,7 @@ impl<T: EthSpec> BeaconState<T> {
/// Set the total slashed balances for some epoch.
///
/// Spec v0.8.1
/// Spec v0.9.1
pub fn set_slashings(&mut self, epoch: Epoch, value: u64) -> Result<(), Error> {
let i = self.get_slashings_index(epoch, AllowNextEpoch::True)?;
self.slashings[i] = value;
@ -742,7 +631,7 @@ impl<T: EthSpec> BeaconState<T> {
/// Get the attestations from the current or previous epoch.
///
/// Spec v0.8.1
/// Spec v0.9.1
pub fn get_matching_source_attestations(
&self,
epoch: Epoch,
@ -756,48 +645,40 @@ impl<T: EthSpec> BeaconState<T> {
}
}
/// Get the current crosslink for a shard.
///
/// Spec v0.8.1
pub fn get_current_crosslink(&self, shard: u64) -> Result<&Crosslink, Error> {
self.current_crosslinks
.get(shard as usize)
.ok_or(Error::ShardOutOfBounds)
}
/// Get the previous crosslink for a shard.
///
/// Spec v0.8.1
pub fn get_previous_crosslink(&self, shard: u64) -> Result<&Crosslink, Error> {
self.previous_crosslinks
.get(shard as usize)
.ok_or(Error::ShardOutOfBounds)
}
/// Generate a seed for the given `epoch`.
///
/// Spec v0.8.0
pub fn get_seed(&self, epoch: Epoch, spec: &ChainSpec) -> Result<Hash256, Error> {
/// Spec v0.9.1
pub fn get_seed(
&self,
epoch: Epoch,
domain_type: Domain,
spec: &ChainSpec,
) -> Result<Hash256, Error> {
// Bypass the safe getter for RANDAO so we can gracefully handle the scenario where `epoch
// == 0`.
let randao = {
let mix = {
let i = epoch + T::EpochsPerHistoricalVector::to_u64() - spec.min_seed_lookahead - 1;
self.randao_mixes[i.as_usize() % self.randao_mixes.len()]
};
let active_index_root = self.get_active_index_root(epoch, spec)?;
let epoch_bytes = int_to_bytes32(epoch.as_u64());
let domain_bytes = int_to_bytes4(spec.get_domain_constant(domain_type));
let epoch_bytes = int_to_bytes8(epoch.as_u64());
let mut preimage = [0; 32 * 3];
preimage[0..32].copy_from_slice(&randao[..]);
preimage[32..64].copy_from_slice(&active_index_root[..]);
preimage[64..].copy_from_slice(&epoch_bytes);
const NUM_DOMAIN_BYTES: usize = 4;
const NUM_EPOCH_BYTES: usize = 8;
const NUM_MIX_BYTES: usize = 32;
let mut preimage = [0; NUM_DOMAIN_BYTES + NUM_EPOCH_BYTES + NUM_MIX_BYTES];
preimage[0..NUM_DOMAIN_BYTES].copy_from_slice(&domain_bytes);
preimage[NUM_DOMAIN_BYTES..NUM_DOMAIN_BYTES + NUM_EPOCH_BYTES]
.copy_from_slice(&epoch_bytes);
preimage[NUM_DOMAIN_BYTES + NUM_EPOCH_BYTES..].copy_from_slice(mix.as_bytes());
Ok(Hash256::from_slice(&hash(&preimage)))
}
/// Return the effective balance (also known as "balance at stake") for a validator with the given ``index``.
///
/// Spec v0.8.1
/// Spec v0.9.1
pub fn get_effective_balance(
&self,
validator_index: usize,
@ -811,43 +692,44 @@ impl<T: EthSpec> BeaconState<T> {
/// Return the epoch at which an activation or exit triggered in ``epoch`` takes effect.
///
/// Spec v0.8.1
/// Spec v0.9.1
pub fn compute_activation_exit_epoch(&self, epoch: Epoch, spec: &ChainSpec) -> Epoch {
epoch + 1 + spec.activation_exit_delay
epoch + 1 + spec.max_seed_lookahead
}
/// Return the churn limit for the current epoch (number of validators who can leave per epoch).
///
/// Uses the epoch cache, and will error if it isn't initialized.
///
/// Spec v0.8.1
/// Spec v0.9.1
pub fn get_churn_limit(&self, spec: &ChainSpec) -> Result<u64, Error> {
Ok(std::cmp::max(
spec.min_per_epoch_churn_limit,
self.cache(RelativeEpoch::Current)?.active_validator_count() as u64
self.committee_cache(RelativeEpoch::Current)?
.active_validator_count() as u64
/ spec.churn_limit_quotient,
))
}
/// Returns the `slot`, `shard` and `committee_index` for which a validator must produce an
/// Returns the `slot`, `index` and `committee_position` for which a validator must produce an
/// attestation.
///
/// Note: Utilizes the cache and will fail if the appropriate cache is not initialized.
///
/// Spec v0.8.1
/// Spec v0.9.1
pub fn get_attestation_duties(
&self,
validator_index: usize,
relative_epoch: RelativeEpoch,
) -> Result<Option<AttestationDuty>, Error> {
let cache = self.cache(relative_epoch)?;
let cache = self.committee_cache(relative_epoch)?;
Ok(cache.get_attestation_duties(validator_index))
}
/// Return the combined effective balance of an array of validators.
///
/// Spec v0.8.1
/// Spec v0.9.1
pub fn get_total_balance(
&self,
validator_indices: &[usize],
@ -887,7 +769,7 @@ impl<T: EthSpec> BeaconState<T> {
relative_epoch: RelativeEpoch,
spec: &ChainSpec,
) -> Result<(), Error> {
let i = Self::cache_index(relative_epoch);
let i = Self::committee_cache_index(relative_epoch);
if self.committee_caches[i]
.is_initialized_at(relative_epoch.into_epoch(self.current_epoch()))
@ -906,7 +788,7 @@ impl<T: EthSpec> BeaconState<T> {
) -> Result<(), Error> {
let epoch = relative_epoch.into_epoch(self.current_epoch());
self.committee_caches[Self::cache_index(relative_epoch)] =
self.committee_caches[Self::committee_cache_index(relative_epoch)] =
CommitteeCache::initialized(&self, epoch, spec)?;
Ok(())
}
@ -917,16 +799,14 @@ impl<T: EthSpec> BeaconState<T> {
///
/// Note: whilst this function will preserve already-built caches, it will not build any.
pub fn advance_caches(&mut self) {
let next = Self::cache_index(RelativeEpoch::Previous);
let current = Self::cache_index(RelativeEpoch::Current);
let caches = &mut self.committee_caches[..];
caches.rotate_left(1);
let next = Self::committee_cache_index(RelativeEpoch::Next);
caches[next] = CommitteeCache::default();
caches[current] = CommitteeCache::default();
}
fn cache_index(relative_epoch: RelativeEpoch) -> usize {
fn committee_cache_index(relative_epoch: RelativeEpoch) -> usize {
match relative_epoch {
RelativeEpoch::Previous => 0,
RelativeEpoch::Current => 1,
@ -934,21 +814,31 @@ impl<T: EthSpec> BeaconState<T> {
}
}
/// Get the committee cache for some `slot`.
///
/// Return an error if the cache for the slot's epoch is not initialized.
fn committee_cache_at_slot(&self, slot: Slot) -> Result<&CommitteeCache, Error> {
let epoch = slot.epoch(T::slots_per_epoch());
let relative_epoch = RelativeEpoch::from_epoch(self.current_epoch(), epoch)?;
self.committee_cache(relative_epoch)
}
/// Returns the cache for some `RelativeEpoch`. Returns an error if the cache has not been
/// initialized.
fn cache(&self, relative_epoch: RelativeEpoch) -> Result<&CommitteeCache, Error> {
let cache = &self.committee_caches[Self::cache_index(relative_epoch)];
fn committee_cache(&self, relative_epoch: RelativeEpoch) -> Result<&CommitteeCache, Error> {
let cache = &self.committee_caches[Self::committee_cache_index(relative_epoch)];
if cache.is_initialized_at(relative_epoch.into_epoch(self.current_epoch())) {
Ok(cache)
} else {
Err(Error::CommitteeCacheUninitialized(relative_epoch))
Err(Error::CommitteeCacheUninitialized(Some(relative_epoch)))
}
}
/// Drops the cache, leaving it in an uninitialized state.
fn drop_committee_cache(&mut self, relative_epoch: RelativeEpoch) {
self.committee_caches[Self::cache_index(relative_epoch)] = CommitteeCache::default();
self.committee_caches[Self::committee_cache_index(relative_epoch)] =
CommitteeCache::default();
}
/// Updates the pubkey cache, if required.

View File

@ -15,22 +15,20 @@ pub struct CommitteeCache {
initialized_epoch: Option<Epoch>,
shuffling: Vec<usize>,
shuffling_positions: Vec<Option<NonZeroUsize>>,
shuffling_start_shard: u64,
shard_count: u64,
committee_count: usize,
committees_per_slot: u64,
slots_per_epoch: u64,
}
impl CommitteeCache {
/// Return a new, fully initialized cache.
///
/// Spec v0.8.1
/// Spec v0.9.1
pub fn initialized<T: EthSpec>(
state: &BeaconState<T>,
epoch: Epoch,
spec: &ChainSpec,
) -> Result<CommitteeCache, Error> {
let relative_epoch = RelativeEpoch::from_epoch(state.current_epoch(), epoch)
RelativeEpoch::from_epoch(state.current_epoch(), epoch)
.map_err(|_| Error::EpochOutOfBounds)?;
// May cause divide-by-zero errors.
@ -44,14 +42,10 @@ impl CommitteeCache {
return Err(Error::InsufficientValidators);
}
let committee_count =
T::get_committee_count(active_validator_indices.len(), spec.target_committee_size)
as usize;
let committees_per_slot =
T::get_committee_count_per_slot(active_validator_indices.len(), spec) as u64;
let shuffling_start_shard =
Self::compute_start_shard(state, relative_epoch, active_validator_indices.len(), spec);
let seed = state.get_seed(epoch, spec)?;
let seed = state.get_seed(epoch, Domain::BeaconAttester, spec)?;
let shuffling = shuffle_list(
active_validator_indices,
@ -73,46 +67,13 @@ impl CommitteeCache {
Ok(CommitteeCache {
initialized_epoch: Some(epoch),
shuffling_start_shard,
shuffling,
shard_count: T::shard_count() as u64,
committee_count,
slots_per_epoch: T::slots_per_epoch(),
shuffling_positions,
committees_per_slot,
slots_per_epoch: T::slots_per_epoch(),
})
}
/// Compute the shard which must be attested to first in a given relative epoch.
///
/// The `active_validator_count` must be the number of validators active at `relative_epoch`.
///
/// Spec v0.8.1
pub fn compute_start_shard<T: EthSpec>(
state: &BeaconState<T>,
relative_epoch: RelativeEpoch,
active_validator_count: usize,
spec: &ChainSpec,
) -> u64 {
match relative_epoch {
RelativeEpoch::Current => state.start_shard,
RelativeEpoch::Previous => {
let shard_delta =
T::get_shard_delta(active_validator_count, spec.target_committee_size);
(state.start_shard + T::ShardCount::to_u64() - shard_delta)
% T::ShardCount::to_u64()
}
RelativeEpoch::Next => {
let current_active_validators =
get_active_validator_count(&state.validators, state.current_epoch());
let shard_delta =
T::get_shard_delta(current_active_validators, spec.target_committee_size);
(state.start_shard + shard_delta) % T::ShardCount::to_u64()
}
}
}
/// Returns `true` if the cache has been initialized at the supplied `epoch`.
///
/// An non-initialized cache does not provide any useful information.
@ -126,7 +87,7 @@ impl CommitteeCache {
///
/// Always returns `&[]` for a non-initialized epoch.
///
/// Spec v0.8.1
/// Spec v0.9.1
pub fn active_validator_indices(&self) -> &[usize] {
&self.shuffling
}
@ -135,34 +96,51 @@ impl CommitteeCache {
///
/// Always returns `&[]` for a non-initialized epoch.
///
/// Spec v0.8.1
/// Spec v0.9.1
pub fn shuffling(&self) -> &[usize] {
&self.shuffling
}
/// Return `Some(CrosslinkCommittee)` if the given shard has a committee during the given
/// `epoch`.
/// Get the Beacon committee for the given `slot` and `index`.
///
/// Always returns `None` for a non-initialized epoch.
///
/// Spec v0.8.1
pub fn get_crosslink_committee_for_shard(&self, shard: Shard) -> Option<CrosslinkCommittee> {
if shard >= self.shard_count || self.initialized_epoch.is_none() {
/// Return `None` if the cache is uninitialized, or the `slot` or `index` is out of range.
pub fn get_beacon_committee(
&self,
slot: Slot,
index: CommitteeIndex,
) -> Option<BeaconCommittee> {
if self.initialized_epoch.is_none()
|| !self.is_initialized_at(slot.epoch(self.slots_per_epoch))
|| index >= self.committees_per_slot
{
return None;
}
let committee_index =
(shard + self.shard_count - self.shuffling_start_shard) % self.shard_count;
(slot.as_u64() % self.slots_per_epoch) * self.committees_per_slot + index;
let committee = self.compute_committee(committee_index as usize)?;
let slot = self.crosslink_slot_for_shard(shard)?;
Some(CrosslinkCommittee {
shard,
committee,
Some(BeaconCommittee {
slot,
index,
committee,
})
}
/// Get all the Beacon committees at a given `slot`.
pub fn get_beacon_committees_at_slot(&self, slot: Slot) -> Result<Vec<BeaconCommittee>, Error> {
if self.initialized_epoch.is_none() {
return Err(Error::CommitteeCacheUninitialized(None));
}
(0..self.committees_per_slot())
.map(|index| {
self.get_beacon_committee(slot, index)
.ok_or(Error::NoCommittee { slot, index })
})
.collect()
}
/// Returns the `AttestationDuty` for the given `validator_index`.
///
/// Returns `None` if the `validator_index` does not exist, does not have duties or `Self` is
@ -170,36 +148,46 @@ impl CommitteeCache {
pub fn get_attestation_duties(&self, validator_index: usize) -> Option<AttestationDuty> {
let i = self.shuffled_position(validator_index)?;
(0..self.committee_count)
(0..self.epoch_committee_count())
.map(|nth_committee| (nth_committee, self.compute_committee_range(nth_committee)))
.find(|(_, range)| {
if let Some(range) = range {
(range.start <= i) && (range.end > i)
range.start <= i && range.end > i
} else {
false
}
})
.and_then(|(nth_committee, range)| {
let shard = (self.shuffling_start_shard + nth_committee as u64) % self.shard_count;
let slot = self.crosslink_slot_for_shard(shard)?;
let (slot, index) = self.convert_to_slot_and_index(nth_committee as u64)?;
let range = range?;
let committee_index = i - range.start;
let committee_position = i - range.start;
let committee_len = range.end - range.start;
Some(AttestationDuty {
slot,
shard,
committee_index,
index,
committee_position,
committee_len,
})
})
}
/// Convert an index addressing the list of all epoch committees into a slot and per-slot index.
fn convert_to_slot_and_index(
&self,
global_committee_index: u64,
) -> Option<(Slot, CommitteeIndex)> {
let epoch_start_slot = self.initialized_epoch?.start_slot(self.slots_per_epoch);
let slot_offset = global_committee_index / self.committees_per_slot;
let index = global_committee_index % self.committees_per_slot;
Some((epoch_start_slot + slot_offset, index))
}
/// Returns the number of active validators in the initialized epoch.
///
/// Always returns `usize::default()` for a non-initialized epoch.
///
/// Spec v0.8.1
/// Spec v0.9.1
pub fn active_validator_count(&self) -> usize {
self.shuffling.len()
}
@ -208,64 +196,19 @@ impl CommitteeCache {
///
/// Always returns `usize::default()` for a non-initialized epoch.
///
/// Spec v0.8.1
/// Spec v0.9.1
pub fn epoch_committee_count(&self) -> usize {
self.committee_count
self.committees_per_slot as usize * self.slots_per_epoch as usize
}
/// Returns the shard assigned to the first committee in the initialized epoch.
///
/// Always returns `u64::default()` for a non-initialized epoch.
pub fn epoch_start_shard(&self) -> u64 {
self.shuffling_start_shard
}
/// Returns all crosslink committees, if any, for the given slot in the initialized epoch.
///
/// Returns `None` if `slot` is not in the initialized epoch, or if `Self` is not initialized.
///
/// Spec v0.8.1
pub fn get_crosslink_committees_for_slot(&self, slot: Slot) -> Option<Vec<CrosslinkCommittee>> {
let position = self
.initialized_epoch?
.position(slot, self.slots_per_epoch)?;
let committees_per_slot = self.committee_count / self.slots_per_epoch as usize;
let position = position * committees_per_slot;
if position >= self.committee_count {
None
} else {
let mut committees = Vec::with_capacity(committees_per_slot);
for index in position..position + committees_per_slot {
let committee = self.compute_committee(index)?;
let shard = (self.shuffling_start_shard + index as u64) % self.shard_count;
committees.push(CrosslinkCommittee {
committee,
shard,
slot,
});
}
Some(committees)
}
}
/// Returns the first committee of the first slot of the initialized epoch.
///
/// Always returns `None` for a non-initialized epoch.
///
/// Spec v0.8.1
pub fn first_committee_at_slot(&self, slot: Slot) -> Option<&[usize]> {
self.get_crosslink_committees_for_slot(slot)?
.first()
.and_then(|cc| Some(cc.committee))
/// Returns the number of committees per slot for this cache's epoch.
pub fn committees_per_slot(&self) -> u64 {
self.committees_per_slot
}
/// Returns a slice of `self.shuffling` that represents the `index`'th committee in the epoch.
///
/// Spec v0.8.1
/// Spec v0.9.1
fn compute_committee(&self, index: usize) -> Option<&[usize]> {
Some(&self.shuffling[self.compute_committee_range(index)?])
}
@ -276,34 +219,20 @@ impl CommitteeCache {
///
/// Will also return `None` if the index is out of bounds.
///
/// Spec v0.8.1
/// Spec v0.9.1
fn compute_committee_range(&self, index: usize) -> Option<Range<usize>> {
if self.committee_count == 0 || index >= self.committee_count {
let count = self.epoch_committee_count();
if count == 0 || index >= count {
return None;
}
let num_validators = self.shuffling.len();
let count = self.committee_count;
let start = (num_validators * index) / count;
let end = (num_validators * (index + 1)) / count;
Some(start..end)
}
/// Returns the `slot` that `shard` will be crosslink-ed in during the initialized epoch.
///
/// Always returns `None` for a non-initialized epoch.
///
/// Spec v0.8.1
fn crosslink_slot_for_shard(&self, shard: u64) -> Option<Slot> {
let offset = (shard + self.shard_count - self.shuffling_start_shard) % self.shard_count;
Some(
self.initialized_epoch?.start_slot(self.slots_per_epoch)
+ offset / (self.committee_count as u64 / self.slots_per_epoch),
)
}
/// Returns the index of some validator in `self.shuffling`.
///
/// Always returns `None` for a non-initialized epoch.
@ -317,7 +246,7 @@ impl CommitteeCache {
/// Returns a list of all `validators` indices where the validator is active at the given
/// `epoch`.
///
/// Spec v0.8.1
/// Spec v0.9.1
pub fn get_active_validator_indices(validators: &[Validator], epoch: Epoch) -> Vec<usize> {
let mut active = Vec::with_capacity(validators.len());
@ -331,11 +260,3 @@ pub fn get_active_validator_indices(validators: &[Validator], epoch: Epoch) -> V
active
}
/// Returns the count of all `validators` indices where the validator is active at the given
/// `epoch`.
///
/// Spec v0.8.1
fn get_active_validator_count(validators: &[Validator], epoch: Epoch) -> usize {
validators.iter().filter(|v| v.is_active_at(epoch)).count()
}

View File

@ -1,8 +1,6 @@
#![cfg(test)]
use super::*;
use crate::{test_utils::*, *};
use serde_derive::{Deserialize, Serialize};
use ssz_types::typenum::*;
#[test]
fn default_values() {
@ -10,13 +8,11 @@ fn default_values() {
assert_eq!(cache.is_initialized_at(Epoch::new(0)), false);
assert!(&cache.active_validator_indices().is_empty());
assert_eq!(cache.get_crosslink_committee_for_shard(0), None);
assert_eq!(cache.get_beacon_committee(Slot::new(0), 0), None);
assert_eq!(cache.get_attestation_duties(0), None);
assert_eq!(cache.active_validator_count(), 0);
assert_eq!(cache.epoch_committee_count(), 0);
assert_eq!(cache.epoch_start_shard(), 0);
assert_eq!(cache.get_crosslink_committees_for_slot(Slot::new(0)), None);
assert_eq!(cache.first_committee_at_slot(Slot::new(0)), None);
assert!(cache.get_beacon_committees_at_slot(Slot::new(0)).is_err());
}
fn new_state<T: EthSpec>(validator_count: usize, slot: Slot) -> BeaconState<T> {
@ -78,9 +74,15 @@ fn shuffles_for_the_right_epoch() {
state.randao_mixes = FixedVector::from(distinct_hashes);
let previous_seed = state.get_seed(state.previous_epoch(), spec).unwrap();
let current_seed = state.get_seed(state.current_epoch(), spec).unwrap();
let next_seed = state.get_seed(state.next_epoch(), spec).unwrap();
let previous_seed = state
.get_seed(state.previous_epoch(), Domain::BeaconAttester, spec)
.unwrap();
let current_seed = state
.get_seed(state.current_epoch(), Domain::BeaconAttester, spec)
.unwrap();
let next_seed = state
.get_seed(state.next_epoch(), Domain::BeaconAttester, spec)
.unwrap();
assert!((previous_seed != current_seed) && (current_seed != next_seed));
@ -116,153 +118,3 @@ fn shuffles_for_the_right_epoch() {
assert_eq!(cache.shuffling, shuffling_with_seed(next_seed));
assert_shuffling_positions_accurate(&cache);
}
#[test]
fn can_start_on_any_shard() {
let num_validators = MinimalEthSpec::minimum_validator_count() * 2;
let epoch = Epoch::new(100_000_000);
let slot = epoch.start_slot(MinimalEthSpec::slots_per_epoch());
let mut state = new_state::<MinimalEthSpec>(num_validators, slot);
let spec = &MinimalEthSpec::default_spec();
let target_committee_size = MinimalEthSpec::default_spec().target_committee_size;
let shard_delta = MinimalEthSpec::get_shard_delta(num_validators, target_committee_size);
let shard_count = MinimalEthSpec::shard_count() as u64;
for i in 0..MinimalEthSpec::shard_count() as u64 {
state.start_shard = i;
let cache = CommitteeCache::initialized(&state, state.current_epoch(), spec).unwrap();
assert_eq!(cache.shuffling_start_shard, i);
let cache = CommitteeCache::initialized(&state, state.previous_epoch(), spec).unwrap();
assert_eq!(
cache.shuffling_start_shard,
(i + shard_count - shard_delta) % shard_count
);
let cache = CommitteeCache::initialized(&state, state.next_epoch(), spec).unwrap();
assert_eq!(cache.shuffling_start_shard, (i + shard_delta) % shard_count);
}
}
/// This spec has more shards than slots in an epoch, permitting epochs where not all shards are
/// included in the committee.
#[derive(Clone, PartialEq, Debug, Default, Serialize, Deserialize)]
pub struct ExcessShardsEthSpec;
impl EthSpec for ExcessShardsEthSpec {
type ShardCount = U128;
type SlotsPerEpoch = U8;
type MaxPendingAttestations = U1024;
params_from_eth_spec!(MinimalEthSpec {
JustificationBitsLength,
MaxValidatorsPerCommittee,
GenesisEpoch,
SlotsPerEth1VotingPeriod,
SlotsPerHistoricalRoot,
EpochsPerHistoricalVector,
EpochsPerSlashingsVector,
HistoricalRootsLimit,
ValidatorRegistryLimit,
MaxProposerSlashings,
MaxAttesterSlashings,
MaxAttestations,
MaxDeposits,
MaxVoluntaryExits,
MaxTransfers
});
fn default_spec() -> ChainSpec {
ChainSpec::minimal()
}
}
#[test]
fn starts_on_the_correct_shard() {
let spec = &ExcessShardsEthSpec::default_spec();
let num_validators = spec.target_committee_size * ExcessShardsEthSpec::shard_count();
let epoch = Epoch::new(100_000_000);
let slot = epoch.start_slot(ExcessShardsEthSpec::slots_per_epoch());
let mut state = new_state::<ExcessShardsEthSpec>(num_validators, slot);
let validator_count = state.validators.len();
let previous_epoch = state.previous_epoch();
let current_epoch = state.current_epoch();
let next_epoch = state.next_epoch();
for (i, mut v) in state.validators.iter_mut().enumerate() {
let epoch = if i < validator_count / 4 {
previous_epoch
} else if i < validator_count / 2 {
current_epoch
} else {
next_epoch
};
v.activation_epoch = epoch;
}
assert_eq!(
get_active_validator_count(&state.validators, previous_epoch),
validator_count / 4
);
assert_eq!(
get_active_validator_count(&state.validators, current_epoch),
validator_count / 2
);
assert_eq!(
get_active_validator_count(&state.validators, next_epoch),
validator_count
);
let previous_shards = ExcessShardsEthSpec::get_committee_count(
get_active_validator_count(&state.validators, previous_epoch),
spec.target_committee_size,
);
let current_shards = ExcessShardsEthSpec::get_committee_count(
get_active_validator_count(&state.validators, current_epoch),
spec.target_committee_size,
);
let next_shards = ExcessShardsEthSpec::get_committee_count(
get_active_validator_count(&state.validators, next_epoch),
spec.target_committee_size,
);
assert_eq!(
previous_shards as usize,
ExcessShardsEthSpec::shard_count() / 4
);
assert_eq!(
current_shards as usize,
ExcessShardsEthSpec::shard_count() / 2
);
assert_eq!(next_shards as usize, ExcessShardsEthSpec::shard_count());
let shard_count = ExcessShardsEthSpec::shard_count();
for i in 0..ExcessShardsEthSpec::shard_count() {
state.start_shard = i as u64;
let cache = CommitteeCache::initialized(&state, state.current_epoch(), spec).unwrap();
assert_eq!(cache.shuffling_start_shard as usize, i);
let cache = CommitteeCache::initialized(&state, state.previous_epoch(), spec).unwrap();
assert_eq!(
cache.shuffling_start_shard as usize,
(i + shard_count - previous_shards) % shard_count
);
let cache = CommitteeCache::initialized(&state, state.next_epoch(), spec).unwrap();
assert_eq!(
cache.shuffling_start_shard as usize,
(i + current_shards) % shard_count
);
}
}

View File

@ -1,7 +1,6 @@
#![cfg(test)]
use super::*;
use crate::test_utils::*;
use std::ops::RangeInclusive;
ssz_tests!(FoundationBeaconState);
@ -19,34 +18,49 @@ fn test_beacon_proposer_index<T: EthSpec>() {
state
};
// Get the i'th candidate proposer for the given state and slot
let ith_candidate = |state: &BeaconState<T>, slot: Slot, i: usize| {
let epoch = slot.epoch(T::slots_per_epoch());
let seed = state.get_beacon_proposer_seed(slot, &spec).unwrap();
let active_validators = state.get_active_validator_indices(epoch);
active_validators[compute_shuffled_index(
i,
active_validators.len(),
&seed,
spec.shuffle_round_count,
)
.unwrap()]
};
// Run a test on the state.
let test = |state: &BeaconState<T>, slot: Slot, shuffling_index: usize| {
let shuffling = state.get_shuffling(relative_epoch).unwrap();
let test = |state: &BeaconState<T>, slot: Slot, candidate_index: usize| {
assert_eq!(
state.get_beacon_proposer_index(slot, relative_epoch, &spec),
Ok(shuffling[shuffling_index])
state.get_beacon_proposer_index(slot, &spec),
Ok(ith_candidate(state, slot, candidate_index))
);
};
// Test where we have one validator per slot
// Test where we have one validator per slot.
// 0th candidate should be chosen every time.
let state = build_state(T::slots_per_epoch() as usize);
for i in 0..T::slots_per_epoch() {
test(&state, Slot::from(i), i as usize);
test(&state, Slot::from(i), 0);
}
// Test where we have two validators per slot
// Test where we have two validators per slot.
// 0th candidate should be chosen every time.
let state = build_state(T::slots_per_epoch() as usize * 2);
for i in 0..T::slots_per_epoch() {
test(&state, Slot::from(i), i as usize * 2);
test(&state, Slot::from(i), 0);
}
// Test with two validators per slot, first validator has zero balance.
let mut state = build_state(T::slots_per_epoch() as usize * 2);
let shuffling = state.get_shuffling(relative_epoch).unwrap().to_vec();
state.validators[shuffling[0]].effective_balance = 0;
let slot0_candidate0 = ith_candidate(&state, Slot::new(0), 0);
state.validators[slot0_candidate0].effective_balance = 0;
test(&state, Slot::new(0), 1);
for i in 1..T::slots_per_epoch() {
test(&state, Slot::from(i), i as usize * 2);
test(&state, Slot::from(i), 0);
}
}
@ -55,72 +69,6 @@ fn beacon_proposer_index() {
test_beacon_proposer_index::<MinimalEthSpec>();
}
/// Should produce (note the set notation brackets):
///
/// (current_epoch - LATEST_ACTIVE_INDEX_ROOTS_LENGTH + ACTIVATION_EXIT_DELAY, current_epoch +
/// ACTIVATION_EXIT_DELAY]
fn active_index_range<T: EthSpec>(current_epoch: Epoch) -> RangeInclusive<Epoch> {
let delay = T::default_spec().activation_exit_delay;
let start: i32 =
current_epoch.as_u64() as i32 - T::epochs_per_historical_vector() as i32 + delay as i32;
let end = current_epoch + delay;
let start: Epoch = if start < 0 {
Epoch::new(0)
} else {
Epoch::from(start as u64 + 1)
};
start..=end
}
/// Test getting an active index root at the start and end of the valid range, and one either side
/// of that range.
fn test_active_index<T: EthSpec>(state_slot: Slot) {
let spec = T::default_spec();
let builder: TestingBeaconStateBuilder<T> =
TestingBeaconStateBuilder::from_default_keypairs_file_if_exists(16, &spec);
let (mut state, _keypairs) = builder.build();
state.slot = state_slot;
let range = active_index_range::<T>(state.current_epoch());
let modulo = |epoch: Epoch| epoch.as_usize() % T::epochs_per_historical_vector();
// Test the start and end of the range.
assert_eq!(
state.get_active_index_root_index(*range.start(), &spec, AllowNextEpoch::False),
Ok(modulo(*range.start()))
);
assert_eq!(
state.get_active_index_root_index(*range.end(), &spec, AllowNextEpoch::False),
Ok(modulo(*range.end()))
);
// One either side of the range.
if state.current_epoch() > 0 {
// Test is invalid on epoch zero, cannot subtract from zero.
assert_eq!(
state.get_active_index_root_index(*range.start() - 1, &spec, AllowNextEpoch::False),
Err(Error::EpochOutOfBounds)
);
}
assert_eq!(
state.get_active_index_root_index(*range.end() + 1, &spec, AllowNextEpoch::False),
Err(Error::EpochOutOfBounds)
);
}
#[test]
fn get_active_index_root_index() {
test_active_index::<MainnetEthSpec>(Slot::new(0));
let epoch = Epoch::from(MainnetEthSpec::epochs_per_historical_vector() * 4);
let slot = epoch.start_slot(MainnetEthSpec::slots_per_epoch());
test_active_index::<MainnetEthSpec>(slot);
}
/// Test that
///
/// 1. Using the cache before it's built fails.
@ -138,28 +86,26 @@ fn test_cache_initialization<'a, T: EthSpec>(
// Assuming the cache isn't already built, assert that a call to a cache-using function fails.
assert_eq!(
state.get_attestation_duties(0, relative_epoch),
Err(BeaconStateError::CommitteeCacheUninitialized(
Err(BeaconStateError::CommitteeCacheUninitialized(Some(
relative_epoch
))
)))
);
// Build the cache.
state.build_committee_cache(relative_epoch, spec).unwrap();
// Assert a call to a cache-using function passes.
let _ = state
.get_beacon_proposer_index(slot, relative_epoch, spec)
.unwrap();
let _ = state.get_beacon_proposer_index(slot, spec).unwrap();
// Drop the cache.
state.drop_committee_cache(relative_epoch);
// Assert a call to a cache-using function fail.
assert_eq!(
state.get_beacon_proposer_index(slot, relative_epoch, spec),
Err(BeaconStateError::CommitteeCacheUninitialized(
state.get_beacon_committee(slot, 0),
Err(BeaconStateError::CommitteeCacheUninitialized(Some(
relative_epoch
))
)))
);
}
@ -212,10 +158,8 @@ mod committees {
spec: &ChainSpec,
) {
let active_indices: Vec<usize> = (0..validator_count).collect();
let seed = state.get_seed(epoch, spec).unwrap();
let seed = state.get_seed(epoch, Domain::BeaconAttester, spec).unwrap();
let relative_epoch = RelativeEpoch::from_epoch(state.current_epoch(), epoch).unwrap();
let start_shard =
CommitteeCache::compute_start_shard(&state, relative_epoch, active_indices.len(), spec);
let mut ordered_indices = state
.get_cached_active_validator_indices(relative_epoch)
@ -231,34 +175,27 @@ mod committees {
shuffle_list(active_indices, spec.shuffle_round_count, &seed[..], false).unwrap();
let mut expected_indices_iter = shuffling.iter();
let mut expected_shards_iter =
(0..T::ShardCount::to_u64()).map(|i| (start_shard + i) % T::ShardCount::to_u64());
// Loop through all slots in the epoch being tested.
for slot in epoch.slot_iter(T::slots_per_epoch()) {
let crosslink_committees = state.get_crosslink_committees_at_slot(slot).unwrap();
let beacon_committees = state.get_beacon_committees_at_slot(slot).unwrap();
// Assert that the number of committees in this slot is consistent with the reported number
// of committees in an epoch.
assert_eq!(
crosslink_committees.len() as u64,
state.get_committee_count(relative_epoch).unwrap() / T::slots_per_epoch()
beacon_committees.len() as u64,
state.get_epoch_committee_count(relative_epoch).unwrap() / T::slots_per_epoch()
);
for cc in crosslink_committees {
// Assert that shards are assigned contiguously across committees.
assert_eq!(expected_shards_iter.next().unwrap(), cc.shard);
for (committee_index, bc) in beacon_committees.iter().enumerate() {
// Assert that indices are assigned sequentially across committees.
assert_eq!(committee_index as u64, bc.index);
// Assert that a committee lookup via slot is identical to a committee lookup via
// shard.
assert_eq!(
state
.get_crosslink_committee_for_shard(cc.shard, relative_epoch)
.unwrap(),
cc
);
// index.
assert_eq!(state.get_beacon_committee(bc.slot, bc.index).unwrap(), *bc);
// Loop through each validator in the committee.
for (committee_i, validator_i) in cc.committee.iter().enumerate() {
for (committee_i, validator_i) in bc.committee.iter().enumerate() {
// Assert the validators are assigned contiguously across committees.
assert_eq!(
*validator_i,
@ -266,24 +203,21 @@ mod committees {
"Non-sequential validators."
);
// Assert a call to `get_attestation_duties` is consistent with a call to
// `get_crosslink_committees_at_slot`
// `get_beacon_committees_at_slot`
let attestation_duty = state
.get_attestation_duties(*validator_i, relative_epoch)
.unwrap()
.unwrap();
assert_eq!(attestation_duty.slot, slot);
assert_eq!(attestation_duty.shard, cc.shard);
assert_eq!(attestation_duty.committee_index, committee_i);
assert_eq!(attestation_duty.committee_len, cc.committee.len());
assert_eq!(attestation_duty.index, bc.index);
assert_eq!(attestation_duty.committee_position, committee_i);
assert_eq!(attestation_duty.committee_len, bc.committee.len());
}
}
}
// Assert that all validators were assigned to a committee.
assert!(expected_indices_iter.next().is_none());
// Assert that all shards were assigned to a committee.
assert!(expected_shards_iter.next().is_none());
}
fn committee_consistency_test<T: EthSpec>(
@ -327,7 +261,10 @@ mod committees {
fn committee_consistency_test_suite<T: EthSpec>(cached_epoch: RelativeEpoch) {
let spec = T::default_spec();
let validator_count = (T::shard_count() * spec.target_committee_size) + 1;
let validator_count = spec.max_committees_per_slot
* T::slots_per_epoch() as usize
* spec.target_committee_size
+ 1;
committee_consistency_test::<T>(validator_count as usize, Epoch::new(0), cached_epoch);

View File

@ -5,19 +5,18 @@ use utils::{u8_from_hex_str, u8_to_hex_str};
/// Each of the BLS signature domains.
///
/// Spec v0.8.1
/// Spec v0.9.1
pub enum Domain {
BeaconProposer,
BeaconAttester,
Randao,
Attestation,
Deposit,
VoluntaryExit,
Transfer,
}
/// Holds all the "constants" for a BeaconChain.
///
/// Spec v0.8.1
/// Spec v0.9.1
#[derive(PartialEq, Debug, Clone, Serialize, Deserialize)]
#[serde(default)]
pub struct ChainSpec {
@ -33,6 +32,7 @@ pub struct ChainSpec {
/*
* Misc
*/
pub max_committees_per_slot: usize,
pub target_committee_size: usize,
pub min_per_epoch_churn_limit: u64,
pub churn_limit_quotient: u64,
@ -61,10 +61,9 @@ pub struct ChainSpec {
pub milliseconds_per_slot: u64,
pub min_attestation_inclusion_delay: u64,
pub min_seed_lookahead: Epoch,
pub activation_exit_delay: u64,
pub max_seed_lookahead: Epoch,
pub min_validator_withdrawability_delay: Epoch,
pub persistent_committee_period: u64,
pub max_epochs_per_crosslink: u64,
pub min_epochs_to_inactivity_penalty: u64,
/*
@ -78,18 +77,17 @@ pub struct ChainSpec {
/*
* Signature domains
*
* Fields should be private to prevent accessing a domain that hasn't been modified to suit
* some `Fork`.
*
* Use `ChainSpec::get_domain(..)` to access these values.
*/
domain_beacon_proposer: u32,
domain_beacon_attester: u32,
domain_randao: u32,
domain_attestation: u32,
domain_deposit: u32,
domain_voluntary_exit: u32,
domain_transfer: u32,
/*
* Fork choice
*/
pub safe_slots_to_update_justified: u64,
/*
* Eth1
@ -103,18 +101,24 @@ pub struct ChainSpec {
}
impl ChainSpec {
/// Get the domain number that represents the fork meta and signature domain.
/// Get the domain number, unmodified by the fork.
///
/// Spec v0.8.1
pub fn get_domain(&self, epoch: Epoch, domain: Domain, fork: &Fork) -> u64 {
let domain_constant = match domain {
/// Spec v0.9.1
pub fn get_domain_constant(&self, domain: Domain) -> u32 {
match domain {
Domain::BeaconProposer => self.domain_beacon_proposer,
Domain::BeaconAttester => self.domain_beacon_attester,
Domain::Randao => self.domain_randao,
Domain::Attestation => self.domain_attestation,
Domain::Deposit => self.domain_deposit,
Domain::VoluntaryExit => self.domain_voluntary_exit,
Domain::Transfer => self.domain_transfer,
};
}
}
/// Get the domain number that represents the fork meta and signature domain.
///
/// Spec v0.9.1
pub fn get_domain(&self, epoch: Epoch, domain: Domain, fork: &Fork) -> u64 {
let domain_constant = self.get_domain_constant(domain);
let mut bytes: Vec<u8> = int_to_bytes4(domain_constant);
bytes.append(&mut fork.get_fork_version(epoch).to_vec());
@ -143,20 +147,21 @@ impl ChainSpec {
/// Returns a `ChainSpec` compatible with the Ethereum Foundation specification.
///
/// Spec v0.8.1
/// Spec v0.9.1
pub fn mainnet() -> Self {
Self {
/*
* Constants
*/
far_future_epoch: Epoch::new(u64::max_value()),
base_rewards_per_epoch: 5,
base_rewards_per_epoch: 4,
deposit_contract_tree_depth: 32,
seconds_per_day: 86400,
/*
* Misc
*/
max_committees_per_slot: 64,
target_committee_size: 128,
min_per_epoch_churn_limit: 4,
churn_limit_quotient: 65_536,
@ -181,13 +186,12 @@ impl ChainSpec {
/*
* Time parameters
*/
milliseconds_per_slot: 6_000,
milliseconds_per_slot: 12_000,
min_attestation_inclusion_delay: 1,
min_seed_lookahead: Epoch::new(1),
activation_exit_delay: 4,
max_seed_lookahead: Epoch::new(4),
min_validator_withdrawability_delay: Epoch::new(256),
persistent_committee_period: 2_048,
max_epochs_per_crosslink: 64,
min_epochs_to_inactivity_penalty: 4,
/*
@ -203,11 +207,15 @@ impl ChainSpec {
* Signature domains
*/
domain_beacon_proposer: 0,
domain_randao: 1,
domain_attestation: 2,
domain_beacon_attester: 1,
domain_randao: 2,
domain_deposit: 3,
domain_voluntary_exit: 4,
domain_transfer: 5,
/*
* Fork choice
*/
safe_slots_to_update_justified: 8,
/*
* Eth1
@ -235,7 +243,7 @@ impl ChainSpec {
///
/// https://github.com/ethereum/eth2.0-specs/blob/v0.8.1/configs/constant_presets/minimal.yaml
///
/// Spec v0.8.1
/// Spec v0.9.1
pub fn minimal() -> Self {
// Note: bootnodes to be updated when static nodes exist.
let boot_nodes = vec![];
@ -244,7 +252,6 @@ impl ChainSpec {
target_committee_size: 4,
shuffle_round_count: 10,
min_genesis_active_validator_count: 64,
max_epochs_per_crosslink: 4,
network_id: 2, // lighthouse testnet network id
boot_nodes,
eth1_follow_distance: 16,
@ -302,10 +309,9 @@ mod tests {
let spec = ChainSpec::mainnet();
test_domain(Domain::BeaconProposer, spec.domain_beacon_proposer, &spec);
test_domain(Domain::BeaconAttester, spec.domain_beacon_attester, &spec);
test_domain(Domain::Randao, spec.domain_randao, &spec);
test_domain(Domain::Attestation, spec.domain_attestation, &spec);
test_domain(Domain::Deposit, spec.domain_deposit, &spec);
test_domain(Domain::VoluntaryExit, spec.domain_voluntary_exit, &spec);
test_domain(Domain::Transfer, spec.domain_transfer, &spec);
}
}

View File

@ -7,7 +7,7 @@ use tree_hash_derive::TreeHash;
/// Casper FFG checkpoint, used in attestations.
///
/// Spec v0.8.0
/// Spec v0.9.1
#[derive(
Debug,
Clone,

View File

@ -1,24 +0,0 @@
use crate::test_utils::TestRandom;
use crate::{EthSpec, PublicKey};
use serde_derive::{Deserialize, Serialize};
use ssz_derive::{Decode, Encode};
use ssz_types::VariableList;
use test_random_derive::TestRandom;
use tree_hash_derive::TreeHash;
/// Spec v0.8.0
#[derive(Clone, Debug, PartialEq, TreeHash, Encode, Decode, Serialize, Deserialize, TestRandom)]
#[serde(bound = "T: EthSpec")]
pub struct CompactCommittee<T: EthSpec> {
pub pubkeys: VariableList<PublicKey, T::MaxValidatorsPerCommittee>,
pub compact_validators: VariableList<u64, T::MaxValidatorsPerCommittee>,
}
impl<T: EthSpec> Default for CompactCommittee<T> {
fn default() -> Self {
Self {
pubkeys: VariableList::empty(),
compact_validators: VariableList::empty(),
}
}
}

View File

@ -1,40 +0,0 @@
use crate::test_utils::TestRandom;
use crate::{Epoch, Hash256};
use serde_derive::{Deserialize, Serialize};
use ssz_derive::{Decode, Encode};
use test_random_derive::TestRandom;
use tree_hash_derive::TreeHash;
/// Specifies the block hash for a shard at an epoch.
///
/// Spec v0.8.0
#[derive(
Debug,
Clone,
PartialEq,
Eq,
Default,
Serialize,
Deserialize,
Hash,
Encode,
Decode,
TreeHash,
TestRandom,
)]
pub struct Crosslink {
pub shard: u64,
pub parent_root: Hash256,
// Crosslinking data
pub start_epoch: Epoch,
pub end_epoch: Epoch,
pub data_root: Hash256,
}
#[cfg(test)]
mod tests {
use super::*;
ssz_tests!(Crosslink);
}

View File

@ -11,7 +11,7 @@ pub const DEPOSIT_TREE_DEPTH: usize = 32;
/// A deposit to potentially become a beacon chain validator.
///
/// Spec v0.8.0
/// Spec v0.9.1
#[derive(Debug, PartialEq, Clone, Serialize, Deserialize, Encode, Decode, TreeHash, TestRandom)]
pub struct Deposit {
pub proof: FixedVector<Hash256, U33>,

View File

@ -11,7 +11,7 @@ use tree_hash_derive::{SignedRoot, TreeHash};
/// The data supplied by the user to the deposit contract.
///
/// Spec v0.8.0
/// Spec v0.9.1
#[derive(
Debug,
PartialEq,
@ -35,7 +35,7 @@ pub struct DepositData {
impl DepositData {
/// Generate the signature for a given DepositData details.
///
/// Spec v0.8.1
/// Spec v0.9.1
pub fn create_signature(&self, secret_key: &SecretKey, spec: &ChainSpec) -> SignatureBytes {
let msg = self.signed_root();
let domain = spec.get_deposit_domain();

View File

@ -8,7 +8,7 @@ use tree_hash_derive::TreeHash;
/// Contains data obtained from the Eth1 chain.
///
/// Spec v0.8.1
/// Spec v0.9.1
#[derive(
Debug,
PartialEq,

View File

@ -1,8 +1,8 @@
use crate::*;
use serde_derive::{Deserialize, Serialize};
use ssz_types::typenum::{
Unsigned, U0, U1, U1024, U1099511627776, U128, U16, U16777216, U4, U4096, U64, U65536, U8,
U8192,
Unsigned, U0, U1, U1024, U1099511627776, U128, U16, U16777216, U2048, U32, U4, U4096, U64,
U65536, U8, U8192,
};
use std::fmt::Debug;
@ -14,7 +14,6 @@ pub trait EthSpec: 'static + Default + Sync + Send + Clone + Debug + PartialEq {
/*
* Misc
*/
type ShardCount: Unsigned + Clone + Sync + Send + Debug + PartialEq;
type MaxValidatorsPerCommittee: Unsigned + Clone + Sync + Send + Debug + PartialEq;
/*
* Initial values
@ -41,7 +40,6 @@ pub trait EthSpec: 'static + Default + Sync + Send + Clone + Debug + PartialEq {
type MaxAttestations: Unsigned + Clone + Sync + Send + Debug + PartialEq;
type MaxDeposits: Unsigned + Clone + Sync + Send + Debug + PartialEq;
type MaxVoluntaryExits: Unsigned + Clone + Sync + Send + Debug + PartialEq;
type MaxTransfers: Unsigned + Clone + Sync + Send + Debug + PartialEq;
/*
* Derived values (set these CAREFULLY)
*/
@ -58,29 +56,21 @@ pub trait EthSpec: 'static + Default + Sync + Send + Clone + Debug + PartialEq {
Epoch::new(Self::GenesisEpoch::to_u64())
}
/// Return the number of committees in one epoch.
/// Return the number of committees per slot.
///
/// Spec v0.8.1
fn get_committee_count(active_validator_count: usize, target_committee_size: usize) -> usize {
let shard_count = Self::shard_count();
let slots_per_epoch = Self::slots_per_epoch() as usize;
/// Note: the number of committees per slot is constant in each epoch, and depends only on
/// the `active_validator_count` during the slot's epoch.
///
/// Spec v0.9.1
fn get_committee_count_per_slot(active_validator_count: usize, spec: &ChainSpec) -> usize {
let slots_per_epoch = Self::SlotsPerEpoch::to_usize();
std::cmp::max(
1,
std::cmp::min(
shard_count / slots_per_epoch,
active_validator_count / slots_per_epoch / target_committee_size,
spec.max_committees_per_slot,
active_validator_count / slots_per_epoch / spec.target_committee_size,
),
) * slots_per_epoch
}
/// Return the number of shards to increment `state.start_shard` by in a given epoch.
///
/// Spec v0.8.1
fn get_shard_delta(active_validator_count: usize, target_committee_size: usize) -> u64 {
std::cmp::min(
Self::get_committee_count(active_validator_count, target_committee_size) as u64,
Self::ShardCount::to_u64() - Self::ShardCount::to_u64() / Self::slots_per_epoch(),
)
}
@ -95,37 +85,30 @@ pub trait EthSpec: 'static + Default + Sync + Send + Clone + Debug + PartialEq {
/// Returns the `SLOTS_PER_EPOCH` constant for this specification.
///
/// Spec v0.8.1
/// Spec v0.9.1
fn slots_per_epoch() -> u64 {
Self::SlotsPerEpoch::to_u64()
}
/// Returns the `SHARD_COUNT` constant for this specification.
///
/// Spec v0.8.1
fn shard_count() -> usize {
Self::ShardCount::to_usize()
}
/// Returns the `SLOTS_PER_HISTORICAL_ROOT` constant for this specification.
///
/// Spec v0.8.1
/// Spec v0.9.1
fn slots_per_historical_root() -> usize {
Self::SlotsPerHistoricalRoot::to_usize()
}
/// Returns the `EPOCHS_PER_HISTORICAL_VECTOR` constant for this specification.
///
/// Spec v0.8.1
/// Spec v0.9.1
fn epochs_per_historical_vector() -> usize {
Self::EpochsPerHistoricalVector::to_usize()
}
/// Returns the `SLOTS_PER_ETH1_VOTING_PERIOD` constant for this specification.
///
/// Spec v0.8.1
/// Spec v0.9.1
fn slots_per_eth1_voting_period() -> usize {
Self::EpochsPerHistoricalVector::to_usize()
Self::SlotsPerEth1VotingPeriod::to_usize()
}
}
@ -139,16 +122,15 @@ macro_rules! params_from_eth_spec {
/// Ethereum Foundation specifications.
///
/// Spec v0.8.0
/// Spec v0.9.1
#[derive(Clone, PartialEq, Debug, Default, Serialize, Deserialize)]
pub struct MainnetEthSpec;
impl EthSpec for MainnetEthSpec {
type JustificationBitsLength = U4;
type ShardCount = U1024;
type MaxValidatorsPerCommittee = U4096;
type MaxValidatorsPerCommittee = U2048;
type GenesisEpoch = U0;
type SlotsPerEpoch = U64;
type SlotsPerEpoch = U32;
type SlotsPerEth1VotingPeriod = U1024;
type SlotsPerHistoricalRoot = U8192;
type EpochsPerHistoricalVector = U65536;
@ -160,8 +142,7 @@ impl EthSpec for MainnetEthSpec {
type MaxAttestations = U128;
type MaxDeposits = U16;
type MaxVoluntaryExits = U16;
type MaxTransfers = U0;
type MaxPendingAttestations = U8192; // 128 max attestations * 64 slots per epoch
type MaxPendingAttestations = U4096; // 128 max attestations * 32 slots per epoch
fn default_spec() -> ChainSpec {
ChainSpec::mainnet()
@ -174,12 +155,11 @@ pub type FoundationBeaconState = BeaconState<MainnetEthSpec>;
///
/// https://github.com/ethereum/eth2.0-specs/blob/v0.8.0/configs/constant_presets/minimal.yaml
///
/// Spec v0.8.0
/// Spec v0.9.1
#[derive(Clone, PartialEq, Debug, Default, Serialize, Deserialize)]
pub struct MinimalEthSpec;
impl EthSpec for MinimalEthSpec {
type ShardCount = U8;
type SlotsPerEpoch = U8;
type SlotsPerEth1VotingPeriod = U16;
type SlotsPerHistoricalRoot = U64;
@ -197,8 +177,7 @@ impl EthSpec for MinimalEthSpec {
MaxAttesterSlashings,
MaxAttestations,
MaxDeposits,
MaxVoluntaryExits,
MaxTransfers
MaxVoluntaryExits
});
fn default_spec() -> ChainSpec {
@ -213,7 +192,6 @@ pub type MinimalBeaconState = BeaconState<MinimalEthSpec>;
pub struct InteropEthSpec;
impl EthSpec for InteropEthSpec {
type ShardCount = U8;
type SlotsPerEpoch = U8;
type SlotsPerHistoricalRoot = U64;
type SlotsPerEth1VotingPeriod = U16;
@ -231,8 +209,7 @@ impl EthSpec for InteropEthSpec {
MaxAttesterSlashings,
MaxAttestations,
MaxDeposits,
MaxVoluntaryExits,
MaxTransfers
MaxVoluntaryExits
});
fn default_spec() -> ChainSpec {

View File

@ -9,7 +9,7 @@ use tree_hash_derive::TreeHash;
/// Specifies a fork of the `BeaconChain`, to prevent replay attacks.
///
/// Spec v0.8.1
/// Spec v0.9.1
#[derive(
Debug, Clone, PartialEq, Default, Serialize, Deserialize, Encode, Decode, TreeHash, TestRandom,
)]
@ -30,7 +30,7 @@ pub struct Fork {
impl Fork {
/// Return the fork version of the given ``epoch``.
///
/// Spec v0.8.1
/// Spec v0.9.1
pub fn get_fork_version(&self, epoch: Epoch) -> [u8; 4] {
if epoch < self.epoch {
return self.previous_version;

View File

@ -9,7 +9,7 @@ use tree_hash_derive::TreeHash;
/// Historical block and state roots.
///
/// Spec v0.8.1
/// Spec v0.9.1
#[derive(Debug, Clone, PartialEq, Serialize, Deserialize, Encode, Decode, TreeHash, TestRandom)]
pub struct HistoricalBatch<T: EthSpec> {
pub block_roots: FixedVector<Hash256, T::SlotsPerHistoricalRoot>,

View File

@ -9,7 +9,7 @@ use tree_hash_derive::{SignedRoot, TreeHash};
///
/// To be included in an `AttesterSlashing`.
///
/// Spec v0.8.0
/// Spec v0.9.1
#[derive(
Debug,
PartialEq,
@ -25,8 +25,7 @@ use tree_hash_derive::{SignedRoot, TreeHash};
#[serde(bound = "T: EthSpec")]
pub struct IndexedAttestation<T: EthSpec> {
/// Lists validator registry indices, not committee indices.
pub custody_bit_0_indices: VariableList<u64, T::MaxValidatorsPerCommittee>,
pub custody_bit_1_indices: VariableList<u64, T::MaxValidatorsPerCommittee>,
pub attesting_indices: VariableList<u64, T::MaxValidatorsPerCommittee>,
pub data: AttestationData,
#[signed_root(skip_hashing)]
pub signature: AggregateSignature,
@ -35,14 +34,14 @@ pub struct IndexedAttestation<T: EthSpec> {
impl<T: EthSpec> IndexedAttestation<T> {
/// Check if ``attestation_data_1`` and ``attestation_data_2`` have the same target.
///
/// Spec v0.8.0
/// Spec v0.9.1
pub fn is_double_vote(&self, other: &Self) -> bool {
self.data.target.epoch == other.data.target.epoch && self.data != other.data
}
/// Check if ``attestation_data_1`` surrounds ``attestation_data_2``.
///
/// Spec v0.8.0
/// Spec v0.9.1
pub fn is_surround_vote(&self, other: &Self) -> bool {
self.data.source.epoch < other.data.source.epoch
&& other.data.target.epoch < self.data.target.epoch

View File

@ -8,18 +8,15 @@ pub mod test_utils;
pub mod attestation;
pub mod attestation_data;
pub mod attestation_data_and_custody_bit;
pub mod attestation_duty;
pub mod attester_slashing;
pub mod beacon_block;
pub mod beacon_block_body;
pub mod beacon_block_header;
pub mod beacon_committee;
pub mod beacon_state;
pub mod chain_spec;
pub mod checkpoint;
pub mod compact_committee;
pub mod crosslink;
pub mod crosslink_committee;
pub mod deposit;
pub mod deposit_data;
pub mod eth1_data;
@ -30,7 +27,6 @@ pub mod historical_batch;
pub mod indexed_attestation;
pub mod pending_attestation;
pub mod proposer_slashing;
pub mod transfer;
pub mod utils;
pub mod voluntary_exit;
#[macro_use]
@ -41,23 +37,19 @@ pub mod slot_height;
mod tree_hash_impls;
pub mod validator;
use ethereum_types::{H160, H256, U256};
use std::collections::HashMap;
use ethereum_types::{H160, H256};
pub use crate::attestation::Attestation;
pub use crate::attestation_data::AttestationData;
pub use crate::attestation_data_and_custody_bit::AttestationDataAndCustodyBit;
pub use crate::attestation_duty::AttestationDuty;
pub use crate::attester_slashing::AttesterSlashing;
pub use crate::beacon_block::BeaconBlock;
pub use crate::beacon_block_body::BeaconBlockBody;
pub use crate::beacon_block_header::BeaconBlockHeader;
pub use crate::beacon_committee::{BeaconCommittee, OwnedBeaconCommittee};
pub use crate::beacon_state::{Error as BeaconStateError, *};
pub use crate::chain_spec::{ChainSpec, Domain};
pub use crate::checkpoint::Checkpoint;
pub use crate::compact_committee::CompactCommittee;
pub use crate::crosslink::Crosslink;
pub use crate::crosslink_committee::{CrosslinkCommittee, OwnedCrosslinkCommittee};
pub use crate::deposit::{Deposit, DEPOSIT_TREE_DEPTH};
pub use crate::deposit_data::DepositData;
pub use crate::eth1_data::Eth1Data;
@ -70,23 +62,12 @@ pub use crate::proposer_slashing::ProposerSlashing;
pub use crate::relative_epoch::{Error as RelativeEpochError, RelativeEpoch};
pub use crate::slot_epoch::{Epoch, Slot};
pub use crate::slot_height::SlotHeight;
pub use crate::transfer::Transfer;
pub use crate::validator::Validator;
pub use crate::voluntary_exit::VoluntaryExit;
pub type Shard = u64;
pub type Committee = Vec<usize>;
pub type CrosslinkCommittees = Vec<(Committee, u64)>;
pub type CommitteeIndex = u64;
pub type Hash256 = H256;
pub type Address = H160;
pub type EthBalance = U256;
/// Maps a (slot, shard_id) to attestation_indices.
pub type AttesterMap = HashMap<(u64, u64), Vec<usize>>;
/// Maps a slot to a block proposer.
pub type ProposerMap = HashMap<u64, usize>;
pub use bls::{
AggregatePublicKey, AggregateSignature, Keypair, PublicKey, PublicKeyBytes, SecretKey,

View File

@ -8,7 +8,7 @@ use tree_hash_derive::TreeHash;
/// An attestation that has been included in the state but not yet fully processed.
///
/// Spec v0.8.0
/// Spec v0.9.1
#[derive(Debug, Clone, PartialEq, Serialize, Deserialize, Encode, Decode, TreeHash, TestRandom)]
pub struct PendingAttestation<T: EthSpec> {
pub aggregation_bits: BitList<T::MaxValidatorsPerCommittee>,

View File

@ -8,7 +8,7 @@ use tree_hash_derive::TreeHash;
/// Two conflicting proposals from the same proposer (validator).
///
/// Spec v0.8.1
/// Spec v0.9.1
#[derive(Debug, PartialEq, Clone, Serialize, Deserialize, Encode, Decode, TreeHash, TestRandom)]
pub struct ProposerSlashing {
pub proposer_index: u64,

View File

@ -9,7 +9,7 @@ pub enum Error {
/// Defines the epochs relative to some epoch. Most useful when referring to the committees prior
/// to and following some epoch.
///
/// Spec v0.8.1
/// Spec v0.9.1
#[derive(Debug, PartialEq, Clone, Copy)]
pub enum RelativeEpoch {
/// The prior epoch.
@ -23,7 +23,7 @@ pub enum RelativeEpoch {
impl RelativeEpoch {
/// Returns the `epoch` that `self` refers to, with respect to the `base` epoch.
///
/// Spec v0.8.1
/// Spec v0.9.1
pub fn into_epoch(self, base: Epoch) -> Epoch {
match self {
// Due to saturating nature of epoch, check for current first.
@ -40,7 +40,7 @@ impl RelativeEpoch {
/// - `EpochTooLow` when `other` is more than 1 prior to `base`.
/// - `EpochTooHigh` when `other` is more than 1 after `base`.
///
/// Spec v0.8.1
/// Spec v0.9.1
pub fn from_epoch(base: Epoch, other: Epoch) -> Result<Self, Error> {
// Due to saturating nature of epoch, check for current first.
if other == base {

View File

@ -6,7 +6,6 @@ mod testing_beacon_state_builder;
mod testing_deposit_builder;
mod testing_pending_attestation_builder;
mod testing_proposer_slashing_builder;
mod testing_transfer_builder;
mod testing_voluntary_exit_builder;
pub use testing_attestation_builder::*;
@ -17,5 +16,4 @@ pub use testing_beacon_state_builder::*;
pub use testing_deposit_builder::*;
pub use testing_pending_attestation_builder::*;
pub use testing_proposer_slashing_builder::*;
pub use testing_transfer_builder::*;
pub use testing_voluntary_exit_builder::*;

View File

@ -13,35 +13,31 @@ pub struct TestingAttestationBuilder<T: EthSpec> {
impl<T: EthSpec> TestingAttestationBuilder<T> {
/// Create a new attestation builder.
pub fn new(
test_task: &AttestationTestTask,
test_task: AttestationTestTask,
state: &BeaconState<T>,
committee: &[usize],
slot: Slot,
shard: u64,
index: u64,
spec: &ChainSpec,
) -> Self {
let data_builder = TestingAttestationDataBuilder::new(test_task, state, shard, slot, spec);
let data_builder = TestingAttestationDataBuilder::new(test_task, state, index, slot, spec);
let mut aggregation_bits_len = committee.len();
let mut custody_bits_len = committee.len();
match test_task {
AttestationTestTask::BadAggregationBitfieldLen => aggregation_bits_len += 1,
AttestationTestTask::BadCustodyBitfieldLen => custody_bits_len += 1,
_ => (),
}
let mut aggregation_bits = BitList::with_capacity(aggregation_bits_len).unwrap();
let mut custody_bits = BitList::with_capacity(custody_bits_len).unwrap();
for i in 0..committee.len() {
custody_bits.set(i, false).unwrap();
aggregation_bits.set(i, false).unwrap();
}
let attestation = Attestation {
aggregation_bits,
data: data_builder.build(),
custody_bits,
signature: AggregateSignature::new(),
};
@ -57,12 +53,11 @@ impl<T: EthSpec> TestingAttestationBuilder<T> {
/// keypair must be that of the first signing validator.
pub fn sign(
&mut self,
test_task: &AttestationTestTask,
test_task: AttestationTestTask,
signing_validators: &[usize],
secret_keys: &[&SecretKey],
fork: &Fork,
spec: &ChainSpec,
mut custody_bit: bool,
) -> &mut Self {
assert_eq!(
signing_validators.len(),
@ -79,7 +74,6 @@ impl<T: EthSpec> TestingAttestationBuilder<T> {
match test_task {
AttestationTestTask::BadIndexedAttestationBadSignature => (),
AttestationTestTask::CustodyBitfieldNotSubset => custody_bit = true,
_ => {
self.attestation
.aggregation_bits
@ -87,29 +81,16 @@ impl<T: EthSpec> TestingAttestationBuilder<T> {
.unwrap();
}
}
match (custody_bit, test_task) {
(true, _) | (_, AttestationTestTask::CustodyBitfieldHasSetBits) => {
self.attestation
.custody_bits
.set(committee_index, true)
.unwrap();
}
(false, _) => (),
}
let message = AttestationDataAndCustodyBit {
data: self.attestation.data.clone(),
custody_bit,
}
.tree_hash_root();
let message = self.attestation.data.tree_hash_root();
let domain = spec.get_domain(
self.attestation.data.target.epoch,
Domain::Attestation,
Domain::BeaconAttester,
fork,
);
let index = if *test_task == AttestationTestTask::BadSignature {
let index = if test_task == AttestationTestTask::BadSignature {
0
} else {
key_index

View File

@ -1,6 +1,5 @@
use crate::test_utils::AttestationTestTask;
use crate::*;
use tree_hash::TreeHash;
/// Builds an `AttestationData` to be used for testing purposes.
///
@ -13,10 +12,10 @@ impl TestingAttestationDataBuilder {
/// Configures a new `AttestationData` which attests to all of the same parameters as the
/// state.
pub fn new<T: EthSpec>(
test_task: &AttestationTestTask,
test_task: AttestationTestTask,
state: &BeaconState<T>,
mut shard: u64,
slot: Slot,
index: u64,
mut slot: Slot,
spec: &ChainSpec,
) -> Self {
let current_epoch = state.current_epoch();
@ -46,34 +45,16 @@ impl TestingAttestationDataBuilder {
}
};
let parent_crosslink = if is_previous_epoch {
state.get_previous_crosslink(shard).unwrap()
} else {
state.get_current_crosslink(shard).unwrap()
};
let mut start = parent_crosslink.end_epoch;
let mut end = std::cmp::min(
target.epoch,
parent_crosslink.end_epoch + spec.max_epochs_per_crosslink,
);
let mut parent_root = Hash256::from_slice(&parent_crosslink.tree_hash_root());
let mut data_root = Hash256::zero();
let beacon_block_root = *state.get_block_root(slot).unwrap();
match test_task {
AttestationTestTask::BadParentCrosslinkStartEpoch => start = Epoch::from(10 as u64),
AttestationTestTask::BadParentCrosslinkEndEpoch => end = Epoch::from(0 as u64),
AttestationTestTask::BadParentCrosslinkHash => parent_root = Hash256::zero(),
AttestationTestTask::NoCommiteeForShard => shard += 2,
AttestationTestTask::BadShard => shard = T::ShardCount::to_u64(),
AttestationTestTask::IncludedTooEarly => shard += 1,
AttestationTestTask::IncludedTooLate => {
target = Checkpoint {
epoch: Epoch::from(3 as u64),
root: Hash256::zero(),
}
// FIXME: re-enable the shard-like tests
// AttestationTestTask::NoCommiteeForShard => index += 2,
// AttestationTestTask::BadShard => index = T::ShardCount::to_u64(),
AttestationTestTask::IncludedTooEarly => {
slot = state.slot - spec.min_attestation_inclusion_delay + 1
}
AttestationTestTask::IncludedTooLate => slot -= T::SlotsPerEpoch::to_u64(),
AttestationTestTask::BadTargetEpoch => {
target = Checkpoint {
epoch: Epoch::from(5 as u64),
@ -98,27 +79,19 @@ impl TestingAttestationDataBuilder {
root: Hash256::zero(),
}
}
AttestationTestTask::BadParentCrosslinkDataRoot => data_root = parent_root,
_ => (),
}
let crosslink = Crosslink {
shard,
parent_root,
start_epoch: start,
end_epoch: end,
data_root,
};
let data = AttestationData {
slot,
index,
// LMD GHOST vote
beacon_block_root,
// FFG Vote
source,
target,
// Crosslink vote
crosslink,
};
Self { data }

View File

@ -19,14 +19,15 @@ impl TestingAttesterSlashingBuilder {
///
/// Where domain is a domain "constant" (e.g., `spec.domain_attestation`).
pub fn double_vote<F, T: EthSpec>(
test_task: &AttesterSlashingTestTask,
test_task: AttesterSlashingTestTask,
validator_indices: &[u64],
signer: F,
) -> AttesterSlashing<T>
where
F: Fn(u64, &[u8], Epoch, Domain) -> Signature,
{
let shard = 0;
let slot = Slot::new(1);
let index = 0;
let epoch_1 = Epoch::new(1);
let epoch_2 = Epoch::new(2);
let hash_1 = Hash256::from_low_u64_le(1);
@ -39,22 +40,16 @@ impl TestingAttesterSlashingBuilder {
epoch: epoch_1,
root: hash_2,
};
let crosslink = Crosslink {
shard,
parent_root: hash_1,
start_epoch: epoch_1,
end_epoch: epoch_2,
data_root: hash_1,
};
let data_1 = AttestationData {
slot,
index,
beacon_block_root: hash_1,
source: checkpoint_1.clone(),
target: checkpoint_1,
crosslink,
};
let data_2 = if *test_task == AttesterSlashingTestTask::NotSlashable {
let data_2 = if test_task == AttesterSlashingTestTask::NotSlashable {
AttestationData { ..data_1.clone() }
} else {
AttestationData {
@ -64,42 +59,39 @@ impl TestingAttesterSlashingBuilder {
};
let mut attestation_1 = IndexedAttestation {
custody_bit_0_indices: validator_indices.to_vec().into(),
custody_bit_1_indices: if *test_task
== AttesterSlashingTestTask::IndexedAttestation1Invalid
attesting_indices: if test_task == AttesterSlashingTestTask::IndexedAttestation1Invalid
{
validator_indices.to_vec().into()
// Trigger bad validator indices ordering error.
vec![1, 0].into()
} else {
VariableList::empty()
validator_indices.to_vec().into()
},
data: data_1,
signature: AggregateSignature::new(),
};
let mut attestation_2 = IndexedAttestation {
custody_bit_0_indices: validator_indices.to_vec().into(),
custody_bit_1_indices: if *test_task
== AttesterSlashingTestTask::IndexedAttestation2Invalid
attesting_indices: if test_task == AttesterSlashingTestTask::IndexedAttestation2Invalid
{
validator_indices.to_vec().into()
// Trigger bad validator indices ordering error.
vec![1, 0].into()
} else {
VariableList::empty()
validator_indices.to_vec().into()
},
data: data_2,
signature: AggregateSignature::new(),
};
let add_signatures = |attestation: &mut IndexedAttestation<T>| {
// All validators sign with a `false` custody bit.
let attestation_data_and_custody_bit = AttestationDataAndCustodyBit {
data: attestation.data.clone(),
custody_bit: false,
};
let message = attestation_data_and_custody_bit.tree_hash_root();
let message = attestation.data.tree_hash_root();
for validator_index in validator_indices {
let signature =
signer(*validator_index, &message[..], epoch_2, Domain::Attestation);
let signature = signer(
*validator_index,
&message[..],
epoch_2,
Domain::BeaconAttester,
);
attestation.signature.add(&signature);
}
};

View File

@ -1,7 +1,7 @@
use crate::{
test_utils::{
TestingAttestationBuilder, TestingAttesterSlashingBuilder, TestingDepositBuilder,
TestingProposerSlashingBuilder, TestingTransferBuilder, TestingVoluntaryExitBuilder,
TestingProposerSlashingBuilder, TestingVoluntaryExitBuilder,
},
typenum::U4294967296,
*,
@ -19,7 +19,7 @@ pub struct TestingBeaconBlockBuilder<T: EthSpec> {
}
/// Enum used for passing test options to builder
#[derive(PartialEq)]
#[derive(PartialEq, Clone, Copy)]
pub enum DepositTestTask {
Valid,
BadPubKey,
@ -29,6 +29,7 @@ pub enum DepositTestTask {
}
/// Enum used for passing test options to builder
#[derive(PartialEq, Clone, Copy)]
pub enum ExitTestTask {
AlreadyInitiated,
AlreadyExited,
@ -39,23 +40,16 @@ pub enum ExitTestTask {
ValidatorUnknown,
}
#[derive(PartialEq)]
/// Enum used for passing test options to builder
#[derive(PartialEq, Clone, Copy)]
pub enum AttestationTestTask {
Valid,
BadParentCrosslinkStartEpoch,
BadParentCrosslinkEndEpoch,
BadParentCrosslinkHash,
NoCommiteeForShard,
WrongJustifiedCheckpoint,
BadTargetTooLow,
BadTargetTooHigh,
BadShard,
BadParentCrosslinkDataRoot,
BadIndexedAttestationBadSignature,
CustodyBitfieldNotSubset,
CustodyBitfieldHasSetBits,
BadCustodyBitfieldLen,
BadAggregationBitfieldLen,
BadSignature,
ValidatorUnknown,
@ -64,8 +58,8 @@ pub enum AttestationTestTask {
BadTargetEpoch,
}
#[derive(PartialEq)]
/// Enum used for passing test options to builder
#[derive(PartialEq, Clone, Copy)]
pub enum AttesterSlashingTestTask {
Valid,
NotSlashable,
@ -74,7 +68,7 @@ pub enum AttesterSlashingTestTask {
}
/// Enum used for passing test options to builder
#[derive(PartialEq)]
#[derive(PartialEq, Clone, Copy)]
pub enum ProposerSlashingTestTask {
Valid,
ProposerUnknown,
@ -131,7 +125,7 @@ impl<T: EthSpec> TestingBeaconBlockBuilder<T> {
/// Inserts a signed, valid `ProposerSlashing` for the validator.
pub fn insert_proposer_slashing(
&mut self,
test_task: &ProposerSlashingTestTask,
test_task: ProposerSlashingTestTask,
validator_index: u64,
secret_key: &SecretKey,
fork: &Fork,
@ -149,7 +143,7 @@ impl<T: EthSpec> TestingBeaconBlockBuilder<T> {
/// Inserts a signed, valid `AttesterSlashing` for each validator index in `validator_indices`.
pub fn insert_attester_slashing(
&mut self,
test_task: &AttesterSlashingTestTask,
test_task: AttesterSlashingTestTask,
validator_indices: &[u64],
secret_keys: &[&SecretKey],
fork: &Fork,
@ -176,7 +170,7 @@ impl<T: EthSpec> TestingBeaconBlockBuilder<T> {
/// to aggregate these split attestations.
pub fn insert_attestations(
&mut self,
test_task: &AttestationTestTask,
test_task: AttestationTestTask,
state: &BeaconState<T>,
secret_keys: &[&SecretKey],
num_attestations: usize,
@ -190,7 +184,7 @@ impl<T: EthSpec> TestingBeaconBlockBuilder<T> {
// - The slot of the committee.
// - A list of all validators in the committee.
// - A list of all validators in the committee that should sign the attestation.
// - The shard of the committee.
// - The index of the committee.
let mut committees: Vec<(Slot, Vec<usize>, Vec<usize>, u64)> = vec![];
if slot < T::slots_per_epoch() {
@ -206,16 +200,16 @@ impl<T: EthSpec> TestingBeaconBlockBuilder<T> {
break;
}
for crosslink_committee in state.get_crosslink_committees_at_slot(slot)? {
for beacon_committee in state.get_beacon_committees_at_slot(slot)? {
if attestations_added >= num_attestations {
break;
}
committees.push((
slot,
crosslink_committee.committee.to_vec(),
crosslink_committee.committee.to_vec(),
crosslink_committee.shard,
beacon_committee.committee.to_vec(),
beacon_committee.committee.to_vec(),
beacon_committee.index,
));
attestations_added += 1;
@ -231,26 +225,26 @@ impl<T: EthSpec> TestingBeaconBlockBuilder<T> {
break;
}
for index in 0..committees.len() {
for i in 0..committees.len() {
if committees.len() >= num_attestations as usize {
break;
}
let (slot, committee, mut signing_validators, shard) = committees[index].clone();
let (slot, committee, mut signing_validators, index) = committees[i].clone();
let new_signing_validators =
signing_validators.split_off(signing_validators.len() / 2);
committees[index] = (slot, committee.clone(), signing_validators, shard);
committees.push((slot, committee, new_signing_validators, shard));
committees[i] = (slot, committee.clone(), signing_validators, index);
committees.push((slot, committee, new_signing_validators, index));
}
}
let attestations: Vec<_> = committees
.par_iter()
.map(|(slot, committee, signing_validators, shard)| {
.map(|(slot, committee, signing_validators, index)| {
let mut builder = TestingAttestationBuilder::new(
test_task, state, committee, *slot, *shard, spec,
test_task, state, committee, *slot, *index, spec,
);
let signing_secret_keys: Vec<&SecretKey> = signing_validators
@ -263,7 +257,6 @@ impl<T: EthSpec> TestingBeaconBlockBuilder<T> {
&signing_secret_keys,
&state.fork,
spec,
false,
);
builder.build()
@ -294,7 +287,7 @@ impl<T: EthSpec> TestingBeaconBlockBuilder<T> {
let keypair = Keypair::random();
let mut builder = TestingDepositBuilder::new(keypair.pk.clone(), amount);
builder.sign(&test_task, &keypair, spec);
builder.sign(test_task, &keypair, spec);
datas.push(builder.build().data);
}
@ -347,7 +340,7 @@ impl<T: EthSpec> TestingBeaconBlockBuilder<T> {
/// Insert a `Valid` exit into the state.
pub fn insert_exit(
&mut self,
test_task: &ExitTestTask,
test_task: ExitTestTask,
state: &mut BeaconState<T>,
mut validator_index: u64,
secret_key: &SecretKey,
@ -362,6 +355,7 @@ impl<T: EthSpec> TestingBeaconBlockBuilder<T> {
ExitTestTask::AlreadyExited => {
state.validators[validator_index as usize].exit_epoch = Epoch::from(314_159 as u64)
}
// FIXME: disabled in v0.9
ExitTestTask::NotActive => {
state.validators[validator_index as usize].activation_epoch =
Epoch::from(314_159 as u64)
@ -381,25 +375,6 @@ impl<T: EthSpec> TestingBeaconBlockBuilder<T> {
.unwrap();
}
/// Insert a `Valid` transfer into the state.
///
/// Note: this will set the validator to be withdrawable by directly modifying the state
/// validator registry. This _may_ cause problems historic hashes, etc.
pub fn insert_transfer(
&mut self,
state: &BeaconState<T>,
from: u64,
to: u64,
amount: u64,
keypair: Keypair,
spec: &ChainSpec,
) {
let mut builder = TestingTransferBuilder::new(from, to, amount, state.slot);
builder.sign::<T>(keypair, &state.fork, spec);
self.block.body.transfers.push(builder.build()).unwrap()
}
/// Signs and returns the block, consuming the builder.
pub fn build(mut self, sk: &SecretKey, fork: &Fork, spec: &ChainSpec) -> BeaconBlock<T> {
self.sign(sk, fork, spec);
@ -416,7 +391,7 @@ impl<T: EthSpec> TestingBeaconBlockBuilder<T> {
///
/// Signs the message using a `BeaconChainHarness`.
fn build_proposer_slashing<T: EthSpec>(
test_task: &ProposerSlashingTestTask,
test_task: ProposerSlashingTestTask,
validator_index: u64,
secret_key: &SecretKey,
fork: &Fork,
@ -434,7 +409,7 @@ fn build_proposer_slashing<T: EthSpec>(
///
/// Signs the message using a `BeaconChainHarness`.
fn build_double_vote_attester_slashing<T: EthSpec>(
test_task: &AttesterSlashingTestTask,
test_task: AttesterSlashingTestTask,
validator_indices: &[u64],
secret_keys: &[&SecretKey],
fork: &Fork,

View File

@ -181,8 +181,6 @@ impl<T: EthSpec> TestingBeaconStateBuilder<T> {
state.slot = slot;
// NOTE: we could update the latest start shard here
state.previous_justified_checkpoint.epoch = epoch - 3;
state.current_justified_checkpoint.epoch = epoch - 2;
state.justification_bits = BitVector::from_bytes(vec![0b0000_1111]).unwrap();
@ -215,23 +213,23 @@ impl<T: EthSpec> TestingBeaconStateBuilder<T> {
for slot in first_slot..=last_slot {
let slot = Slot::from(slot);
let committees: Vec<OwnedCrosslinkCommittee> = state
.get_crosslink_committees_at_slot(slot)
let committees: Vec<OwnedBeaconCommittee> = state
.get_beacon_committees_at_slot(slot)
.unwrap()
.into_iter()
.map(|c| c.clone().into_owned())
.collect();
for crosslink_committee in committees {
for beacon_committee in committees {
let mut builder = TestingPendingAttestationBuilder::new(
&AttestationTestTask::Valid,
AttestationTestTask::Valid,
state,
crosslink_committee.shard,
beacon_committee.index,
slot,
spec,
);
// The entire committee should have signed the pending attestation.
let signers = vec![true; crosslink_committee.committee.len()];
let signers = vec![true; beacon_committee.committee.len()];
builder.add_committee_participation(signers);
let attestation = builder.build();

View File

@ -30,7 +30,7 @@ impl TestingDepositBuilder {
/// - `pubkey` to the signing pubkey.
/// - `withdrawal_credentials` to the signing pubkey.
/// - `proof_of_possession`
pub fn sign(&mut self, test_task: &DepositTestTask, keypair: &Keypair, spec: &ChainSpec) {
pub fn sign(&mut self, test_task: DepositTestTask, keypair: &Keypair, spec: &ChainSpec) {
let new_key = Keypair::random();
let mut pubkeybytes = PublicKeyBytes::from(keypair.pk.clone());
let mut secret_key = keypair.sk.clone();

View File

@ -13,23 +13,18 @@ impl<T: EthSpec> TestingPendingAttestationBuilder<T> {
///
/// The `inclusion_delay` will be set to `MIN_ATTESTATION_INCLUSION_DELAY`.
///
/// * The aggregation and custody bitfields will all be empty, they need to be set with
/// * The aggregation bitfield will be empty, it needs to be set with
/// `Self::add_committee_participation`.
pub fn new(
test_task: &AttestationTestTask,
test_task: AttestationTestTask,
state: &BeaconState<T>,
shard: u64,
index: u64,
slot: Slot,
spec: &ChainSpec,
) -> Self {
let data_builder = TestingAttestationDataBuilder::new(test_task, state, shard, slot, spec);
let data_builder = TestingAttestationDataBuilder::new(test_task, state, index, slot, spec);
let relative_epoch =
RelativeEpoch::from_epoch(state.current_epoch(), slot.epoch(T::slots_per_epoch()))
.expect("epoch out of bounds");
let proposer_index = state
.get_beacon_proposer_index(slot, relative_epoch, spec)
.unwrap() as u64;
let proposer_index = state.get_beacon_proposer_index(slot, spec).unwrap() as u64;
let pending_attestation = PendingAttestation {
aggregation_bits: BitList::with_capacity(T::MaxValidatorsPerCommittee::to_usize())

View File

@ -19,7 +19,7 @@ impl TestingProposerSlashingBuilder {
///
/// Where domain is a domain "constant" (e.g., `spec.domain_attestation`).
pub fn double_vote<T, F>(
test_task: &ProposerSlashingTestTask,
test_task: ProposerSlashingTestTask,
mut proposer_index: u64,
signer: F,
) -> ProposerSlashing
@ -29,7 +29,7 @@ impl TestingProposerSlashingBuilder {
{
let slot = Slot::new(0);
let hash_1 = Hash256::from([1; 32]);
let hash_2 = if *test_task == ProposerSlashingTestTask::ProposalsIdentical {
let hash_2 = if test_task == ProposerSlashingTestTask::ProposalsIdentical {
hash_1.clone()
} else {
Hash256::from([2; 32])
@ -43,7 +43,7 @@ impl TestingProposerSlashingBuilder {
signature: Signature::empty_signature(),
};
let slot_2 = if *test_task == ProposerSlashingTestTask::ProposalEpochMismatch {
let slot_2 = if test_task == ProposerSlashingTestTask::ProposalEpochMismatch {
Slot::new(128)
} else {
Slot::new(0)
@ -57,21 +57,21 @@ impl TestingProposerSlashingBuilder {
let epoch = slot.epoch(T::slots_per_epoch());
if *test_task != ProposerSlashingTestTask::BadProposal1Signature {
if test_task != ProposerSlashingTestTask::BadProposal1Signature {
header_1.signature = {
let message = header_1.signed_root();
signer(proposer_index, &message[..], epoch, Domain::BeaconProposer)
};
}
if *test_task != ProposerSlashingTestTask::BadProposal2Signature {
if test_task != ProposerSlashingTestTask::BadProposal2Signature {
header_2.signature = {
let message = header_2.signed_root();
signer(proposer_index, &message[..], epoch, Domain::BeaconProposer)
};
}
if *test_task == ProposerSlashingTestTask::ProposerUnknown {
if test_task == ProposerSlashingTestTask::ProposerUnknown {
proposer_index = 3_141_592;
}

View File

@ -1,45 +0,0 @@
use crate::*;
use tree_hash::SignedRoot;
/// Builds a transfer to be used for testing purposes.
///
/// This struct should **never be used for production purposes.**
pub struct TestingTransferBuilder {
transfer: Transfer,
}
impl TestingTransferBuilder {
/// Instantiates a new builder.
pub fn new(sender: u64, recipient: u64, amount: u64, slot: Slot) -> Self {
let keypair = Keypair::random();
let transfer = Transfer {
sender,
recipient,
amount,
fee: 0,
slot,
pubkey: keypair.pk,
signature: Signature::empty_signature(),
};
Self { transfer }
}
/// Signs the transfer.
///
/// The keypair must match that of the `from` validator index.
pub fn sign<T: EthSpec>(&mut self, keypair: Keypair, fork: &Fork, spec: &ChainSpec) {
self.transfer.pubkey = keypair.pk;
let message = self.transfer.signed_root();
let epoch = self.transfer.slot.epoch(T::slots_per_epoch());
let domain = spec.get_domain(epoch, Domain::Transfer, fork);
self.transfer.signature = Signature::new(&message, domain, &keypair.sk);
}
/// Builds the transfer, consuming the builder.
pub fn build(self) -> Transfer {
self.transfer
}
}

View File

@ -1,45 +0,0 @@
use super::Slot;
use crate::test_utils::TestRandom;
use bls::{PublicKey, Signature};
use derivative::Derivative;
use serde_derive::{Deserialize, Serialize};
use ssz_derive::{Decode, Encode};
use test_random_derive::TestRandom;
use tree_hash::TreeHash;
use tree_hash_derive::{SignedRoot, TreeHash};
/// The data submitted to the deposit contract.
///
/// Spec v0.8.0
#[derive(
Debug,
Clone,
Serialize,
Deserialize,
Encode,
Decode,
TreeHash,
TestRandom,
SignedRoot,
Derivative,
)]
#[derivative(PartialEq, Eq, Hash)]
pub struct Transfer {
pub sender: u64,
pub recipient: u64,
pub amount: u64,
pub fee: u64,
pub slot: Slot,
pub pubkey: PublicKey,
#[derivative(Hash = "ignore")]
#[signed_root(skip_hashing)]
pub signature: Signature,
}
#[cfg(test)]
mod tests {
use super::*;
ssz_tests!(Transfer);
}

View File

@ -7,7 +7,7 @@ use tree_hash_derive::TreeHash;
/// Information about a `BeaconChain` validator.
///
/// Spec v0.8.0
/// Spec v0.9.1
#[derive(Debug, Clone, PartialEq, Serialize, Deserialize, Encode, Decode, TestRandom, TreeHash)]
pub struct Validator {
pub pubkey: PublicKey,

View File

@ -9,7 +9,7 @@ use tree_hash_derive::{SignedRoot, TreeHash};
/// An exit voluntarily submitted a validator who wishes to withdraw.
///
/// Spec v0.8.0
/// Spec v0.9.1
#[derive(
Debug,
PartialEq,

View File

@ -19,7 +19,17 @@ const TOTAL_SIZE: usize = SEED_SIZE + ROUND_SIZE + POSITION_WINDOW_SIZE;
/// It holds that: shuffle_list(shuffle_list(l, r, s, true), r, s, false) == l
/// and: shuffle_list(shuffle_list(l, r, s, false), r, s, true) == l
///
/// TODO forwards is around the wrong way - denote?
/// The Eth2.0 spec mostly uses shuffling with `forwards == false`, because backwards
/// shuffled lists are slightly easier to specify, and slightly easier to compute.
///
/// The forwards shuffling of a list is equivalent to:
///
/// `[indices[x] for i in 0..n, where compute_shuffled_index(x) = i]`
///
/// Whereas the backwards shuffling of a list is:
///
/// `[indices[compute_shuffled_index(i)] for i in 0..n]`
///
/// Returns `None` under any of the following conditions:
/// - `list_size == 0`
/// - `list_size > 2**24`

View File

@ -1,5 +1,5 @@
# Bump the test tag here and in .gitlab-ci.yml and CI will take care of updating the cached tarballs
TESTS_TAG := v0.8.3
TESTS_TAG := v0.9.1
TESTS = general minimal mainnet
TARBALLS = $(patsubst %,%-$(TESTS_TAG).tar.gz,$(TESTS))

View File

@ -6,8 +6,8 @@ use crate::type_name;
use crate::type_name::TypeName;
use serde_derive::Deserialize;
use state_processing::per_epoch_processing::{
errors::EpochProcessingError, process_crosslinks, process_final_updates,
process_justification_and_finalization, process_registry_updates, process_slashings,
errors::EpochProcessingError, process_final_updates, process_justification_and_finalization,
process_registry_updates, process_rewards_and_penalties, process_slashings,
validator_statuses::ValidatorStatuses,
};
use std::marker::PhantomData;
@ -38,7 +38,7 @@ pub trait EpochTransition<E: EthSpec>: TypeName + Debug + Sync {
#[derive(Debug)]
pub struct JustificationAndFinalization;
#[derive(Debug)]
pub struct Crosslinks;
pub struct RewardsAndPenalties;
#[derive(Debug)]
pub struct RegistryUpdates;
#[derive(Debug)]
@ -50,7 +50,7 @@ type_name!(
JustificationAndFinalization,
"justification_and_finalization"
);
type_name!(Crosslinks, "crosslinks");
type_name!(RewardsAndPenalties, "rewards_and_penalties");
type_name!(RegistryUpdates, "registry_updates");
type_name!(Slashings, "slashings");
type_name!(FinalUpdates, "final_updates");
@ -63,10 +63,11 @@ impl<E: EthSpec> EpochTransition<E> for JustificationAndFinalization {
}
}
impl<E: EthSpec> EpochTransition<E> for Crosslinks {
impl<E: EthSpec> EpochTransition<E> for RewardsAndPenalties {
fn run(state: &mut BeaconState<E>, spec: &ChainSpec) -> Result<(), EpochProcessingError> {
process_crosslinks(state, spec)?;
Ok(())
let mut validator_statuses = ValidatorStatuses::new(state, spec)?;
validator_statuses.process_attestations(state, spec)?;
process_rewards_and_penalties(state, &mut validator_statuses, spec)
}
}

View File

@ -8,13 +8,13 @@ use ssz::Decode;
use state_processing::per_block_processing::{
errors::BlockProcessingError, process_attestations, process_attester_slashings,
process_block_header, process_deposits, process_exits, process_proposer_slashings,
process_transfers, VerifySignatures,
VerifySignatures,
};
use std::fmt::Debug;
use std::path::Path;
use types::{
Attestation, AttesterSlashing, BeaconBlock, BeaconState, ChainSpec, Deposit, EthSpec,
ProposerSlashing, Transfer, VoluntaryExit,
ProposerSlashing, VoluntaryExit,
};
#[derive(Debug, Clone, Default, Deserialize)]
@ -95,16 +95,6 @@ impl<E: EthSpec> Operation<E> for ProposerSlashing {
}
}
impl<E: EthSpec> Operation<E> for Transfer {
fn apply_to(
&self,
state: &mut BeaconState<E>,
spec: &ChainSpec,
) -> Result<(), BlockProcessingError> {
process_transfers(state, &[self.clone()], VerifySignatures::True, spec)
}
}
impl<E: EthSpec> Operation<E> for VoluntaryExit {
fn handler_name() -> String {
"voluntary_exit".into()

View File

@ -3,7 +3,7 @@ use types::EthSpec;
pub use case_result::CaseResult;
pub use cases::Case;
pub use cases::{
Crosslinks, FinalUpdates, JustificationAndFinalization, RegistryUpdates, Slashings,
FinalUpdates, JustificationAndFinalization, RegistryUpdates, RewardsAndPenalties, Slashings,
};
pub use error::Error;
pub use handler::*;

View File

@ -38,15 +38,12 @@ type_name!(MainnetEthSpec, "mainnet");
type_name_generic!(Attestation);
type_name!(AttestationData);
type_name!(AttestationDataAndCustodyBit);
type_name_generic!(AttesterSlashing);
type_name_generic!(BeaconBlock);
type_name_generic!(BeaconBlockBody);
type_name!(BeaconBlockHeader);
type_name_generic!(BeaconState);
type_name!(Checkpoint);
type_name_generic!(CompactCommittee);
type_name!(Crosslink);
type_name!(Deposit);
type_name!(DepositData);
type_name!(Eth1Data);
@ -55,6 +52,5 @@ type_name_generic!(HistoricalBatch);
type_name_generic!(IndexedAttestation);
type_name_generic!(PendingAttestation);
type_name!(ProposerSlashing);
type_name!(Transfer);
type_name!(Validator);
type_name!(VoluntaryExit);

View File

@ -15,12 +15,6 @@ fn operations_deposit() {
OperationsHandler::<MainnetEthSpec, Deposit>::run();
}
#[test]
fn operations_transfer() {
OperationsHandler::<MinimalEthSpec, Transfer>::run();
// Note: there are no transfer tests for mainnet
}
#[test]
fn operations_exit() {
OperationsHandler::<MinimalEthSpec, VoluntaryExit>::run();
@ -139,10 +133,6 @@ mod ssz_static {
ssz_static_test!(attestation, Attestation<_>, SR);
ssz_static_test!(attestation_data, AttestationData);
ssz_static_test!(
attestation_data_and_custody_bit,
AttestationDataAndCustodyBit
);
ssz_static_test!(attester_slashing, AttesterSlashing<_>);
ssz_static_test!(beacon_block, BeaconBlock<_>, SR);
ssz_static_test!(beacon_block_body, BeaconBlockBody<_>);
@ -155,8 +145,6 @@ mod ssz_static {
}
);
ssz_static_test!(checkpoint, Checkpoint);
ssz_static_test!(compact_committee, CompactCommittee<_>);
ssz_static_test!(crosslink, Crosslink);
ssz_static_test!(deposit, Deposit);
ssz_static_test!(deposit_data, DepositData, SR);
ssz_static_test!(eth1_data, Eth1Data);
@ -165,7 +153,6 @@ mod ssz_static {
ssz_static_test!(indexed_attestation, IndexedAttestation<_>, SR);
ssz_static_test!(pending_attestation, PendingAttestation<_>);
ssz_static_test!(proposer_slashing, ProposerSlashing);
ssz_static_test!(transfer, Transfer, SR);
ssz_static_test!(validator, Validator);
ssz_static_test!(voluntary_exit, VoluntaryExit, SR);
}
@ -187,9 +174,9 @@ fn epoch_processing_justification_and_finalization() {
}
#[test]
fn epoch_processing_crosslinks() {
EpochProcessingHandler::<MinimalEthSpec, Crosslinks>::run();
EpochProcessingHandler::<MainnetEthSpec, Crosslinks>::run();
fn epoch_processing_rewards_and_penalties() {
EpochProcessingHandler::<MinimalEthSpec, RewardsAndPenalties>::run();
// Note: there are no reward and penalty tests for mainnet yet
}
#[test]

View File

@ -8,11 +8,11 @@ Node (BN) and fulfils the roles of a validator.
The VC is responsible for the following tasks:
- Requesting validator duties (a.k.a. shuffling) from the BN.
- Prompting the BN to produce a new block, when a validators block production
- Prompting the BN to produce a new block, when a validator's block production
duties require.
- Completing all the fields on a new block (e.g., RANDAO reveal, signature) and
publishing the block to a BN.
- Prompting the BN to produce a new shard attestation as per a validators
- Prompting the BN to produce a new attestation as per a validator's
duties.
- Ensuring that no slashable messages are signed by a validator private key.
- Keeping track of the system clock and how it relates to slots/epochs.

Some files were not shown because too many files have changed in this diff Show More