Process exits and slashings off the network (#1253)

* Process exits and slashings off the network

* Fix rest_api tests

* Add op verification tests

* Add tests for pruning of slashings in the op pool

* Address Paul's review comments
This commit is contained in:
Michael Sproul 2020-06-18 21:06:34 +10:00 committed by GitHub
parent 3199b1a6f2
commit bcb6afa0aa
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
27 changed files with 956 additions and 273 deletions

15
Cargo.lock generated
View File

@ -237,6 +237,7 @@ dependencies = [
"bitvec",
"bls",
"bus",
"derivative",
"environment",
"eth1",
"eth2_config",
@ -267,6 +268,7 @@ dependencies = [
"slog",
"sloggers",
"slot_clock",
"smallvec 1.4.0",
"state_processing",
"store",
"tempfile",
@ -925,6 +927,17 @@ dependencies = [
"types",
]
[[package]]
name = "derivative"
version = "2.1.1"
source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "cb582b60359da160a9477ee80f15c8d784c477e69c217ef2cdd4169c24ea380f"
dependencies = [
"proc-macro2",
"quote",
"syn",
]
[[package]]
name = "derive_arbitrary"
version = "0.4.4"
@ -2904,6 +2917,7 @@ dependencies = [
"sloggers",
"slot_clock",
"smallvec 1.4.0",
"state_processing",
"store",
"tempfile",
"tokio 0.2.21",
@ -5254,6 +5268,7 @@ dependencies = [
"compare_fields",
"compare_fields_derive",
"criterion",
"derivative",
"dirs",
"eth2_hashing",
"eth2_interop_keypairs",

View File

@ -15,6 +15,7 @@ merkle_proof = { path = "../../consensus/merkle_proof" }
store = { path = "../store" }
parking_lot = "0.10.2"
lazy_static = "1.4.0"
smallvec = "1.4.0"
lighthouse_metrics = { path = "../../common/lighthouse_metrics" }
log = "0.4.8"
operation_pool = { path = "../operation_pool" }
@ -49,4 +50,5 @@ safe_arith = { path = "../../consensus/safe_arith" }
fork_choice = { path = "../../consensus/fork_choice" }
environment = { path = "../../lighthouse/environment" }
bus = "2.2.3"
derivative = "2.1.1"
itertools = "0.9.0"

View File

@ -16,6 +16,7 @@ use crate::naive_aggregation_pool::{Error as NaiveAggregationError, NaiveAggrega
use crate::observed_attestations::{Error as AttestationObservationError, ObservedAttestations};
use crate::observed_attesters::{ObservedAggregators, ObservedAttesters};
use crate::observed_block_producers::ObservedBlockProducers;
use crate::observed_operations::{ObservationOutcome, ObservedOperations};
use crate::persisted_beacon_chain::PersistedBeaconChain;
use crate::persisted_fork_choice::PersistedForkChoice;
use crate::shuffling_cache::ShufflingCache;
@ -31,13 +32,10 @@ use parking_lot::RwLock;
use slog::{crit, debug, error, info, trace, warn, Logger};
use slot_clock::SlotClock;
use state_processing::{
common::get_indexed_attestation,
per_block_processing::errors::{
AttestationValidationError, AttesterSlashingValidationError, ExitValidationError,
ProposerSlashingValidationError,
},
common::get_indexed_attestation, per_block_processing,
per_block_processing::errors::AttestationValidationError, per_slot_processing,
BlockSignatureStrategy, SigVerifiedOp,
};
use state_processing::{per_block_processing, per_slot_processing, BlockSignatureStrategy};
use std::borrow::Cow;
use std::cmp::Ordering;
use std::collections::HashMap;
@ -193,6 +191,12 @@ pub struct BeaconChain<T: BeaconChainTypes> {
pub observed_aggregators: ObservedAggregators<T::EthSpec>,
/// Maintains a record of which validators have proposed blocks for each slot.
pub observed_block_producers: ObservedBlockProducers<T::EthSpec>,
/// Maintains a record of which validators have submitted voluntary exits.
pub observed_voluntary_exits: ObservedOperations<SignedVoluntaryExit, T::EthSpec>,
/// Maintains a record of which validators we've seen proposer slashings for.
pub observed_proposer_slashings: ObservedOperations<ProposerSlashing, T::EthSpec>,
/// Maintains a record of which validators we've seen attester slashings for.
pub observed_attester_slashings: ObservedOperations<AttesterSlashing<T::EthSpec>, T::EthSpec>,
/// Provides information from the Ethereum 1 (PoW) chain.
pub eth1_chain: Option<Eth1Chain<T::Eth1Chain, T::EthSpec>>,
/// Stores a "snapshot" of the chain at the time the head-of-the-chain block was received.
@ -1075,81 +1079,68 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
}
}
/// Accept some exit and queue it for inclusion in an appropriate block.
pub fn process_voluntary_exit(
/// Verify a voluntary exit before allowing it to propagate on the gossip network.
pub fn verify_voluntary_exit_for_gossip(
&self,
exit: SignedVoluntaryExit,
) -> Result<(), ExitValidationError> {
match self.wall_clock_state() {
Ok(state) => {
if self.eth1_chain.is_some() {
self.op_pool.insert_voluntary_exit(exit, &state, &self.spec)
} else {
Ok(())
}
}
Err(e) => {
error!(
&self.log,
"Unable to process voluntary exit";
"error" => format!("{:?}", e),
"reason" => "no state"
);
Ok(())
}
) -> Result<ObservationOutcome<SignedVoluntaryExit>, Error> {
// NOTE: this could be more efficient if it avoided cloning the head state
let wall_clock_state = self.wall_clock_state()?;
Ok(self
.observed_voluntary_exits
.verify_and_observe(exit, &wall_clock_state, &self.spec)?)
}
/// Accept a pre-verified exit and queue it for inclusion in an appropriate block.
pub fn import_voluntary_exit(&self, exit: SigVerifiedOp<SignedVoluntaryExit>) {
if self.eth1_chain.is_some() {
self.op_pool.insert_voluntary_exit(exit)
}
}
/// Verify a proposer slashing before allowing it to propagate on the gossip network.
pub fn verify_proposer_slashing_for_gossip(
&self,
proposer_slashing: ProposerSlashing,
) -> Result<ObservationOutcome<ProposerSlashing>, Error> {
let wall_clock_state = self.wall_clock_state()?;
Ok(self.observed_proposer_slashings.verify_and_observe(
proposer_slashing,
&wall_clock_state,
&self.spec,
)?)
}
/// Accept some proposer slashing and queue it for inclusion in an appropriate block.
pub fn process_proposer_slashing(
&self,
proposer_slashing: ProposerSlashing,
) -> Result<(), ProposerSlashingValidationError> {
match self.wall_clock_state() {
Ok(state) => {
if self.eth1_chain.is_some() {
self.op_pool
.insert_proposer_slashing(proposer_slashing, &state, &self.spec)
} else {
Ok(())
}
}
Err(e) => {
error!(
&self.log,
"Unable to process proposer slashing";
"error" => format!("{:?}", e),
"reason" => "no state"
);
Ok(())
}
pub fn import_proposer_slashing(&self, proposer_slashing: SigVerifiedOp<ProposerSlashing>) {
if self.eth1_chain.is_some() {
self.op_pool.insert_proposer_slashing(proposer_slashing)
}
}
/// Accept some attester slashing and queue it for inclusion in an appropriate block.
pub fn process_attester_slashing(
/// Verify an attester slashing before allowing it to propagate on the gossip network.
pub fn verify_attester_slashing_for_gossip(
&self,
attester_slashing: AttesterSlashing<T::EthSpec>,
) -> Result<(), AttesterSlashingValidationError> {
match self.wall_clock_state() {
Ok(state) => {
if self.eth1_chain.is_some() {
self.op_pool
.insert_attester_slashing(attester_slashing, &state, &self.spec)
} else {
Ok(())
}
}
Err(e) => {
error!(
&self.log,
"Unable to process attester slashing";
"error" => format!("{:?}", e),
"reason" => "no state"
);
Ok(())
}
) -> Result<ObservationOutcome<AttesterSlashing<T::EthSpec>>, Error> {
let wall_clock_state = self.wall_clock_state()?;
Ok(self.observed_attester_slashings.verify_and_observe(
attester_slashing,
&wall_clock_state,
&self.spec,
)?)
}
/// Accept some attester slashing and queue it for inclusion in an appropriate block.
pub fn import_attester_slashing(
&self,
attester_slashing: SigVerifiedOp<AttesterSlashing<T::EthSpec>>,
) -> Result<(), Error> {
if self.eth1_chain.is_some() {
self.op_pool
.insert_attester_slashing(attester_slashing, self.head_info()?.fork)
}
Ok(())
}
/// Attempt to verify and import a chain of blocks to `self`.
@ -1637,8 +1628,7 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
let mut graffiti: [u8; 32] = [0; 32];
graffiti.copy_from_slice(GRAFFITI.as_bytes());
let (proposer_slashings, attester_slashings) =
self.op_pool.get_slashings(&state, &self.spec);
let (proposer_slashings, attester_slashings) = self.op_pool.get_slashings(&state);
let eth1_data = eth1_chain.eth1_data_for_block_production(&state, &self.spec)?;
let deposits = eth1_chain
@ -1930,7 +1920,8 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
.get_state(&finalized_block.state_root, Some(finalized_block.slot))?
.ok_or_else(|| Error::MissingBeaconState(finalized_block.state_root))?;
self.op_pool.prune_all(&finalized_state, &self.spec);
self.op_pool
.prune_all(&finalized_state, self.head_info()?.fork);
// TODO: configurable max finality distance
let max_finality_distance = 0;

View File

@ -283,8 +283,8 @@ where
store
.get_item::<PersistedOperationPool<TEthSpec>>(&Hash256::from_slice(&OP_POOL_DB_KEY))
.map_err(|e| format!("DB error whilst reading persisted op pool: {:?}", e))?
.map(|persisted| persisted.into_operation_pool(&head_state, &self.spec))
.unwrap_or_else(|| OperationPool::new()),
.map(PersistedOperationPool::into_operation_pool)
.unwrap_or_else(OperationPool::new),
);
let finalized_block_root = head_state.finalized_checkpoint.root;
@ -500,6 +500,10 @@ where
observed_aggregators: <_>::default(),
// TODO: allow for persisting and loading the pool from disk.
observed_block_producers: <_>::default(),
// TODO: allow for persisting and loading the pool from disk.
observed_voluntary_exits: <_>::default(),
observed_proposer_slashings: <_>::default(),
observed_attester_slashings: <_>::default(),
eth1_chain: self.eth1_chain,
genesis_validators_root: canonical_head.beacon_state.genesis_validators_root,
canonical_head: TimeoutRwLock::new(canonical_head.clone()),

View File

@ -10,8 +10,12 @@ use ssz::DecodeError;
use ssz_types::Error as SszTypesError;
use state_processing::{
block_signature_verifier::Error as BlockSignatureVerifierError,
per_block_processing::errors::AttestationValidationError,
signature_sets::Error as SignatureSetError, BlockProcessingError, SlotProcessingError,
per_block_processing::errors::{
AttestationValidationError, AttesterSlashingValidationError, ExitValidationError,
ProposerSlashingValidationError,
},
signature_sets::Error as SignatureSetError,
BlockProcessingError, SlotProcessingError,
};
use std::time::Duration;
use types::*;
@ -50,6 +54,9 @@ pub enum BeaconChainError {
},
CannotAttestToFutureState,
AttestationValidationError(AttestationValidationError),
ExitValidationError(ExitValidationError),
ProposerSlashingValidationError(ProposerSlashingValidationError),
AttesterSlashingValidationError(AttesterSlashingValidationError),
StateSkipTooLarge {
start_slot: Slot,
requested_slot: Slot,
@ -78,6 +85,9 @@ pub enum BeaconChainError {
easy_from_to!(SlotProcessingError, BeaconChainError);
easy_from_to!(AttestationValidationError, BeaconChainError);
easy_from_to!(ExitValidationError, BeaconChainError);
easy_from_to!(ProposerSlashingValidationError, BeaconChainError);
easy_from_to!(AttesterSlashingValidationError, BeaconChainError);
easy_from_to!(SszTypesError, BeaconChainError);
easy_from_to!(OpPoolError, BeaconChainError);
easy_from_to!(NaiveAggregationError, BeaconChainError);

View File

@ -18,6 +18,7 @@ mod naive_aggregation_pool;
mod observed_attestations;
mod observed_attesters;
mod observed_block_producers;
pub mod observed_operations;
mod persisted_beacon_chain;
mod persisted_fork_choice;
mod shuffling_cache;

View File

@ -0,0 +1,104 @@
use derivative::Derivative;
use parking_lot::Mutex;
use smallvec::SmallVec;
use state_processing::{SigVerifiedOp, VerifyOperation};
use std::collections::HashSet;
use std::iter::FromIterator;
use std::marker::PhantomData;
use types::{
AttesterSlashing, BeaconState, ChainSpec, EthSpec, ProposerSlashing, SignedVoluntaryExit,
};
/// Number of validator indices to store on the stack in `observed_validators`.
pub const SMALL_VEC_SIZE: usize = 8;
/// Stateful tracker for exit/slashing operations seen on the network.
///
/// Implements the conditions for gossip verification of exits and slashings from the P2P spec.
#[derive(Debug, Derivative)]
#[derivative(Default(bound = "T: ObservableOperation<E>, E: EthSpec"))]
pub struct ObservedOperations<T: ObservableOperation<E>, E: EthSpec> {
/// Indices of validators for whom we have already seen an instance of an operation `T`.
///
/// For voluntary exits, this is the set of all `signed_voluntary_exit.message.validator_index`.
/// For proposer slashings, this is the set of all `proposer_slashing.index`.
/// For attester slashings, this is the set of all validators who would be slashed by
/// previously seen attester slashings, i.e. those validators in the intersection of
/// `attestation_1.attester_indices` and `attestation_2.attester_indices`.
observed_validator_indices: Mutex<HashSet<u64>>,
_phantom: PhantomData<(T, E)>,
}
/// Was the observed operation new and valid for further processing, or a useless duplicate?
#[derive(Debug, PartialEq, Eq, Clone)]
pub enum ObservationOutcome<T> {
New(SigVerifiedOp<T>),
AlreadyKnown,
}
/// Trait for exits and slashings which can be observed using `ObservedOperations`.
pub trait ObservableOperation<E: EthSpec>: VerifyOperation<E> + Sized {
/// The set of validator indices involved in this operation.
///
/// See the comment on `observed_validator_indices` above for detail.
fn observed_validators(&self) -> SmallVec<[u64; SMALL_VEC_SIZE]>;
}
impl<E: EthSpec> ObservableOperation<E> for SignedVoluntaryExit {
fn observed_validators(&self) -> SmallVec<[u64; SMALL_VEC_SIZE]> {
std::iter::once(self.message.validator_index).collect()
}
}
impl<E: EthSpec> ObservableOperation<E> for ProposerSlashing {
fn observed_validators(&self) -> SmallVec<[u64; SMALL_VEC_SIZE]> {
std::iter::once(self.signed_header_1.message.proposer_index).collect()
}
}
impl<E: EthSpec> ObservableOperation<E> for AttesterSlashing<E> {
fn observed_validators(&self) -> SmallVec<[u64; SMALL_VEC_SIZE]> {
let attestation_1_indices =
HashSet::<u64>::from_iter(self.attestation_1.attesting_indices.iter().copied());
let attestation_2_indices =
HashSet::<u64>::from_iter(self.attestation_2.attesting_indices.iter().copied());
attestation_1_indices
.intersection(&attestation_2_indices)
.copied()
.collect()
}
}
impl<T: ObservableOperation<E>, E: EthSpec> ObservedOperations<T, E> {
pub fn verify_and_observe(
&self,
op: T,
head_state: &BeaconState<E>,
spec: &ChainSpec,
) -> Result<ObservationOutcome<T>, T::Error> {
let mut observed_validator_indices = self.observed_validator_indices.lock();
let new_validator_indices = op.observed_validators();
// If all of the new validator indices have been previously observed, short-circuit
// the validation. This implements the uniqueness check part of the spec, which for attester
// slashings reads:
//
// At least one index in the intersection of the attesting indices of each attestation has
// not yet been seen in any prior attester_slashing.
if new_validator_indices
.iter()
.all(|index| observed_validator_indices.contains(index))
{
return Ok(ObservationOutcome::AlreadyKnown);
}
// Validate the op using operation-specific logic (`verify_attester_slashing`, etc).
let verified_op = op.validate(head_state, spec)?;
// Add the relevant indices to the set of known indices to prevent processing of duplicates
// in the future.
observed_validator_indices.extend(new_validator_indices);
Ok(ObservationOutcome::New(verified_op))
}
}

View File

@ -0,0 +1,271 @@
//! Tests for gossip verification of voluntary exits, propser slashings and attester slashings.
#![cfg(not(debug_assertions))]
#[macro_use]
extern crate lazy_static;
use beacon_chain::observed_operations::ObservationOutcome;
use beacon_chain::test_utils::{
AttestationStrategy, BeaconChainHarness, BlockStrategy, DiskHarnessType,
};
use sloggers::{null::NullLoggerBuilder, Build};
use std::sync::Arc;
use store::{LevelDB, StoreConfig};
use tempfile::{tempdir, TempDir};
use types::test_utils::{
AttesterSlashingTestTask, ProposerSlashingTestTask, TestingAttesterSlashingBuilder,
TestingProposerSlashingBuilder, TestingVoluntaryExitBuilder,
};
use types::*;
pub const VALIDATOR_COUNT: usize = 24;
lazy_static! {
/// A cached set of keys.
static ref KEYPAIRS: Vec<Keypair> =
types::test_utils::generate_deterministic_keypairs(VALIDATOR_COUNT);
}
type E = MinimalEthSpec;
type TestHarness = BeaconChainHarness<DiskHarnessType<E>>;
type HotColdDB = store::HotColdDB<E, LevelDB<E>, LevelDB<E>>;
fn get_store(db_path: &TempDir) -> Arc<HotColdDB> {
let spec = E::default_spec();
let hot_path = db_path.path().join("hot_db");
let cold_path = db_path.path().join("cold_db");
let config = StoreConfig::default();
let log = NullLoggerBuilder.build().expect("logger should build");
Arc::new(
HotColdDB::open(&hot_path, &cold_path, config, spec, log)
.expect("disk store should initialize"),
)
}
fn get_harness(store: Arc<HotColdDB>, validator_count: usize) -> TestHarness {
let harness = BeaconChainHarness::new_with_disk_store(
MinimalEthSpec,
store,
KEYPAIRS[0..validator_count].to_vec(),
);
harness.advance_slot();
harness
}
#[test]
fn voluntary_exit() {
let db_path = tempdir().unwrap();
let store = get_store(&db_path);
let harness = get_harness(store.clone(), VALIDATOR_COUNT);
let spec = &harness.chain.spec;
harness.extend_chain(
(E::slots_per_epoch() * (spec.shard_committee_period + 1)) as usize,
BlockStrategy::OnCanonicalHead,
AttestationStrategy::AllValidators,
);
let head_info = harness.chain.head_info().unwrap();
let make_exit = |validator_index: usize, exit_epoch: u64| {
TestingVoluntaryExitBuilder::new(Epoch::new(exit_epoch), validator_index as u64).build(
&KEYPAIRS[validator_index].sk,
&head_info.fork,
head_info.genesis_validators_root,
spec,
)
};
let validator_index1 = VALIDATOR_COUNT - 1;
let validator_index2 = VALIDATOR_COUNT - 2;
let exit1 = make_exit(validator_index1, spec.shard_committee_period);
// First verification should show it to be fresh.
assert!(matches!(
harness
.chain
.verify_voluntary_exit_for_gossip(exit1.clone())
.unwrap(),
ObservationOutcome::New(_)
));
// Second should not.
assert!(matches!(
harness
.chain
.verify_voluntary_exit_for_gossip(exit1.clone()),
Ok(ObservationOutcome::AlreadyKnown)
));
// A different exit for the same validator should also be detected as a duplicate.
let exit2 = make_exit(validator_index1, spec.shard_committee_period + 1);
assert!(matches!(
harness.chain.verify_voluntary_exit_for_gossip(exit2),
Ok(ObservationOutcome::AlreadyKnown)
));
// Exit for a different validator should be fine.
let exit3 = make_exit(validator_index2, spec.shard_committee_period);
assert!(matches!(
harness
.chain
.verify_voluntary_exit_for_gossip(exit3)
.unwrap(),
ObservationOutcome::New(_)
));
}
#[test]
fn proposer_slashing() {
let db_path = tempdir().unwrap();
let store = get_store(&db_path);
let harness = get_harness(store.clone(), VALIDATOR_COUNT);
let spec = &harness.chain.spec;
let head_info = harness.chain.head_info().unwrap();
let validator_index1 = VALIDATOR_COUNT - 1;
let validator_index2 = VALIDATOR_COUNT - 2;
let make_slashing = |validator_index: usize| {
TestingProposerSlashingBuilder::double_vote::<E>(
ProposerSlashingTestTask::Valid,
validator_index as u64,
&KEYPAIRS[validator_index].sk,
&head_info.fork,
head_info.genesis_validators_root,
spec,
)
};
let slashing1 = make_slashing(validator_index1);
// First slashing for this proposer should be allowed.
assert!(matches!(
harness
.chain
.verify_proposer_slashing_for_gossip(slashing1.clone())
.unwrap(),
ObservationOutcome::New(_)
));
// Duplicate slashing should be detected.
assert!(matches!(
harness
.chain
.verify_proposer_slashing_for_gossip(slashing1.clone())
.unwrap(),
ObservationOutcome::AlreadyKnown
));
// Different slashing for the same index should be rejected
let slashing2 = ProposerSlashing {
signed_header_1: slashing1.signed_header_2,
signed_header_2: slashing1.signed_header_1,
};
assert!(matches!(
harness
.chain
.verify_proposer_slashing_for_gossip(slashing2)
.unwrap(),
ObservationOutcome::AlreadyKnown
));
// Proposer slashing for a different index should be accepted
let slashing3 = make_slashing(validator_index2);
assert!(matches!(
harness
.chain
.verify_proposer_slashing_for_gossip(slashing3)
.unwrap(),
ObservationOutcome::New(_)
));
}
#[test]
fn attester_slashing() {
let db_path = tempdir().unwrap();
let store = get_store(&db_path);
let harness = get_harness(store.clone(), VALIDATOR_COUNT);
let spec = &harness.chain.spec;
let head_info = harness.chain.head_info().unwrap();
// First third of the validators
let first_third = (0..VALIDATOR_COUNT as u64 / 3).collect::<Vec<_>>();
// First half of the validators
let first_half = (0..VALIDATOR_COUNT as u64 / 2).collect::<Vec<_>>();
// Last third of the validators
let last_third = (2 * VALIDATOR_COUNT as u64 / 3..VALIDATOR_COUNT as u64).collect::<Vec<_>>();
// Last half of the validators
let second_half = (VALIDATOR_COUNT as u64 / 2..VALIDATOR_COUNT as u64).collect::<Vec<_>>();
let signer = |idx: u64, message: &[u8]| Signature::new(message, &KEYPAIRS[idx as usize].sk);
let make_slashing = |validators| {
TestingAttesterSlashingBuilder::double_vote::<_, E>(
AttesterSlashingTestTask::Valid,
validators,
signer,
&head_info.fork,
head_info.genesis_validators_root,
spec,
)
};
// Slashing for first third of validators should be accepted.
let slashing1 = make_slashing(&first_third);
assert!(matches!(
harness
.chain
.verify_attester_slashing_for_gossip(slashing1.clone())
.unwrap(),
ObservationOutcome::New(_)
));
// Overlapping slashing for first half of validators should also be accepted.
let slashing2 = make_slashing(&first_half);
assert!(matches!(
harness
.chain
.verify_attester_slashing_for_gossip(slashing2.clone())
.unwrap(),
ObservationOutcome::New(_)
));
// Repeating slashing1 or slashing2 should be rejected
assert!(matches!(
harness
.chain
.verify_attester_slashing_for_gossip(slashing1.clone())
.unwrap(),
ObservationOutcome::AlreadyKnown
));
assert!(matches!(
harness
.chain
.verify_attester_slashing_for_gossip(slashing2.clone())
.unwrap(),
ObservationOutcome::AlreadyKnown
));
// Slashing for last half of validators should be accepted (distinct from all existing)
let slashing3 = make_slashing(&second_half);
assert!(matches!(
harness
.chain
.verify_attester_slashing_for_gossip(slashing3)
.unwrap(),
ObservationOutcome::New(_)
));
// Slashing for last third (contained in last half) should be rejected.
let slashing4 = make_slashing(&last_third);
assert!(matches!(
harness
.chain
.verify_attester_slashing_for_gossip(slashing4)
.unwrap(),
ObservationOutcome::AlreadyKnown
));
}

View File

@ -354,8 +354,6 @@ fn roundtrip_operation_pool() {
.persist_op_pool()
.expect("should persist op pool");
let head_state = harness.chain.head().expect("should get head").beacon_state;
let key = Hash256::from_slice(&OP_POOL_DB_KEY);
let restored_op_pool = harness
.chain
@ -363,7 +361,7 @@ fn roundtrip_operation_pool() {
.get_item::<PersistedOperationPool<MinimalEthSpec>>(&key)
.expect("should read db")
.expect("should find op pool")
.into_operation_pool(&head_state, &harness.spec);
.into_operation_pool();
assert_eq!(harness.chain.op_pool, restored_op_pool);
}

View File

@ -9,7 +9,7 @@ use std::boxed::Box;
use types::SubnetId;
use types::{
Attestation, AttesterSlashing, EthSpec, ProposerSlashing, SignedAggregateAndProof,
SignedBeaconBlock, VoluntaryExit,
SignedBeaconBlock, SignedVoluntaryExit,
};
#[derive(Debug, Clone, PartialEq)]
@ -21,7 +21,7 @@ pub enum PubsubMessage<T: EthSpec> {
/// Gossipsub message providing notification of a raw un-aggregated attestation with its shard id.
Attestation(Box<(SubnetId, Attestation<T>)>),
/// Gossipsub message providing notification of a voluntary exit.
VoluntaryExit(Box<VoluntaryExit>),
VoluntaryExit(Box<SignedVoluntaryExit>),
/// Gossipsub message providing notification of a new proposer slashing.
ProposerSlashing(Box<ProposerSlashing>),
/// Gossipsub message providing notification of a new attester slashing.
@ -111,8 +111,9 @@ impl<T: EthSpec> PubsubMessage<T> {
return Ok(PubsubMessage::BeaconBlock(Box::new(beacon_block)));
}
GossipKind::VoluntaryExit => {
let voluntary_exit = VoluntaryExit::from_ssz_bytes(decompressed_data)
.map_err(|e| format!("{:?}", e))?;
let voluntary_exit =
SignedVoluntaryExit::from_ssz_bytes(decompressed_data)
.map_err(|e| format!("{:?}", e))?;
return Ok(PubsubMessage::VoluntaryExit(Box::new(voluntary_exit)));
}
GossipKind::ProposerSlashing => {

View File

@ -19,6 +19,7 @@ eth2-libp2p = { path = "../eth2-libp2p" }
hashset_delay = { path = "../../common/hashset_delay" }
rest_types = { path = "../../common/rest_types" }
types = { path = "../../consensus/types" }
state_processing = { path = "../../consensus/state_processing" }
slot_clock = { path = "../../common/slot_clock" }
slog = { version = "2.5.2", features = ["max_level_trace"] }
hex = "0.4.2"

View File

@ -257,23 +257,45 @@ impl<T: BeaconChainTypes> Router<T> {
}
}
}
PubsubMessage::VoluntaryExit(_exit) => {
// TODO: Apply more sophisticated validation
self.propagate_message(id, peer_id.clone());
// TODO: Handle exits
debug!(self.log, "Received a voluntary exit"; "peer_id" => format!("{}", peer_id) );
PubsubMessage::VoluntaryExit(exit) => {
debug!(self.log, "Received a voluntary exit"; "peer_id" => format!("{}", peer_id));
if let Some(verified_exit) = self
.processor
.verify_voluntary_exit_for_gossip(&peer_id, *exit)
{
self.propagate_message(id, peer_id.clone());
self.processor.import_verified_voluntary_exit(verified_exit);
}
}
PubsubMessage::ProposerSlashing(_proposer_slashing) => {
// TODO: Apply more sophisticated validation
self.propagate_message(id, peer_id.clone());
// TODO: Handle proposer slashings
debug!(self.log, "Received a proposer slashing"; "peer_id" => format!("{}", peer_id) );
PubsubMessage::ProposerSlashing(proposer_slashing) => {
debug!(
self.log,
"Received a proposer slashing";
"peer_id" => format!("{}", peer_id)
);
if let Some(verified_proposer_slashing) = self
.processor
.verify_proposer_slashing_for_gossip(&peer_id, *proposer_slashing)
{
self.propagate_message(id, peer_id.clone());
self.processor
.import_verified_proposer_slashing(verified_proposer_slashing);
}
}
PubsubMessage::AttesterSlashing(_attester_slashing) => {
// TODO: Apply more sophisticated validation
self.propagate_message(id, peer_id.clone());
// TODO: Handle attester slashings
debug!(self.log, "Received an attester slashing"; "peer_id" => format!("{}", peer_id) );
PubsubMessage::AttesterSlashing(attester_slashing) => {
debug!(
self.log,
"Received a attester slashing";
"peer_id" => format!("{}", peer_id)
);
if let Some(verified_attester_slashing) = self
.processor
.verify_attester_slashing_for_gossip(&peer_id, *attester_slashing)
{
self.propagate_message(id, peer_id.clone());
self.processor
.import_verified_attester_slashing(verified_attester_slashing);
}
}
}
}

View File

@ -5,6 +5,7 @@ use beacon_chain::{
Error as AttnError, SignatureVerifiedAttestation, VerifiedAggregatedAttestation,
VerifiedUnaggregatedAttestation,
},
observed_operations::ObservationOutcome,
BeaconChain, BeaconChainError, BeaconChainTypes, BlockError, BlockProcessingOutcome,
ForkChoiceError, GossipVerifiedBlock,
};
@ -13,11 +14,12 @@ use eth2_libp2p::{NetworkGlobals, PeerId, Request, Response};
use itertools::process_results;
use slog::{debug, error, o, trace, warn};
use ssz::Encode;
use state_processing::SigVerifiedOp;
use std::sync::Arc;
use tokio::sync::mpsc;
use types::{
Attestation, ChainSpec, Epoch, EthSpec, Hash256, SignedAggregateAndProof, SignedBeaconBlock,
Slot, SubnetId,
Attestation, AttesterSlashing, ChainSpec, Epoch, EthSpec, Hash256, ProposerSlashing,
SignedAggregateAndProof, SignedBeaconBlock, SignedVoluntaryExit, Slot, SubnetId,
};
//TODO: Rate limit requests
@ -927,6 +929,142 @@ impl<T: BeaconChainTypes> Processor<T> {
}
}
}
/// Verify a voluntary exit before gossiping or processing it.
///
/// Errors are logged at debug level.
pub fn verify_voluntary_exit_for_gossip(
&self,
peer_id: &PeerId,
voluntary_exit: SignedVoluntaryExit,
) -> Option<SigVerifiedOp<SignedVoluntaryExit>> {
let validator_index = voluntary_exit.message.validator_index;
match self.chain.verify_voluntary_exit_for_gossip(voluntary_exit) {
Ok(ObservationOutcome::New(sig_verified_exit)) => Some(sig_verified_exit),
Ok(ObservationOutcome::AlreadyKnown) => {
debug!(
self.log,
"Dropping exit for already exiting validator";
"validator_index" => validator_index,
"peer" => format!("{:?}", peer_id)
);
None
}
Err(e) => {
debug!(
self.log,
"Dropping invalid exit";
"validator_index" => validator_index,
"peer" => format!("{:?}", peer_id),
"error" => format!("{:?}", e)
);
None
}
}
}
/// Import a verified exit into the op pool.
pub fn import_verified_voluntary_exit(
&self,
verified_voluntary_exit: SigVerifiedOp<SignedVoluntaryExit>,
) {
self.chain.import_voluntary_exit(verified_voluntary_exit);
debug!(self.log, "Successfully imported voluntary exit");
}
/// Verify a proposer slashing before gossiping or processing it.
///
/// Errors are logged at debug level.
pub fn verify_proposer_slashing_for_gossip(
&self,
peer_id: &PeerId,
proposer_slashing: ProposerSlashing,
) -> Option<SigVerifiedOp<ProposerSlashing>> {
let validator_index = proposer_slashing.signed_header_1.message.proposer_index;
match self
.chain
.verify_proposer_slashing_for_gossip(proposer_slashing)
{
Ok(ObservationOutcome::New(verified_slashing)) => Some(verified_slashing),
Ok(ObservationOutcome::AlreadyKnown) => {
debug!(
self.log,
"Dropping proposer slashing";
"reason" => "Already seen a proposer slashing for that validator",
"validator_index" => validator_index,
"peer" => format!("{:?}", peer_id)
);
None
}
Err(e) => {
debug!(
self.log,
"Dropping invalid proposer slashing";
"validator_index" => validator_index,
"peer" => format!("{:?}", peer_id),
"error" => format!("{:?}", e)
);
None
}
}
}
/// Import a verified proposer slashing into the op pool.
pub fn import_verified_proposer_slashing(
&self,
proposer_slashing: SigVerifiedOp<ProposerSlashing>,
) {
self.chain.import_proposer_slashing(proposer_slashing);
debug!(self.log, "Successfully imported proposer slashing");
}
/// Verify an attester slashing before gossiping or processing it.
///
/// Errors are logged at debug level.
pub fn verify_attester_slashing_for_gossip(
&self,
peer_id: &PeerId,
attester_slashing: AttesterSlashing<T::EthSpec>,
) -> Option<SigVerifiedOp<AttesterSlashing<T::EthSpec>>> {
match self
.chain
.verify_attester_slashing_for_gossip(attester_slashing)
{
Ok(ObservationOutcome::New(verified_slashing)) => Some(verified_slashing),
Ok(ObservationOutcome::AlreadyKnown) => {
debug!(
self.log,
"Dropping attester slashing";
"reason" => "Slashings already known for all slashed validators",
"peer" => format!("{:?}", peer_id)
);
None
}
Err(e) => {
debug!(
self.log,
"Dropping invalid attester slashing";
"peer" => format!("{:?}", peer_id),
"error" => format!("{:?}", e)
);
None
}
}
}
/// Import a verified attester slashing into the op pool.
pub fn import_verified_attester_slashing(
&self,
attester_slashing: SigVerifiedOp<AttesterSlashing<T::EthSpec>>,
) {
if let Err(e) = self.chain.import_attester_slashing(attester_slashing) {
debug!(self.log, "Error importing attester slashing"; "error" => format!("{:?}", e));
} else {
debug!(self.log, "Successfully imported attester slashing");
}
}
}
/// Build a `StatusMessage` representing the state of the given `beacon_chain`.

View File

@ -9,29 +9,27 @@ use attestation::AttMaxCover;
use attestation_id::AttestationId;
use max_cover::maximum_cover;
use parking_lot::RwLock;
use state_processing::per_block_processing::errors::{
AttestationValidationError, AttesterSlashingValidationError, ExitValidationError,
ProposerSlashingValidationError,
};
use state_processing::per_block_processing::errors::AttestationValidationError;
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, VerifySignatures,
get_slashable_indices, get_slashable_indices_modular, verify_attestation_for_block_inclusion,
verify_exit, VerifySignatures,
};
use state_processing::SigVerifiedOp;
use std::collections::{hash_map, HashMap, HashSet};
use std::marker::PhantomData;
use std::ptr;
use types::{
typenum::Unsigned, Attestation, AttesterSlashing, BeaconState, BeaconStateError, ChainSpec,
EthSpec, Fork, Hash256, ProposerSlashing, RelativeEpoch, SignedVoluntaryExit, Validator,
EthSpec, Fork, ForkVersion, Hash256, ProposerSlashing, RelativeEpoch, SignedVoluntaryExit,
Validator,
};
#[derive(Default, Debug)]
pub struct OperationPool<T: EthSpec + Default> {
/// Map from attestation ID (see below) to vectors of attestations.
attestations: RwLock<HashMap<AttestationId, Vec<Attestation<T>>>>,
/// Map from two attestation IDs to a slashing for those IDs.
attester_slashings: RwLock<HashMap<(AttestationId, AttestationId), AttesterSlashing<T>>>,
/// Set of attester slashings, and the fork version they were verified against.
attester_slashings: RwLock<HashSet<(AttesterSlashing<T>, ForkVersion)>>,
/// Map from proposer index to slashing.
proposer_slashings: RwLock<HashMap<u64, ProposerSlashing>>,
/// Map from exiting validator to their exit data.
@ -175,54 +173,23 @@ impl<T: EthSpec> OperationPool<T> {
/// Insert a proposer slashing into the pool.
pub fn insert_proposer_slashing(
&self,
slashing: ProposerSlashing,
state: &BeaconState<T>,
spec: &ChainSpec,
) -> Result<(), ProposerSlashingValidationError> {
// TODO: should maybe insert anyway if the proposer is unknown in the validator index,
// because they could *become* known later
verify_proposer_slashing(&slashing, state, VerifySignatures::True, spec)?;
verified_proposer_slashing: SigVerifiedOp<ProposerSlashing>,
) {
let slashing = verified_proposer_slashing.into_inner();
self.proposer_slashings
.write()
.insert(slashing.signed_header_1.message.proposer_index, slashing);
Ok(())
}
/// Compute the tuple ID that is used to identify an attester slashing.
///
/// Depends on the fork field of the state, but not on the state's epoch.
fn attester_slashing_id(
slashing: &AttesterSlashing<T>,
state: &BeaconState<T>,
spec: &ChainSpec,
) -> (AttestationId, AttestationId) {
(
AttestationId::from_data(
&slashing.attestation_1.data,
&state.fork,
state.genesis_validators_root,
spec,
),
AttestationId::from_data(
&slashing.attestation_2.data,
&state.fork,
state.genesis_validators_root,
spec,
),
)
}
/// Insert an attester slashing into the pool.
pub fn insert_attester_slashing(
&self,
slashing: AttesterSlashing<T>,
state: &BeaconState<T>,
spec: &ChainSpec,
) -> Result<(), AttesterSlashingValidationError> {
verify_attester_slashing(state, &slashing, VerifySignatures::True, spec)?;
let id = Self::attester_slashing_id(&slashing, state, spec);
self.attester_slashings.write().insert(id, slashing);
Ok(())
verified_slashing: SigVerifiedOp<AttesterSlashing<T>>,
fork: Fork,
) {
self.attester_slashings
.write()
.insert((verified_slashing.into_inner(), fork.current_version));
}
/// Get proposer and attester slashings for inclusion in a block.
@ -233,7 +200,6 @@ impl<T: EthSpec> OperationPool<T> {
pub fn get_slashings(
&self,
state: &BeaconState<T>,
spec: &ChainSpec,
) -> (Vec<ProposerSlashing>, Vec<AttesterSlashing<T>>) {
let proposer_slashings = filter_limit_operations(
self.proposer_slashings.read().values(),
@ -258,11 +224,11 @@ impl<T: EthSpec> OperationPool<T> {
.attester_slashings
.read()
.iter()
.filter(|(id, slashing)| {
// Check the fork.
Self::attester_slashing_id(slashing, state, spec) == **id
})
.filter(|(_, slashing)| {
.filter(|(slashing, fork)| {
if *fork != state.fork.previous_version && *fork != state.fork.current_version {
return false;
}
// Take all slashings that will slash 1 or more validators.
let slashed_validators =
get_slashable_indices_modular(state, slashing, |index, validator| {
@ -279,7 +245,7 @@ impl<T: EthSpec> OperationPool<T> {
}
})
.take(T::MaxAttesterSlashings::to_usize())
.map(|(_, slashing)| slashing.clone())
.map(|(slashing, _)| slashing.clone())
.collect();
(proposer_slashings, attester_slashings)
@ -298,17 +264,20 @@ impl<T: EthSpec> OperationPool<T> {
/// Prune attester slashings for all slashed or withdrawn validators, or attestations on another
/// fork.
pub fn prune_attester_slashings(&self, finalized_state: &BeaconState<T>, spec: &ChainSpec) {
self.attester_slashings.write().retain(|id, slashing| {
let fork_ok = &Self::attester_slashing_id(slashing, finalized_state, spec) == id;
let curr_epoch = finalized_state.current_epoch();
let slashing_ok =
get_slashable_indices_modular(finalized_state, slashing, |_, validator| {
validator.slashed || validator.is_withdrawable_at(curr_epoch)
})
.is_ok();
fork_ok && slashing_ok
});
pub fn prune_attester_slashings(&self, finalized_state: &BeaconState<T>, head_fork: Fork) {
self.attester_slashings
.write()
.retain(|(slashing, fork_version)| {
// Any slashings for forks older than the finalized state's previous fork can be
// discarded. We allow the head_fork's current version too in case a fork has
// occurred between the finalized state and the head.
let fork_ok = *fork_version == finalized_state.fork.previous_version
|| *fork_version == finalized_state.fork.current_version
|| *fork_version == head_fork.current_version;
// Slashings that don't slash any validators can also be dropped.
let slashing_ok = get_slashable_indices(finalized_state, slashing).is_ok();
fork_ok && slashing_ok
});
}
/// Total number of attester slashings in the pool.
@ -321,18 +290,12 @@ impl<T: EthSpec> OperationPool<T> {
self.proposer_slashings.read().len()
}
/// Insert a voluntary exit, validating it almost-entirely (future exits are permitted).
pub fn insert_voluntary_exit(
&self,
exit: SignedVoluntaryExit,
state: &BeaconState<T>,
spec: &ChainSpec,
) -> Result<(), ExitValidationError> {
verify_exit_time_independent_only(state, &exit, VerifySignatures::True, spec)?;
/// Insert a voluntary exit that has previously been checked elsewhere.
pub fn insert_voluntary_exit(&self, verified_exit: SigVerifiedOp<SignedVoluntaryExit>) {
let exit = verified_exit.into_inner();
self.voluntary_exits
.write()
.insert(exit.message.validator_index, exit);
Ok(())
}
/// Get a list of voluntary exits for inclusion in a block.
@ -357,11 +320,11 @@ impl<T: EthSpec> OperationPool<T> {
);
}
/// Prune all types of transactions given the latest finalized state.
pub fn prune_all(&self, finalized_state: &BeaconState<T>, spec: &ChainSpec) {
/// Prune all types of transactions given the latest finalized state and head fork.
pub fn prune_all(&self, finalized_state: &BeaconState<T>, head_fork: Fork) {
self.prune_attestations(finalized_state);
self.prune_proposer_slashings(finalized_state);
self.prune_attester_slashings(finalized_state, spec);
self.prune_attester_slashings(finalized_state, head_fork);
self.prune_voluntary_exits(finalized_state);
}
@ -424,7 +387,10 @@ impl<T: EthSpec + Default> PartialEq for OperationPool<T> {
mod release_tests {
use super::attestation::earliest_attestation_validators;
use super::*;
use state_processing::common::{get_attesting_indices, get_base_reward};
use state_processing::{
common::{get_attesting_indices, get_base_reward},
VerifyOperation,
};
use std::collections::BTreeSet;
use std::iter::FromIterator;
use types::test_utils::*;
@ -895,43 +861,99 @@ mod release_tests {
}
}
struct TestContext {
spec: ChainSpec,
state: BeaconState<MainnetEthSpec>,
keypairs: Vec<Keypair>,
op_pool: OperationPool<MainnetEthSpec>,
}
impl TestContext {
fn new() -> Self {
let spec = MainnetEthSpec::default_spec();
let num_validators = 32;
let mut state_builder =
TestingBeaconStateBuilder::<MainnetEthSpec>::from_deterministic_keypairs(
num_validators,
&spec,
);
state_builder.build_caches(&spec).unwrap();
let (state, keypairs) = state_builder.build();
let op_pool = OperationPool::new();
TestContext {
spec,
state,
keypairs,
op_pool,
}
}
fn proposer_slashing(&self, proposer_index: u64) -> ProposerSlashing {
TestingProposerSlashingBuilder::double_vote::<MainnetEthSpec>(
ProposerSlashingTestTask::Valid,
proposer_index,
&self.keypairs[proposer_index as usize].sk,
&self.state.fork,
self.state.genesis_validators_root,
&self.spec,
)
}
fn attester_slashing(&self, slashed_indices: &[u64]) -> AttesterSlashing<MainnetEthSpec> {
let signer =
|idx: u64, message: &[u8]| Signature::new(message, &self.keypairs[idx as usize].sk);
TestingAttesterSlashingBuilder::double_vote(
AttesterSlashingTestTask::Valid,
slashed_indices,
signer,
&self.state.fork,
self.state.genesis_validators_root,
&self.spec,
)
}
}
/// Insert two slashings for the same proposer and ensure only one is returned.
#[test]
fn duplicate_proposer_slashing() {
let spec = MainnetEthSpec::default_spec();
let num_validators = 32;
let mut state_builder =
TestingBeaconStateBuilder::<MainnetEthSpec>::from_deterministic_keypairs(
num_validators,
&spec,
);
state_builder.build_caches(&spec).unwrap();
let (state, keypairs) = state_builder.build();
let op_pool = OperationPool::new();
let ctxt = TestContext::new();
let (op_pool, state, spec) = (&ctxt.op_pool, &ctxt.state, &ctxt.spec);
let proposer_index = 0;
let slashing1 = TestingProposerSlashingBuilder::double_vote::<MainnetEthSpec>(
ProposerSlashingTestTask::Valid,
proposer_index,
&keypairs[proposer_index as usize].sk,
&state.fork,
state.genesis_validators_root,
&spec,
);
let slashing1 = ctxt.proposer_slashing(proposer_index);
let slashing2 = ProposerSlashing {
signed_header_1: slashing1.signed_header_2.clone(),
signed_header_2: slashing1.signed_header_1.clone(),
};
// Both slashings should be accepted by the pool.
op_pool
.insert_proposer_slashing(slashing1.clone(), &state, &spec)
.unwrap();
op_pool
.insert_proposer_slashing(slashing2.clone(), &state, &spec)
.unwrap();
// Both slashings should be valid and accepted by the pool.
op_pool.insert_proposer_slashing(slashing1.clone().validate(state, spec).unwrap());
op_pool.insert_proposer_slashing(slashing2.clone().validate(state, spec).unwrap());
// Should only get the second slashing back.
assert_eq!(op_pool.get_slashings(&state, &spec).0, vec![slashing2]);
assert_eq!(op_pool.get_slashings(state).0, vec![slashing2]);
}
// Sanity check on the pruning of proposer slashings
#[test]
fn prune_proposer_slashing_noop() {
let ctxt = TestContext::new();
let (op_pool, state, spec) = (&ctxt.op_pool, &ctxt.state, &ctxt.spec);
let slashing = ctxt.proposer_slashing(0);
op_pool.insert_proposer_slashing(slashing.clone().validate(state, spec).unwrap());
op_pool.prune_proposer_slashings(state);
assert_eq!(op_pool.get_slashings(state).0, vec![slashing]);
}
// Sanity check on the pruning of attester slashings
#[test]
fn prune_attester_slashing_noop() {
let ctxt = TestContext::new();
let (op_pool, state, spec) = (&ctxt.op_pool, &ctxt.state, &ctxt.spec);
let slashing = ctxt.attester_slashing(&[1, 3, 5, 7, 9]);
op_pool
.insert_attester_slashing(slashing.clone().validate(state, spec).unwrap(), state.fork);
op_pool.prune_attester_slashings(state, state.fork);
assert_eq!(op_pool.get_slashings(state).1, vec![slashing]);
}
}

View File

@ -19,7 +19,7 @@ pub struct PersistedOperationPool<T: EthSpec> {
// be difficult to make that roundtrip due to eager aggregation.
attestations: Vec<(AttestationId, Vec<Attestation<T>>)>,
/// Attester slashings.
attester_slashings: Vec<AttesterSlashing<T>>,
attester_slashings: Vec<(AttesterSlashing<T>, ForkVersion)>,
/// Proposer slashings.
proposer_slashings: Vec<ProposerSlashing>,
/// Voluntary exits.
@ -40,7 +40,7 @@ impl<T: EthSpec> PersistedOperationPool<T> {
.attester_slashings
.read()
.iter()
.map(|(_, slashing)| slashing.clone())
.cloned()
.collect();
let proposer_slashings = operation_pool
@ -66,19 +66,9 @@ impl<T: EthSpec> PersistedOperationPool<T> {
}
/// Reconstruct an `OperationPool`.
pub fn into_operation_pool(self, state: &BeaconState<T>, spec: &ChainSpec) -> OperationPool<T> {
pub fn into_operation_pool(self) -> OperationPool<T> {
let attestations = RwLock::new(self.attestations.into_iter().collect());
let attester_slashings = RwLock::new(
self.attester_slashings
.into_iter()
.map(|slashing| {
(
OperationPool::attester_slashing_id(&slashing, state, spec),
slashing,
)
})
.collect(),
);
let attester_slashings = RwLock::new(self.attester_slashings.into_iter().collect());
let proposer_slashings = RwLock::new(
self.proposer_slashings
.into_iter()

View File

@ -2,7 +2,9 @@ use crate::helpers::*;
use crate::response_builder::ResponseBuilder;
use crate::validator::get_state_for_epoch;
use crate::{ApiError, ApiResult, UrlQuery};
use beacon_chain::{BeaconChain, BeaconChainTypes, StateSkipConfig};
use beacon_chain::{
observed_operations::ObservationOutcome, BeaconChain, BeaconChainTypes, StateSkipConfig,
};
use bus::BusReader;
use futures::executor::block_on;
use hyper::body::Bytes;
@ -504,31 +506,23 @@ pub async fn proposer_slashing<T: BeaconChainTypes>(
.map_err(|e| ApiError::ServerError(format!("Unable to get request body: {:?}", e)))?;
serde_json::from_slice::<ProposerSlashing>(&chunks)
.map_err(|e| {
ApiError::BadRequest(format!(
"Unable to parse JSON into ProposerSlashing: {:?}",
e
))
})
.map_err(|e| format!("Unable to parse JSON into ProposerSlashing: {:?}", e))
.and_then(move |proposer_slashing| {
let spec = &beacon_chain.spec;
let state = &beacon_chain.head().unwrap().beacon_state;
if beacon_chain.eth1_chain.is_some() {
beacon_chain
.op_pool
.insert_proposer_slashing(proposer_slashing, state, spec)
.map_err(|e| {
ApiError::BadRequest(format!(
"Error while inserting proposer slashing: {:?}",
e
))
})
let obs_outcome = beacon_chain
.verify_proposer_slashing_for_gossip(proposer_slashing)
.map_err(|e| format!("Error while verifying proposer slashing: {:?}", e))?;
if let ObservationOutcome::New(verified_proposer_slashing) = obs_outcome {
beacon_chain.import_proposer_slashing(verified_proposer_slashing);
Ok(())
} else {
Err("Proposer slashing for that validator index already known".into())
}
} else {
return Err(ApiError::BadRequest(
"Cannot insert proposer slashing on node without Eth1 connection.".to_string(),
));
Err("Cannot insert proposer slashing on node without Eth1 connection.".to_string())
}
})
.map_err(ApiError::BadRequest)
.and_then(|_| response_builder?.body(&true))
}
@ -551,18 +545,24 @@ pub async fn attester_slashing<T: BeaconChainTypes>(
))
})
.and_then(move |attester_slashing| {
let spec = &beacon_chain.spec;
let state = &beacon_chain.head().unwrap().beacon_state;
if beacon_chain.eth1_chain.is_some() {
beacon_chain
.op_pool
.insert_attester_slashing(attester_slashing, state, spec)
.map_err(|e| {
ApiError::BadRequest(format!(
"Error while inserting attester slashing: {:?}",
e
))
.verify_attester_slashing_for_gossip(attester_slashing)
.map_err(|e| format!("Error while verifying attester slashing: {:?}", e))
.and_then(|outcome| {
if let ObservationOutcome::New(verified_attester_slashing) = outcome {
beacon_chain
.import_attester_slashing(verified_attester_slashing)
.map_err(|e| {
format!("Error while importing attester slashing: {:?}", e)
})
} else {
Err(format!(
"Attester slashing only covers already slashed indices"
))
}
})
.map_err(ApiError::BadRequest)
} else {
Err(ApiError::BadRequest(
"Cannot insert attester slashing on node without Eth1 connection.".to_string(),

View File

@ -1019,7 +1019,7 @@ fn proposer_slashing() {
let spec = &chain.spec;
// Check that there are no proposer slashings before insertion
let (proposer_slashings, _attester_slashings) = chain.op_pool.get_slashings(&state, spec);
let (proposer_slashings, _attester_slashings) = chain.op_pool.get_slashings(&state);
assert_eq!(proposer_slashings.len(), 0);
let slot = state.slot;
@ -1050,7 +1050,7 @@ fn proposer_slashing() {
assert!(result, true);
// Length should be just one as we've inserted only one proposer slashing
let (proposer_slashings, _attester_slashings) = chain.op_pool.get_slashings(&state, spec);
let (proposer_slashings, _attester_slashings) = chain.op_pool.get_slashings(&state);
assert_eq!(proposer_slashings.len(), 1);
assert_eq!(proposer_slashing.clone(), proposer_slashings[0]);
@ -1073,7 +1073,7 @@ fn proposer_slashing() {
assert!(result.is_err());
// Length should still be one as we've inserted nothing since last time.
let (proposer_slashings, _attester_slashings) = chain.op_pool.get_slashings(&state, spec);
let (proposer_slashings, _attester_slashings) = chain.op_pool.get_slashings(&state);
assert_eq!(proposer_slashings.len(), 1);
assert_eq!(proposer_slashing, proposer_slashings[0]);
}
@ -1106,7 +1106,7 @@ fn attester_slashing() {
let fork = &state.fork;
// Checking there are no attester slashings before insertion
let (_proposer_slashings, attester_slashings) = chain.op_pool.get_slashings(&state, spec);
let (_proposer_slashings, attester_slashings) = chain.op_pool.get_slashings(&state);
assert_eq!(attester_slashings.len(), 0);
let attester_slashing = build_double_vote_attester_slashing(
@ -1130,7 +1130,7 @@ fn attester_slashing() {
assert!(result, true);
// Length should be just one as we've inserted only one attester slashing
let (_proposer_slashings, attester_slashings) = chain.op_pool.get_slashings(&state, spec);
let (_proposer_slashings, attester_slashings) = chain.op_pool.get_slashings(&state);
assert_eq!(attester_slashings.len(), 1);
assert_eq!(attester_slashing, attester_slashings[0]);
@ -1151,10 +1151,10 @@ fn attester_slashing() {
.beacon()
.attester_slashing(invalid_attester_slashing),
);
assert!(result.is_err());
result.unwrap_err();
// Length should still be one as we've failed to insert the attester slashing.
let (_proposer_slashings, attester_slashings) = chain.op_pool.get_slashings(&state, spec);
let (_proposer_slashings, attester_slashings) = chain.op_pool.get_slashings(&state);
assert_eq!(attester_slashings.len(), 1);
assert_eq!(attester_slashing, attester_slashings[0]);
}

View File

@ -46,7 +46,7 @@ pub use typenum;
/// // Push a value to if it _does_ exceed the maximum.
/// assert!(long.push(6).is_err());
/// ```
#[derive(Debug, PartialEq, Clone, Serialize, Deserialize)]
#[derive(Debug, PartialEq, Eq, Hash, Clone, Serialize, Deserialize)]
#[serde(transparent)]
pub struct VariableList<T, N> {
vec: Vec<T>,

View File

@ -9,6 +9,7 @@ pub mod per_block_processing;
pub mod per_epoch_processing;
pub mod per_slot_processing;
pub mod test_utils;
pub mod verify_operation;
pub use genesis::{
eth2_genesis_time, initialize_beacon_state_from_eth1, is_valid_genesis_state,
@ -20,3 +21,4 @@ pub use per_block_processing::{
};
pub use per_epoch_processing::{errors::EpochProcessingError, per_epoch_processing};
pub use per_slot_processing::{per_slot_processing, Error as SlotProcessingError};
pub use verify_operation::{SigVerifiedOp, VerifyOperation};

View File

@ -0,0 +1,73 @@
use crate::per_block_processing::{
errors::{
AttesterSlashingValidationError, ExitValidationError, ProposerSlashingValidationError,
},
verify_attester_slashing, verify_exit, verify_proposer_slashing,
};
use crate::VerifySignatures;
use types::{
AttesterSlashing, BeaconState, ChainSpec, EthSpec, ProposerSlashing, SignedVoluntaryExit,
};
/// Wrapper around an operation type that acts as proof that its signature has been checked.
///
/// The inner field is private, meaning instances of this type can only be constructed
/// by calling `validate`.
#[derive(Debug, PartialEq, Eq, Clone)]
pub struct SigVerifiedOp<T>(T);
impl<T> SigVerifiedOp<T> {
pub fn into_inner(self) -> T {
self.0
}
}
/// Trait for operations that can be verified and transformed into a `SigVerifiedOp`.
pub trait VerifyOperation<E: EthSpec>: Sized {
type Error;
fn validate(
self,
state: &BeaconState<E>,
spec: &ChainSpec,
) -> Result<SigVerifiedOp<Self>, Self::Error>;
}
impl<E: EthSpec> VerifyOperation<E> for SignedVoluntaryExit {
type Error = ExitValidationError;
fn validate(
self,
state: &BeaconState<E>,
spec: &ChainSpec,
) -> Result<SigVerifiedOp<Self>, Self::Error> {
verify_exit(state, &self, VerifySignatures::True, spec)?;
Ok(SigVerifiedOp(self))
}
}
impl<E: EthSpec> VerifyOperation<E> for AttesterSlashing<E> {
type Error = AttesterSlashingValidationError;
fn validate(
self,
state: &BeaconState<E>,
spec: &ChainSpec,
) -> Result<SigVerifiedOp<Self>, Self::Error> {
verify_attester_slashing(state, &self, VerifySignatures::True, spec)?;
Ok(SigVerifiedOp(self))
}
}
impl<E: EthSpec> VerifyOperation<E> for ProposerSlashing {
type Error = ProposerSlashingValidationError;
fn validate(
self,
state: &BeaconState<E>,
spec: &ChainSpec,
) -> Result<SigVerifiedOp<Self>, Self::Error> {
verify_proposer_slashing(&self, state, VerifySignatures::True, spec)?;
Ok(SigVerifiedOp(self))
}
}

View File

@ -37,6 +37,7 @@ rand_xorshift = "0.2.0"
cached_tree_hash = { path = "../cached_tree_hash" }
serde_yaml = "0.8.11"
tempfile = "3.1.0"
derivative = "2.1.1"
rusqlite = { version = "0.23.1", features = ["bundled"], optional = true }
arbitrary = { version = "0.4.4", features = ["derive"], optional = true }

View File

@ -1,5 +1,6 @@
use crate::{test_utils::TestRandom, EthSpec, IndexedAttestation};
use derivative::Derivative;
use serde_derive::{Deserialize, Serialize};
use ssz_derive::{Decode, Encode};
use test_random_derive::TestRandom;
@ -9,7 +10,20 @@ use tree_hash_derive::TreeHash;
///
/// Spec v0.11.1
#[cfg_attr(feature = "arbitrary-fuzz", derive(arbitrary::Arbitrary))]
#[derive(Debug, PartialEq, Clone, Serialize, Deserialize, Encode, Decode, TreeHash, TestRandom)]
#[derive(
Derivative,
Debug,
PartialEq,
Eq,
Clone,
Serialize,
Deserialize,
Encode,
Decode,
TreeHash,
TestRandom,
)]
#[derivative(Hash(bound = "T: EthSpec"))]
#[serde(bound = "T: EthSpec")]
pub struct AttesterSlashing<T: EthSpec> {
pub attestation_1: IndexedAttestation<T>,

View File

@ -8,7 +8,7 @@ use ssz_types::typenum::{
};
use std::fmt::Debug;
pub trait EthSpec: 'static + Default + Sync + Send + Clone + Debug + PartialEq {
pub trait EthSpec: 'static + Default + Sync + Send + Clone + Debug + PartialEq + Eq {
/*
* Constants
*/
@ -18,7 +18,7 @@ pub trait EthSpec: 'static + Default + Sync + Send + Clone + Debug + PartialEq {
/*
* Misc
*/
type MaxValidatorsPerCommittee: Unsigned + Clone + Sync + Send + Debug + PartialEq;
type MaxValidatorsPerCommittee: Unsigned + Clone + Sync + Send + Debug + PartialEq + Eq;
/*
* Time parameters
*/
@ -135,7 +135,7 @@ macro_rules! params_from_eth_spec {
///
/// Spec v0.11.1
#[cfg_attr(feature = "arbitrary-fuzz", derive(arbitrary::Arbitrary))]
#[derive(Clone, PartialEq, Debug, Default, Serialize, Deserialize)]
#[derive(Clone, PartialEq, Eq, Debug, Default, Serialize, Deserialize)]
pub struct MainnetEthSpec;
impl EthSpec for MainnetEthSpec {
@ -173,7 +173,7 @@ pub type FoundationBeaconState = BeaconState<MainnetEthSpec>;
///
/// Spec v0.11.1
#[cfg_attr(feature = "arbitrary-fuzz", derive(arbitrary::Arbitrary))]
#[derive(Clone, PartialEq, Debug, Default, Serialize, Deserialize)]
#[derive(Clone, PartialEq, Eq, Debug, Default, Serialize, Deserialize)]
pub struct MinimalEthSpec;
impl EthSpec for MinimalEthSpec {
@ -212,7 +212,7 @@ pub type MinimalBeaconState = BeaconState<MinimalEthSpec>;
/// Interop testnet spec
#[cfg_attr(feature = "arbitrary-fuzz", derive(arbitrary::Arbitrary))]
#[derive(Clone, PartialEq, Debug, Default, Serialize, Deserialize)]
#[derive(Clone, PartialEq, Eq, Debug, Default, Serialize, Deserialize)]
pub struct InteropEthSpec;
impl EthSpec for InteropEthSpec {

View File

@ -12,7 +12,17 @@ use tree_hash_derive::TreeHash;
/// Spec v0.11.1
#[cfg_attr(feature = "arbitrary-fuzz", derive(arbitrary::Arbitrary))]
#[derive(
Debug, Clone, PartialEq, Default, Serialize, Deserialize, Encode, Decode, TreeHash, TestRandom,
Debug,
Clone,
Copy,
PartialEq,
Default,
Serialize,
Deserialize,
Encode,
Decode,
TreeHash,
TestRandom,
)]
pub struct Fork {
#[serde(

View File

@ -1,7 +1,8 @@
use crate::{test_utils::TestRandom, AggregateSignature, AttestationData, EthSpec, VariableList};
use serde_derive::{Deserialize, Serialize};
use ssz::Encode;
use ssz_derive::{Decode, Encode};
use std::hash::{Hash, Hasher};
use test_random_derive::TestRandom;
use tree_hash_derive::TreeHash;
@ -11,7 +12,9 @@ use tree_hash_derive::TreeHash;
///
/// Spec v0.11.1
#[cfg_attr(feature = "arbitrary-fuzz", derive(arbitrary::Arbitrary))]
#[derive(Debug, PartialEq, Clone, Serialize, Deserialize, Encode, Decode, TreeHash, TestRandom)]
#[derive(
Debug, PartialEq, Eq, Clone, Serialize, Deserialize, Encode, Decode, TreeHash, TestRandom,
)]
#[serde(bound = "T: EthSpec")]
pub struct IndexedAttestation<T: EthSpec> {
/// Lists validator registry indices, not committee indices.
@ -37,6 +40,19 @@ impl<T: EthSpec> IndexedAttestation<T> {
}
}
/// Implementation of non-crypto-secure `Hash`, for use with `HashMap` and `HashSet`.
///
/// Guarantees `att1 == att2 -> hash(att1) == hash(att2)`.
///
/// Used in the operation pool.
impl<T: EthSpec> Hash for IndexedAttestation<T> {
fn hash<H: Hasher>(&self, state: &mut H) {
self.attesting_indices.hash(state);
self.data.hash(state);
self.signature.as_ssz_bytes().hash(state);
}
}
#[cfg(test)]
mod tests {
use super::*;

View File

@ -93,6 +93,7 @@ pub use crate::voluntary_exit::VoluntaryExit;
pub type CommitteeIndex = u64;
pub type Hash256 = H256;
pub type Address = H160;
pub type ForkVersion = [u8; 4];
pub use bls::{
AggregatePublicKey, AggregateSignature, Keypair, PublicKey, PublicKeyBytes, SecretKey,

View File

@ -13,10 +13,6 @@ impl TestingAttesterSlashingBuilder {
///
/// - `validator_index: u64`
/// - `message: &[u8]`
/// - `epoch: Epoch`
/// - `domain: Domain`
///
/// Where domain is a domain "constant" (e.g., `spec.domain_attestation`).
pub fn double_vote<F, T: EthSpec>(
test_task: AttesterSlashingTestTask,
validator_indices: &[u64],