[Altair] Sync committee pools (#2321)

Add pools supporting sync committees:
- naive sync aggregation pool
- observed sync contributions pool
- observed sync contributors pool
- observed sync aggregators pool

Add SSZ types and tests related to sync committee signatures.

Co-authored-by: Michael Sproul <michael@sigmaprime.io>
Co-authored-by: realbigsean <seananderson33@gmail.com>
This commit is contained in:
realbigsean 2021-07-15 00:52:02 +00:00
parent 8fa6e463ca
commit a3a7f39b0d
59 changed files with 5277 additions and 933 deletions

2
Cargo.lock generated
View File

@ -4429,6 +4429,7 @@ name = "operation_pool"
version = "0.2.0"
dependencies = [
"beacon_chain",
"derivative",
"eth2_ssz",
"eth2_ssz_derive",
"int_to_bytes",
@ -4442,6 +4443,7 @@ dependencies = [
"serde_derive",
"state_processing",
"store",
"superstruct",
"types",
]

View File

@ -27,11 +27,9 @@
//! ```
use crate::{
beacon_chain::{
HEAD_LOCK_TIMEOUT, MAXIMUM_GOSSIP_CLOCK_DISPARITY, VALIDATOR_PUBKEY_CACHE_LOCK_TIMEOUT,
},
beacon_chain::{MAXIMUM_GOSSIP_CLOCK_DISPARITY, VALIDATOR_PUBKEY_CACHE_LOCK_TIMEOUT},
metrics,
observed_attestations::ObserveOutcome,
observed_aggregates::ObserveOutcome,
observed_attesters::Error as ObservedAttestersError,
BeaconChain, BeaconChainError, BeaconChainTypes,
};
@ -430,7 +428,7 @@ impl<T: BeaconChainTypes> VerifiedAggregatedAttestation<T> {
match chain
.observed_aggregators
.read()
.validator_has_been_observed(attestation, aggregator_index as usize)
.validator_has_been_observed(attestation.data.target.epoch, aggregator_index as usize)
{
Ok(true) => Err(Error::AggregatorAlreadyKnown(aggregator_index)),
Ok(false) => Ok(()),
@ -485,7 +483,7 @@ impl<T: BeaconChainTypes> VerifiedAggregatedAttestation<T> {
if let ObserveOutcome::AlreadyKnown = chain
.observed_attestations
.write()
.observe_attestation(attestation, Some(attestation_root))
.observe_item(attestation, Some(attestation_root))
.map_err(|e| Error::BeaconChainError(e.into()))?
{
return Err(Error::AttestationAlreadyKnown(attestation_root));
@ -498,7 +496,7 @@ impl<T: BeaconChainTypes> VerifiedAggregatedAttestation<T> {
if chain
.observed_aggregators
.write()
.observe_validator(&attestation, aggregator_index as usize)
.observe_validator(attestation.data.target.epoch, aggregator_index as usize)
.map_err(BeaconChainError::from)?
{
return Err(Error::PriorAttestationKnown {
@ -689,7 +687,7 @@ impl<T: BeaconChainTypes> VerifiedUnaggregatedAttestation<T> {
if chain
.observed_attesters
.read()
.validator_has_been_observed(&attestation, validator_index as usize)
.validator_has_been_observed(attestation.data.target.epoch, validator_index as usize)
.map_err(BeaconChainError::from)?
{
return Err(Error::PriorAttestationKnown {
@ -716,7 +714,7 @@ impl<T: BeaconChainTypes> VerifiedUnaggregatedAttestation<T> {
if chain
.observed_attesters
.write()
.observe_validator(&attestation, validator_index as usize)
.observe_validator(attestation.data.target.epoch, validator_index as usize)
.map_err(BeaconChainError::from)?
{
return Err(Error::PriorAttestationKnown {
@ -923,10 +921,8 @@ pub fn verify_attestation_signature<T: BeaconChainTypes>(
.ok_or(BeaconChainError::ValidatorPubkeyCacheLockTimeout)?;
let fork = chain
.canonical_head
.try_read_for(HEAD_LOCK_TIMEOUT)
.ok_or(BeaconChainError::CanonicalHeadLockTimeout)
.map(|head| head.beacon_state.fork())?;
.spec
.fork_at_epoch(indexed_attestation.data.target.epoch);
let signature_set = indexed_attestation_signature_set_from_pubkeys(
|validator_index| pubkey_cache.get(validator_index).map(Cow::Borrowed),
@ -1029,10 +1025,8 @@ pub fn verify_signed_aggregate_signatures<T: BeaconChainTypes>(
}
let fork = chain
.canonical_head
.try_read_for(HEAD_LOCK_TIMEOUT)
.ok_or(BeaconChainError::CanonicalHeadLockTimeout)
.map(|head| head.beacon_state.fork())?;
.spec
.fork_at_epoch(indexed_attestation.data.target.epoch);
let signature_sets = vec![
signed_aggregate_selection_proof_signature_set(

View File

@ -14,15 +14,25 @@ use crate::eth1_chain::{Eth1Chain, Eth1ChainBackend};
use crate::events::ServerSentEventHandler;
use crate::head_tracker::HeadTracker;
use crate::migrate::BackgroundMigrator;
use crate::naive_aggregation_pool::{Error as NaiveAggregationError, NaiveAggregationPool};
use crate::observed_attestations::{Error as AttestationObservationError, ObservedAttestations};
use crate::observed_attesters::{ObservedAggregators, ObservedAttesters};
use crate::naive_aggregation_pool::{
AggregatedAttestationMap, Error as NaiveAggregationError, NaiveAggregationPool,
SyncContributionAggregateMap,
};
use crate::observed_aggregates::{
Error as AttestationObservationError, ObservedAggregateAttestations, ObservedSyncContributions,
};
use crate::observed_attesters::{
ObservedAggregators, ObservedAttesters, ObservedSyncAggregators, ObservedSyncContributors,
};
use crate::observed_block_producers::ObservedBlockProducers;
use crate::observed_operations::{ObservationOutcome, ObservedOperations};
use crate::persisted_beacon_chain::{PersistedBeaconChain, DUMMY_CANONICAL_HEAD_BLOCK_ROOT};
use crate::persisted_fork_choice::PersistedForkChoice;
use crate::shuffling_cache::{BlockShufflingIds, ShufflingCache};
use crate::snapshot_cache::SnapshotCache;
use crate::sync_committee_verification::{
Error as SyncCommitteeError, VerifiedSyncCommitteeMessage, VerifiedSyncContribution,
};
use crate::timeout_rw_lock::TimeoutRwLock;
use crate::validator_monitor::{
get_block_delay_ms, get_slot_delay_ms, timestamp_now, ValidatorMonitor,
@ -39,6 +49,7 @@ use itertools::process_results;
use itertools::Itertools;
use operation_pool::{OperationPool, PersistedOperationPool};
use parking_lot::{Mutex, RwLock};
use safe_arith::SafeArith;
use slasher::Slasher;
use slog::{crit, debug, error, info, trace, warn, Logger};
use slot_clock::SlotClock;
@ -221,14 +232,28 @@ pub struct BeaconChain<T: BeaconChainTypes> {
///
/// This pool accepts `Attestation` objects that only have one aggregation bit set and provides
/// a method to get an aggregated `Attestation` for some `AttestationData`.
pub naive_aggregation_pool: RwLock<NaiveAggregationPool<T::EthSpec>>,
pub naive_aggregation_pool: RwLock<NaiveAggregationPool<AggregatedAttestationMap<T::EthSpec>>>,
/// A pool of `SyncCommitteeContribution` dedicated to the "naive aggregation strategy" defined in the eth2
/// specs.
///
/// This pool accepts `SyncCommitteeContribution` objects that only have one aggregation bit set and provides
/// a method to get an aggregated `SyncCommitteeContribution` for some `SyncCommitteeContributionData`.
pub naive_sync_aggregation_pool:
RwLock<NaiveAggregationPool<SyncContributionAggregateMap<T::EthSpec>>>,
/// Contains a store of attestations which have been observed by the beacon chain.
pub(crate) observed_attestations: RwLock<ObservedAttestations<T::EthSpec>>,
pub(crate) observed_attestations: RwLock<ObservedAggregateAttestations<T::EthSpec>>,
/// Contains a store of sync contributions which have been observed by the beacon chain.
pub(crate) observed_sync_contributions: RwLock<ObservedSyncContributions<T::EthSpec>>,
/// Maintains a record of which validators have been seen to attest in recent epochs.
pub(crate) observed_attesters: RwLock<ObservedAttesters<T::EthSpec>>,
/// Maintains a record of which validators have been seen sending sync messages in recent epochs.
pub(crate) observed_sync_contributors: RwLock<ObservedSyncContributors<T::EthSpec>>,
/// Maintains a record of which validators have been seen to create `SignedAggregateAndProofs`
/// in recent epochs.
pub(crate) observed_aggregators: RwLock<ObservedAggregators<T::EthSpec>>,
/// Maintains a record of which validators have been seen to create `SignedContributionAndProofs`
/// in recent epochs.
pub(crate) observed_sync_aggregators: RwLock<ObservedSyncAggregators<T::EthSpec>>,
/// Maintains a record of which validators have proposed blocks for each slot.
pub(crate) observed_block_producers: RwLock<ObservedBlockProducers<T::EthSpec>>,
/// Maintains a record of which validators have submitted voluntary exits.
@ -823,6 +848,80 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
})
}
/// Return the sync committee at `slot + 1` from the canonical chain.
///
/// This is useful when dealing with sync committee messages, because messages are signed
/// and broadcast one slot prior to the slot of the sync committee (which is relevant at
/// sync committee period boundaries).
pub fn sync_committee_at_next_slot(
&self,
slot: Slot,
) -> Result<Arc<SyncCommittee<T::EthSpec>>, Error> {
let epoch = slot.safe_add(1)?.epoch(T::EthSpec::slots_per_epoch());
self.sync_committee_at_epoch(epoch)
}
/// Return the sync committee at `epoch` from the canonical chain.
pub fn sync_committee_at_epoch(
&self,
epoch: Epoch,
) -> Result<Arc<SyncCommittee<T::EthSpec>>, Error> {
// Try to read a committee from the head. This will work most of the time, but will fail
// for faraway committees, or if there are skipped slots at the transition to Altair.
let spec = &self.spec;
let committee_from_head =
self.with_head(
|head| match head.beacon_state.get_built_sync_committee(epoch, spec) {
Ok(committee) => Ok(Some(committee.clone())),
Err(BeaconStateError::SyncCommitteeNotKnown { .. })
| Err(BeaconStateError::IncorrectStateVariant) => Ok(None),
Err(e) => Err(Error::from(e)),
},
)?;
if let Some(committee) = committee_from_head {
Ok(committee)
} else {
// Slow path: load a state (or advance the head).
let sync_committee_period = epoch.sync_committee_period(spec)?;
let committee = self
.state_for_sync_committee_period(sync_committee_period)?
.get_built_sync_committee(epoch, spec)?
.clone();
Ok(committee)
}
}
/// Load a state suitable for determining the sync committee for the given period.
///
/// Specifically, the state at the start of the *previous* sync committee period.
///
/// This is sufficient for historical duties, and efficient in the case where the head
/// is lagging the current period and we need duties for the next period (because we only
/// have to transition the head to start of the current period).
///
/// We also need to ensure that the load slot is after the Altair fork.
///
/// **WARNING**: the state returned will have dummy state roots. It should only be used
/// for its sync committees (determining duties, etc).
pub fn state_for_sync_committee_period(
&self,
sync_committee_period: u64,
) -> Result<BeaconState<T::EthSpec>, Error> {
let altair_fork_epoch = self
.spec
.altair_fork_epoch
.ok_or(Error::AltairForkDisabled)?;
let load_slot = std::cmp::max(
self.spec.epochs_per_sync_committee_period * sync_committee_period.saturating_sub(1),
altair_fork_epoch,
)
.start_slot(T::EthSpec::slots_per_epoch());
self.state_at_slot(load_slot, StateSkipConfig::WithoutStateRoots)
}
/// Returns info representing the head block and state.
///
/// A summarized version of `Self::head` that involves less cloning.
@ -1270,6 +1369,36 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
})
}
/// Accepts some `SyncCommitteeMessage` from the network and attempts to verify it, returning `Ok(_)` if
/// it is valid to be (re)broadcast on the gossip network.
pub fn verify_sync_committee_message_for_gossip(
&self,
sync_message: SyncCommitteeMessage,
subnet_id: SyncSubnetId,
) -> Result<VerifiedSyncCommitteeMessage, SyncCommitteeError> {
metrics::inc_counter(&metrics::SYNC_MESSAGE_PROCESSING_REQUESTS);
let _timer = metrics::start_timer(&metrics::SYNC_MESSAGE_GOSSIP_VERIFICATION_TIMES);
VerifiedSyncCommitteeMessage::verify(sync_message, subnet_id, self).map(|v| {
metrics::inc_counter(&metrics::SYNC_MESSAGE_PROCESSING_SUCCESSES);
v
})
}
/// Accepts some `SignedContributionAndProof` from the network and attempts to verify it,
/// returning `Ok(_)` if it is valid to be (re)broadcast on the gossip network.
pub fn verify_sync_contribution_for_gossip(
&self,
sync_contribution: SignedContributionAndProof<T::EthSpec>,
) -> Result<VerifiedSyncContribution<T>, SyncCommitteeError> {
metrics::inc_counter(&metrics::SYNC_CONTRIBUTION_PROCESSING_REQUESTS);
let _timer = metrics::start_timer(&metrics::SYNC_CONTRIBUTION_GOSSIP_VERIFICATION_TIMES);
VerifiedSyncContribution::verify(sync_contribution, self).map(|v| {
metrics::inc_counter(&metrics::SYNC_CONTRIBUTION_PROCESSING_SUCCESSES);
v
})
}
/// Accepts some attestation-type object and attempts to verify it in the context of fork
/// choice. If it is valid it is applied to `self.fork_choice`.
///
@ -1339,6 +1468,70 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
Ok(unaggregated_attestation)
}
/// Accepts a `VerifiedSyncCommitteeMessage` and attempts to apply it to the "naive
/// aggregation pool".
///
/// The naive aggregation pool is used by local validators to produce
/// `SignedContributionAndProof`.
///
/// If the sync message is too old (low slot) to be included in the pool it is simply dropped
/// and no error is returned.
pub fn add_to_naive_sync_aggregation_pool(
&self,
verified_sync_committee_message: VerifiedSyncCommitteeMessage,
) -> Result<VerifiedSyncCommitteeMessage, SyncCommitteeError> {
let sync_message = verified_sync_committee_message.sync_message();
let positions_by_subnet_id: &HashMap<SyncSubnetId, Vec<usize>> =
verified_sync_committee_message.subnet_positions();
for (subnet_id, positions) in positions_by_subnet_id.iter() {
for position in positions {
let _timer =
metrics::start_timer(&metrics::SYNC_CONTRIBUTION_PROCESSING_APPLY_TO_AGG_POOL);
let contribution = SyncCommitteeContribution::from_message(
sync_message,
subnet_id.into(),
*position,
)?;
match self
.naive_sync_aggregation_pool
.write()
.insert(&contribution)
{
Ok(outcome) => trace!(
self.log,
"Stored unaggregated sync committee message";
"outcome" => ?outcome,
"index" => sync_message.validator_index,
"slot" => sync_message.slot.as_u64(),
),
Err(NaiveAggregationError::SlotTooLow {
slot,
lowest_permissible_slot,
}) => {
trace!(
self.log,
"Refused to store unaggregated sync committee message";
"lowest_permissible_slot" => lowest_permissible_slot.as_u64(),
"slot" => slot.as_u64(),
);
}
Err(e) => {
error!(
self.log,
"Failed to store unaggregated sync committee message";
"error" => ?e,
"index" => sync_message.validator_index,
"slot" => sync_message.slot.as_u64(),
);
return Err(Error::from(e).into());
}
};
}
}
Ok(verified_sync_committee_message)
}
/// Accepts a `VerifiedAggregatedAttestation` and attempts to apply it to `self.op_pool`.
///
/// The op pool is used by local block producers to pack blocks with operations.
@ -1368,6 +1561,26 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
Ok(signed_aggregate)
}
/// Accepts a `VerifiedSyncContribution` and attempts to apply it to `self.op_pool`.
///
/// The op pool is used by local block producers to pack blocks with operations.
pub fn add_contribution_to_block_inclusion_pool(
&self,
contribution: VerifiedSyncContribution<T>,
) -> Result<(), SyncCommitteeError> {
let _timer = metrics::start_timer(&metrics::SYNC_CONTRIBUTION_PROCESSING_APPLY_TO_OP_POOL);
// If there's no eth1 chain then it's impossible to produce blocks and therefore
// useless to put things in the op pool.
if self.eth1_chain.is_some() {
self.op_pool
.insert_sync_contribution(contribution.contribution())
.map_err(Error::from)?;
}
Ok(())
}
/// Filter an attestation from the op pool for shuffling compatibility.
///
/// Use the provided `filter_cache` map to memoize results.
@ -1818,11 +2031,7 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
// Iterate through the attestations in the block and register them as an "observed
// attestation". This will stop us from propagating them on the gossip network.
for a in signed_block.message().body().attestations() {
match self
.observed_attestations
.write()
.observe_attestation(a, None)
{
match self.observed_attestations.write().observe_item(a, None) {
// If the observation was successful or if the slot for the attestation was too
// low, continue.
//

View File

@ -251,6 +251,13 @@ where
.get_item::<PersistedOperationPool<TEthSpec>>(&OP_POOL_DB_KEY)
.map_err(|e| format!("DB error whilst reading persisted op pool: {:?}", e))?
.map(PersistedOperationPool::into_operation_pool)
.transpose()
.map_err(|e| {
format!(
"Error while creating the op pool from the persisted op pool: {:?}",
e
)
})?
.unwrap_or_else(OperationPool::new),
);
@ -506,12 +513,20 @@ where
// TODO: allow for persisting and loading the pool from disk.
naive_aggregation_pool: <_>::default(),
// TODO: allow for persisting and loading the pool from disk.
naive_sync_aggregation_pool: <_>::default(),
// TODO: allow for persisting and loading the pool from disk.
observed_attestations: <_>::default(),
// TODO: allow for persisting and loading the pool from disk.
observed_sync_contributions: <_>::default(),
// TODO: allow for persisting and loading the pool from disk.
observed_attesters: <_>::default(),
// TODO: allow for persisting and loading the pool from disk.
observed_sync_contributors: <_>::default(),
// TODO: allow for persisting and loading the pool from disk.
observed_aggregators: <_>::default(),
// TODO: allow for persisting and loading the pool from disk.
observed_sync_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(),

View File

@ -3,7 +3,7 @@ use crate::beacon_fork_choice_store::Error as ForkChoiceStoreError;
use crate::eth1_chain::Error as Eth1ChainError;
use crate::migrate::PruningError;
use crate::naive_aggregation_pool::Error as NaiveAggregationError;
use crate::observed_attestations::Error as ObservedAttestationsError;
use crate::observed_aggregates::Error as ObservedAttestationsError;
use crate::observed_attesters::Error as ObservedAttestersError;
use crate::observed_block_producers::Error as ObservedBlockProducersError;
use futures::channel::mpsc::TrySendError;
@ -14,7 +14,7 @@ use state_processing::{
block_signature_verifier::Error as BlockSignatureVerifierError,
per_block_processing::errors::{
AttestationValidationError, AttesterSlashingValidationError, ExitValidationError,
ProposerSlashingValidationError,
ProposerSlashingValidationError, SyncCommitteeMessageValidationError,
},
signature_sets::Error as SignatureSetError,
state_advance::Error as StateAdvanceError,
@ -60,6 +60,7 @@ pub enum BeaconChainError {
},
CannotAttestToFutureState,
AttestationValidationError(AttestationValidationError),
SyncCommitteeMessageValidationError(SyncCommitteeMessageValidationError),
ExitValidationError(ExitValidationError),
ProposerSlashingValidationError(ProposerSlashingValidationError),
AttesterSlashingValidationError(AttesterSlashingValidationError),
@ -121,10 +122,12 @@ pub enum BeaconChainError {
old_slot: Slot,
new_slot: Slot,
},
AltairForkDisabled,
}
easy_from_to!(SlotProcessingError, BeaconChainError);
easy_from_to!(AttestationValidationError, BeaconChainError);
easy_from_to!(SyncCommitteeMessageValidationError, BeaconChainError);
easy_from_to!(ExitValidationError, BeaconChainError);
easy_from_to!(ProposerSlashingValidationError, BeaconChainError);
easy_from_to!(AttesterSlashingValidationError, BeaconChainError);

View File

@ -14,7 +14,7 @@ mod head_tracker;
mod metrics;
pub mod migrate;
mod naive_aggregation_pool;
mod observed_attestations;
mod observed_aggregates;
mod observed_attesters;
mod observed_block_producers;
pub mod observed_operations;
@ -24,6 +24,7 @@ pub mod schema_change;
mod shuffling_cache;
mod snapshot_cache;
pub mod state_advance_timer;
pub mod sync_committee_verification;
pub mod test_utils;
mod timeout_rw_lock;
pub mod validator_monitor;

View File

@ -1,3 +1,5 @@
use crate::observed_attesters::SlotSubcommitteeIndex;
use crate::types::consts::altair::SYNC_COMMITTEE_SUBNET_COUNT;
use crate::{BeaconChain, BeaconChainError, BeaconChainTypes};
use lazy_static::lazy_static;
pub use lighthouse_metrics::*;
@ -144,10 +146,6 @@ lazy_static! {
"beacon_attestation_processing_apply_to_agg_pool",
"Time spent applying an attestation to the naive aggregation pool"
);
pub static ref ATTESTATION_PROCESSING_AGG_POOL_MAPS_WRITE_LOCK: Result<Histogram> = try_create_histogram(
"beacon_attestation_processing_agg_pool_maps_write_lock",
"Time spent waiting for the maps write lock when adding to the agg poll"
);
pub static ref ATTESTATION_PROCESSING_AGG_POOL_PRUNE: Result<Histogram> = try_create_histogram(
"beacon_attestation_processing_agg_pool_prune",
"Time spent for the agg pool to prune"
@ -329,6 +327,8 @@ lazy_static! {
try_create_int_gauge("beacon_op_pool_proposer_slashings_total", "Count of proposer slashings in the op pool");
pub static ref OP_POOL_NUM_VOLUNTARY_EXITS: Result<IntGauge> =
try_create_int_gauge("beacon_op_pool_voluntary_exits_total", "Count of voluntary exits in the op pool");
pub static ref OP_POOL_NUM_SYNC_CONTRIBUTIONS: Result<IntGauge> =
try_create_int_gauge("beacon_op_pool_sync_contributions_total", "Count of sync contributions in the op pool");
/*
* Participation Metrics
@ -357,6 +357,18 @@ lazy_static! {
"beacon_attn_observation_epoch_aggregators",
"Count of aggregators that have been seen by the beacon chain in the previous epoch"
);
/*
* Sync Committee Observation Metrics
*/
pub static ref SYNC_COMM_OBSERVATION_PREV_SLOT_SIGNERS: Result<IntGauge> = try_create_int_gauge(
"beacon_sync_comm_observation_slot_signers",
"Count of sync committee contributors that have been seen by the beacon chain in the previous slot"
);
pub static ref SYNC_COMM_OBSERVATION_PREV_SLOT_AGGREGATORS: Result<IntGauge> = try_create_int_gauge(
"beacon_sync_comm_observation_slot_aggregators",
"Count of sync committee aggregators that have been seen by the beacon chain in the previous slot"
);
}
// Third lazy-static block is used to account for macro recursion limit.
@ -649,6 +661,93 @@ lazy_static! {
);
}
// Fourth lazy-static block is used to account for macro recursion limit.
lazy_static! {
/*
* Sync Committee Message Verification
*/
pub static ref SYNC_MESSAGE_PROCESSING_REQUESTS: Result<IntCounter> = try_create_int_counter(
"beacon_sync_committee_message_processing_requests_total",
"Count of all sync messages submitted for processing"
);
pub static ref SYNC_MESSAGE_PROCESSING_SUCCESSES: Result<IntCounter> = try_create_int_counter(
"beacon_sync_committee_message_processing_successes_total",
"Number of sync messages verified for gossip"
);
pub static ref SYNC_MESSAGE_GOSSIP_VERIFICATION_TIMES: Result<Histogram> = try_create_histogram(
"beacon_sync_committee_message_gossip_verification_seconds",
"Full runtime of sync contribution gossip verification"
);
/*
* Sync Committee Contribution Verification
*/
pub static ref SYNC_CONTRIBUTION_PROCESSING_REQUESTS: Result<IntCounter> = try_create_int_counter(
"beacon_sync_contribution_processing_requests_total",
"Count of all sync contributions submitted for processing"
);
pub static ref SYNC_CONTRIBUTION_PROCESSING_SUCCESSES: Result<IntCounter> = try_create_int_counter(
"beacon_sync_contribution_processing_successes_total",
"Number of sync contributions verified for gossip"
);
pub static ref SYNC_CONTRIBUTION_GOSSIP_VERIFICATION_TIMES: Result<Histogram> = try_create_histogram(
"beacon_sync_contribution_gossip_verification_seconds",
"Full runtime of sync contribution gossip verification"
);
/*
* General Sync Committee Contribution Processing
*/
pub static ref SYNC_CONTRIBUTION_PROCESSING_APPLY_TO_AGG_POOL: Result<Histogram> = try_create_histogram(
"beacon_sync_contribution_processing_apply_to_agg_pool",
"Time spent applying a sync contribution to the naive aggregation pool"
);
pub static ref SYNC_CONTRIBUTION_PROCESSING_AGG_POOL_PRUNE: Result<Histogram> = try_create_histogram(
"beacon_sync_contribution_processing_agg_pool_prune",
"Time spent for the agg pool to prune"
);
pub static ref SYNC_CONTRIBUTION_PROCESSING_AGG_POOL_INSERT: Result<Histogram> = try_create_histogram(
"beacon_sync_contribution_processing_agg_pool_insert",
"Time spent for the outer pool.insert() function of agg pool"
);
pub static ref SYNC_CONTRIBUTION_PROCESSING_AGG_POOL_CORE_INSERT: Result<Histogram> = try_create_histogram(
"beacon_sync_contribution_processing_agg_pool_core_insert",
"Time spent for the core map.insert() function of agg pool"
);
pub static ref SYNC_CONTRIBUTION_PROCESSING_AGG_POOL_AGGREGATION: Result<Histogram> = try_create_histogram(
"beacon_sync_contribution_processing_agg_pool_aggregation",
"Time spent doing signature aggregation when adding to the agg poll"
);
pub static ref SYNC_CONTRIBUTION_PROCESSING_AGG_POOL_CREATE_MAP: Result<Histogram> = try_create_histogram(
"beacon_sync_contribution_processing_agg_pool_create_map",
"Time spent for creating a map for a new slot"
);
pub static ref SYNC_CONTRIBUTION_PROCESSING_APPLY_TO_OP_POOL: Result<Histogram> = try_create_histogram(
"beacon_sync_contribution_processing_apply_to_op_pool",
"Time spent applying a sync contribution to the block inclusion pool"
);
pub static ref SYNC_CONTRIBUTION_PROCESSING_SIGNATURE_SETUP_TIMES: Result<Histogram> = try_create_histogram(
"beacon_sync_contribution_processing_signature_setup_seconds",
"Time spent on setting up for the signature verification of sync contribution processing"
);
pub static ref SYNC_CONTRIBUTION_PROCESSING_SIGNATURE_TIMES: Result<Histogram> = try_create_histogram(
"beacon_sync_contribution_processing_signature_seconds",
"Time spent on the signature verification of sync contribution processing"
);
/*
* General Sync Committee Contribution Processing
*/
pub static ref SYNC_MESSAGE_PROCESSING_SIGNATURE_SETUP_TIMES: Result<Histogram> = try_create_histogram(
"beacon_sync_committee_message_processing_signature_setup_seconds",
"Time spent on setting up for the signature verification of sync message processing"
);
pub static ref SYNC_MESSAGE_PROCESSING_SIGNATURE_TIMES: Result<Histogram> = try_create_histogram(
"beacon_sync_committee_message_processing_signature_seconds",
"Time spent on the signature verification of sync message processing"
);
}
/// Scrape the `beacon_chain` for metrics that are not constantly updated (e.g., the present slot,
/// head state info, etc) and update the Prometheus `DEFAULT_REGISTRY`.
pub fn scrape_for_metrics<T: BeaconChainTypes>(beacon_chain: &BeaconChain<T>) {
@ -659,6 +758,7 @@ pub fn scrape_for_metrics<T: BeaconChainTypes>(beacon_chain: &BeaconChain<T>) {
if let Some(slot) = beacon_chain.slot_clock.now() {
scrape_attestation_observation(slot, beacon_chain);
scrape_sync_committee_observation(slot, beacon_chain);
}
set_gauge_by_usize(
@ -677,6 +777,10 @@ pub fn scrape_for_metrics<T: BeaconChainTypes>(beacon_chain: &BeaconChain<T>) {
&OP_POOL_NUM_VOLUNTARY_EXITS,
beacon_chain.op_pool.num_voluntary_exits(),
);
set_gauge_by_usize(
&OP_POOL_NUM_SYNC_CONTRIBUTIONS,
beacon_chain.op_pool.num_sync_contributions(),
);
beacon_chain
.validator_monitor
@ -771,6 +875,34 @@ fn scrape_attestation_observation<T: BeaconChainTypes>(slot_now: Slot, chain: &B
}
}
fn scrape_sync_committee_observation<T: BeaconChainTypes>(slot_now: Slot, chain: &BeaconChain<T>) {
let prev_slot = slot_now - 1;
let contributors = chain.observed_sync_contributors.read();
let mut contributor_sum = 0;
for i in 0..SYNC_COMMITTEE_SUBNET_COUNT {
if let Some(count) =
contributors.observed_validator_count(SlotSubcommitteeIndex::new(prev_slot, i))
{
contributor_sum += count;
}
}
drop(contributors);
set_gauge_by_usize(&SYNC_COMM_OBSERVATION_PREV_SLOT_SIGNERS, contributor_sum);
let sync_aggregators = chain.observed_sync_aggregators.read();
let mut aggregator_sum = 0;
for i in 0..SYNC_COMMITTEE_SUBNET_COUNT {
if let Some(count) =
sync_aggregators.observed_validator_count(SlotSubcommitteeIndex::new(prev_slot, i))
{
aggregator_sum += count;
}
}
drop(sync_aggregators);
set_gauge_by_usize(&SYNC_COMM_OBSERVATION_PREV_SLOT_AGGREGATORS, aggregator_sum);
}
fn set_gauge_by_slot(gauge: &Result<IntGauge>, value: Slot) {
set_gauge(gauge, value.as_u64() as i64);
}

View File

@ -1,13 +1,18 @@
use crate::metrics;
use std::collections::HashMap;
use tree_hash::TreeHash;
use types::{Attestation, AttestationData, EthSpec, Hash256, Slot};
use types::consts::altair::SYNC_COMMITTEE_SUBNET_COUNT;
use types::slot_data::SlotData;
use types::sync_committee_contribution::SyncContributionData;
use types::{Attestation, AttestationData, EthSpec, Hash256, Slot, SyncCommitteeContribution};
type AttestationDataRoot = Hash256;
type SyncDataRoot = Hash256;
/// The number of slots that will be stored in the pool.
///
/// For example, if `SLOTS_RETAINED == 3` and the pool is pruned at slot `6`, then all attestations
/// at slots less than `4` will be dropped and any future attestation with a slot less than `4`
/// For example, if `SLOTS_RETAINED == 3` and the pool is pruned at slot `6`, then all items
/// at slots less than `4` will be dropped and any future item with a slot less than `4`
/// will be refused.
const SLOTS_RETAINED: usize = 3;
@ -16,51 +21,98 @@ const SLOTS_RETAINED: usize = 3;
/// This is a DoS protection measure.
const MAX_ATTESTATIONS_PER_SLOT: usize = 16_384;
/// Returned upon successfully inserting an attestation into the pool.
/// Returned upon successfully inserting an item into the pool.
#[derive(Debug, PartialEq)]
pub enum InsertOutcome {
/// The `attestation.data` had not been seen before and was added to the pool.
NewAttestationData { committee_index: usize },
/// A validator signature for the given `attestation.data` was already known. No changes were
/// The item had not been seen before and was added to the pool.
NewItemInserted { committee_index: usize },
/// A validator signature for the given item's `Data` was already known. No changes were
/// made.
SignatureAlreadyKnown { committee_index: usize },
/// The `attestation.data` was known, but a signature for the given validator was not yet
/// The item's `Data` was known, but a signature for the given validator was not yet
/// known. The signature was aggregated into the pool.
SignatureAggregated { committee_index: usize },
}
#[derive(Debug, PartialEq)]
pub enum Error {
/// The given `attestation.data.slot` was too low to be stored. No changes were made.
/// The given `data.slot` was too low to be stored. No changes were made.
SlotTooLow {
slot: Slot,
lowest_permissible_slot: Slot,
},
/// The given `attestation.aggregation_bits` field was empty.
/// The given `aggregation_bits` field was empty.
NoAggregationBitsSet,
/// The given `attestation.aggregation_bits` field had more than one signature. The number of
/// The given `aggregation_bits` field had more than one signature. The number of
/// signatures found is included.
MoreThanOneAggregationBitSet(usize),
/// We have reached the maximum number of unique `AttestationData` that can be stored in a
/// We have reached the maximum number of unique items that can be stored in a
/// slot. This is a DoS protection function.
ReachedMaxAttestationsPerSlot(usize),
/// The given `attestation.aggregation_bits` field had a different length to the one currently
ReachedMaxItemsPerSlot(usize),
/// The given `aggregation_bits` field had a different length to the one currently
/// stored. This indicates a fairly serious error somewhere in the code that called this
/// function.
InconsistentBitfieldLengths,
/// The given `attestation` was for the incorrect slot. This is an internal error.
IncorrectSlot { expected: Slot, attestation: Slot },
/// The given item was for the incorrect slot. This is an internal error.
IncorrectSlot { expected: Slot, actual: Slot },
}
/// Implemented for items in the `NaiveAggregationPool`. Requires that items implement `SlotData`,
/// which means they have an associated slot. This handles aggregation of items that are inserted.
pub trait AggregateMap {
/// `Key` should be a hash of `Data`.
type Key;
/// The item stored in the map
type Value: Clone + SlotData;
/// The unique fields of `Value`, hashed to create `Key`.
type Data: SlotData;
/// Create a new `AggregateMap` with capacity `initial_capacity`.
fn new(initial_capacity: usize) -> Self;
/// Insert a `Value` into `Self`, returning a result.
fn insert(&mut self, value: &Self::Value) -> Result<InsertOutcome, Error>;
/// Get a `Value` from `Self` based on `Data`.
fn get(&self, data: &Self::Data) -> Option<Self::Value>;
/// Get a reference to the inner `HashMap`.
fn get_map(&self) -> &HashMap<Self::Key, Self::Value>;
/// Get a `Value` from `Self` based on `Key`, which is a hash of `Data`.
fn get_by_root(&self, root: &Self::Key) -> Option<&Self::Value>;
/// The number of items store in `Self`.
fn len(&self) -> usize;
/// Start a timer observing inserts.
fn start_insert_timer() -> Option<metrics::HistogramTimer>;
/// Start a timer observing the time it takes to create a new map for a new slot.
fn start_create_map_timer() -> Option<metrics::HistogramTimer>;
/// Start a timer observing the time it takes to prune the pool.
fn start_prune_timer() -> Option<metrics::HistogramTimer>;
/// The default capacity of `Self`.
fn default_capacity() -> usize;
}
/// A collection of `Attestation` objects, keyed by their `attestation.data`. Enforces that all
/// `attestation` are from the same slot.
struct AggregatedAttestationMap<E: EthSpec> {
pub struct AggregatedAttestationMap<E: EthSpec> {
map: HashMap<AttestationDataRoot, Attestation<E>>,
}
impl<E: EthSpec> AggregatedAttestationMap<E> {
impl<E: EthSpec> AggregateMap for AggregatedAttestationMap<E> {
type Key = AttestationDataRoot;
type Value = Attestation<E>;
type Data = AttestationData;
/// Create an empty collection with the given `initial_capacity`.
pub fn new(initial_capacity: usize) -> Self {
fn new(initial_capacity: usize) -> Self {
Self {
map: HashMap::with_capacity(initial_capacity),
}
@ -69,7 +121,7 @@ impl<E: EthSpec> AggregatedAttestationMap<E> {
/// Insert an attestation into `self`, aggregating it into the pool.
///
/// The given attestation (`a`) must only have one signature.
pub fn insert(&mut self, a: &Attestation<E>) -> Result<InsertOutcome, Error> {
fn insert(&mut self, a: &Self::Value) -> Result<InsertOutcome, Error> {
let _timer = metrics::start_timer(&metrics::ATTESTATION_PROCESSING_AGG_POOL_CORE_INSERT);
let set_bits = a
@ -106,65 +158,190 @@ impl<E: EthSpec> AggregatedAttestationMap<E> {
}
} else {
if self.map.len() >= MAX_ATTESTATIONS_PER_SLOT {
return Err(Error::ReachedMaxAttestationsPerSlot(
MAX_ATTESTATIONS_PER_SLOT,
));
return Err(Error::ReachedMaxItemsPerSlot(MAX_ATTESTATIONS_PER_SLOT));
}
self.map.insert(attestation_data_root, a.clone());
Ok(InsertOutcome::NewAttestationData { committee_index })
Ok(InsertOutcome::NewItemInserted { committee_index })
}
}
/// Returns an aggregated `Attestation` with the given `data`, if any.
///
/// The given `a.data.slot` must match the slot that `self` was initialized with.
pub fn get(&self, data: &AttestationData) -> Option<Attestation<E>> {
fn get(&self, data: &Self::Data) -> Option<Self::Value> {
self.map.get(&data.tree_hash_root()).cloned()
}
fn get_map(&self) -> &HashMap<Self::Key, Self::Value> {
&self.map
}
/// Returns an aggregated `Attestation` with the given `root`, if any.
pub fn get_by_root(&self, root: &AttestationDataRoot) -> Option<&Attestation<E>> {
fn get_by_root(&self, root: &Self::Key) -> Option<&Self::Value> {
self.map.get(root)
}
/// Iterate all attestations in `self`.
pub fn iter(&self) -> impl Iterator<Item = &Attestation<E>> {
self.map.iter().map(|(_key, attestation)| attestation)
fn len(&self) -> usize {
self.map.len()
}
pub fn len(&self) -> usize {
self.map.len()
fn start_insert_timer() -> Option<metrics::HistogramTimer> {
metrics::start_timer(&metrics::ATTESTATION_PROCESSING_AGG_POOL_INSERT)
}
fn start_create_map_timer() -> Option<metrics::HistogramTimer> {
metrics::start_timer(&metrics::ATTESTATION_PROCESSING_AGG_POOL_CREATE_MAP)
}
fn start_prune_timer() -> Option<metrics::HistogramTimer> {
metrics::start_timer(&metrics::ATTESTATION_PROCESSING_AGG_POOL_PRUNE)
}
/// Use the `TARGET_COMMITTEE_SIZE`.
///
/// Note: hard-coded until `TARGET_COMMITTEE_SIZE` is available via `EthSpec`.
fn default_capacity() -> usize {
128
}
}
/// A pool of `Attestation` that is specially designed to store "unaggregated" attestations from
/// the native aggregation scheme.
/// A collection of `SyncCommitteeContribution`, keyed by their `SyncContributionData`. Enforces that all
/// contributions are from the same slot.
pub struct SyncContributionAggregateMap<E: EthSpec> {
map: HashMap<SyncDataRoot, SyncCommitteeContribution<E>>,
}
impl<E: EthSpec> AggregateMap for SyncContributionAggregateMap<E> {
type Key = SyncDataRoot;
type Value = SyncCommitteeContribution<E>;
type Data = SyncContributionData;
/// Create an empty collection with the given `initial_capacity`.
fn new(initial_capacity: usize) -> Self {
Self {
map: HashMap::with_capacity(initial_capacity),
}
}
/// Insert a sync committee contribution into `self`, aggregating it into the pool.
///
/// The given sync contribution must only have one signature.
fn insert(
&mut self,
contribution: &SyncCommitteeContribution<E>,
) -> Result<InsertOutcome, Error> {
let _timer =
metrics::start_timer(&metrics::SYNC_CONTRIBUTION_PROCESSING_AGG_POOL_CORE_INSERT);
let set_bits = contribution
.aggregation_bits
.iter()
.enumerate()
.filter(|(_i, bit)| *bit)
.map(|(i, _bit)| i)
.collect::<Vec<_>>();
let committee_index = set_bits
.first()
.copied()
.ok_or(Error::NoAggregationBitsSet)?;
if set_bits.len() > 1 {
return Err(Error::MoreThanOneAggregationBitSet(set_bits.len()));
}
let sync_data_root = SyncContributionData::from_contribution(contribution).tree_hash_root();
if let Some(existing_contribution) = self.map.get_mut(&sync_data_root) {
if existing_contribution
.aggregation_bits
.get(committee_index)
.map_err(|_| Error::InconsistentBitfieldLengths)?
{
Ok(InsertOutcome::SignatureAlreadyKnown { committee_index })
} else {
let _timer = metrics::start_timer(
&metrics::SYNC_CONTRIBUTION_PROCESSING_AGG_POOL_AGGREGATION,
);
existing_contribution.aggregate(contribution);
Ok(InsertOutcome::SignatureAggregated { committee_index })
}
} else {
if self.map.len() >= E::sync_committee_size() {
return Err(Error::ReachedMaxItemsPerSlot(E::sync_committee_size()));
}
self.map.insert(sync_data_root, contribution.clone());
Ok(InsertOutcome::NewItemInserted { committee_index })
}
}
/// Returns an aggregated `SyncCommitteeContribution` with the given `data`, if any.
///
/// The given `data.slot` must match the slot that `self` was initialized with.
fn get(&self, data: &SyncContributionData) -> Option<SyncCommitteeContribution<E>> {
self.map.get(&data.tree_hash_root()).cloned()
}
fn get_map(&self) -> &HashMap<SyncDataRoot, SyncCommitteeContribution<E>> {
&self.map
}
/// Returns an aggregated `SyncCommitteeContribution` with the given `root`, if any.
fn get_by_root(&self, root: &SyncDataRoot) -> Option<&SyncCommitteeContribution<E>> {
self.map.get(root)
}
fn len(&self) -> usize {
self.map.len()
}
fn start_insert_timer() -> Option<metrics::HistogramTimer> {
metrics::start_timer(&metrics::SYNC_CONTRIBUTION_PROCESSING_AGG_POOL_INSERT)
}
fn start_create_map_timer() -> Option<metrics::HistogramTimer> {
metrics::start_timer(&metrics::SYNC_CONTRIBUTION_PROCESSING_AGG_POOL_CREATE_MAP)
}
fn start_prune_timer() -> Option<metrics::HistogramTimer> {
metrics::start_timer(&metrics::SYNC_CONTRIBUTION_PROCESSING_AGG_POOL_PRUNE)
}
/// Default to `SYNC_COMMITTEE_SUBNET_COUNT`.
fn default_capacity() -> usize {
SYNC_COMMITTEE_SUBNET_COUNT as usize
}
}
/// A pool of `Attestation` or `SyncCommitteeContribution` that is specially designed to store
/// "unaggregated" messages from the native aggregation scheme.
///
/// **The `NaiveAggregationPool` does not do any signature or attestation verification. It assumes
/// that all `Attestation` objects provided are valid.**
/// **The `NaiveAggregationPool` does not do any verification. It assumes that all `Attestation`
/// or `SyncCommitteeContribution` objects provided are valid.**
///
/// ## Details
///
/// The pool sorts the `Attestation` by `attestation.data.slot`, then by `attestation.data`.
/// The pool sorts the items by `slot`, then by `Data`.
///
/// As each unaggregated attestation is added it is aggregated with any existing `attestation` with
/// the same `AttestationData`. Considering that the pool only accepts attestations with a single
/// As each item is added it is aggregated with any existing item with the same `Data`. Considering
/// that the pool only accepts attestations or sync contributions with a single
/// signature, there should only ever be a single aggregated `Attestation` for any given
/// `AttestationData`.
/// `AttestationData` or a single `SyncCommitteeContribution` for any given `SyncContributionData`.
///
/// The pool has a capacity for `SLOTS_RETAINED` slots, when a new `attestation.data.slot` is
/// The pool has a capacity for `SLOTS_RETAINED` slots, when a new `slot` is
/// provided, the oldest slot is dropped and replaced with the new slot. The pool can also be
/// pruned by supplying a `current_slot`; all existing attestations with a slot lower than
/// `current_slot - SLOTS_RETAINED` will be removed and any future attestation with a slot lower
/// than that will also be refused. Pruning is done automatically based upon the attestations it
/// pruned by supplying a `current_slot`; all existing items with a slot lower than
/// `current_slot - SLOTS_RETAINED` will be removed and any future item with a slot lower
/// than that will also be refused. Pruning is done automatically based upon the items it
/// receives and it can be triggered manually.
pub struct NaiveAggregationPool<E: EthSpec> {
pub struct NaiveAggregationPool<T: AggregateMap> {
lowest_permissible_slot: Slot,
maps: HashMap<Slot, AggregatedAttestationMap<E>>,
maps: HashMap<Slot, T>,
}
impl<E: EthSpec> Default for NaiveAggregationPool<E> {
impl<T: AggregateMap> Default for NaiveAggregationPool<T> {
fn default() -> Self {
Self {
lowest_permissible_slot: Slot::new(0),
@ -173,20 +350,20 @@ impl<E: EthSpec> Default for NaiveAggregationPool<E> {
}
}
impl<E: EthSpec> NaiveAggregationPool<E> {
/// Insert an attestation into `self`, aggregating it into the pool.
impl<T: AggregateMap> NaiveAggregationPool<T> {
/// Insert an item into `self`, aggregating it into the pool.
///
/// The given attestation (`a`) must only have one signature and have an
/// `attestation.data.slot` that is not lower than `self.lowest_permissible_slot`.
/// The given item must only have one signature and have an
/// `slot` that is not lower than `self.lowest_permissible_slot`.
///
/// The pool may be pruned if the given `attestation.data` has a slot higher than any
/// The pool may be pruned if the given item has a slot higher than any
/// previously seen.
pub fn insert(&mut self, attestation: &Attestation<E>) -> Result<InsertOutcome, Error> {
let _timer = metrics::start_timer(&metrics::ATTESTATION_PROCESSING_AGG_POOL_INSERT);
let slot = attestation.data.slot;
pub fn insert(&mut self, item: &T::Value) -> Result<InsertOutcome, Error> {
let _timer = T::start_insert_timer();
let slot = item.get_slot();
let lowest_permissible_slot = self.lowest_permissible_slot;
// Reject any attestations that are too old.
// Reject any items that are too old.
if slot < lowest_permissible_slot {
return Err(Error::SlotTooLow {
slot,
@ -194,14 +371,10 @@ impl<E: EthSpec> NaiveAggregationPool<E> {
});
}
let lock_timer =
metrics::start_timer(&metrics::ATTESTATION_PROCESSING_AGG_POOL_MAPS_WRITE_LOCK);
drop(lock_timer);
let outcome = if let Some(map) = self.maps.get_mut(&slot) {
map.insert(attestation)
map.insert(item)
} else {
let _timer = metrics::start_timer(&metrics::ATTESTATION_PROCESSING_AGG_POOL_CREATE_MAP);
let _timer = T::start_create_map_timer();
// To avoid re-allocations, try and determine a rough initial capacity for the new item
// by obtaining the mean size of all items in earlier epoch.
let (count, sum) = self
@ -213,12 +386,11 @@ impl<E: EthSpec> NaiveAggregationPool<E> {
.map(|(_slot, map)| map.len())
.fold((0, 0), |(count, sum), len| (count + 1, sum + len));
// Use the mainnet default committee size if we can't determine an average.
let initial_capacity = sum.checked_div(count).unwrap_or(128);
let initial_capacity = sum.checked_div(count).unwrap_or_else(T::default_capacity);
let mut item = AggregatedAttestationMap::new(initial_capacity);
let outcome = item.insert(attestation);
self.maps.insert(slot, item);
let mut aggregate_map = T::new(initial_capacity);
let outcome = aggregate_map.insert(item);
self.maps.insert(slot, aggregate_map);
outcome
};
@ -228,39 +400,39 @@ impl<E: EthSpec> NaiveAggregationPool<E> {
outcome
}
/// Returns the total number of attestations stored in `self`.
pub fn num_attestations(&self) -> usize {
/// Returns the total number of items stored in `self`.
pub fn num_items(&self) -> usize {
self.maps.iter().map(|(_, map)| map.len()).sum()
}
/// Returns an aggregated `Attestation` with the given `data`, if any.
pub fn get(&self, data: &AttestationData) -> Option<Attestation<E>> {
self.maps.get(&data.slot).and_then(|map| map.get(data))
/// Returns an aggregated `T::Value` with the given `T::Data`, if any.
pub fn get(&self, data: &T::Data) -> Option<T::Value> {
self.maps
.get(&data.get_slot())
.and_then(|map| map.get(data))
}
/// Returns an aggregated `Attestation` with the given `data`, if any.
pub fn get_by_slot_and_root(
&self,
slot: Slot,
root: &AttestationDataRoot,
) -> Option<Attestation<E>> {
/// Returns an aggregated `T::Value` with the given `slot` and `root`, if any.
pub fn get_by_slot_and_root(&self, slot: Slot, root: &T::Key) -> Option<T::Value> {
self.maps
.get(&slot)
.and_then(|map| map.get_by_root(root).cloned())
}
/// Iterate all attestations in all slots of `self`.
pub fn iter(&self) -> impl Iterator<Item = &Attestation<E>> {
self.maps.iter().map(|(_slot, map)| map.iter()).flatten()
/// Iterate all items in all slots of `self`.
pub fn iter(&self) -> impl Iterator<Item = &T::Value> {
self.maps
.iter()
.map(|(_slot, map)| map.get_map().iter().map(|(_key, value)| value))
.flatten()
}
/// Removes any attestations with a slot lower than `current_slot` and bars any future
/// attestations with a slot lower than `current_slot - SLOTS_RETAINED`.
/// Removes any items with a slot lower than `current_slot` and bars any future
/// items with a slot lower than `current_slot - SLOTS_RETAINED`.
pub fn prune(&mut self, current_slot: Slot) {
let _timer = metrics::start_timer(&metrics::ATTESTATION_PROCESSING_AGG_POOL_PRUNE);
let _timer = T::start_prune_timer();
// Taking advantage of saturating subtraction on `Slot`.
let lowest_permissible_slot = current_slot - Slot::from(SLOTS_RETAINED);
let lowest_permissible_slot = current_slot.saturating_sub(Slot::from(SLOTS_RETAINED));
// No need to prune if the lowest permissible slot has not changed and the queue length is
// less than the maximum
@ -301,9 +473,10 @@ impl<E: EthSpec> NaiveAggregationPool<E> {
mod tests {
use super::*;
use ssz_types::BitList;
use store::BitVector;
use types::{
test_utils::{generate_deterministic_keypair, test_random_instance},
Fork, Hash256,
Fork, Hash256, SyncCommitteeMessage,
};
type E = types::MainnetEthSpec;
@ -315,7 +488,14 @@ mod tests {
a
}
fn sign(a: &mut Attestation<E>, i: usize, genesis_validators_root: Hash256) {
fn get_sync_contribution(slot: Slot) -> SyncCommitteeContribution<E> {
let mut a: SyncCommitteeContribution<E> = test_random_instance();
a.slot = slot;
a.aggregation_bits = BitVector::new();
a
}
fn sign_attestation(a: &mut Attestation<E>, i: usize, genesis_validators_root: Hash256) {
a.sign(
&generate_deterministic_keypair(i).sk,
i,
@ -326,187 +506,294 @@ mod tests {
.expect("should sign attestation");
}
fn unset_bit(a: &mut Attestation<E>, i: usize) {
fn sign_sync_contribution(
a: &mut SyncCommitteeContribution<E>,
i: usize,
genesis_validators_root: Hash256,
) {
let sync_message = SyncCommitteeMessage::new::<E>(
a.slot,
a.beacon_block_root,
i as u64,
&generate_deterministic_keypair(i).sk,
&Fork::default(),
genesis_validators_root,
&E::default_spec(),
);
let signed_contribution: SyncCommitteeContribution<E> =
SyncCommitteeContribution::from_message(&sync_message, a.subcommittee_index, i)
.unwrap();
a.aggregate(&signed_contribution);
}
fn unset_attestation_bit(a: &mut Attestation<E>, i: usize) {
a.aggregation_bits
.set(i, false)
.expect("should unset aggregation bit")
}
#[test]
fn single_attestation() {
let mut a = get_attestation(Slot::new(0));
let mut pool = NaiveAggregationPool::default();
assert_eq!(
pool.insert(&a),
Err(Error::NoAggregationBitsSet),
"should not accept attestation without any signatures"
);
sign(&mut a, 0, Hash256::random());
assert_eq!(
pool.insert(&a),
Ok(InsertOutcome::NewAttestationData { committee_index: 0 }),
"should accept new attestation"
);
assert_eq!(
pool.insert(&a),
Ok(InsertOutcome::SignatureAlreadyKnown { committee_index: 0 }),
"should acknowledge duplicate signature"
);
let retrieved = pool
.get(&a.data)
.expect("should not error while getting attestation");
assert_eq!(
retrieved, a,
"retrieved attestation should equal the one inserted"
);
sign(&mut a, 1, Hash256::random());
assert_eq!(
pool.insert(&a),
Err(Error::MoreThanOneAggregationBitSet(2)),
"should not accept attestation with multiple signatures"
);
fn unset_sync_contribution_bit(a: &mut SyncCommitteeContribution<E>, i: usize) {
a.aggregation_bits
.set(i, false)
.expect("should unset aggregation bit")
}
#[test]
fn multiple_attestations() {
let mut a_0 = get_attestation(Slot::new(0));
let mut a_1 = a_0.clone();
let genesis_validators_root = Hash256::random();
sign(&mut a_0, 0, genesis_validators_root);
sign(&mut a_1, 1, genesis_validators_root);
let mut pool = NaiveAggregationPool::default();
assert_eq!(
pool.insert(&a_0),
Ok(InsertOutcome::NewAttestationData { committee_index: 0 }),
"should accept a_0"
);
assert_eq!(
pool.insert(&a_1),
Ok(InsertOutcome::SignatureAggregated { committee_index: 1 }),
"should accept a_1"
);
let retrieved = pool
.get(&a_0.data)
.expect("should not error while getting attestation");
let mut a_01 = a_0.clone();
a_01.aggregate(&a_1);
assert_eq!(
retrieved, a_01,
"retrieved attestation should be aggregated"
);
/*
* Throw a different attestation data in there and ensure it isn't aggregated
*/
let mut a_different = a_0.clone();
let different_root = Hash256::from_low_u64_be(1337);
unset_bit(&mut a_different, 0);
sign(&mut a_different, 2, genesis_validators_root);
assert_ne!(a_different.data.beacon_block_root, different_root);
a_different.data.beacon_block_root = different_root;
assert_eq!(
pool.insert(&a_different),
Ok(InsertOutcome::NewAttestationData { committee_index: 2 }),
"should accept a_different"
);
assert_eq!(
pool.get(&a_0.data)
.expect("should not error while getting attestation"),
retrieved,
"should not have aggregated different attestation data"
);
fn mutate_attestation_block_root(a: &mut Attestation<E>, block_root: Hash256) {
a.data.beacon_block_root = block_root
}
#[test]
fn auto_pruning() {
let mut base = get_attestation(Slot::new(0));
sign(&mut base, 0, Hash256::random());
fn mutate_attestation_slot(a: &mut Attestation<E>, slot: Slot) {
a.data.slot = slot
}
let mut pool = NaiveAggregationPool::default();
fn attestation_block_root_comparator(a: &Attestation<E>, block_root: Hash256) -> bool {
a.data.beacon_block_root == block_root
}
for i in 0..SLOTS_RETAINED * 2 {
let slot = Slot::from(i);
let mut a = base.clone();
a.data.slot = slot;
fn key_from_attestation(a: &Attestation<E>) -> AttestationData {
a.data.clone()
}
assert_eq!(
pool.insert(&a),
Ok(InsertOutcome::NewAttestationData { committee_index: 0 }),
"should accept new attestation"
);
fn mutate_sync_contribution_block_root(
a: &mut SyncCommitteeContribution<E>,
block_root: Hash256,
) {
a.beacon_block_root = block_root
}
if i < SLOTS_RETAINED {
let len = i + 1;
assert_eq!(pool.maps.len(), len, "the pool should have length {}", len);
} else {
assert_eq!(
pool.maps.len(),
SLOTS_RETAINED,
"the pool should have length SLOTS_RETAINED"
);
fn mutate_sync_contribution_slot(a: &mut SyncCommitteeContribution<E>, slot: Slot) {
a.slot = slot
}
let mut pool_slots = pool
.maps
.iter()
.map(|(slot, _map)| *slot)
.collect::<Vec<_>>();
fn sync_contribution_block_root_comparator(
a: &SyncCommitteeContribution<E>,
block_root: Hash256,
) -> bool {
a.beacon_block_root == block_root
}
pool_slots.sort_unstable();
fn key_from_sync_contribution(a: &SyncCommitteeContribution<E>) -> SyncContributionData {
SyncContributionData::from_contribution(&a)
}
macro_rules! test_suite {
(
$mod_name: ident,
$get_method_name: ident,
$sign_method_name: ident,
$unset_method_name: ident,
$block_root_mutator: ident,
$slot_mutator: ident,
$block_root_comparator: ident,
$key_getter: ident,
$map_type: ident,
$item_limit: expr
) => {
#[cfg(test)]
mod $mod_name {
use super::*;
#[test]
fn single_item() {
let mut a = $get_method_name(Slot::new(0));
let mut pool: NaiveAggregationPool<$map_type<E>> =
NaiveAggregationPool::default();
for (j, pool_slot) in pool_slots.iter().enumerate() {
let expected_slot = slot - (SLOTS_RETAINED - 1 - j) as u64;
assert_eq!(
*pool_slot, expected_slot,
"the slot of the map should be {}",
expected_slot
)
pool.insert(&a),
Err(Error::NoAggregationBitsSet),
"should not accept item without any signatures"
);
$sign_method_name(&mut a, 0, Hash256::random());
assert_eq!(
pool.insert(&a),
Ok(InsertOutcome::NewItemInserted { committee_index: 0 }),
"should accept new item"
);
assert_eq!(
pool.insert(&a),
Ok(InsertOutcome::SignatureAlreadyKnown { committee_index: 0 }),
"should acknowledge duplicate signature"
);
let retrieved = pool
.get(&$key_getter(&a))
.expect("should not error while getting item");
assert_eq!(retrieved, a, "retrieved item should equal the one inserted");
$sign_method_name(&mut a, 1, Hash256::random());
assert_eq!(
pool.insert(&a),
Err(Error::MoreThanOneAggregationBitSet(2)),
"should not accept item with multiple signatures"
);
}
#[test]
fn multiple_items() {
let mut a_0 = $get_method_name(Slot::new(0));
let mut a_1 = a_0.clone();
let genesis_validators_root = Hash256::random();
$sign_method_name(&mut a_0, 0, genesis_validators_root);
$sign_method_name(&mut a_1, 1, genesis_validators_root);
let mut pool: NaiveAggregationPool<$map_type<E>> =
NaiveAggregationPool::default();
assert_eq!(
pool.insert(&a_0),
Ok(InsertOutcome::NewItemInserted { committee_index: 0 }),
"should accept a_0"
);
assert_eq!(
pool.insert(&a_1),
Ok(InsertOutcome::SignatureAggregated { committee_index: 1 }),
"should accept a_1"
);
let retrieved = pool
.get(&$key_getter(&a_0))
.expect("should not error while getting attestation");
let mut a_01 = a_0.clone();
a_01.aggregate(&a_1);
assert_eq!(retrieved, a_01, "retrieved item should be aggregated");
/*
* Throw different data in there and ensure it isn't aggregated
*/
let mut a_different = a_0.clone();
let different_root = Hash256::from_low_u64_be(1337);
$unset_method_name(&mut a_different, 0);
$sign_method_name(&mut a_different, 2, genesis_validators_root);
assert!(!$block_root_comparator(&a_different, different_root));
$block_root_mutator(&mut a_different, different_root);
assert_eq!(
pool.insert(&a_different),
Ok(InsertOutcome::NewItemInserted { committee_index: 2 }),
"should accept a_different"
);
assert_eq!(
pool.get(&$key_getter(&a_0))
.expect("should not error while getting item"),
retrieved,
"should not have aggregated different items with different data"
);
}
#[test]
fn auto_pruning_item() {
let mut base = $get_method_name(Slot::new(0));
$sign_method_name(&mut base, 0, Hash256::random());
let mut pool: NaiveAggregationPool<$map_type<E>> =
NaiveAggregationPool::default();
for i in 0..SLOTS_RETAINED * 2 {
let slot = Slot::from(i);
let mut a = base.clone();
$slot_mutator(&mut a, slot);
assert_eq!(
pool.insert(&a),
Ok(InsertOutcome::NewItemInserted { committee_index: 0 }),
"should accept new item"
);
if i < SLOTS_RETAINED {
let len = i + 1;
assert_eq!(pool.maps.len(), len, "the pool should have length {}", len);
} else {
assert_eq!(
pool.maps.len(),
SLOTS_RETAINED,
"the pool should have length SLOTS_RETAINED"
);
let mut pool_slots = pool
.maps
.iter()
.map(|(slot, _map)| *slot)
.collect::<Vec<_>>();
pool_slots.sort_unstable();
for (j, pool_slot) in pool_slots.iter().enumerate() {
let expected_slot = slot - (SLOTS_RETAINED - 1 - j) as u64;
assert_eq!(
*pool_slot, expected_slot,
"the slot of the map should be {}",
expected_slot
)
}
}
}
}
#[test]
fn max_items() {
let mut base = $get_method_name(Slot::new(0));
$sign_method_name(&mut base, 0, Hash256::random());
let mut pool: NaiveAggregationPool<$map_type<E>> =
NaiveAggregationPool::default();
for i in 0..=$item_limit {
let mut a = base.clone();
$block_root_mutator(&mut a, Hash256::from_low_u64_be(i as u64));
if i < $item_limit {
assert_eq!(
pool.insert(&a),
Ok(InsertOutcome::NewItemInserted { committee_index: 0 }),
"should accept item below limit"
);
} else {
assert_eq!(
pool.insert(&a),
Err(Error::ReachedMaxItemsPerSlot($item_limit)),
"should not accept item above limit"
);
}
}
}
}
}
};
}
#[test]
fn max_attestations() {
let mut base = get_attestation(Slot::new(0));
sign(&mut base, 0, Hash256::random());
test_suite! {
attestation_tests,
get_attestation,
sign_attestation,
unset_attestation_bit,
mutate_attestation_block_root,
mutate_attestation_slot,
attestation_block_root_comparator,
key_from_attestation,
AggregatedAttestationMap,
MAX_ATTESTATIONS_PER_SLOT
}
let mut pool = NaiveAggregationPool::default();
for i in 0..=MAX_ATTESTATIONS_PER_SLOT {
let mut a = base.clone();
a.data.beacon_block_root = Hash256::from_low_u64_be(i as u64);
if i < MAX_ATTESTATIONS_PER_SLOT {
assert_eq!(
pool.insert(&a),
Ok(InsertOutcome::NewAttestationData { committee_index: 0 }),
"should accept attestation below limit"
);
} else {
assert_eq!(
pool.insert(&a),
Err(Error::ReachedMaxAttestationsPerSlot(
MAX_ATTESTATIONS_PER_SLOT
)),
"should not accept attestation above limit"
);
}
}
test_suite! {
sync_contribution_tests,
get_sync_contribution,
sign_sync_contribution,
unset_sync_contribution_bit,
mutate_sync_contribution_block_root,
mutate_sync_contribution_slot,
sync_contribution_block_root_comparator,
key_from_sync_contribution,
SyncContributionAggregateMap,
E::sync_committee_size()
}
}

View File

@ -0,0 +1,507 @@
//! Provides an `ObservedAggregates` struct which allows us to reject aggregated attestations or
//! sync committee contributions if we've already seen them.
use std::collections::HashSet;
use std::marker::PhantomData;
use tree_hash::TreeHash;
use types::consts::altair::{
SYNC_COMMITTEE_SUBNET_COUNT, TARGET_AGGREGATORS_PER_SYNC_SUBCOMMITTEE,
};
use types::slot_data::SlotData;
use types::{Attestation, EthSpec, Hash256, Slot, SyncCommitteeContribution};
pub type ObservedSyncContributions<E> = ObservedAggregates<SyncCommitteeContribution<E>, E>;
pub type ObservedAggregateAttestations<E> = ObservedAggregates<Attestation<E>, E>;
/// A trait use to associate capacity constants with the type being stored in `ObservedAggregates`.
pub trait Consts {
/// The default capacity of items stored per slot, in a single `SlotHashSet`.
const DEFAULT_PER_SLOT_CAPACITY: usize;
/// The maximum number of slots
fn max_slot_capacity() -> usize;
/// The maximum number of items stored per slot, in a single `SlotHashSet`.
fn max_per_slot_capacity() -> usize;
}
impl<T: EthSpec> Consts for Attestation<T> {
/// Use 128 as it's the target committee size for the mainnet spec. This is perhaps a little
/// wasteful for the minimal spec, but considering it's approx. 128 * 32 bytes we're not wasting
/// much.
const DEFAULT_PER_SLOT_CAPACITY: usize = 128;
/// We need to keep attestations for each slot of the current epoch.
fn max_slot_capacity() -> usize {
T::slots_per_epoch() as usize
}
/// As a DoS protection measure, the maximum number of distinct `Attestations` or
/// `SyncCommitteeContributions` that will be recorded for each slot.
///
/// Currently this is set to ~524k. If we say that each entry is 40 bytes (Hash256 (32 bytes) + an
/// 8 byte hash) then this comes to about 20mb per slot. If we're storing 34 of these slots, then
/// we're at 680mb. This is a lot of memory usage, but probably not a show-stopper for most
/// reasonable hardware.
///
/// Upstream conditions should strongly restrict the amount of attestations that can show up in
/// this pool. The maximum size with respect to upstream restrictions is more likely on the order
/// of the number of validators.
fn max_per_slot_capacity() -> usize {
1 << 19 // 524,288
}
}
impl<T: EthSpec> Consts for SyncCommitteeContribution<T> {
/// Set to `TARGET_AGGREGATORS_PER_SYNC_SUBCOMMITTEE * SYNC_COMMITTEE_SUBNET_COUNT`. This is the
/// expected number of aggregators per slot across all subcommittees.
const DEFAULT_PER_SLOT_CAPACITY: usize =
(SYNC_COMMITTEE_SUBNET_COUNT * TARGET_AGGREGATORS_PER_SYNC_SUBCOMMITTEE) as usize;
/// We only need to keep contributions related to the current slot.
fn max_slot_capacity() -> usize {
1
}
/// We should never receive more aggregates than there are sync committee participants.
fn max_per_slot_capacity() -> usize {
T::sync_committee_size()
}
}
#[derive(Debug, PartialEq)]
pub enum ObserveOutcome {
/// This item was already known.
AlreadyKnown,
/// This was the first time this item was observed.
New,
}
#[derive(Debug, PartialEq)]
pub enum Error {
SlotTooLow {
slot: Slot,
lowest_permissible_slot: Slot,
},
/// The function to obtain a set index failed, this is an internal error.
InvalidSetIndex(usize),
/// We have reached the maximum number of unique items that can be observed in a slot.
/// This is a DoS protection function.
ReachedMaxObservationsPerSlot(usize),
IncorrectSlot {
expected: Slot,
attestation: Slot,
},
}
/// A `HashSet` that contains entries related to some `Slot`.
struct SlotHashSet {
set: HashSet<Hash256>,
slot: Slot,
max_capacity: usize,
}
impl SlotHashSet {
pub fn new(slot: Slot, initial_capacity: usize, max_capacity: usize) -> Self {
Self {
slot,
set: HashSet::with_capacity(initial_capacity),
max_capacity,
}
}
/// Store the items in self so future observations recognise its existence.
pub fn observe_item<T: SlotData>(
&mut self,
item: &T,
root: Hash256,
) -> Result<ObserveOutcome, Error> {
if item.get_slot() != self.slot {
return Err(Error::IncorrectSlot {
expected: self.slot,
attestation: item.get_slot(),
});
}
if self.set.contains(&root) {
Ok(ObserveOutcome::AlreadyKnown)
} else {
// Here we check to see if this slot has reached the maximum observation count.
//
// The resulting behaviour is that we are no longer able to successfully observe new
// items, however we will continue to return `is_known` values. We could also
// disable `is_known`, however then we would stop forwarding items across the
// gossip network and I think that this is a worse case than sending some invalid ones.
// The underlying libp2p network is responsible for removing duplicate messages, so
// this doesn't risk a broadcast loop.
if self.set.len() >= self.max_capacity {
return Err(Error::ReachedMaxObservationsPerSlot(self.max_capacity));
}
self.set.insert(root);
Ok(ObserveOutcome::New)
}
}
/// Indicates if `item` has been observed before.
pub fn is_known<T: SlotData>(&self, item: &T, root: Hash256) -> Result<bool, Error> {
if item.get_slot() != self.slot {
return Err(Error::IncorrectSlot {
expected: self.slot,
attestation: item.get_slot(),
});
}
Ok(self.set.contains(&root))
}
/// The number of observed items in `self`.
pub fn len(&self) -> usize {
self.set.len()
}
}
/// Stores the roots of objects for some number of `Slots`, so we can determine if
/// these have previously been seen on the network.
pub struct ObservedAggregates<T: TreeHash + SlotData + Consts, E: EthSpec> {
lowest_permissible_slot: Slot,
sets: Vec<SlotHashSet>,
_phantom_spec: PhantomData<E>,
_phantom_tree_hash: PhantomData<T>,
}
impl<T: TreeHash + SlotData + Consts, E: EthSpec> Default for ObservedAggregates<T, E> {
fn default() -> Self {
Self {
lowest_permissible_slot: Slot::new(0),
sets: vec![],
_phantom_spec: PhantomData,
_phantom_tree_hash: PhantomData,
}
}
}
impl<T: TreeHash + SlotData + Consts, E: EthSpec> ObservedAggregates<T, E> {
/// Store the root of `item` in `self`.
///
/// `root` must equal `item.tree_hash_root()`.
pub fn observe_item(
&mut self,
item: &T,
root_opt: Option<Hash256>,
) -> Result<ObserveOutcome, Error> {
let index = self.get_set_index(item.get_slot())?;
let root = root_opt.unwrap_or_else(|| item.tree_hash_root());
self.sets
.get_mut(index)
.ok_or(Error::InvalidSetIndex(index))
.and_then(|set| set.observe_item(item, root))
}
/// Check to see if the `root` of `item` is in self.
///
/// `root` must equal `a.tree_hash_root()`.
pub fn is_known(&mut self, item: &T, root: Hash256) -> Result<bool, Error> {
let index = self.get_set_index(item.get_slot())?;
self.sets
.get(index)
.ok_or(Error::InvalidSetIndex(index))
.and_then(|set| set.is_known(item, root))
}
/// The maximum number of slots that items are stored for.
fn max_capacity(&self) -> u64 {
// We add `2` in order to account for one slot either side of the range due to
// `MAXIMUM_GOSSIP_CLOCK_DISPARITY`.
(T::max_slot_capacity() + 2) as u64
}
/// Removes any items with a slot lower than `current_slot` and bars any future
/// item with a slot lower than `current_slot - SLOTS_RETAINED`.
pub fn prune(&mut self, current_slot: Slot) {
let lowest_permissible_slot = current_slot.saturating_sub(self.max_capacity() - 1);
self.sets.retain(|set| set.slot >= lowest_permissible_slot);
self.lowest_permissible_slot = lowest_permissible_slot;
}
/// Returns the index of `self.set` that matches `slot`.
///
/// If there is no existing set for this slot one will be created. If `self.sets.len() >=
/// Self::max_capacity()`, the set with the lowest slot will be replaced.
fn get_set_index(&mut self, slot: Slot) -> Result<usize, Error> {
let lowest_permissible_slot = self.lowest_permissible_slot;
if slot < lowest_permissible_slot {
return Err(Error::SlotTooLow {
slot,
lowest_permissible_slot,
});
}
// Prune the pool if this item indicates that the current slot has advanced.
if lowest_permissible_slot + self.max_capacity() < slot + 1 {
self.prune(slot)
}
if let Some(index) = self.sets.iter().position(|set| set.slot == slot) {
return Ok(index);
}
// To avoid re-allocations, try and determine a rough initial capacity for the new set
// by obtaining the mean size of all items in earlier epoch.
let (count, sum) = self
.sets
.iter()
// Only include slots that are less than the given slot in the average. This should
// generally avoid including recent slots that are still "filling up".
.filter(|set| set.slot < slot)
.map(|set| set.len())
.fold((0, 0), |(count, sum), len| (count + 1, sum + len));
// If we are unable to determine an average, just use the `self.default_per_slot_capacity`.
let initial_capacity = sum
.checked_div(count)
.unwrap_or(T::DEFAULT_PER_SLOT_CAPACITY);
if self.sets.len() < self.max_capacity() as usize || self.sets.is_empty() {
let index = self.sets.len();
self.sets.push(SlotHashSet::new(
slot,
initial_capacity,
T::max_per_slot_capacity(),
));
return Ok(index);
}
let index = self
.sets
.iter()
.enumerate()
.min_by_key(|(_i, set)| set.slot)
.map(|(i, _set)| i)
.expect("sets cannot be empty due to previous .is_empty() check");
self.sets[index] = SlotHashSet::new(slot, initial_capacity, T::max_per_slot_capacity());
Ok(index)
}
}
#[cfg(test)]
#[cfg(not(debug_assertions))]
mod tests {
use super::*;
use tree_hash::TreeHash;
use types::{test_utils::test_random_instance, Hash256};
type E = types::MainnetEthSpec;
fn get_attestation(slot: Slot, beacon_block_root: u64) -> Attestation<E> {
let mut a: Attestation<E> = test_random_instance();
a.data.slot = slot;
a.data.beacon_block_root = Hash256::from_low_u64_be(beacon_block_root);
a
}
fn get_sync_contribution(slot: Slot, beacon_block_root: u64) -> SyncCommitteeContribution<E> {
let mut a: SyncCommitteeContribution<E> = test_random_instance();
a.slot = slot;
a.beacon_block_root = Hash256::from_low_u64_be(beacon_block_root);
a
}
macro_rules! test_suite {
($mod_name: ident, $type: ident, $method_name: ident) => {
#[cfg(test)]
mod $mod_name {
use super::*;
const NUM_ELEMENTS: usize = 8;
fn single_slot_test(store: &mut $type<E>, slot: Slot) {
let items = (0..NUM_ELEMENTS as u64)
.map(|i| $method_name(slot, i))
.collect::<Vec<_>>();
for a in &items {
assert_eq!(
store.is_known(a, a.tree_hash_root()),
Ok(false),
"should indicate an unknown attestation is unknown"
);
assert_eq!(
store.observe_item(a, None),
Ok(ObserveOutcome::New),
"should observe new attestation"
);
}
for a in &items {
assert_eq!(
store.is_known(a, a.tree_hash_root()),
Ok(true),
"should indicate a known attestation is known"
);
assert_eq!(
store.observe_item(a, Some(a.tree_hash_root())),
Ok(ObserveOutcome::AlreadyKnown),
"should acknowledge an existing attestation"
);
}
}
#[test]
fn single_slot() {
let mut store = $type::default();
single_slot_test(&mut store, Slot::new(0));
assert_eq!(store.sets.len(), 1, "should have a single set stored");
assert_eq!(
store.sets[0].len(),
NUM_ELEMENTS,
"set should have NUM_ELEMENTS elements"
);
}
#[test]
fn mulitple_contiguous_slots() {
let mut store = $type::default();
let max_cap = store.max_capacity();
for i in 0..max_cap * 3 {
let slot = Slot::new(i);
single_slot_test(&mut store, slot);
/*
* Ensure that the number of sets is correct.
*/
if i < max_cap {
assert_eq!(
store.sets.len(),
i as usize + 1,
"should have a {} sets stored",
i + 1
);
} else {
assert_eq!(
store.sets.len(),
max_cap as usize,
"should have max_capacity sets stored"
);
}
/*
* Ensure that each set contains the correct number of elements.
*/
for set in &store.sets[..] {
assert_eq!(
set.len(),
NUM_ELEMENTS,
"each store should have NUM_ELEMENTS elements"
)
}
/*
* Ensure that all the sets have the expected slots
*/
let mut store_slots =
store.sets.iter().map(|set| set.slot).collect::<Vec<_>>();
assert!(
store_slots.len() <= store.max_capacity() as usize,
"store size should not exceed max"
);
store_slots.sort_unstable();
let expected_slots = (i.saturating_sub(max_cap - 1)..=i)
.map(Slot::new)
.collect::<Vec<_>>();
assert_eq!(expected_slots, store_slots, "should have expected slots");
}
}
#[test]
fn mulitple_non_contiguous_slots() {
let mut store = $type::default();
let max_cap = store.max_capacity();
let to_skip = vec![1_u64, 2, 3, 5, 6, 29, 30, 31, 32, 64];
let slots = (0..max_cap * 3)
.into_iter()
.filter(|i| !to_skip.contains(i))
.collect::<Vec<_>>();
for &i in &slots {
if to_skip.contains(&i) {
continue;
}
let slot = Slot::from(i);
single_slot_test(&mut store, slot);
/*
* Ensure that each set contains the correct number of elements.
*/
for set in &store.sets[..] {
assert_eq!(
set.len(),
NUM_ELEMENTS,
"each store should have NUM_ELEMENTS elements"
)
}
/*
* Ensure that all the sets have the expected slots
*/
let mut store_slots =
store.sets.iter().map(|set| set.slot).collect::<Vec<_>>();
store_slots.sort_unstable();
assert!(
store_slots.len() <= store.max_capacity() as usize,
"store size should not exceed max"
);
let lowest = store.lowest_permissible_slot.as_u64();
let highest = slot.as_u64();
let expected_slots = (lowest..=highest)
.filter(|i| !to_skip.contains(i))
.map(Slot::new)
.collect::<Vec<_>>();
assert_eq!(
expected_slots,
&store_slots[..],
"should have expected slots"
);
}
}
}
};
}
test_suite!(
observed_sync_aggregates,
ObservedSyncContributions,
get_sync_contribution
);
test_suite!(
observed_aggregate_attestations,
ObservedAggregateAttestations,
get_attestation
);
}

View File

@ -1,424 +0,0 @@
//! Provides an `ObservedAttestations` struct which allows us to reject aggregated attestations if
//! we've already seen the aggregated attestation.
use std::collections::HashSet;
use std::marker::PhantomData;
use tree_hash::TreeHash;
use types::{Attestation, EthSpec, Hash256, Slot};
/// As a DoS protection measure, the maximum number of distinct `Attestations` that will be
/// recorded for each slot.
///
/// Currently this is set to ~524k. If we say that each entry is 40 bytes (Hash256 (32 bytes) + an
/// 8 byte hash) then this comes to about 20mb per slot. If we're storing 34 of these slots, then
/// we're at 680mb. This is a lot of memory usage, but probably not a show-stopper for most
/// reasonable hardware.
///
/// Upstream conditions should strongly restrict the amount of attestations that can show up in
/// this pool. The maximum size with respect to upstream restrictions is more likely on the order
/// of the number of validators.
const MAX_OBSERVATIONS_PER_SLOT: usize = 1 << 19; // 524,288
#[derive(Debug, PartialEq)]
pub enum ObserveOutcome {
/// This attestation was already known.
AlreadyKnown,
/// This was the first time this attestation was observed.
New,
}
#[derive(Debug, PartialEq)]
pub enum Error {
SlotTooLow {
slot: Slot,
lowest_permissible_slot: Slot,
},
/// The function to obtain a set index failed, this is an internal error.
InvalidSetIndex(usize),
/// We have reached the maximum number of unique `Attestation` that can be observed in a slot.
/// This is a DoS protection function.
ReachedMaxObservationsPerSlot(usize),
IncorrectSlot {
expected: Slot,
attestation: Slot,
},
}
/// A `HashSet` that contains entries related to some `Slot`.
struct SlotHashSet {
set: HashSet<Hash256>,
slot: Slot,
}
impl SlotHashSet {
pub fn new(slot: Slot, initial_capacity: usize) -> Self {
Self {
slot,
set: HashSet::with_capacity(initial_capacity),
}
}
/// Store the attestation in self so future observations recognise its existence.
pub fn observe_attestation<E: EthSpec>(
&mut self,
a: &Attestation<E>,
root: Hash256,
) -> Result<ObserveOutcome, Error> {
if a.data.slot != self.slot {
return Err(Error::IncorrectSlot {
expected: self.slot,
attestation: a.data.slot,
});
}
if self.set.contains(&root) {
Ok(ObserveOutcome::AlreadyKnown)
} else {
// Here we check to see if this slot has reached the maximum observation count.
//
// The resulting behaviour is that we are no longer able to successfully observe new
// attestations, however we will continue to return `is_known` values. We could also
// disable `is_known`, however then we would stop forwarding attestations across the
// gossip network and I think that this is a worse case than sending some invalid ones.
// The underlying libp2p network is responsible for removing duplicate messages, so
// this doesn't risk a broadcast loop.
if self.set.len() >= MAX_OBSERVATIONS_PER_SLOT {
return Err(Error::ReachedMaxObservationsPerSlot(
MAX_OBSERVATIONS_PER_SLOT,
));
}
self.set.insert(root);
Ok(ObserveOutcome::New)
}
}
/// Indicates if `a` has been observed before.
pub fn is_known<E: EthSpec>(&self, a: &Attestation<E>, root: Hash256) -> Result<bool, Error> {
if a.data.slot != self.slot {
return Err(Error::IncorrectSlot {
expected: self.slot,
attestation: a.data.slot,
});
}
Ok(self.set.contains(&root))
}
/// The number of observed attestations in `self`.
pub fn len(&self) -> usize {
self.set.len()
}
}
/// Stores the roots of `Attestation` objects for some number of `Slots`, so we can determine if
/// these have previously been seen on the network.
pub struct ObservedAttestations<E: EthSpec> {
lowest_permissible_slot: Slot,
sets: Vec<SlotHashSet>,
_phantom: PhantomData<E>,
}
impl<E: EthSpec> Default for ObservedAttestations<E> {
fn default() -> Self {
Self {
lowest_permissible_slot: Slot::new(0),
sets: vec![],
_phantom: PhantomData,
}
}
}
impl<E: EthSpec> ObservedAttestations<E> {
/// Store the root of `a` in `self`.
///
/// `root` must equal `a.tree_hash_root()`.
pub fn observe_attestation(
&mut self,
a: &Attestation<E>,
root_opt: Option<Hash256>,
) -> Result<ObserveOutcome, Error> {
let index = self.get_set_index(a.data.slot)?;
let root = root_opt.unwrap_or_else(|| a.tree_hash_root());
self.sets
.get_mut(index)
.ok_or(Error::InvalidSetIndex(index))
.and_then(|set| set.observe_attestation(a, root))
}
/// Check to see if the `root` of `a` is in self.
///
/// `root` must equal `a.tree_hash_root()`.
pub fn is_known(&mut self, a: &Attestation<E>, root: Hash256) -> Result<bool, Error> {
let index = self.get_set_index(a.data.slot)?;
self.sets
.get(index)
.ok_or(Error::InvalidSetIndex(index))
.and_then(|set| set.is_known(a, root))
}
/// The maximum number of slots that attestations are stored for.
fn max_capacity(&self) -> u64 {
// We add `2` in order to account for one slot either side of the range due to
// `MAXIMUM_GOSSIP_CLOCK_DISPARITY`.
E::slots_per_epoch() + 2
}
/// Removes any attestations with a slot lower than `current_slot` and bars any future
/// attestations with a slot lower than `current_slot - SLOTS_RETAINED`.
pub fn prune(&mut self, current_slot: Slot) {
// Taking advantage of saturating subtraction on `Slot`.
let lowest_permissible_slot = current_slot - (self.max_capacity() - 1);
self.sets.retain(|set| set.slot >= lowest_permissible_slot);
self.lowest_permissible_slot = lowest_permissible_slot;
}
/// Returns the index of `self.set` that matches `slot`.
///
/// If there is no existing set for this slot one will be created. If `self.sets.len() >=
/// Self::max_capacity()`, the set with the lowest slot will be replaced.
fn get_set_index(&mut self, slot: Slot) -> Result<usize, Error> {
let lowest_permissible_slot = self.lowest_permissible_slot;
if slot < lowest_permissible_slot {
return Err(Error::SlotTooLow {
slot,
lowest_permissible_slot,
});
}
// Prune the pool if this attestation indicates that the current slot has advanced.
if lowest_permissible_slot + self.max_capacity() < slot + 1 {
self.prune(slot)
}
if let Some(index) = self.sets.iter().position(|set| set.slot == slot) {
return Ok(index);
}
// To avoid re-allocations, try and determine a rough initial capacity for the new set
// by obtaining the mean size of all items in earlier epoch.
let (count, sum) = self
.sets
.iter()
// Only include slots that are less than the given slot in the average. This should
// generally avoid including recent slots that are still "filling up".
.filter(|set| set.slot < slot)
.map(|set| set.len())
.fold((0, 0), |(count, sum), len| (count + 1, sum + len));
// If we are unable to determine an average, just use 128 as it's the target committee
// size for the mainnet spec. This is perhaps a little wasteful for the minimal spec,
// but considering it's approx. 128 * 32 bytes we're not wasting much.
let initial_capacity = sum.checked_div(count).unwrap_or(128);
if self.sets.len() < self.max_capacity() as usize || self.sets.is_empty() {
let index = self.sets.len();
self.sets.push(SlotHashSet::new(slot, initial_capacity));
return Ok(index);
}
let index = self
.sets
.iter()
.enumerate()
.min_by_key(|(_i, set)| set.slot)
.map(|(i, _set)| i)
.expect("sets cannot be empty due to previous .is_empty() check");
self.sets[index] = SlotHashSet::new(slot, initial_capacity);
Ok(index)
}
}
#[cfg(test)]
#[cfg(not(debug_assertions))]
mod tests {
use super::*;
use tree_hash::TreeHash;
use types::{test_utils::test_random_instance, Hash256};
type E = types::MainnetEthSpec;
const NUM_ELEMENTS: usize = 8;
fn get_attestation(slot: Slot, beacon_block_root: u64) -> Attestation<E> {
let mut a: Attestation<E> = test_random_instance();
a.data.slot = slot;
a.data.beacon_block_root = Hash256::from_low_u64_be(beacon_block_root);
a
}
fn single_slot_test(store: &mut ObservedAttestations<E>, slot: Slot) {
let attestations = (0..NUM_ELEMENTS as u64)
.map(|i| get_attestation(slot, i))
.collect::<Vec<_>>();
for a in &attestations {
assert_eq!(
store.is_known(a, a.tree_hash_root()),
Ok(false),
"should indicate an unknown attestation is unknown"
);
assert_eq!(
store.observe_attestation(a, None),
Ok(ObserveOutcome::New),
"should observe new attestation"
);
}
for a in &attestations {
assert_eq!(
store.is_known(a, a.tree_hash_root()),
Ok(true),
"should indicate a known attestation is known"
);
assert_eq!(
store.observe_attestation(a, Some(a.tree_hash_root())),
Ok(ObserveOutcome::AlreadyKnown),
"should acknowledge an existing attestation"
);
}
}
#[test]
fn single_slot() {
let mut store = ObservedAttestations::default();
single_slot_test(&mut store, Slot::new(0));
assert_eq!(store.sets.len(), 1, "should have a single set stored");
assert_eq!(
store.sets[0].len(),
NUM_ELEMENTS,
"set should have NUM_ELEMENTS elements"
);
}
#[test]
fn mulitple_contiguous_slots() {
let mut store = ObservedAttestations::default();
let max_cap = store.max_capacity();
for i in 0..max_cap * 3 {
let slot = Slot::new(i);
single_slot_test(&mut store, slot);
/*
* Ensure that the number of sets is correct.
*/
if i < max_cap {
assert_eq!(
store.sets.len(),
i as usize + 1,
"should have a {} sets stored",
i + 1
);
} else {
assert_eq!(
store.sets.len(),
max_cap as usize,
"should have max_capacity sets stored"
);
}
/*
* Ensure that each set contains the correct number of elements.
*/
for set in &store.sets[..] {
assert_eq!(
set.len(),
NUM_ELEMENTS,
"each store should have NUM_ELEMENTS elements"
)
}
/*
* Ensure that all the sets have the expected slots
*/
let mut store_slots = store.sets.iter().map(|set| set.slot).collect::<Vec<_>>();
assert!(
store_slots.len() <= store.max_capacity() as usize,
"store size should not exceed max"
);
store_slots.sort_unstable();
let expected_slots = (i.saturating_sub(max_cap - 1)..=i)
.map(Slot::new)
.collect::<Vec<_>>();
assert_eq!(expected_slots, store_slots, "should have expected slots");
}
}
#[test]
fn mulitple_non_contiguous_slots() {
let mut store = ObservedAttestations::default();
let max_cap = store.max_capacity();
let to_skip = vec![1_u64, 2, 3, 5, 6, 29, 30, 31, 32, 64];
let slots = (0..max_cap * 3)
.into_iter()
.filter(|i| !to_skip.contains(i))
.collect::<Vec<_>>();
for &i in &slots {
if to_skip.contains(&i) {
continue;
}
let slot = Slot::from(i);
single_slot_test(&mut store, slot);
/*
* Ensure that each set contains the correct number of elements.
*/
for set in &store.sets[..] {
assert_eq!(
set.len(),
NUM_ELEMENTS,
"each store should have NUM_ELEMENTS elements"
)
}
/*
* Ensure that all the sets have the expected slots
*/
let mut store_slots = store.sets.iter().map(|set| set.slot).collect::<Vec<_>>();
store_slots.sort_unstable();
assert!(
store_slots.len() <= store.max_capacity() as usize,
"store size should not exceed max"
);
let lowest = store.lowest_permissible_slot.as_u64();
let highest = slot.as_u64();
let expected_slots = (lowest..=highest)
.filter(|i| !to_skip.contains(i))
.map(Slot::new)
.collect::<Vec<_>>();
assert_eq!(
expected_slots,
&store_slots[..],
"should have expected slots"
);
}
}
}

View File

@ -5,14 +5,29 @@
//! the same epoch.
//! - `ObservedAggregators`: allows filtering aggregated attestations from the same aggregators in
//! the same epoch
//!
//! Provides an additional two structs that help us filter out sync committee message and
//! contribution gossip from validators that have already published messages this slot:
//!
//! - `ObservedSyncContributors`: allows filtering sync committee messages from the same validator in
//! the same slot.
//! - `ObservedSyncAggregators`: allows filtering sync committee contributions from the same aggregators in
//! the same slot and in the same subcommittee.
use crate::types::consts::altair::TARGET_AGGREGATORS_PER_SYNC_SUBCOMMITTEE;
use bitvec::vec::BitVec;
use std::collections::{HashMap, HashSet};
use std::hash::Hash;
use std::marker::PhantomData;
use types::{Attestation, Epoch, EthSpec, Unsigned};
use types::slot_data::SlotData;
use types::{Epoch, EthSpec, Slot, Unsigned};
pub type ObservedAttesters<E> = AutoPruningContainer<EpochBitfield, E>;
pub type ObservedAggregators<E> = AutoPruningContainer<EpochHashSet, E>;
pub type ObservedAttesters<E> = AutoPruningEpochContainer<EpochBitfield, E>;
pub type ObservedSyncContributors<E> =
AutoPruningSlotContainer<SlotSubcommitteeIndex, SyncContributorSlotHashSet<E>, E>;
pub type ObservedAggregators<E> = AutoPruningEpochContainer<EpochHashSet, E>;
pub type ObservedSyncAggregators<E> =
AutoPruningSlotContainer<SlotSubcommitteeIndex, SyncAggregatorSlotHashSet, E>;
#[derive(Debug, PartialEq)]
pub enum Error {
@ -20,7 +35,11 @@ pub enum Error {
epoch: Epoch,
lowest_permissible_epoch: Epoch,
},
/// We have reached the maximum number of unique `Attestation` that can be observed in a slot.
SlotTooLow {
slot: Slot,
lowest_permissible_slot: Slot,
},
/// We have reached the maximum number of unique items that can be observed in a slot.
/// This is a DoS protection function.
ReachedMaxObservationsPerSlot(usize),
/// The function to obtain a set index failed, this is an internal error.
@ -48,7 +67,8 @@ pub trait Item {
fn contains(&self, validator_index: usize) -> bool;
}
/// Stores a `BitVec` that represents which validator indices have attested during an epoch.
/// Stores a `BitVec` that represents which validator indices have attested or sent sync committee
/// signatures during an epoch.
pub struct EpochBitfield {
bitfield: BitVec,
}
@ -99,7 +119,7 @@ impl Item for EpochBitfield {
}
}
/// Stores a `HashSet` of which validator indices have created an aggregate attestation during an
/// Stores a `HashSet` of which validator indices have created an aggregate during an
/// epoch.
pub struct EpochHashSet {
set: HashSet<usize>,
@ -138,6 +158,84 @@ impl Item for EpochHashSet {
}
}
/// Stores a `HashSet` of which validator indices have created a sync aggregate during a
/// slot.
pub struct SyncContributorSlotHashSet<E> {
set: HashSet<usize>,
phantom: PhantomData<E>,
}
impl<E: EthSpec> Item for SyncContributorSlotHashSet<E> {
fn with_capacity(capacity: usize) -> Self {
Self {
set: HashSet::with_capacity(capacity),
phantom: PhantomData,
}
}
/// Defaults to the `SYNC_SUBCOMMITTEE_SIZE`.
fn default_capacity() -> usize {
E::sync_subcommittee_size()
}
fn len(&self) -> usize {
self.set.len()
}
fn validator_count(&self) -> usize {
self.set.len()
}
/// Inserts the `validator_index` in the set. Returns `true` if the `validator_index` was
/// already in the set.
fn insert(&mut self, validator_index: usize) -> bool {
!self.set.insert(validator_index)
}
/// Returns `true` if the `validator_index` is in the set.
fn contains(&self, validator_index: usize) -> bool {
self.set.contains(&validator_index)
}
}
/// Stores a `HashSet` of which validator indices have created a sync aggregate during a
/// slot.
pub struct SyncAggregatorSlotHashSet {
set: HashSet<usize>,
}
impl Item for SyncAggregatorSlotHashSet {
fn with_capacity(capacity: usize) -> Self {
Self {
set: HashSet::with_capacity(capacity),
}
}
/// Defaults to the `TARGET_AGGREGATORS_PER_SYNC_SUBCOMMITTEE`.
fn default_capacity() -> usize {
TARGET_AGGREGATORS_PER_SYNC_SUBCOMMITTEE as usize
}
fn len(&self) -> usize {
self.set.len()
}
fn validator_count(&self) -> usize {
self.set.len()
}
/// Inserts the `validator_index` in the set. Returns `true` if the `validator_index` was
/// already in the set.
fn insert(&mut self, validator_index: usize) -> bool {
!self.set.insert(validator_index)
}
/// Returns `true` if the `validator_index` is in the set.
fn contains(&self, validator_index: usize) -> bool {
self.set.contains(&validator_index)
}
}
/// A container that stores some number of `T` items.
///
/// This container is "auto-pruning" since it gets an idea of the current slot by which
@ -146,13 +244,13 @@ impl Item for EpochHashSet {
/// attestations with an epoch prior to `a.data.target.epoch - 32` will be cleared from the cache.
///
/// `T` should be set to a `EpochBitfield` or `EpochHashSet`.
pub struct AutoPruningContainer<T, E: EthSpec> {
pub struct AutoPruningEpochContainer<T, E: EthSpec> {
lowest_permissible_epoch: Epoch,
items: HashMap<Epoch, T>,
_phantom: PhantomData<E>,
}
impl<T, E: EthSpec> Default for AutoPruningContainer<T, E> {
impl<T, E: EthSpec> Default for AutoPruningEpochContainer<T, E> {
fn default() -> Self {
Self {
lowest_permissible_epoch: Epoch::new(0),
@ -162,22 +260,20 @@ impl<T, E: EthSpec> Default for AutoPruningContainer<T, E> {
}
}
impl<T: Item, E: EthSpec> AutoPruningContainer<T, E> {
impl<T: Item, E: EthSpec> AutoPruningEpochContainer<T, E> {
/// Observe that `validator_index` has produced attestation `a`. Returns `Ok(true)` if `a` has
/// previously been observed for `validator_index`.
///
/// ## Errors
///
/// - `validator_index` is higher than `VALIDATOR_REGISTRY_LIMIT`.
/// - `a.data.target.slot` is earlier than `self.earliest_permissible_slot`.
/// - `a.data.target.slot` is earlier than `self.lowest_permissible_slot`.
pub fn observe_validator(
&mut self,
a: &Attestation<E>,
epoch: Epoch,
validator_index: usize,
) -> Result<bool, Error> {
self.sanitize_request(a, validator_index)?;
let epoch = a.data.target.epoch;
self.sanitize_request(epoch, validator_index)?;
self.prune(epoch);
@ -211,17 +307,17 @@ impl<T: Item, E: EthSpec> AutoPruningContainer<T, E> {
/// ## Errors
///
/// - `validator_index` is higher than `VALIDATOR_REGISTRY_LIMIT`.
/// - `a.data.target.slot` is earlier than `self.earliest_permissible_slot`.
/// - `a.data.target.slot` is earlier than `self.lowest_permissible_slot`.
pub fn validator_has_been_observed(
&self,
a: &Attestation<E>,
epoch: Epoch,
validator_index: usize,
) -> Result<bool, Error> {
self.sanitize_request(a, validator_index)?;
self.sanitize_request(epoch, validator_index)?;
let exists = self
.items
.get(&a.data.target.epoch)
.get(&epoch)
.map_or(false, |item| item.contains(validator_index));
Ok(exists)
@ -233,12 +329,11 @@ impl<T: Item, E: EthSpec> AutoPruningContainer<T, E> {
self.items.get(&epoch).map(|item| item.validator_count())
}
fn sanitize_request(&self, a: &Attestation<E>, validator_index: usize) -> Result<(), Error> {
fn sanitize_request(&self, epoch: Epoch, validator_index: usize) -> Result<(), Error> {
if validator_index > E::ValidatorRegistryLimit::to_usize() {
return Err(Error::ValidatorIndexTooHigh(validator_index));
}
let epoch = a.data.target.epoch;
let lowest_permissible_epoch = self.lowest_permissible_epoch;
if epoch < lowest_permissible_epoch {
return Err(Error::EpochTooLow {
@ -272,84 +367,240 @@ impl<T: Item, E: EthSpec> AutoPruningContainer<T, E> {
/// Also sets `self.lowest_permissible_epoch` with relation to `current_epoch` and
/// `Self::max_capacity`.
pub fn prune(&mut self, current_epoch: Epoch) {
// Taking advantage of saturating subtraction on `Slot`.
let lowest_permissible_epoch = current_epoch - (self.max_capacity().saturating_sub(1));
let lowest_permissible_epoch =
current_epoch.saturating_sub(self.max_capacity().saturating_sub(1));
self.lowest_permissible_epoch = lowest_permissible_epoch;
self.items
.retain(|epoch, _item| *epoch >= lowest_permissible_epoch);
}
#[allow(dead_code)]
/// Returns the `lowest_permissible_epoch`. Used in tests.
pub(crate) fn get_lowest_permissible(&self) -> Epoch {
self.lowest_permissible_epoch
}
}
/// A container that stores some number of `V` items.
///
/// This container is "auto-pruning" since it gets an idea of the current slot by which
/// sync contributions are provided to it and prunes old entries based upon that. For example, if
/// `Self::max_capacity == 3` and an attestation with `data.slot` is supplied, then all
/// sync contributions with an epoch prior to `data.slot - 3` will be cleared from the cache.
///
/// `V` should be set to a `SyncAggregatorSlotHashSet` or a `SyncContributorSlotHashSet`.
pub struct AutoPruningSlotContainer<K: SlotData + Eq + Hash, V, E: EthSpec> {
lowest_permissible_slot: Slot,
items: HashMap<K, V>,
_phantom: PhantomData<E>,
}
impl<K: SlotData + Eq + Hash, V, E: EthSpec> Default for AutoPruningSlotContainer<K, V, E> {
fn default() -> Self {
Self {
lowest_permissible_slot: Slot::new(0),
items: HashMap::new(),
_phantom: PhantomData,
}
}
}
impl<K: SlotData + Eq + Hash, V: Item, E: EthSpec> AutoPruningSlotContainer<K, V, E> {
/// Observe that `validator_index` has produced a sync committee message. Returns `Ok(true)` if
/// the sync committee message has previously been observed for `validator_index`.
///
/// ## Errors
///
/// - `validator_index` is higher than `VALIDATOR_REGISTRY_LIMIT`.
/// - `key.slot` is earlier than `self.lowest_permissible_slot`.
pub fn observe_validator(&mut self, key: K, validator_index: usize) -> Result<bool, Error> {
let slot = key.get_slot();
self.sanitize_request(slot, validator_index)?;
self.prune(slot);
if let Some(item) = self.items.get_mut(&key) {
Ok(item.insert(validator_index))
} else {
// To avoid re-allocations, try and determine a rough initial capacity for the new item
// by obtaining the mean size of all items in earlier slot.
let (count, sum) = self
.items
.iter()
// Only include slots that are less than the given slot in the average. This should
// generally avoid including recent slots that are still "filling up".
.filter(|(item_key, _item)| item_key.get_slot() < slot)
.map(|(_, item)| item.len())
.fold((0, 0), |(count, sum), len| (count + 1, sum + len));
let initial_capacity = sum.checked_div(count).unwrap_or_else(V::default_capacity);
let mut item = V::with_capacity(initial_capacity);
item.insert(validator_index);
self.items.insert(key, item);
Ok(false)
}
}
/// Returns `Ok(true)` if the `validator_index` has already produced a conflicting sync committee message.
///
/// ## Errors
///
/// - `validator_index` is higher than `VALIDATOR_REGISTRY_LIMIT`.
/// - `key.slot` is earlier than `self.lowest_permissible_slot`.
pub fn validator_has_been_observed(
&self,
key: K,
validator_index: usize,
) -> Result<bool, Error> {
self.sanitize_request(key.get_slot(), validator_index)?;
let exists = self
.items
.get(&key)
.map_or(false, |item| item.contains(validator_index));
Ok(exists)
}
/// Returns the number of validators that have been observed at the given `slot`. Returns
/// `None` if `self` does not have a cache for that slot.
pub fn observed_validator_count(&self, key: K) -> Option<usize> {
self.items.get(&key).map(|item| item.validator_count())
}
fn sanitize_request(&self, slot: Slot, validator_index: usize) -> Result<(), Error> {
if validator_index > E::ValidatorRegistryLimit::to_usize() {
return Err(Error::ValidatorIndexTooHigh(validator_index));
}
let lowest_permissible_slot = self.lowest_permissible_slot;
if slot < lowest_permissible_slot {
return Err(Error::SlotTooLow {
slot,
lowest_permissible_slot,
});
}
Ok(())
}
/// The maximum number of slots stored in `self`.
fn max_capacity(&self) -> u64 {
// The next, current and previous slots. We require the next slot due to the
// `MAXIMUM_GOSSIP_CLOCK_DISPARITY`.
3
}
/// Updates `self` with the current slot, removing all sync committee messages that become expired
/// relative to `Self::max_capacity`.
///
/// Also sets `self.lowest_permissible_slot` with relation to `current_slot` and
/// `Self::max_capacity`.
pub fn prune(&mut self, current_slot: Slot) {
let lowest_permissible_slot =
current_slot.saturating_sub(self.max_capacity().saturating_sub(1));
self.lowest_permissible_slot = lowest_permissible_slot;
self.items
.retain(|key, _item| key.get_slot() >= lowest_permissible_slot);
}
#[allow(dead_code)]
/// Returns the `lowest_permissible_slot`. Used in tests.
pub(crate) fn get_lowest_permissible(&self) -> Slot {
self.lowest_permissible_slot
}
}
/// This is used to key information about sync committee aggregators. We require the
/// `subcommittee_index` because it is possible that a validator can aggregate for multiple
/// subcommittees in the same slot.
#[derive(Eq, PartialEq, Hash, Clone, Copy, PartialOrd, Ord, Debug)]
pub struct SlotSubcommitteeIndex {
slot: Slot,
subcommittee_index: u64,
}
impl SlotData for SlotSubcommitteeIndex {
fn get_slot(&self) -> Slot {
self.slot
}
}
impl SlotSubcommitteeIndex {
pub fn new(slot: Slot, subcommittee_index: u64) -> Self {
Self {
slot,
subcommittee_index,
}
}
}
#[cfg(test)]
mod tests {
use super::*;
macro_rules! test_suite {
type E = types::MainnetEthSpec;
macro_rules! test_suite_epoch {
($mod_name: ident, $type: ident) => {
#[cfg(test)]
mod $mod_name {
use super::*;
use types::test_utils::test_random_instance;
type E = types::MainnetEthSpec;
fn single_period_test(store: &mut $type<E>, period: Epoch) {
let validator_indices = [0, 1, 2, 3, 5, 6, 7, 18, 22];
fn get_attestation(epoch: Epoch) -> Attestation<E> {
let mut a: Attestation<E> = test_random_instance();
a.data.target.epoch = epoch;
a
}
fn single_epoch_test(store: &mut $type<E>, epoch: Epoch) {
let attesters = [0, 1, 2, 3, 5, 6, 7, 18, 22];
let a = &get_attestation(epoch);
for &i in &attesters {
for &i in &validator_indices {
assert_eq!(
store.validator_has_been_observed(a, i),
store.validator_has_been_observed(period, i),
Ok(false),
"should indicate an unknown attestation is unknown"
"should indicate an unknown item is unknown"
);
assert_eq!(
store.observe_validator(a, i),
store.observe_validator(period, i),
Ok(false),
"should observe new attestation"
"should observe new item"
);
}
for &i in &attesters {
for &i in &validator_indices {
assert_eq!(
store.validator_has_been_observed(a, i),
store.validator_has_been_observed(period, i),
Ok(true),
"should indicate a known attestation is known"
"should indicate a known item is known"
);
assert_eq!(
store.observe_validator(a, i),
store.observe_validator(period, i),
Ok(true),
"should acknowledge an existing attestation"
"should acknowledge an existing item"
);
}
}
#[test]
fn single_epoch() {
fn single_period() {
let mut store = $type::default();
single_epoch_test(&mut store, Epoch::new(0));
single_period_test(&mut store, Epoch::new(0));
assert_eq!(store.items.len(), 1, "should have a single bitfield stored");
}
#[test]
fn mulitple_contiguous_epochs() {
fn mulitple_contiguous_periods() {
let mut store = $type::default();
let max_cap = store.max_capacity();
for i in 0..max_cap * 3 {
let epoch = Epoch::new(i);
let period = Epoch::new(i);
single_epoch_test(&mut store, epoch);
single_period_test(&mut store, period);
/*
* Ensure that the number of sets is correct.
@ -374,74 +625,77 @@ mod tests {
* Ensure that all the sets have the expected slots
*/
let mut store_epochs = store
let mut store_periods = store
.items
.iter()
.map(|(epoch, _set)| *epoch)
.map(|(period, _set)| *period)
.collect::<Vec<_>>();
assert!(
store_epochs.len() <= store.max_capacity() as usize,
store_periods.len() <= store.max_capacity() as usize,
"store size should not exceed max"
);
store_epochs.sort_unstable();
store_periods.sort_unstable();
let expected_epochs = (i.saturating_sub(max_cap - 1)..=i)
let expected_periods = (i.saturating_sub(max_cap - 1)..=i)
.map(Epoch::new)
.collect::<Vec<_>>();
assert_eq!(expected_epochs, store_epochs, "should have expected slots");
assert_eq!(
expected_periods, store_periods,
"should have expected slots"
);
}
}
#[test]
fn mulitple_non_contiguous_epochs() {
fn mulitple_non_contiguous_periods() {
let mut store = $type::default();
let max_cap = store.max_capacity();
let to_skip = vec![1_u64, 3, 4, 5];
let epochs = (0..max_cap * 3)
let periods = (0..max_cap * 3)
.into_iter()
.filter(|i| !to_skip.contains(i))
.collect::<Vec<_>>();
for &i in &epochs {
for &i in &periods {
if to_skip.contains(&i) {
continue;
}
let epoch = Epoch::from(i);
let period = Epoch::from(i);
single_epoch_test(&mut store, epoch);
single_period_test(&mut store, period);
/*
* Ensure that all the sets have the expected slots
*/
let mut store_epochs = store
let mut store_periods = store
.items
.iter()
.map(|(epoch, _)| *epoch)
.map(|(period, _)| *period)
.collect::<Vec<_>>();
store_epochs.sort_unstable();
store_periods.sort_unstable();
assert!(
store_epochs.len() <= store.max_capacity() as usize,
store_periods.len() <= store.max_capacity() as usize,
"store size should not exceed max"
);
let lowest = store.lowest_permissible_epoch.as_u64();
let highest = epoch.as_u64();
let expected_epochs = (lowest..=highest)
let lowest = store.get_lowest_permissible().as_u64();
let highest = period.as_u64();
let expected_periods = (lowest..=highest)
.filter(|i| !to_skip.contains(i))
.map(Epoch::new)
.collect::<Vec<_>>();
assert_eq!(
expected_epochs,
&store_epochs[..],
expected_periods,
&store_periods[..],
"should have expected epochs"
);
}
@ -450,6 +704,285 @@ mod tests {
};
}
test_suite!(observed_attesters, ObservedAttesters);
test_suite!(observed_aggregators, ObservedAggregators);
test_suite_epoch!(observed_attesters, ObservedAttesters);
test_suite_epoch!(observed_aggregators, ObservedAggregators);
macro_rules! test_suite_slot {
($mod_name: ident, $type: ident) => {
#[cfg(test)]
mod $mod_name {
use super::*;
fn single_period_test(store: &mut $type<E>, key: SlotSubcommitteeIndex) {
let validator_indices = [0, 1, 2, 3, 5, 6, 7, 18, 22];
for &i in &validator_indices {
assert_eq!(
store.validator_has_been_observed(key, i),
Ok(false),
"should indicate an unknown item is unknown"
);
assert_eq!(
store.observe_validator(key, i),
Ok(false),
"should observe new item"
);
}
for &i in &validator_indices {
assert_eq!(
store.validator_has_been_observed(key, i),
Ok(true),
"should indicate a known item is known"
);
assert_eq!(
store.observe_validator(key, i),
Ok(true),
"should acknowledge an existing item"
);
}
}
#[test]
fn single_period() {
let mut store = $type::default();
single_period_test(&mut store, SlotSubcommitteeIndex::new(Slot::new(0), 0));
assert_eq!(store.items.len(), 1, "should have a single bitfield stored");
}
#[test]
fn single_period_multiple_subcommittees() {
let mut store = $type::default();
single_period_test(&mut store, SlotSubcommitteeIndex::new(Slot::new(0), 0));
single_period_test(&mut store, SlotSubcommitteeIndex::new(Slot::new(0), 1));
single_period_test(&mut store, SlotSubcommitteeIndex::new(Slot::new(0), 2));
assert_eq!(store.items.len(), 3, "should have three hash sets stored");
}
#[test]
fn mulitple_contiguous_periods_same_subcommittee() {
let mut store = $type::default();
let max_cap = store.max_capacity();
for i in 0..max_cap * 3 {
let period = SlotSubcommitteeIndex::new(Slot::new(i), 0);
single_period_test(&mut store, period);
/*
* Ensure that the number of sets is correct.
*/
if i < max_cap {
assert_eq!(
store.items.len(),
i as usize + 1,
"should have a {} items stored",
i + 1
);
} else {
assert_eq!(
store.items.len(),
max_cap as usize,
"should have max_capacity items stored"
);
}
/*
* Ensure that all the sets have the expected slots
*/
let mut store_periods = store
.items
.iter()
.map(|(period, _set)| *period)
.collect::<Vec<_>>();
assert!(
store_periods.len() <= store.max_capacity() as usize,
"store size should not exceed max"
);
store_periods.sort_unstable();
let expected_periods = (i.saturating_sub(max_cap - 1)..=i)
.map(|i| SlotSubcommitteeIndex::new(Slot::new(i), 0))
.collect::<Vec<_>>();
assert_eq!(
expected_periods, store_periods,
"should have expected slots"
);
}
}
#[test]
fn mulitple_non_contiguous_periods_same_subcommitte() {
let mut store = $type::default();
let max_cap = store.max_capacity();
let to_skip = vec![1_u64, 3, 4, 5];
let periods = (0..max_cap * 3)
.into_iter()
.filter(|i| !to_skip.contains(i))
.collect::<Vec<_>>();
for &i in &periods {
if to_skip.contains(&i) {
continue;
}
let period = SlotSubcommitteeIndex::new(Slot::from(i), 0);
single_period_test(&mut store, period);
/*
* Ensure that all the sets have the expected slots
*/
let mut store_periods = store
.items
.iter()
.map(|(period, _)| *period)
.collect::<Vec<_>>();
store_periods.sort_unstable();
assert!(
store_periods.len() <= store.max_capacity() as usize,
"store size should not exceed max"
);
let lowest = store.get_lowest_permissible().as_u64();
let highest = period.slot.as_u64();
let expected_periods = (lowest..=highest)
.filter(|i| !to_skip.contains(i))
.map(|i| SlotSubcommitteeIndex::new(Slot::new(i), 0))
.collect::<Vec<_>>();
assert_eq!(
expected_periods,
&store_periods[..],
"should have expected epochs"
);
}
}
#[test]
fn mulitple_contiguous_periods_different_subcommittee() {
let mut store = $type::default();
let max_cap = store.max_capacity();
for i in 0..max_cap * 3 {
let period = SlotSubcommitteeIndex::new(Slot::new(i), i);
single_period_test(&mut store, period);
/*
* Ensure that the number of sets is correct.
*/
if i < max_cap {
assert_eq!(
store.items.len(),
i as usize + 1,
"should have a {} items stored",
i + 1
);
} else {
assert_eq!(
store.items.len(),
max_cap as usize,
"should have max_capacity items stored"
);
}
/*
* Ensure that all the sets have the expected slots
*/
let mut store_periods = store
.items
.iter()
.map(|(period, _set)| *period)
.collect::<Vec<_>>();
assert!(
store_periods.len() <= store.max_capacity() as usize,
"store size should not exceed max"
);
store_periods.sort_unstable();
let expected_periods = (i.saturating_sub(max_cap - 1)..=i)
.map(|i| SlotSubcommitteeIndex::new(Slot::new(i), i))
.collect::<Vec<_>>();
assert_eq!(
expected_periods, store_periods,
"should have expected slots"
);
}
}
#[test]
fn mulitple_non_contiguous_periods_different_subcommitte() {
let mut store = $type::default();
let max_cap = store.max_capacity();
let to_skip = vec![1_u64, 3, 4, 5];
let periods = (0..max_cap * 3)
.into_iter()
.filter(|i| !to_skip.contains(i))
.collect::<Vec<_>>();
for &i in &periods {
if to_skip.contains(&i) {
continue;
}
let period = SlotSubcommitteeIndex::new(Slot::from(i), i);
single_period_test(&mut store, period);
/*
* Ensure that all the sets have the expected slots
*/
let mut store_periods = store
.items
.iter()
.map(|(period, _)| *period)
.collect::<Vec<_>>();
store_periods.sort_unstable();
assert!(
store_periods.len() <= store.max_capacity() as usize,
"store size should not exceed max"
);
let lowest = store.get_lowest_permissible().as_u64();
let highest = period.slot.as_u64();
let expected_periods = (lowest..=highest)
.filter(|i| !to_skip.contains(i))
.map(|i| SlotSubcommitteeIndex::new(Slot::new(i), i))
.collect::<Vec<_>>();
assert_eq!(
expected_periods,
&store_periods[..],
"should have expected epochs"
);
}
}
}
};
}
test_suite_slot!(observed_sync_contributors, ObservedSyncContributors);
test_suite_slot!(observed_sync_aggregators, ObservedSyncAggregators);
}

View File

@ -1,6 +1,7 @@
//! Utilities for managing database schema changes.
use crate::beacon_chain::BeaconChainTypes;
use crate::beacon_chain::{BeaconChainTypes, OP_POOL_DB_KEY};
use crate::validator_pubkey_cache::ValidatorPubkeyCache;
use operation_pool::{PersistedOperationPool, PersistedOperationPoolBase};
use std::fs;
use std::path::Path;
use std::sync::Arc;
@ -54,6 +55,24 @@ pub fn migrate_schema<T: BeaconChainTypes>(
Ok(())
}
// Migration for adding sync committee contributions to the persisted op pool.
(SchemaVersion(3), SchemaVersion(4)) => {
// Deserialize from what exists in the database using the `PersistedOperationPoolBase`
// variant and convert it to the Altair variant.
let pool_opt = db
.get_item::<PersistedOperationPoolBase<T::EthSpec>>(&OP_POOL_DB_KEY)?
.map(PersistedOperationPool::Base)
.map(PersistedOperationPool::base_to_altair);
if let Some(pool) = pool_opt {
// Store the converted pool under the same key.
db.put_item::<PersistedOperationPool<T::EthSpec>>(&OP_POOL_DB_KEY, &pool)?;
}
db.store_schema_version(to)?;
Ok(())
}
// Anything else is an error.
(_, _) => Err(HotColdDBError::UnsupportedSchemaVersion {
target_version: to,

View File

@ -0,0 +1,658 @@
//! Provides verification for the following sync committee messages:
//!
//! - "Unaggregated" `SyncCommitteeMessage` received from either gossip or the HTTP API.
//! - "Aggregated" `SignedContributionAndProof` received from gossip or the HTTP API.
//!
//! For clarity, we define:
//!
//! - Unaggregated: a `SyncCommitteeMessage` object.
//! - Aggregated: a `SignedContributionAndProof` which has zero or more signatures.
//! - Note: "zero or more" may soon change to "one or more".
//!
//! Similar to the `crate::block_verification` module, we try to avoid doing duplicate verification
//! work as a sync committee message passes through different stages of verification. We represent these
//! different stages of verification with wrapper types. These wrapper-types flow in a particular
//! pattern:
//!
//! ```ignore
//! types::SyncCommitteeMessage types::SignedContributionAndProof
//! | |
//! ▼ ▼
//! VerifiedSyncCommitteeMessage VerifiedSyncContribution
//! | |
//! -------------------------------------
//! |
//! ▼
//! impl SignatureVerifiedSyncContribution
//! ```
use crate::observed_attesters::SlotSubcommitteeIndex;
use crate::{
beacon_chain::{MAXIMUM_GOSSIP_CLOCK_DISPARITY, VALIDATOR_PUBKEY_CACHE_LOCK_TIMEOUT},
metrics,
observed_aggregates::ObserveOutcome,
BeaconChain, BeaconChainError, BeaconChainTypes,
};
use bls::{verify_signature_sets, PublicKeyBytes};
use derivative::Derivative;
use safe_arith::ArithError;
use slot_clock::SlotClock;
use state_processing::per_block_processing::errors::SyncCommitteeMessageValidationError;
use state_processing::signature_sets::{
signed_sync_aggregate_selection_proof_signature_set, signed_sync_aggregate_signature_set,
sync_committee_contribution_signature_set_from_pubkeys,
sync_committee_message_set_from_pubkeys,
};
use std::borrow::Cow;
use std::collections::HashMap;
use strum::AsRefStr;
use tree_hash::TreeHash;
use types::consts::altair::SYNC_COMMITTEE_SUBNET_COUNT;
use types::slot_data::SlotData;
use types::sync_committee::Error as SyncCommitteeError;
use types::{
sync_committee_contribution::Error as ContributionError, AggregateSignature, BeaconStateError,
EthSpec, Hash256, SignedContributionAndProof, Slot, SyncCommitteeContribution,
SyncCommitteeMessage, SyncSelectionProof, SyncSubnetId,
};
/// Returned when a sync committee contribution was not successfully verified. It might not have been verified for
/// two reasons:
///
/// - The sync committee message is malformed or inappropriate for the context (indicated by all variants
/// other than `BeaconChainError`).
/// - The application encountered an internal error whilst attempting to determine validity
/// (the `BeaconChainError` variant)
#[derive(Debug, AsRefStr)]
pub enum Error {
/// The sync committee message is from a slot that is later than the current slot (with respect to the
/// gossip clock disparity).
///
/// ## Peer scoring
///
/// Assuming the local clock is correct, the peer has sent an invalid message.
FutureSlot {
message_slot: Slot,
latest_permissible_slot: Slot,
},
/// The sync committee message is from a slot that is prior to the earliest permissible slot (with
/// respect to the gossip clock disparity).
///
/// ## Peer scoring
///
/// Assuming the local clock is correct, the peer has sent an invalid message.
PastSlot {
message_slot: Slot,
earliest_permissible_slot: Slot,
},
/// The sync committee message's aggregation bits were empty when they shouldn't be.
///
/// ## Peer scoring
///
/// The peer has sent an invalid message.
EmptyAggregationBitfield,
/// The `selection_proof` on the sync contribution does not elect it as an aggregator.
///
/// ## Peer scoring
///
/// The peer has sent an invalid message.
InvalidSelectionProof { aggregator_index: u64 },
/// The `selection_proof` on the sync committee contribution selects it as a validator, however the
/// aggregator index is not in the committee for that sync contribution.
///
/// ## Peer scoring
///
/// The peer has sent an invalid message.
AggregatorNotInCommittee { aggregator_index: u64 },
/// The aggregator index refers to a validator index that we have not seen.
///
/// ## Peer scoring
///
/// The peer has sent an invalid message.
AggregatorPubkeyUnknown(u64),
/// The sync contribution has been seen before; either in a block, on the gossip network or from a
/// local validator.
///
/// ## Peer scoring
///
/// It's unclear if this sync contribution is valid, however we have already observed it and do not
/// need to observe it again.
SyncContributionAlreadyKnown(Hash256),
/// There has already been an aggregation observed for this validator, we refuse to process a
/// second.
///
/// ## Peer scoring
///
/// It's unclear if this sync committee message is valid, however we have already observed an aggregate
/// sync committee message from this validator for this epoch and should not observe another.
AggregatorAlreadyKnown(u64),
/// The aggregator index is higher than the maximum possible validator count.
///
/// ## Peer scoring
///
/// The peer has sent an invalid message.
UnknownValidatorIndex(usize),
/// The public key of the validator has not been seen locally.
///
/// ## Peer scoring
///
/// It's unclear if this sync committee message is valid, however we have already observed an aggregate
/// sync committee message from this validator for this epoch and should not observe another.
UnknownValidatorPubkey(PublicKeyBytes),
/// A signature on the sync committee message is invalid.
///
/// ## Peer scoring
///
/// The peer has sent an invalid message.
InvalidSignature,
/// We have already observed a signature for the `validator_index` and refuse to process
/// another.
///
/// ## Peer scoring
///
/// It's unclear if this sync message is valid, however we have already observed a
/// signature from this validator for this slot and should not observe
/// another.
PriorSyncCommitteeMessageKnown { validator_index: u64, slot: Slot },
/// The sync committee message was received on an invalid sync committee message subnet.
///
/// ## Peer scoring
///
/// The peer has sent an invalid message.
InvalidSubnetId {
received: SyncSubnetId,
expected: Vec<SyncSubnetId>,
},
/// The sync message failed the `state_processing` verification stage.
///
/// ## Peer scoring
///
/// The peer has sent an invalid message.
Invalid(SyncCommitteeMessageValidationError),
/// There was an error whilst processing the sync contribution. It is not known if it is valid or invalid.
///
/// ## Peer scoring
///
/// We were unable to process this sync committee message due to an internal error. It's unclear if the
/// sync committee message is valid.
BeaconChainError(BeaconChainError),
/// There was an error whilst processing the sync contribution. It is not known if it is valid or invalid.
///
/// ## Peer scoring
///
/// We were unable to process this sync committee message due to an internal error. It's unclear if the
/// sync committee message is valid.
BeaconStateError(BeaconStateError),
/// There was an error whilst processing the sync contribution. It is not known if it is valid or invalid.
///
/// ## Peer scoring
///
/// We were unable to process this sync committee message due to an internal error. It's unclear if the
/// sync committee message is valid.
InvalidSubcommittee {
subcommittee_index: u64,
subcommittee_size: u64,
},
/// There was an error whilst processing the sync contribution. It is not known if it is valid or invalid.
///
/// ## Peer scoring
///
/// We were unable to process this sync committee message due to an internal error. It's unclear if the
/// sync committee message is valid.
ArithError(ArithError),
/// There was an error whilst processing the sync contribution. It is not known if it is valid or invalid.
///
/// ## Peer scoring
///
/// We were unable to process this sync committee message due to an internal error. It's unclear if the
/// sync committee message is valid.
ContributionError(ContributionError),
/// There was an error whilst processing the sync contribution. It is not known if it is valid or invalid.
///
/// ## Peer scoring
///
/// We were unable to process this sync committee message due to an internal error. It's unclear if the
/// sync committee message is valid.
SyncCommitteeError(SyncCommitteeError),
}
impl From<BeaconChainError> for Error {
fn from(e: BeaconChainError) -> Self {
Error::BeaconChainError(e)
}
}
impl From<BeaconStateError> for Error {
fn from(e: BeaconStateError) -> Self {
Error::BeaconStateError(e)
}
}
impl From<SyncCommitteeError> for Error {
fn from(e: SyncCommitteeError) -> Self {
Error::SyncCommitteeError(e)
}
}
impl From<ArithError> for Error {
fn from(e: ArithError) -> Self {
Error::ArithError(e)
}
}
impl From<ContributionError> for Error {
fn from(e: ContributionError) -> Self {
Error::ContributionError(e)
}
}
/// Wraps a `SignedContributionAndProof` that has been verified for propagation on the gossip network.\
#[derive(Derivative)]
#[derivative(Clone(bound = "T: BeaconChainTypes"))]
pub struct VerifiedSyncContribution<T: BeaconChainTypes> {
signed_aggregate: SignedContributionAndProof<T::EthSpec>,
}
/// Wraps a `SyncCommitteeMessage` that has been verified for propagation on the gossip network.
#[derive(Clone)]
pub struct VerifiedSyncCommitteeMessage {
sync_message: SyncCommitteeMessage,
subnet_positions: HashMap<SyncSubnetId, Vec<usize>>,
}
impl<T: BeaconChainTypes> VerifiedSyncContribution<T> {
/// Returns `Ok(Self)` if the `signed_aggregate` is valid to be (re)published on the gossip
/// network.
pub fn verify(
signed_aggregate: SignedContributionAndProof<T::EthSpec>,
chain: &BeaconChain<T>,
) -> Result<Self, Error> {
let aggregator_index = signed_aggregate.message.aggregator_index;
let contribution = &signed_aggregate.message.contribution;
let subcommittee_index = contribution.subcommittee_index as usize;
// Ensure sync committee contribution is within the MAXIMUM_GOSSIP_CLOCK_DISPARITY allowance.
verify_propagation_slot_range(chain, contribution)?;
// Validate subcommittee index.
if contribution.subcommittee_index >= SYNC_COMMITTEE_SUBNET_COUNT {
return Err(Error::InvalidSubcommittee {
subcommittee_index: contribution.subcommittee_index,
subcommittee_size: SYNC_COMMITTEE_SUBNET_COUNT,
});
}
// Ensure that the sync committee message has participants.
if contribution.aggregation_bits.is_zero() {
return Err(Error::EmptyAggregationBitfield);
}
// Ensure the aggregator's pubkey is in the declared subcommittee of the current sync committee
let pubkey_bytes = chain
.validator_pubkey_bytes(aggregator_index as usize)?
.ok_or(Error::UnknownValidatorIndex(aggregator_index as usize))?;
let sync_subcommittee_pubkeys = chain
.sync_committee_at_next_slot(contribution.get_slot())?
.get_subcommittee_pubkeys(subcommittee_index)?;
if !sync_subcommittee_pubkeys.contains(&pubkey_bytes) {
return Err(Error::AggregatorNotInCommittee { aggregator_index });
};
// Ensure the valid sync contribution has not already been seen locally.
let contribution_root = contribution.tree_hash_root();
if chain
.observed_sync_contributions
.write()
.is_known(contribution, contribution_root)
.map_err(|e| Error::BeaconChainError(e.into()))?
{
return Err(Error::SyncContributionAlreadyKnown(contribution_root));
}
// Ensure there has been no other observed aggregate for the given `aggregator_index`.
//
// Note: do not observe yet, only observe once the sync contribution has been verified.
let observed_key =
SlotSubcommitteeIndex::new(contribution.slot, contribution.subcommittee_index);
match chain
.observed_sync_aggregators
.read()
.validator_has_been_observed(observed_key, aggregator_index as usize)
{
Ok(true) => Err(Error::AggregatorAlreadyKnown(aggregator_index)),
Ok(false) => Ok(()),
Err(e) => Err(BeaconChainError::from(e).into()),
}?;
// Note: this clones the signature which is known to be a relatively slow operation.
//
// Future optimizations should remove this clone.
let selection_proof =
SyncSelectionProof::from(signed_aggregate.message.selection_proof.clone());
if !selection_proof
.is_aggregator::<T::EthSpec>()
.map_err(|e| Error::BeaconChainError(e.into()))?
{
return Err(Error::InvalidSelectionProof { aggregator_index });
}
// Gather all validator pubkeys that signed this contribution.
let participant_pubkeys = sync_subcommittee_pubkeys
.into_iter()
.zip(contribution.aggregation_bits.iter())
.filter_map(|(pubkey, bit)| bit.then(|| pubkey))
.collect::<Vec<_>>();
// Ensure that all signatures are valid.
if !verify_signed_aggregate_signatures(
chain,
&signed_aggregate,
participant_pubkeys.as_slice(),
)? {
return Err(Error::InvalidSignature);
}
let contribution = &signed_aggregate.message.contribution;
let aggregator_index = signed_aggregate.message.aggregator_index;
// Observe the valid sync contribution so we do not re-process it.
//
// It's important to double check that the contribution is not already known, otherwise two
// contribution processed at the same time could be published.
if let ObserveOutcome::AlreadyKnown = chain
.observed_sync_contributions
.write()
.observe_item(contribution, Some(contribution_root))
.map_err(|e| Error::BeaconChainError(e.into()))?
{
return Err(Error::SyncContributionAlreadyKnown(contribution_root));
}
// Observe the aggregator so we don't process another aggregate from them.
//
// It's important to double check that the sync committee message is not already known, otherwise two
// sync committee messages processed at the same time could be published.
if chain
.observed_sync_aggregators
.write()
.observe_validator(observed_key, aggregator_index as usize)
.map_err(BeaconChainError::from)?
{
return Err(Error::PriorSyncCommitteeMessageKnown {
validator_index: aggregator_index,
slot: contribution.slot,
});
}
Ok(VerifiedSyncContribution { signed_aggregate })
}
/// A helper function to add this aggregate to `beacon_chain.op_pool`.
pub fn add_to_pool(self, chain: &BeaconChain<T>) -> Result<(), Error> {
chain.add_contribution_to_block_inclusion_pool(self)
}
/// Returns the underlying `contribution` for the `signed_aggregate`.
pub fn contribution(self) -> SyncCommitteeContribution<T::EthSpec> {
self.signed_aggregate.message.contribution
}
/// Returns the underlying `signed_aggregate`.
pub fn aggregate(&self) -> &SignedContributionAndProof<T::EthSpec> {
&self.signed_aggregate
}
}
impl VerifiedSyncCommitteeMessage {
/// Returns `Ok(Self)` if the `sync_message` is valid to be (re)published on the gossip
/// network.
///
/// `subnet_id` is the subnet from which we received this sync message. This function will
/// verify that it was received on the correct subnet.
pub fn verify<T: BeaconChainTypes>(
sync_message: SyncCommitteeMessage,
subnet_id: SyncSubnetId,
chain: &BeaconChain<T>,
) -> Result<Self, Error> {
// Ensure sync committee message is for the current slot (within a
// MAXIMUM_GOSSIP_CLOCK_DISPARITY allowance).
//
// We do not queue future sync committee messages for later processing.
verify_propagation_slot_range(chain, &sync_message)?;
// Ensure the `subnet_id` is valid for the given validator.
let pubkey = chain
.validator_pubkey_bytes(sync_message.validator_index as usize)?
.ok_or(Error::UnknownValidatorIndex(
sync_message.validator_index as usize,
))?;
let sync_committee = chain.sync_committee_at_next_slot(sync_message.get_slot())?;
let subnet_positions = sync_committee.subcommittee_positions_for_public_key(&pubkey)?;
if !subnet_positions.contains_key(&subnet_id) {
return Err(Error::InvalidSubnetId {
received: subnet_id,
expected: subnet_positions.keys().cloned().collect::<Vec<_>>(),
});
}
// The sync committee message is the first valid message received for the participating validator
// for the slot, sync_message.slot.
let validator_index = sync_message.validator_index;
if chain
.observed_sync_contributors
.read()
.validator_has_been_observed(
SlotSubcommitteeIndex::new(sync_message.slot, subnet_id.into()),
validator_index as usize,
)
.map_err(BeaconChainError::from)?
{
return Err(Error::PriorSyncCommitteeMessageKnown {
validator_index,
slot: sync_message.slot,
});
}
// The aggregate signature of the sync committee message is valid.
verify_sync_committee_message(chain, &sync_message, &pubkey)?;
// Now that the sync committee message has been fully verified, store that we have received a valid
// sync committee message from this validator.
//
// It's important to double check that the sync committee message still hasn't been observed, since
// there can be a race-condition if we receive two sync committee messages at the same time and
// process them in different threads.
if chain
.observed_sync_contributors
.write()
.observe_validator(
SlotSubcommitteeIndex::new(sync_message.slot, subnet_id.into()),
validator_index as usize,
)
.map_err(BeaconChainError::from)?
{
return Err(Error::PriorSyncCommitteeMessageKnown {
validator_index,
slot: sync_message.slot,
});
}
Ok(Self {
sync_message,
subnet_positions,
})
}
/// A helper function to add this sync committee message to `beacon_chain.naive_sync_aggregation_pool`.
pub fn add_to_pool<T: BeaconChainTypes>(self, chain: &BeaconChain<T>) -> Result<Self, Error> {
chain.add_to_naive_sync_aggregation_pool(self)
}
/// Returns the subcommittee positions for the sync message, keyed on the `SyncSubnetId` for
/// the subnets the signature should be sent on.
pub fn subnet_positions(&self) -> &HashMap<SyncSubnetId, Vec<usize>> {
&self.subnet_positions
}
/// Returns the wrapped `SyncCommitteeMessage`.
pub fn sync_message(&self) -> &SyncCommitteeMessage {
&self.sync_message
}
}
/// Verify that the `sync_contribution` is within the acceptable gossip propagation range, with reference
/// to the current slot of the `chain`.
///
/// Accounts for `MAXIMUM_GOSSIP_CLOCK_DISPARITY`.
pub fn verify_propagation_slot_range<T: BeaconChainTypes, U: SlotData>(
chain: &BeaconChain<T>,
sync_contribution: &U,
) -> Result<(), Error> {
let message_slot = sync_contribution.get_slot();
let latest_permissible_slot = chain
.slot_clock
.now_with_future_tolerance(MAXIMUM_GOSSIP_CLOCK_DISPARITY)
.ok_or(BeaconChainError::UnableToReadSlot)?;
if message_slot > latest_permissible_slot {
return Err(Error::FutureSlot {
message_slot,
latest_permissible_slot,
});
}
let earliest_permissible_slot = chain
.slot_clock
.now_with_past_tolerance(MAXIMUM_GOSSIP_CLOCK_DISPARITY)
.ok_or(BeaconChainError::UnableToReadSlot)?;
if message_slot < earliest_permissible_slot {
return Err(Error::PastSlot {
message_slot,
earliest_permissible_slot,
});
}
Ok(())
}
/// Verifies all the signatures in a `SignedContributionAndProof` using BLS batch verification. This
/// includes three signatures:
///
/// - `signed_aggregate.signature`
/// - `signed_aggregate.message.selection_proof`
/// - `signed_aggregate.message.aggregate.signature`
///
/// # Returns
///
/// - `Ok(true)`: if all signatures are valid.
/// - `Ok(false)`: if one or more signatures are invalid.
/// - `Err(e)`: if there was an error preventing signature verification.
pub fn verify_signed_aggregate_signatures<T: BeaconChainTypes>(
chain: &BeaconChain<T>,
signed_aggregate: &SignedContributionAndProof<T::EthSpec>,
participant_pubkeys: &[PublicKeyBytes],
) -> Result<bool, Error> {
let pubkey_cache = chain
.validator_pubkey_cache
.try_read_for(VALIDATOR_PUBKEY_CACHE_LOCK_TIMEOUT)
.ok_or(BeaconChainError::ValidatorPubkeyCacheLockTimeout)?;
let aggregator_index = signed_aggregate.message.aggregator_index;
if aggregator_index >= pubkey_cache.len() as u64 {
return Err(Error::AggregatorPubkeyUnknown(aggregator_index));
}
let next_slot_epoch =
(signed_aggregate.message.contribution.slot + 1).epoch(T::EthSpec::slots_per_epoch());
let fork = chain.spec.fork_at_epoch(next_slot_epoch);
let signature_sets = vec![
signed_sync_aggregate_selection_proof_signature_set(
|validator_index| pubkey_cache.get(validator_index).map(Cow::Borrowed),
&signed_aggregate,
&fork,
chain.genesis_validators_root,
&chain.spec,
)
.map_err(BeaconChainError::SignatureSetError)?,
signed_sync_aggregate_signature_set(
|validator_index| pubkey_cache.get(validator_index).map(Cow::Borrowed),
&signed_aggregate,
&fork,
chain.genesis_validators_root,
&chain.spec,
)
.map_err(BeaconChainError::SignatureSetError)?,
sync_committee_contribution_signature_set_from_pubkeys::<T::EthSpec, _>(
|validator_index| {
pubkey_cache
.get_pubkey_from_pubkey_bytes(validator_index)
.map(Cow::Borrowed)
},
participant_pubkeys,
&signed_aggregate.message.contribution.signature,
signed_aggregate
.message
.contribution
.slot
.epoch(T::EthSpec::slots_per_epoch()),
signed_aggregate.message.contribution.beacon_block_root,
&fork,
chain.genesis_validators_root,
&chain.spec,
)
.map_err(BeaconChainError::SignatureSetError)?,
];
Ok(verify_signature_sets(signature_sets.iter()))
}
/// Verifies that the signature of the `sync_message` is valid.
pub fn verify_sync_committee_message<T: BeaconChainTypes>(
chain: &BeaconChain<T>,
sync_message: &SyncCommitteeMessage,
pubkey_bytes: &PublicKeyBytes,
) -> Result<(), Error> {
let signature_setup_timer =
metrics::start_timer(&metrics::SYNC_MESSAGE_PROCESSING_SIGNATURE_SETUP_TIMES);
let pubkey_cache = chain
.validator_pubkey_cache
.try_read_for(VALIDATOR_PUBKEY_CACHE_LOCK_TIMEOUT)
.ok_or(BeaconChainError::ValidatorPubkeyCacheLockTimeout)?;
let pubkey = pubkey_cache
.get_pubkey_from_pubkey_bytes(pubkey_bytes)
.map(Cow::Borrowed)
.ok_or_else(|| Error::UnknownValidatorPubkey(*pubkey_bytes))?;
let next_slot_epoch = (sync_message.get_slot() + 1).epoch(T::EthSpec::slots_per_epoch());
let fork = chain.spec.fork_at_epoch(next_slot_epoch);
let agg_sig = AggregateSignature::from(&sync_message.signature);
let signature_set = sync_committee_message_set_from_pubkeys::<T::EthSpec>(
pubkey,
&agg_sig,
sync_message.slot.epoch(T::EthSpec::slots_per_epoch()),
sync_message.beacon_block_root,
&fork,
chain.genesis_validators_root,
&chain.spec,
)
.map_err(BeaconChainError::SignatureSetError)?;
metrics::stop_timer(signature_setup_timer);
let _signature_verification_timer =
metrics::start_timer(&metrics::SYNC_MESSAGE_PROCESSING_SIGNATURE_TIMES);
if signature_set.verify() {
Ok(())
} else {
Err(Error::InvalidSignature)
}
}

View File

@ -31,17 +31,18 @@ use store::{config::StoreConfig, BlockReplay, HotColdDB, ItemStore, LevelDB, Mem
use task_executor::ShutdownReason;
use tempfile::{tempdir, TempDir};
use tree_hash::TreeHash;
use types::sync_selection_proof::SyncSelectionProof;
pub use types::test_utils::generate_deterministic_keypairs;
use types::{
typenum::U4294967296, AggregateSignature, Attestation, AttestationData, AttesterSlashing,
BeaconBlock, BeaconState, BeaconStateHash, ChainSpec, Checkpoint, Deposit, DepositData, Domain,
Epoch, EthSpec, ForkName, Graffiti, Hash256, IndexedAttestation, Keypair, ProposerSlashing,
PublicKeyBytes, SelectionProof, SignatureBytes, SignedAggregateAndProof, SignedBeaconBlock,
SignedBeaconBlockHash, SignedRoot, SignedVoluntaryExit, Slot, SubnetId, VariableList,
SignedBeaconBlockHash, SignedContributionAndProof, SignedRoot, SignedVoluntaryExit, Slot,
SubnetId, SyncCommittee, SyncCommitteeContribution, SyncCommitteeMessage, VariableList,
VoluntaryExit,
};
pub use types::test_utils::generate_deterministic_keypairs;
// 4th September 2019
pub const HARNESS_GENESIS_TIME: u64 = 1_567_552_690;
// This parameter is required by a builder but not used because we use the `TestingSlotClock`.
@ -87,6 +88,14 @@ pub enum AttestationStrategy {
SomeValidators(Vec<usize>),
}
/// Indicates whether the `BeaconChainHarness` should use the `state.current_sync_committee` or
/// `state.next_sync_committee` when creating sync messages or contributions.
#[derive(Clone, Debug)]
pub enum RelativeSyncCommittee {
Current,
Next,
}
fn make_rng() -> Mutex<StdRng> {
// Nondeterminism in tests is a highly undesirable thing. Seed the RNG to some arbitrary
// but fixed value for reproducibility.
@ -155,6 +164,11 @@ pub type HarnessAttestations<E> = Vec<(
Option<SignedAggregateAndProof<E>>,
)>;
pub type HarnessSyncContributions<E> = Vec<(
Vec<(SyncCommitteeMessage, usize)>,
Option<SignedContributionAndProof<E>>,
)>;
impl<E: EthSpec> BeaconChainHarness<EphemeralHarnessType<E>> {
pub fn new(
eth_spec_instance: E,
@ -596,6 +610,57 @@ where
.collect()
}
/// A list of sync messages for the given state.
pub fn make_sync_committee_messages(
&self,
state: &BeaconState<E>,
head_block_root: Hash256,
message_slot: Slot,
relative_sync_committee: RelativeSyncCommittee,
) -> Vec<Vec<(SyncCommitteeMessage, usize)>> {
let sync_committee: Arc<SyncCommittee<E>> = match relative_sync_committee {
RelativeSyncCommittee::Current => state
.current_sync_committee()
.expect("should be called on altair beacon state")
.clone(),
RelativeSyncCommittee::Next => state
.next_sync_committee()
.expect("should be called on altair beacon state")
.clone(),
};
sync_committee
.pubkeys
.as_ref()
.chunks(E::sync_subcommittee_size())
.map(|subcommittee| {
subcommittee
.iter()
.enumerate()
.map(|(subcommittee_position, pubkey)| {
let validator_index = self
.chain
.validator_index(pubkey)
.expect("should find validator index")
.expect("pubkey should exist in the beacon chain");
let sync_message = SyncCommitteeMessage::new::<E>(
message_slot,
head_block_root,
validator_index as u64,
&self.validator_keypairs[validator_index].sk,
&state.fork(),
state.genesis_validators_root(),
&self.spec,
);
(sync_message, subcommittee_position)
})
.collect()
})
.collect()
}
/// Deprecated: Use make_unaggregated_attestations() instead.
///
/// A list of attestations for each committee for the given slot.
@ -712,6 +777,94 @@ where
.collect()
}
pub fn make_sync_contributions(
&self,
state: &BeaconState<E>,
block_hash: Hash256,
slot: Slot,
relative_sync_committee: RelativeSyncCommittee,
) -> HarnessSyncContributions<E> {
let sync_messages =
self.make_sync_committee_messages(&state, block_hash, slot, relative_sync_committee);
let sync_contributions: Vec<Option<SignedContributionAndProof<E>>> = sync_messages
.iter()
.enumerate()
.map(|(subnet_id, committee_messages)| {
// If there are any sync messages in this committee, create an aggregate.
if let Some((sync_message, subcommittee_position)) = committee_messages.first() {
let sync_committee: Arc<SyncCommittee<E>> = state
.current_sync_committee()
.expect("should be called on altair beacon state")
.clone();
let aggregator_index = sync_committee
.get_subcommittee_pubkeys(subnet_id)
.unwrap()
.iter()
.find_map(|pubkey| {
let validator_index = self
.chain
.validator_index(pubkey)
.expect("should find validator index")
.expect("pubkey should exist in the beacon chain");
let selection_proof = SyncSelectionProof::new::<E>(
slot,
subnet_id as u64,
&self.validator_keypairs[validator_index].sk,
&state.fork(),
state.genesis_validators_root(),
&self.spec,
);
selection_proof
.is_aggregator::<E>()
.expect("should determine aggregator")
.then(|| validator_index)
})?;
let default = SyncCommitteeContribution::from_message(
&sync_message,
subnet_id as u64,
*subcommittee_position,
)
.expect("should derive sync contribution");
let aggregate = committee_messages.iter().skip(1).fold(
default,
|mut agg, (sig, position)| {
let contribution = SyncCommitteeContribution::from_message(
sig,
subnet_id as u64,
*position,
)
.expect("should derive sync contribution");
agg.aggregate(&contribution);
agg
},
);
let signed_aggregate = SignedContributionAndProof::from_aggregate(
aggregator_index as u64,
aggregate,
None,
&self.validator_keypairs[aggregator_index].sk,
&state.fork(),
state.genesis_validators_root(),
&self.spec,
);
Some(signed_aggregate)
} else {
None
}
})
.collect();
sync_messages.into_iter().zip(sync_contributions).collect()
}
pub fn make_attester_slashing(&self, validator_indices: Vec<u64>) -> AttesterSlashing<E> {
let mut attestation_1 = IndexedAttestation {
attesting_indices: VariableList::new(validator_indices).unwrap(),

View File

@ -173,6 +173,13 @@ impl<T: BeaconChainTypes> ValidatorPubkeyCache<T> {
self.pubkeys.get(i)
}
/// Get the `PublicKey` for a validator with `PublicKeyBytes`.
pub fn get_pubkey_from_pubkey_bytes(&self, pubkey: &PublicKeyBytes) -> Option<&PublicKey> {
self.get_index(pubkey)
.map(|index| self.get(index))
.flatten()
}
/// Get the public key (in bytes form) for a validator with index `i`.
pub fn get_pubkey_bytes(&self, i: usize) -> Option<&PublicKeyBytes> {
self.pubkey_bytes.get(i)

View File

@ -0,0 +1,668 @@
#![cfg(not(debug_assertions))]
#[macro_use]
extern crate lazy_static;
use beacon_chain::sync_committee_verification::Error as SyncCommitteeError;
use beacon_chain::test_utils::{BeaconChainHarness, EphemeralHarnessType, RelativeSyncCommittee};
use int_to_bytes::int_to_bytes32;
use safe_arith::SafeArith;
use store::{SignedContributionAndProof, SyncCommitteeMessage};
use tree_hash::TreeHash;
use types::consts::altair::SYNC_COMMITTEE_SUBNET_COUNT;
use types::{
AggregateSignature, Epoch, EthSpec, Hash256, Keypair, MainnetEthSpec, SecretKey, Slot,
SyncSelectionProof, SyncSubnetId, Unsigned,
};
pub type E = MainnetEthSpec;
pub const VALIDATOR_COUNT: usize = 256;
lazy_static! {
/// A cached set of keys.
static ref KEYPAIRS: Vec<Keypair> = types::test_utils::generate_deterministic_keypairs(VALIDATOR_COUNT);
}
/// Returns a beacon chain harness.
fn get_harness(validator_count: usize) -> BeaconChainHarness<EphemeralHarnessType<E>> {
let mut spec = E::default_spec();
spec.altair_fork_epoch = Some(Epoch::new(0));
let harness = BeaconChainHarness::new(
MainnetEthSpec,
Some(spec),
KEYPAIRS[0..validator_count].to_vec(),
);
harness.advance_slot();
harness
}
/// Returns a sync message that is valid for some slot in the given `chain`.
///
/// Also returns some info about who created it.
fn get_valid_sync_committee_message(
harness: &BeaconChainHarness<EphemeralHarnessType<E>>,
slot: Slot,
relative_sync_committee: RelativeSyncCommittee,
) -> (SyncCommitteeMessage, usize, SecretKey, SyncSubnetId) {
let head_state = harness
.chain
.head_beacon_state()
.expect("should get head state");
let head_block_root = harness
.chain
.head()
.expect("should get head state")
.beacon_block_root;
let (signature, _) = harness
.make_sync_committee_messages(&head_state, head_block_root, slot, relative_sync_committee)
.get(0)
.expect("sync messages should exist")
.get(0)
.expect("first sync message should exist")
.clone();
(
signature.clone(),
signature.validator_index as usize,
harness.validator_keypairs[signature.validator_index as usize]
.sk
.clone(),
SyncSubnetId::new(0),
)
}
fn get_valid_sync_contribution(
harness: &BeaconChainHarness<EphemeralHarnessType<E>>,
relative_sync_committee: RelativeSyncCommittee,
) -> (SignedContributionAndProof<E>, usize, SecretKey) {
let head_state = harness
.chain
.head_beacon_state()
.expect("should get head state");
let head_block_root = harness
.chain
.head()
.expect("should get head state")
.beacon_block_root;
let sync_contributions = harness.make_sync_contributions(
&head_state,
head_block_root,
head_state.slot(),
relative_sync_committee,
);
let (_, contribution_opt) = sync_contributions
.get(0)
.expect("sync contributions should exist");
let contribution = contribution_opt
.as_ref()
.cloned()
.expect("signed contribution and proof should exist");
let aggregator_index = contribution.message.aggregator_index as usize;
(
contribution,
aggregator_index,
harness.validator_keypairs[aggregator_index].sk.clone(),
)
}
/// Returns a proof and index for a validator that is **not** an aggregator for the current sync period.
fn get_non_aggregator(
harness: &BeaconChainHarness<EphemeralHarnessType<E>>,
slot: Slot,
) -> (usize, SecretKey) {
let state = &harness.chain.head().expect("should get head").beacon_state;
let sync_subcommittee_size = E::sync_committee_size()
.safe_div(SYNC_COMMITTEE_SUBNET_COUNT as usize)
.expect("should determine sync subcommittee size");
let sync_committee = state
.current_sync_committee()
.expect("should use altair state")
.clone();
let non_aggregator_index = sync_committee
.pubkeys
.chunks(sync_subcommittee_size)
.enumerate()
.find_map(|(subcommittee_index, subcommittee)| {
subcommittee.iter().find_map(|pubkey| {
let validator_index = harness
.chain
.validator_index(&pubkey)
.expect("should get validator index")
.expect("pubkey should exist in beacon chain");
let selection_proof = SyncSelectionProof::new::<E>(
slot,
subcommittee_index as u64,
&harness.validator_keypairs[validator_index].sk,
&state.fork(),
state.genesis_validators_root(),
&harness.spec,
);
if !selection_proof
.is_aggregator::<E>()
.expect("should determine aggregator")
{
Some(validator_index)
} else {
None
}
})
})
.expect("should find at least one non-aggregator");
let aggregator_sk = harness.validator_keypairs[non_aggregator_index].sk.clone();
(non_aggregator_index, aggregator_sk)
}
/// Tests verification of `SignedContributionAndProof` from the gossip network.
#[test]
fn aggregated_gossip_verification() {
let harness = get_harness(VALIDATOR_COUNT);
let state = harness.get_current_state();
harness.add_attested_blocks_at_slots(
state,
Hash256::zero(),
&[Slot::new(1), Slot::new(2)],
(0..VALIDATOR_COUNT).collect::<Vec<_>>().as_slice(),
);
let current_slot = harness.chain.slot().expect("should get slot");
let (valid_aggregate, aggregator_index, aggregator_sk) =
get_valid_sync_contribution(&harness, RelativeSyncCommittee::Current);
macro_rules! assert_invalid {
($desc: tt, $attn_getter: expr, $($error: pat) |+ $( if $guard: expr )?) => {
assert!(
matches!(
harness
.chain
.verify_sync_contribution_for_gossip($attn_getter)
.err()
.expect(&format!(
"{} should error during verify_sync_contribution_for_gossip",
$desc
)),
$( $error ) |+ $( if $guard )?
),
"case: {}",
$desc,
);
};
}
/*
* The following two tests ensure:
*
* The contribution's slot is for the current slot, i.e. contribution.slot == current_slot
* (with a MAXIMUM_GOSSIP_CLOCK_DISPARITY allowance).
*/
let future_slot = current_slot + 1;
assert_invalid!(
"aggregate from future slot",
{
let mut a = valid_aggregate.clone();
a.message.contribution.slot = future_slot;
a
},
SyncCommitteeError::FutureSlot { message_slot, latest_permissible_slot }
if message_slot == future_slot && latest_permissible_slot == current_slot
);
let early_slot = current_slot
.as_u64()
// Subtract an additional slot since the harness will be exactly on the start of the
// slot and the propagation tolerance will allow an extra slot.
.checked_sub(2)
.expect("chain is not sufficiently deep for test")
.into();
assert_invalid!(
"aggregate from past slot",
{
let mut a = valid_aggregate.clone();
a.message.contribution.slot = early_slot;
a
},
SyncCommitteeError::PastSlot {
message_slot,
earliest_permissible_slot
}
if message_slot == early_slot
&& earliest_permissible_slot == current_slot - 1
);
/*
* The following test ensures:
*
* The subcommittee index is in the allowed range,
* i.e. `contribution.subcommittee_index < SYNC_COMMITTEE_SUBNET_COUNT`.
*/
assert_invalid!(
"subcommittee index out of range",
{
let mut a = valid_aggregate.clone();
a.message.contribution.subcommittee_index = SYNC_COMMITTEE_SUBNET_COUNT;
a
},
SyncCommitteeError::InvalidSubcommittee {
subcommittee_index,
subcommittee_size,
}
if subcommittee_index == SYNC_COMMITTEE_SUBNET_COUNT && subcommittee_size == SYNC_COMMITTEE_SUBNET_COUNT
);
/*
* The following test ensures:
*
* The sync contribution has participants.
*/
assert_invalid!(
"aggregate with no participants",
{
let mut a = valid_aggregate.clone();
let aggregation_bits = &mut a.message.contribution.aggregation_bits;
aggregation_bits.difference_inplace(&aggregation_bits.clone());
assert!(aggregation_bits.is_zero());
a.message.contribution.signature = AggregateSignature::infinity();
a
},
SyncCommitteeError::EmptyAggregationBitfield
);
/*
* This test ensures:
*
* The aggregator signature, signed_contribution_and_proof.signature, is valid.
*/
assert_invalid!(
"aggregate with bad signature",
{
let mut a = valid_aggregate.clone();
a.signature = aggregator_sk.sign(Hash256::from_low_u64_be(42));
a
},
SyncCommitteeError::InvalidSignature
);
/*
* The following test ensures:
*
* The contribution_and_proof.selection_proof is a valid signature of the `SyncAggregatorSelectionData`
* derived from the contribution by the validator with index `contribution_and_proof.aggregator_index`.
*/
assert_invalid!(
"aggregate with bad selection proof signature",
{
let mut a = valid_aggregate.clone();
// Generate some random signature until happens to be a valid selection proof. We need
// this in order to reach the signature verification code.
//
// Could run for ever, but that seems _really_ improbable.
let mut i: u64 = 0;
a.message.selection_proof = loop {
i += 1;
let proof: SyncSelectionProof = aggregator_sk
.sign(Hash256::from_slice(&int_to_bytes32(i)))
.into();
if proof
.is_aggregator::<E>()
.expect("should determine aggregator")
{
break proof.into();
}
};
a
},
SyncCommitteeError::InvalidSignature
);
/*
* The following test ensures:
*
* The aggregate signature is valid for the message `beacon_block_root` and aggregate pubkey
* derived from the participation info in `aggregation_bits` for the subcommittee specified by
* the `contribution.subcommittee_index`.
*/
assert_invalid!(
"aggregate with bad aggregate signature",
{
let mut a = valid_aggregate.clone();
let mut agg_sig = AggregateSignature::infinity();
agg_sig.add_assign(&aggregator_sk.sign(Hash256::from_low_u64_be(42)));
a.message.contribution.signature = agg_sig;
a
},
SyncCommitteeError::InvalidSignature
);
let too_high_index = <E as EthSpec>::ValidatorRegistryLimit::to_u64() + 1;
assert_invalid!(
"aggregate with too-high aggregator index",
{
let mut a = valid_aggregate.clone();
a.message.aggregator_index = too_high_index;
a
},
SyncCommitteeError::UnknownValidatorIndex(index)
if index == too_high_index as usize
);
/*
* The following test ensures:
*
* The aggregator's validator index is in the declared subcommittee of the current sync
* committee -- i.e. state.validators[contribution_and_proof.aggregator_index].pubkey in
* get_sync_subcommittee_pubkeys(state, contribution.subcommittee_index).
*/
assert_invalid!(
"aggregate with unknown aggregator index",
{
let mut a = valid_aggregate.clone();
a.message.contribution.subcommittee_index +=1;
a
},
SyncCommitteeError::AggregatorNotInCommittee {
aggregator_index
}
if aggregator_index == valid_aggregate.message.aggregator_index as u64
);
/*
* The following test ensures:
*
* `contribution_and_proof.selection_proof` selects the validator as an aggregator for the
* slot -- i.e. is_sync_committee_aggregator(contribution_and_proof.selection_proof) returns True.
*/
let (non_aggregator_index, non_aggregator_sk) = get_non_aggregator(&harness, current_slot);
assert_invalid!(
"aggregate from non-aggregator",
{
SignedContributionAndProof::from_aggregate(
non_aggregator_index as u64,
valid_aggregate.message.contribution.clone(),
None,
&non_aggregator_sk,
&harness.chain.head_info().expect("should get head info").fork,
harness.chain.genesis_validators_root,
&harness.chain.spec,
)
},
SyncCommitteeError::InvalidSelectionProof {
aggregator_index: index
}
if index == non_aggregator_index as u64
);
// NOTE: from here on, the tests are stateful, and rely on the valid sync contribution having been
// seen. A refactor to give each test case its own state might be nice at some point
harness
.chain
.verify_sync_contribution_for_gossip(valid_aggregate.clone())
.expect("should verify sync contribution");
/*
* The following test ensures:
*
* The sync committee contribution is the first valid contribution received for the aggregator
* with index contribution_and_proof.aggregator_index for the slot contribution.slot and
* subcommittee index contribution.subcommittee_index.
*/
assert_invalid!(
"aggregate that has already been seen",
valid_aggregate.clone(),
SyncCommitteeError::SyncContributionAlreadyKnown(hash)
if hash == valid_aggregate.message.contribution.tree_hash_root()
);
/*
* The following test ensures:
*
* The sync committee contribution is the first valid contribution received for the aggregator
* with index `contribution_and_proof.aggregator_index` for the slot `contribution.slot` and
* subcommittee index `contribution.subcommittee_index`.
*/
assert_invalid!(
"aggregate from aggregator and subcommittee that has already been seen",
{
let mut a = valid_aggregate;
a.message.contribution.beacon_block_root = Hash256::from_low_u64_le(42);
a
},
SyncCommitteeError::AggregatorAlreadyKnown(index)
if index == aggregator_index as u64
);
/*
* The following test ensures that:
*
* A sync committee contribution for the slot before the sync committee period boundary is verified
* using the `head_state.next_sync_committee`.
*/
// Advance to the slot before the 3rd sync committee period because `current_sync_committee = next_sync_committee`
// at genesis.
let state = harness.get_current_state();
let target_slot = Slot::new(
(2 * harness.spec.epochs_per_sync_committee_period.as_u64() * E::slots_per_epoch()) - 1,
);
harness
.add_attested_block_at_slot(target_slot, state, Hash256::zero(), &[])
.expect("should add block");
// **Incorrectly** create a sync contribution using the current sync committee
let (next_valid_contribution, _, _) =
get_valid_sync_contribution(&harness, RelativeSyncCommittee::Current);
assert_invalid!(
"sync contribution created with incorrect sync committee",
next_valid_contribution.clone(),
SyncCommitteeError::InvalidSignature
);
}
/// Tests the verification conditions for sync committee messages on the gossip network.
#[test]
fn unaggregated_gossip_verification() {
let harness = get_harness(VALIDATOR_COUNT);
let state = harness.get_current_state();
harness.add_attested_blocks_at_slots(
state,
Hash256::zero(),
&[Slot::new(1), Slot::new(2)],
(0..VALIDATOR_COUNT).collect::<Vec<_>>().as_slice(),
);
let current_slot = harness.chain.slot().expect("should get slot");
let (valid_sync_committee_message, expected_validator_index, validator_sk, subnet_id) =
get_valid_sync_committee_message(&harness, current_slot, RelativeSyncCommittee::Current);
macro_rules! assert_invalid {
($desc: tt, $attn_getter: expr, $subnet_getter: expr, $($error: pat) |+ $( if $guard: expr )?) => {
assert!(
matches!(
harness
.chain
.verify_sync_committee_message_for_gossip($attn_getter, $subnet_getter)
.err()
.expect(&format!(
"{} should error during verify_sync_committee_message_for_gossip",
$desc
)),
$( $error ) |+ $( if $guard )?
),
"case: {}",
$desc,
);
};
}
/*
* The following test ensures:
*
* The subnet_id is valid for the given validator, i.e. subnet_id in
* compute_subnets_for_sync_committee(state, sync_committee_message.validator_index).
*/
let id: u64 = subnet_id.into();
let invalid_subnet_id = SyncSubnetId::new(id + 1);
assert_invalid!(
"invalid subnet id",
{
valid_sync_committee_message.clone()
},
invalid_subnet_id,
SyncCommitteeError::InvalidSubnetId {
received,
expected,
}
if received == invalid_subnet_id && expected.contains(&subnet_id)
);
/*
* The following two tests ensure:
*
* This signature is within a MAXIMUM_GOSSIP_CLOCK_DISPARITY allowance from the current slot.
*/
let future_slot = current_slot + 1;
assert_invalid!(
"sync message from future slot",
{
let mut signature = valid_sync_committee_message.clone();
signature.slot = future_slot;
signature
},
subnet_id,
SyncCommitteeError::FutureSlot {
message_slot,
latest_permissible_slot,
}
if message_slot == future_slot && latest_permissible_slot == current_slot
);
// Subtract an additional slot since the harness will be exactly on the start of the
// slot and the propagation tolerance will allow an extra slot.
let early_slot = current_slot
.as_u64()
.checked_sub(2)
.expect("chain is not sufficiently deep for test")
.into();
assert_invalid!(
"sync message from past slot",
{
let mut signature = valid_sync_committee_message.clone();
signature.slot = early_slot;
signature
},
subnet_id,
SyncCommitteeError::PastSlot {
message_slot,
earliest_permissible_slot,
}
if message_slot == early_slot && earliest_permissible_slot == current_slot - 1
);
/*
* The following test ensures that:
*
* The signature is valid for the message beacon_block_root for the validator referenced by
* validator_index.
*/
assert_invalid!(
"sync message with bad signature",
{
let mut sync_message = valid_sync_committee_message.clone();
sync_message.signature = validator_sk.sign(Hash256::from_low_u64_le(424242));
sync_message
},
subnet_id,
SyncCommitteeError::InvalidSignature
);
harness
.chain
.verify_sync_committee_message_for_gossip(valid_sync_committee_message.clone(), subnet_id)
.expect("valid sync message should be verified");
/*
* The following test ensures that:
*
* There has been no other valid sync committee message for the declared slot for the
* validator referenced by sync_committee_message.validator_index.
*/
assert_invalid!(
"sync message that has already been seen",
valid_sync_committee_message,
subnet_id,
SyncCommitteeError::PriorSyncCommitteeMessageKnown {
validator_index,
slot,
}
if validator_index == expected_validator_index as u64 && slot == current_slot
);
/*
* The following test ensures that:
*
* A sync committee message for the slot before the sync committee period boundary is verified
* using the `head_state.next_sync_committee`.
*/
// Advance to the slot before the 3rd sync committee period because `current_sync_committee = next_sync_committee`
// at genesis.
let state = harness.get_current_state();
let target_slot = Slot::new(
(2 * harness.spec.epochs_per_sync_committee_period.as_u64() * E::slots_per_epoch()) - 1,
);
harness
.add_attested_block_at_slot(target_slot, state, Hash256::zero(), &[])
.expect("should add block");
// **Incorrectly** create a sync message using the current sync committee
let (next_valid_sync_committee_message, _, _, next_subnet_id) =
get_valid_sync_committee_message(&harness, target_slot, RelativeSyncCommittee::Current);
assert_invalid!(
"sync message on incorrect subnet",
next_valid_sync_committee_message.clone(),
next_subnet_id,
SyncCommitteeError::InvalidSubnetId {
received,
expected,
}
if received == subnet_id && !expected.contains(&subnet_id)
);
}

View File

@ -436,7 +436,8 @@ fn roundtrip_operation_pool() {
.get_item::<PersistedOperationPool<MinimalEthSpec>>(&OP_POOL_DB_KEY)
.expect("should read db")
.expect("should find op pool")
.into_operation_pool();
.into_operation_pool()
.unwrap();
assert_eq!(harness.chain.op_pool, restored_op_pool);
}

View File

@ -553,14 +553,14 @@ fn import_gossip_block_at_current_slot() {
fn import_gossip_attestation() {
let mut rig = TestRig::new(SMALL_CHAIN);
let initial_attns = rig.chain.naive_aggregation_pool.read().num_attestations();
let initial_attns = rig.chain.naive_aggregation_pool.read().num_items();
rig.enqueue_unaggregated_attestation();
rig.assert_event_journal(&[GOSSIP_ATTESTATION, WORKER_FREED, NOTHING_TO_DO]);
assert_eq!(
rig.chain.naive_aggregation_pool.read().num_attestations(),
rig.chain.naive_aggregation_pool.read().num_items(),
initial_attns + 1,
"op pool should have one more attestation"
);
@ -578,14 +578,14 @@ fn attestation_to_unknown_block_processed(import_method: BlockImportMethod) {
// Send the attestation but not the block, and check that it was not imported.
let initial_attns = rig.chain.naive_aggregation_pool.read().num_attestations();
let initial_attns = rig.chain.naive_aggregation_pool.read().num_items();
rig.enqueue_next_block_unaggregated_attestation();
rig.assert_event_journal(&[GOSSIP_ATTESTATION, WORKER_FREED, NOTHING_TO_DO]);
assert_eq!(
rig.chain.naive_aggregation_pool.read().num_attestations(),
rig.chain.naive_aggregation_pool.read().num_items(),
initial_attns,
"Attestation should not have been included."
);
@ -616,7 +616,7 @@ fn attestation_to_unknown_block_processed(import_method: BlockImportMethod) {
);
assert_eq!(
rig.chain.naive_aggregation_pool.read().num_attestations(),
rig.chain.naive_aggregation_pool.read().num_items(),
initial_attns + 1,
"Attestation should have been included."
);
@ -707,14 +707,14 @@ fn requeue_unknown_block_gossip_attestation_without_import() {
// Send the attestation but not the block, and check that it was not imported.
let initial_attns = rig.chain.naive_aggregation_pool.read().num_attestations();
let initial_attns = rig.chain.naive_aggregation_pool.read().num_items();
rig.enqueue_next_block_unaggregated_attestation();
rig.assert_event_journal(&[GOSSIP_ATTESTATION, WORKER_FREED, NOTHING_TO_DO]);
assert_eq!(
rig.chain.naive_aggregation_pool.read().num_attestations(),
rig.chain.naive_aggregation_pool.read().num_items(),
initial_attns,
"Attestation should not have been included."
);
@ -727,7 +727,7 @@ fn requeue_unknown_block_gossip_attestation_without_import() {
);
assert_eq!(
rig.chain.naive_aggregation_pool.read().num_attestations(),
rig.chain.naive_aggregation_pool.read().num_items(),
initial_attns,
"Attestation should not have been included."
);
@ -748,7 +748,7 @@ fn requeue_unknown_block_gossip_aggregated_attestation_without_import() {
rig.assert_event_journal(&[GOSSIP_AGGREGATE, WORKER_FREED, NOTHING_TO_DO]);
assert_eq!(
rig.chain.naive_aggregation_pool.read().num_attestations(),
rig.chain.naive_aggregation_pool.read().num_items(),
initial_attns,
"Attestation should not have been included."
);

View File

@ -5,6 +5,7 @@ authors = ["Michael Sproul <michael@sigmaprime.io>"]
edition = "2018"
[dependencies]
derivative = "2.1.1"
itertools = "0.10.0"
int_to_bytes = { path = "../../consensus/int_to_bytes" }
lazy_static = "1.4.0"
@ -18,6 +19,7 @@ rayon = "1.5.0"
serde = "1.0.116"
serde_derive = "1.0.116"
store = { path = "../store" }
superstruct = "0.2.0"
[dev-dependencies]
rand = "0.7.3"

View File

@ -4,9 +4,13 @@ mod attester_slashing;
mod max_cover;
mod metrics;
mod persistence;
mod sync_aggregate_id;
pub use persistence::PersistedOperationPool;
pub use persistence::{
PersistedOperationPool, PersistedOperationPoolAltair, PersistedOperationPoolBase,
};
use crate::sync_aggregate_id::SyncAggregateId;
use attestation::AttMaxCover;
use attestation_id::AttestationId;
use attester_slashing::AttesterSlashingMaxCover;
@ -18,18 +22,24 @@ use state_processing::per_block_processing::{
VerifySignatures,
};
use state_processing::SigVerifiedOp;
use std::collections::{hash_map, HashMap, HashSet};
use std::collections::{hash_map::Entry, HashMap, HashSet};
use std::marker::PhantomData;
use std::ptr;
use types::{
typenum::Unsigned, Attestation, AttesterSlashing, BeaconState, BeaconStateError, ChainSpec,
Epoch, EthSpec, Fork, ForkVersion, Hash256, ProposerSlashing, RelativeEpoch,
SignedVoluntaryExit, Validator,
sync_aggregate::Error as SyncAggregateError, typenum::Unsigned, Attestation, AttesterSlashing,
BeaconState, BeaconStateError, ChainSpec, Epoch, EthSpec, Fork, ForkVersion, Hash256,
ProposerSlashing, RelativeEpoch, SignedVoluntaryExit, Slot, SyncAggregate,
SyncCommitteeContribution, Validator,
};
type SyncContributions<T> = RwLock<HashMap<SyncAggregateId, Vec<SyncCommitteeContribution<T>>>>;
#[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 sync aggregate ID to the best `SyncCommitteeContribution`s seen for that ID.
sync_contributions: SyncContributions<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.
@ -42,6 +52,15 @@ pub struct OperationPool<T: EthSpec + Default> {
#[derive(Debug, PartialEq)]
pub enum OpPoolError {
GetAttestationsTotalBalanceError(BeaconStateError),
GetBlockRootError(BeaconStateError),
SyncAggregateError(SyncAggregateError),
IncorrectOpPoolVariant,
}
impl From<SyncAggregateError> for OpPoolError {
fn from(e: SyncAggregateError) -> Self {
OpPoolError::SyncAggregateError(e)
}
}
impl<T: EthSpec> OperationPool<T> {
@ -50,6 +69,97 @@ impl<T: EthSpec> OperationPool<T> {
Self::default()
}
/// Insert a sync contribution into the pool. We don't aggregate these contributions until they
/// are retrieved from the pool.
///
/// ## Note
///
/// This function assumes the given `contribution` is valid.
pub fn insert_sync_contribution(
&self,
contribution: SyncCommitteeContribution<T>,
) -> Result<(), OpPoolError> {
let aggregate_id = SyncAggregateId::new(contribution.slot, contribution.beacon_block_root);
let mut contributions = self.sync_contributions.write();
match contributions.entry(aggregate_id) {
Entry::Vacant(entry) => {
// If no contributions exist for the key, insert the given contribution.
entry.insert(vec![contribution]);
}
Entry::Occupied(mut entry) => {
// If contributions exists for this key, check whether there exists a contribution
// with a matching `subcommittee_index`. If one exists, check whether the new or
// old contribution has more aggregation bits set. If the new one does, add it to the
// pool in place of the old one.
let existing_contributions = entry.get_mut();
match existing_contributions
.iter_mut()
.find(|existing_contribution| {
existing_contribution.subcommittee_index == contribution.subcommittee_index
}) {
Some(existing_contribution) => {
// Only need to replace the contribution if the new contribution has more
// bits set.
if existing_contribution.aggregation_bits.num_set_bits()
< contribution.aggregation_bits.num_set_bits()
{
*existing_contribution = contribution;
}
}
None => {
// If there has been no previous sync contribution for this subcommittee index,
// add it to the pool.
existing_contributions.push(contribution);
}
}
}
};
Ok(())
}
/// Calculate the `SyncAggregate` from the sync contributions that exist in the pool for the
/// slot previous to the slot associated with `state`. Return the calculated `SyncAggregate` if
/// contributions exist at this slot, or else `None`.
pub fn get_sync_aggregate(
&self,
state: &BeaconState<T>,
) -> Result<Option<SyncAggregate<T>>, OpPoolError> {
// Sync aggregates are formed from the contributions from the previous slot.
let slot = state.slot().saturating_sub(1u64);
let block_root = *state
.get_block_root(slot)
.map_err(OpPoolError::GetBlockRootError)?;
let id = SyncAggregateId::new(slot, block_root);
self.sync_contributions
.read()
.get(&id)
.map(|contributions| SyncAggregate::from_contributions(contributions))
.transpose()
.map_err(|e| e.into())
}
/// Total number of sync contributions in the pool.
pub fn num_sync_contributions(&self) -> usize {
self.sync_contributions
.read()
.values()
.map(|contributions| contributions.len())
.sum()
}
/// Remove sync contributions which are too old to be included in a block.
pub fn prune_sync_contributions(&self, current_slot: Slot) {
// Prune sync contributions that are from before the previous slot.
self.sync_contributions.write().retain(|_, contributions| {
// All the contributions in this bucket have the same data, so we only need to
// check the first one.
contributions.first().map_or(false, |contribution| {
current_slot <= contribution.slot.saturating_add(Slot::new(1))
})
});
}
/// Insert an attestation into the pool, aggregating it with existing attestations if possible.
///
/// ## Note
@ -68,11 +178,11 @@ impl<T: EthSpec> OperationPool<T> {
let mut attestations = self.attestations.write();
let existing_attestations = match attestations.entry(id) {
hash_map::Entry::Vacant(entry) => {
Entry::Vacant(entry) => {
entry.insert(vec![attestation]);
return Ok(());
}
hash_map::Entry::Occupied(entry) => entry.into_mut(),
Entry::Occupied(entry) => entry.into_mut(),
};
let mut aggregated = false;
@ -376,6 +486,7 @@ impl<T: EthSpec> OperationPool<T> {
/// Prune all types of transactions given the latest head state and head fork.
pub fn prune_all(&self, head_state: &BeaconState<T>, current_epoch: Epoch) {
self.prune_attestations(current_epoch);
self.prune_sync_contributions(head_state.slot());
self.prune_proposer_slashings(head_state);
self.prune_attester_slashings(head_state);
self.prune_voluntary_exits(head_state);
@ -498,18 +609,19 @@ impl<T: EthSpec + Default> PartialEq for OperationPool<T> {
#[cfg(all(test, not(debug_assertions)))]
mod release_tests {
use lazy_static::lazy_static;
use super::attestation::earliest_attestation_validators;
use super::*;
use beacon_chain::test_utils::{BeaconChainHarness, EphemeralHarnessType};
use beacon_chain::test_utils::{
BeaconChainHarness, EphemeralHarnessType, RelativeSyncCommittee,
};
use lazy_static::lazy_static;
use state_processing::{
common::{base::get_base_reward, get_attesting_indices},
VerifyOperation,
};
use std::collections::BTreeSet;
use std::iter::FromIterator;
use store::StoreConfig;
use types::consts::altair::SYNC_COMMITTEE_SUBNET_COUNT;
use types::*;
pub const MAX_VALIDATOR_COUNT: usize = 4 * 32 * 128;
@ -521,13 +633,10 @@ mod release_tests {
fn get_harness<E: EthSpec>(
validator_count: usize,
spec: Option<ChainSpec>,
) -> BeaconChainHarness<EphemeralHarnessType<E>> {
let harness = BeaconChainHarness::new_with_store_config(
E::default(),
None,
KEYPAIRS[0..validator_count].to_vec(),
StoreConfig::default(),
);
let harness =
BeaconChainHarness::new(E::default(), spec, KEYPAIRS[0..validator_count].to_vec());
harness.advance_slot();
@ -542,14 +651,30 @@ mod release_tests {
let num_validators =
num_committees * E::slots_per_epoch() as usize * spec.target_committee_size;
let harness = get_harness::<E>(num_validators);
let harness = get_harness::<E>(num_validators, None);
let slot_offset = 5 * E::slots_per_epoch() + E::slots_per_epoch() / 2;
(harness, spec)
}
// advance until we have finalized and justified epochs
for _ in 0..slot_offset {
harness.advance_slot();
}
/// Test state for sync contribution-related tests.
fn sync_contribution_test_state<E: EthSpec>(
num_committees: usize,
) -> (BeaconChainHarness<EphemeralHarnessType<E>>, ChainSpec) {
let mut spec = E::default_spec();
spec.altair_fork_epoch = Some(Epoch::new(0));
let num_validators =
num_committees * E::slots_per_epoch() as usize * spec.target_committee_size;
let harness = get_harness::<E>(num_validators, Some(spec.clone()));
let state = harness.get_current_state();
harness.add_attested_blocks_at_slots(
state,
Hash256::zero(),
&[Slot::new(1)],
(0..num_validators).collect::<Vec<_>>().as_slice(),
);
(harness, spec)
}
@ -558,7 +683,7 @@ mod release_tests {
fn test_earliest_attestation() {
let (harness, ref spec) = attestation_test_state::<MainnetEthSpec>(1);
let mut state = harness.get_current_state();
let slot = state.slot() - 1;
let slot = state.slot();
let committees = state
.get_beacon_committees_at_slot(slot)
.unwrap()
@ -629,7 +754,7 @@ mod release_tests {
let op_pool = OperationPool::<MainnetEthSpec>::new();
let mut state = harness.get_current_state();
let slot = state.slot() - 1;
let slot = state.slot();
let committees = state
.get_beacon_committees_at_slot(slot)
.unwrap()
@ -666,7 +791,6 @@ mod release_tests {
assert_eq!(op_pool.num_attestations(), committees.len());
// Before the min attestation inclusion delay, get_attestations shouldn't return anything.
*state.slot_mut() -= 1;
assert_eq!(
op_pool
.get_attestations(&state, |_| true, |_| true, spec)
@ -709,7 +833,7 @@ mod release_tests {
let op_pool = OperationPool::<MainnetEthSpec>::new();
let slot = state.slot() - 1;
let slot = state.slot();
let committees = state
.get_beacon_committees_at_slot(slot)
.unwrap()
@ -755,7 +879,7 @@ mod release_tests {
let op_pool = OperationPool::<MainnetEthSpec>::new();
let slot = state.slot() - 1;
let slot = state.slot();
let committees = state
.get_beacon_committees_at_slot(slot)
.unwrap()
@ -852,7 +976,7 @@ mod release_tests {
let op_pool = OperationPool::<MainnetEthSpec>::new();
let slot = state.slot() - 1;
let slot = state.slot();
let committees = state
.get_beacon_committees_at_slot(slot)
.unwrap()
@ -941,7 +1065,7 @@ mod release_tests {
let mut state = harness.get_current_state();
let op_pool = OperationPool::<MainnetEthSpec>::new();
let slot = state.slot() - 1;
let slot = state.slot();
let committees = state
.get_beacon_committees_at_slot(slot)
.unwrap()
@ -1071,7 +1195,7 @@ mod release_tests {
/// Insert two slashings for the same proposer and ensure only one is returned.
#[test]
fn duplicate_proposer_slashing() {
let harness = get_harness(32);
let harness = get_harness(32, None);
let state = harness.get_current_state();
let op_pool = OperationPool::<MainnetEthSpec>::new();
@ -1096,7 +1220,7 @@ mod release_tests {
// Sanity check on the pruning of proposer slashings
#[test]
fn prune_proposer_slashing_noop() {
let harness = get_harness(32);
let harness = get_harness(32, None);
let state = harness.get_current_state();
let op_pool = OperationPool::<MainnetEthSpec>::new();
@ -1109,7 +1233,7 @@ mod release_tests {
// Sanity check on the pruning of attester slashings
#[test]
fn prune_attester_slashing_noop() {
let harness = get_harness(32);
let harness = get_harness(32, None);
let spec = &harness.spec;
let state = harness.get_current_state();
let op_pool = OperationPool::<MainnetEthSpec>::new();
@ -1126,7 +1250,7 @@ mod release_tests {
// Check that we get maximum coverage for attester slashings (highest qty of validators slashed)
#[test]
fn simple_max_cover_attester_slashing() {
let harness = get_harness(32);
let harness = get_harness(32, None);
let spec = &harness.spec;
let state = harness.get_current_state();
let op_pool = OperationPool::<MainnetEthSpec>::new();
@ -1160,7 +1284,7 @@ mod release_tests {
// Check that we get maximum coverage for attester slashings with overlapping indices
#[test]
fn overlapping_max_cover_attester_slashing() {
let harness = get_harness(32);
let harness = get_harness(32, None);
let spec = &harness.spec;
let state = harness.get_current_state();
let op_pool = OperationPool::<MainnetEthSpec>::new();
@ -1194,7 +1318,7 @@ mod release_tests {
// Max coverage of attester slashings taking into account proposer slashings
#[test]
fn max_coverage_attester_proposer_slashings() {
let harness = get_harness(32);
let harness = get_harness(32, None);
let spec = &harness.spec;
let state = harness.get_current_state();
let op_pool = OperationPool::<MainnetEthSpec>::new();
@ -1225,7 +1349,7 @@ mod release_tests {
//Max coverage checking that non overlapping indices are still recognized for their value
#[test]
fn max_coverage_different_indices_set() {
let harness = get_harness(32);
let harness = get_harness(32, None);
let spec = &harness.spec;
let state = harness.get_current_state();
let op_pool = OperationPool::<MainnetEthSpec>::new();
@ -1257,7 +1381,7 @@ mod release_tests {
//Max coverage should be affected by the overall effective balances
#[test]
fn max_coverage_effective_balances() {
let harness = get_harness(32);
let harness = get_harness(32, None);
let spec = &harness.spec;
let mut state = harness.get_current_state();
let op_pool = OperationPool::<MainnetEthSpec>::new();
@ -1285,4 +1409,268 @@ mod release_tests {
let best_slashings = op_pool.get_slashings(&state);
assert_eq!(best_slashings.1, vec![slashing_2, slashing_3]);
}
/// End-to-end test of basic sync contribution handling.
#[test]
fn sync_contribution_aggregation_insert_get_prune() {
let (harness, _) = sync_contribution_test_state::<MainnetEthSpec>(1);
let op_pool = OperationPool::<MainnetEthSpec>::new();
let state = harness.get_current_state();
let block_root = *state
.get_block_root(state.slot() - Slot::new(1))
.ok()
.expect("block root should exist at slot");
let contributions = harness.make_sync_contributions(
&state,
block_root,
state.slot() - Slot::new(1),
RelativeSyncCommittee::Current,
);
for (_, contribution_and_proof) in contributions {
let contribution = contribution_and_proof
.expect("contribution exists for committee")
.message
.contribution;
op_pool.insert_sync_contribution(contribution).unwrap();
}
assert_eq!(op_pool.sync_contributions.read().len(), 1);
assert_eq!(
op_pool.num_sync_contributions(),
SYNC_COMMITTEE_SUBNET_COUNT as usize
);
let sync_aggregate = op_pool
.get_sync_aggregate(&state)
.expect("Should calculate the sync aggregate")
.expect("Should have block sync aggregate");
assert_eq!(
sync_aggregate.sync_committee_bits.num_set_bits(),
MainnetEthSpec::sync_committee_size()
);
// Prune sync contributions shouldn't do anything at this point.
op_pool.prune_sync_contributions(state.slot() - Slot::new(1));
assert_eq!(
op_pool.num_sync_contributions(),
SYNC_COMMITTEE_SUBNET_COUNT as usize
);
op_pool.prune_sync_contributions(state.slot());
assert_eq!(
op_pool.num_sync_contributions(),
SYNC_COMMITTEE_SUBNET_COUNT as usize
);
// But once we advance to more than one slot after the contribution, it should prune it
// out of existence.
op_pool.prune_sync_contributions(state.slot() + Slot::new(1));
assert_eq!(op_pool.num_sync_contributions(), 0);
}
/// Adding a sync contribution already in the pool should not increase the size of the pool.
#[test]
fn sync_contribution_duplicate() {
let (harness, _) = sync_contribution_test_state::<MainnetEthSpec>(1);
let op_pool = OperationPool::<MainnetEthSpec>::new();
let state = harness.get_current_state();
let block_root = *state
.get_block_root(state.slot() - Slot::new(1))
.ok()
.expect("block root should exist at slot");
let contributions = harness.make_sync_contributions(
&state,
block_root,
state.slot() - Slot::new(1),
RelativeSyncCommittee::Current,
);
for (_, contribution_and_proof) in contributions {
let contribution = contribution_and_proof
.expect("contribution exists for committee")
.message
.contribution;
op_pool
.insert_sync_contribution(contribution.clone())
.unwrap();
op_pool.insert_sync_contribution(contribution).unwrap();
}
assert_eq!(op_pool.sync_contributions.read().len(), 1);
assert_eq!(
op_pool.num_sync_contributions(),
SYNC_COMMITTEE_SUBNET_COUNT as usize
);
}
/// Adding a sync contribution already in the pool with more bits set should increase the
/// number of bits set in the aggregate.
#[test]
fn sync_contribution_with_more_bits() {
let (harness, _) = sync_contribution_test_state::<MainnetEthSpec>(1);
let op_pool = OperationPool::<MainnetEthSpec>::new();
let state = harness.get_current_state();
let block_root = *state
.get_block_root(state.slot() - Slot::new(1))
.ok()
.expect("block root should exist at slot");
let contributions = harness.make_sync_contributions(
&state,
block_root,
state.slot() - Slot::new(1),
RelativeSyncCommittee::Current,
);
let expected_bits = MainnetEthSpec::sync_committee_size() - (2 * contributions.len());
let mut first_contribution = contributions[0]
.1
.as_ref()
.unwrap()
.message
.contribution
.clone();
// Add all contributions, but unset the first two bits of each.
for (_, contribution_and_proof) in contributions {
let mut contribution_fewer_bits = contribution_and_proof
.expect("contribution exists for committee")
.message
.contribution;
// Unset the first two bits of each contribution.
contribution_fewer_bits
.aggregation_bits
.set(0, false)
.expect("set bit");
contribution_fewer_bits
.aggregation_bits
.set(1, false)
.expect("set bit");
op_pool
.insert_sync_contribution(contribution_fewer_bits)
.unwrap();
}
let sync_aggregate = op_pool
.get_sync_aggregate(&state)
.expect("Should calculate the sync aggregate")
.expect("Should have block sync aggregate");
assert_eq!(
sync_aggregate.sync_committee_bits.num_set_bits(),
expected_bits
);
// Unset the first bit of the first contribution and re-insert it. This should increase the
// number of bits set in the sync aggregate by one.
first_contribution
.aggregation_bits
.set(0, false)
.expect("set bit");
op_pool
.insert_sync_contribution(first_contribution)
.unwrap();
// The sync aggregate should now include the additional set bit.
let sync_aggregate = op_pool
.get_sync_aggregate(&state)
.expect("Should calculate the sync aggregate")
.expect("Should have block sync aggregate");
assert_eq!(
sync_aggregate.sync_committee_bits.num_set_bits(),
expected_bits + 1
);
}
/// Adding a sync contribution already in the pool with fewer bits set should not increase the
/// number of bits set in the aggregate.
#[test]
fn sync_contribution_with_fewer_bits() {
let (harness, _) = sync_contribution_test_state::<MainnetEthSpec>(1);
let op_pool = OperationPool::<MainnetEthSpec>::new();
let state = harness.get_current_state();
let block_root = *state
.get_block_root(state.slot() - Slot::new(1))
.ok()
.expect("block root should exist at slot");
let contributions = harness.make_sync_contributions(
&state,
block_root,
state.slot() - Slot::new(1),
RelativeSyncCommittee::Current,
);
let expected_bits = MainnetEthSpec::sync_committee_size() - (2 * contributions.len());
let mut first_contribution = contributions[0]
.1
.as_ref()
.unwrap()
.message
.contribution
.clone();
// Add all contributions, but unset the first two bits of each.
for (_, contribution_and_proof) in contributions {
let mut contribution_fewer_bits = contribution_and_proof
.expect("contribution exists for committee")
.message
.contribution;
// Unset the first two bits of each contribution.
contribution_fewer_bits
.aggregation_bits
.set(0, false)
.expect("set bit");
contribution_fewer_bits
.aggregation_bits
.set(1, false)
.expect("set bit");
op_pool
.insert_sync_contribution(contribution_fewer_bits)
.unwrap();
}
let sync_aggregate = op_pool
.get_sync_aggregate(&state)
.expect("Should calculate the sync aggregate")
.expect("Should have block sync aggregate");
assert_eq!(
sync_aggregate.sync_committee_bits.num_set_bits(),
expected_bits
);
// Unset the first three bits of the first contribution and re-insert it. This should
// not affect the number of bits set in the sync aggregate.
first_contribution
.aggregation_bits
.set(0, false)
.expect("set bit");
first_contribution
.aggregation_bits
.set(1, false)
.expect("set bit");
first_contribution
.aggregation_bits
.set(2, false)
.expect("set bit");
op_pool
.insert_sync_contribution(first_contribution)
.unwrap();
// The sync aggregate should still have the same number of set bits.
let sync_aggregate = op_pool
.get_sync_aggregate(&state)
.expect("Should calculate the sync aggregate")
.expect("Should have block sync aggregate");
assert_eq!(
sync_aggregate.sync_committee_bits.num_set_bits(),
expected_bits
);
}
}

View File

@ -1,5 +1,8 @@
use crate::attestation_id::AttestationId;
use crate::sync_aggregate_id::SyncAggregateId;
use crate::OpPoolError;
use crate::OperationPool;
use derivative::Derivative;
use parking_lot::RwLock;
use serde_derive::{Deserialize, Serialize};
use ssz::{Decode, Encode};
@ -7,17 +10,32 @@ use ssz_derive::{Decode, Encode};
use store::{DBColumn, Error as StoreError, StoreItem};
use types::*;
type PersistedSyncContributions<T> = Vec<(SyncAggregateId, Vec<SyncCommitteeContribution<T>>)>;
/// SSZ-serializable version of `OperationPool`.
///
/// Operations are stored in arbitrary order, so it's not a good idea to compare instances
/// of this type (or its encoded form) for equality. Convert back to an `OperationPool` first.
#[derive(Clone, PartialEq, Debug, Encode, Decode, Serialize, Deserialize)]
#[superstruct(
variants(Base, Altair),
variant_attributes(
derive(Derivative, PartialEq, Debug, Serialize, Deserialize, Encode, Decode),
serde(bound = "T: EthSpec", deny_unknown_fields),
derivative(Clone),
),
partial_getter_error(ty = "OpPoolError", expr = "OpPoolError::IncorrectOpPoolVariant")
)]
#[derive(PartialEq, Debug, Serialize, Deserialize, Encode)]
#[serde(untagged)]
#[serde(bound = "T: EthSpec")]
pub struct PersistedOperationPool<T: EthSpec> {
/// Mapping from attestation ID to attestation mappings.
// We could save space by not storing the attestation ID, but it might
// be difficult to make that roundtrip due to eager aggregation.
attestations: Vec<(AttestationId, Vec<Attestation<T>>)>,
/// Mapping from sync contribution ID to sync contributions and aggregate.
#[superstruct(only(Altair))]
sync_contributions: PersistedSyncContributions<T>,
/// Attester slashings.
attester_slashings: Vec<(AttesterSlashing<T>, ForkVersion)>,
/// Proposer slashings.
@ -27,7 +45,9 @@ pub struct PersistedOperationPool<T: EthSpec> {
}
impl<T: EthSpec> PersistedOperationPool<T> {
/// Convert an `OperationPool` into serializable form.
/// Convert an `OperationPool` into serializable form. Always converts to
/// `PersistedOperationPool::Altair` because the v3 to v4 database schema migration ensures
/// the op pool is always persisted as the Altair variant.
pub fn from_operation_pool(operation_pool: &OperationPool<T>) -> Self {
let attestations = operation_pool
.attestations
@ -36,6 +56,13 @@ impl<T: EthSpec> PersistedOperationPool<T> {
.map(|(att_id, att)| (att_id.clone(), att.clone()))
.collect();
let sync_contributions = operation_pool
.sync_contributions
.read()
.iter()
.map(|(id, contribution)| (id.clone(), contribution.clone()))
.collect();
let attester_slashings = operation_pool
.attester_slashings
.read()
@ -57,42 +84,82 @@ impl<T: EthSpec> PersistedOperationPool<T> {
.map(|(_, exit)| exit.clone())
.collect();
Self {
PersistedOperationPool::Altair(PersistedOperationPoolAltair {
attestations,
sync_contributions,
attester_slashings,
proposer_slashings,
voluntary_exits,
}
})
}
/// Reconstruct an `OperationPool`.
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().collect());
/// Reconstruct an `OperationPool`. Sets `sync_contributions` to its `Default` if `self` matches
/// `PersistedOperationPool::Base`.
pub fn into_operation_pool(self) -> Result<OperationPool<T>, OpPoolError> {
let attestations = RwLock::new(self.attestations().to_vec().into_iter().collect());
let attester_slashings =
RwLock::new(self.attester_slashings().to_vec().into_iter().collect());
let proposer_slashings = RwLock::new(
self.proposer_slashings
self.proposer_slashings()
.to_vec()
.into_iter()
.map(|slashing| (slashing.signed_header_1.message.proposer_index, slashing))
.collect(),
);
let voluntary_exits = RwLock::new(
self.voluntary_exits
self.voluntary_exits()
.to_vec()
.into_iter()
.map(|exit| (exit.message.validator_index, exit))
.collect(),
);
let op_pool = match self {
PersistedOperationPool::Base(_) => OperationPool {
attestations,
sync_contributions: <_>::default(),
attester_slashings,
proposer_slashings,
voluntary_exits,
_phantom: Default::default(),
},
PersistedOperationPool::Altair(_) => {
let sync_contributions =
RwLock::new(self.sync_contributions()?.to_vec().into_iter().collect());
OperationPool {
attestations,
attester_slashings,
proposer_slashings,
voluntary_exits,
_phantom: Default::default(),
OperationPool {
attestations,
sync_contributions,
attester_slashings,
proposer_slashings,
voluntary_exits,
_phantom: Default::default(),
}
}
};
Ok(op_pool)
}
/// Convert the `PersistedOperationPool::Base` variant to `PersistedOperationPool::Altair` by
/// setting `sync_contributions` to its default.
pub fn base_to_altair(self) -> Self {
match self {
PersistedOperationPool::Base(_) => {
PersistedOperationPool::Altair(PersistedOperationPoolAltair {
attestations: self.attestations().to_vec(),
sync_contributions: <_>::default(),
attester_slashings: self.attester_slashings().to_vec(),
proposer_slashings: self.proposer_slashings().to_vec(),
voluntary_exits: self.voluntary_exits().to_vec(),
})
}
PersistedOperationPool::Altair(_) => self,
}
}
}
impl<T: EthSpec> StoreItem for PersistedOperationPool<T> {
/// This `StoreItem` implementation is necessary for migrating the `PersistedOperationPool`
/// in the v3 to v4 database schema migration.
impl<T: EthSpec> StoreItem for PersistedOperationPoolBase<T> {
fn db_column() -> DBColumn {
DBColumn::OpPool
}
@ -105,3 +172,23 @@ impl<T: EthSpec> StoreItem for PersistedOperationPool<T> {
Self::from_ssz_bytes(bytes).map_err(Into::into)
}
}
/// Deserialization for `PersistedOperationPool` defaults to `PersistedOperationPool::Altair`
/// because the v3 to v4 database schema migration ensures the persisted op pool is always stored
/// in the Altair format.
impl<T: EthSpec> StoreItem for PersistedOperationPool<T> {
fn db_column() -> DBColumn {
DBColumn::OpPool
}
fn as_store_bytes(&self) -> Vec<u8> {
self.as_ssz_bytes()
}
fn from_store_bytes(bytes: &[u8]) -> Result<Self, StoreError> {
// Default deserialization to the Altair variant.
PersistedOperationPoolAltair::from_ssz_bytes(bytes)
.map(Self::Altair)
.map_err(Into::into)
}
}

View File

@ -0,0 +1,21 @@
use serde_derive::{Deserialize, Serialize};
use ssz_derive::{Decode, Encode};
use types::{Hash256, Slot};
/// Used to key `SyncAggregate`s in the `naive_sync_aggregation_pool`.
#[derive(
PartialEq, Eq, Clone, Hash, Debug, PartialOrd, Ord, Encode, Decode, Serialize, Deserialize,
)]
pub struct SyncAggregateId {
pub slot: Slot,
pub beacon_block_root: Hash256,
}
impl SyncAggregateId {
pub fn new(slot: Slot, beacon_block_root: Hash256) -> Self {
Self {
slot,
beacon_block_root,
}
}
}

View File

@ -2,7 +2,7 @@ use crate::{DBColumn, Error, StoreItem};
use ssz::{Decode, Encode};
use types::{Checkpoint, Hash256};
pub const CURRENT_SCHEMA_VERSION: SchemaVersion = SchemaVersion(3);
pub const CURRENT_SCHEMA_VERSION: SchemaVersion = SchemaVersion(4);
// All the keys that get stored under the `BeaconMeta` column.
//

View File

@ -6,6 +6,7 @@ use crate::{get_key_for_col, DBColumn, Error, KeyValueStore, KeyValueStoreOp};
use ssz::{Decode, DecodeError, Encode};
use ssz_derive::{Decode, Encode};
use std::convert::TryInto;
use std::sync::Arc;
use types::superstruct;
use types::*;
@ -85,9 +86,9 @@ where
// Light-client sync committees
#[superstruct(only(Altair))]
pub current_sync_committee: SyncCommittee<T>,
pub current_sync_committee: Arc<SyncCommittee<T>>,
#[superstruct(only(Altair))]
pub next_sync_committee: SyncCommittee<T>,
pub next_sync_committee: Arc<SyncCommittee<T>>,
}
/// Implement the conversion function from BeaconState -> PartialBeaconState.

View File

@ -54,14 +54,14 @@
//! }
//! ```
use prometheus::{HistogramOpts, HistogramTimer, Opts};
use prometheus::{HistogramOpts, Opts};
use std::time::Duration;
use prometheus::core::{Atomic, GenericGauge, GenericGaugeVec};
pub use prometheus::{
proto::{Metric, MetricFamily, MetricType},
Encoder, Gauge, GaugeVec, Histogram, HistogramVec, IntCounter, IntCounterVec, IntGauge,
IntGaugeVec, Result, TextEncoder,
Encoder, Gauge, GaugeVec, Histogram, HistogramTimer, HistogramVec, IntCounter, IntCounterVec,
IntGauge, IntGaugeVec, Result, TextEncoder,
};
/// Collect all the metrics for reporting.

View File

@ -2,6 +2,7 @@ use super::*;
use core::num::NonZeroUsize;
use ethereum_types::{H256, U128, U256};
use smallvec::SmallVec;
use std::sync::Arc;
macro_rules! impl_decodable_for_uint {
($type: ident, $bit_size: expr) => {
@ -271,6 +272,20 @@ impl<T: Decode> Decode for Option<T> {
}
}
impl<T: Decode> Decode for Arc<T> {
fn is_ssz_fixed_len() -> bool {
T::is_ssz_fixed_len()
}
fn ssz_fixed_len() -> usize {
T::ssz_fixed_len()
}
fn from_ssz_bytes(bytes: &[u8]) -> Result<Self, DecodeError> {
T::from_ssz_bytes(bytes).map(Arc::new)
}
}
impl Decode for H256 {
fn is_ssz_fixed_len() -> bool {
true

View File

@ -2,6 +2,7 @@ use super::*;
use core::num::NonZeroUsize;
use ethereum_types::{H256, U128, U256};
use smallvec::SmallVec;
use std::sync::Arc;
macro_rules! impl_encodable_for_uint {
($type: ident, $bit_size: expr) => {
@ -231,6 +232,24 @@ impl<T: Encode> Encode for Option<T> {
}
}
impl<T: Encode> Encode for Arc<T> {
fn is_ssz_fixed_len() -> bool {
T::is_ssz_fixed_len()
}
fn ssz_fixed_len() -> usize {
T::ssz_fixed_len()
}
fn ssz_append(&self, buf: &mut Vec<u8>) {
self.as_ref().ssz_append(buf)
}
fn ssz_bytes_len(&self) -> usize {
self.as_ref().ssz_bytes_len()
}
}
macro_rules! impl_for_vec {
($type: ty) => {
impl<T: Encode> Encode for $type {

View File

@ -267,6 +267,32 @@ impl<N: Unsigned + Clone> Bitfield<Fixed<N>> {
pub fn from_bytes(bytes: Vec<u8>) -> Result<Self, Error> {
Self::from_raw_bytes(bytes, Self::capacity())
}
/// Compute the intersection of two fixed-length `Bitfield`s.
///
/// Return a new fixed-length `Bitfield`.
pub fn intersection(&self, other: &Self) -> Self {
let mut result = Self::new();
// Bitwise-and the bytes together, starting from the left of each vector. This takes care
// of masking out any entries beyond `min_len` as well, assuming the bitfield doesn't
// contain any set bits beyond its length.
for i in 0..result.bytes.len() {
result.bytes[i] = self.bytes[i] & other.bytes[i];
}
result
}
/// Compute the union of two fixed-length `Bitfield`s.
///
/// Return a new fixed-length `Bitfield`.
pub fn union(&self, other: &Self) -> Self {
let mut result = Self::new();
for i in 0..result.bytes.len() {
result.bytes[i] =
self.bytes.get(i).copied().unwrap_or(0) | other.bytes.get(i).copied().unwrap_or(0);
}
result
}
}
impl<N: Unsigned + Clone> Default for Bitfield<Fixed<N>> {
@ -700,6 +726,58 @@ mod bitvector {
assert!(BitVector16::from_ssz_bytes(&[1, 0b0000_0000, 0b0000_0000]).is_err());
}
#[test]
fn intersection() {
let a = BitVector16::from_raw_bytes(vec![0b1100, 0b0001], 16).unwrap();
let b = BitVector16::from_raw_bytes(vec![0b1011, 0b1001], 16).unwrap();
let c = BitVector16::from_raw_bytes(vec![0b1000, 0b0001], 16).unwrap();
assert_eq!(a.intersection(&b), c);
assert_eq!(b.intersection(&a), c);
assert_eq!(a.intersection(&c), c);
assert_eq!(b.intersection(&c), c);
assert_eq!(a.intersection(&a), a);
assert_eq!(b.intersection(&b), b);
assert_eq!(c.intersection(&c), c);
}
#[test]
fn intersection_diff_length() {
let a = BitVector16::from_bytes(vec![0b0010_1110, 0b0010_1011]).unwrap();
let b = BitVector16::from_bytes(vec![0b0010_1101, 0b0000_0001]).unwrap();
let c = BitVector16::from_bytes(vec![0b0010_1100, 0b0000_0001]).unwrap();
assert_eq!(a.len(), 16);
assert_eq!(b.len(), 16);
assert_eq!(c.len(), 16);
assert_eq!(a.intersection(&b), c);
assert_eq!(b.intersection(&a), c);
}
#[test]
fn union() {
let a = BitVector16::from_raw_bytes(vec![0b1100, 0b0001], 16).unwrap();
let b = BitVector16::from_raw_bytes(vec![0b1011, 0b1001], 16).unwrap();
let c = BitVector16::from_raw_bytes(vec![0b1111, 0b1001], 16).unwrap();
assert_eq!(a.union(&b), c);
assert_eq!(b.union(&a), c);
assert_eq!(a.union(&a), a);
assert_eq!(b.union(&b), b);
assert_eq!(c.union(&c), c);
}
#[test]
fn union_diff_length() {
let a = BitVector16::from_bytes(vec![0b0010_1011, 0b0010_1110]).unwrap();
let b = BitVector16::from_bytes(vec![0b0000_0001, 0b0010_1101]).unwrap();
let c = BitVector16::from_bytes(vec![0b0010_1011, 0b0010_1111]).unwrap();
assert_eq!(a.len(), c.len());
assert_eq!(a.union(&b), c);
assert_eq!(b.union(&a), c);
}
#[test]
fn ssz_round_trip() {
assert_round_trip(BitVector0::new());

View File

@ -144,6 +144,7 @@ pub type HeaderValidationError = BlockOperationError<HeaderInvalid>;
pub type AttesterSlashingValidationError = BlockOperationError<AttesterSlashingInvalid>;
pub type ProposerSlashingValidationError = BlockOperationError<ProposerSlashingInvalid>;
pub type AttestationValidationError = BlockOperationError<AttestationInvalid>;
pub type SyncCommitteeMessageValidationError = BlockOperationError<SyncAggregateInvalid>;
pub type DepositValidationError = BlockOperationError<DepositInvalid>;
pub type ExitValidationError = BlockOperationError<ExitInvalid>;

View File

@ -8,9 +8,10 @@ use std::borrow::Cow;
use tree_hash::TreeHash;
use types::{
AggregateSignature, AttesterSlashing, BeaconBlockRef, BeaconState, BeaconStateError, ChainSpec,
DepositData, Domain, EthSpec, Fork, Hash256, InconsistentFork, IndexedAttestation,
ProposerSlashing, PublicKey, Signature, SignedAggregateAndProof, SignedBeaconBlock,
SignedBeaconBlockHeader, SignedRoot, SignedVoluntaryExit, SigningData,
DepositData, Domain, Epoch, EthSpec, Fork, Hash256, InconsistentFork, IndexedAttestation,
ProposerSlashing, PublicKey, PublicKeyBytes, Signature, SignedAggregateAndProof,
SignedBeaconBlock, SignedBeaconBlockHeader, SignedContributionAndProof, SignedRoot,
SignedVoluntaryExit, SigningData, SyncAggregatorSelectionData, Unsigned,
};
pub type Result<T> = std::result::Result<T, Error>;
@ -25,6 +26,9 @@ pub enum Error {
/// Attempted to find the public key of a validator that does not exist. You cannot distinguish
/// between an error and an invalid block in this case.
ValidatorUnknown(u64),
/// Attempted to find the public key of a validator that does not exist. You cannot distinguish
/// between an error and an invalid block in this case.
ValidatorPubkeyUnknown(PublicKeyBytes),
/// The `BeaconBlock` has a `proposer_index` that does not match the index we computed locally.
///
/// The block is invalid.
@ -396,3 +400,120 @@ where
message,
))
}
pub fn signed_sync_aggregate_selection_proof_signature_set<'a, T, F>(
get_pubkey: F,
signed_contribution_and_proof: &'a SignedContributionAndProof<T>,
fork: &Fork,
genesis_validators_root: Hash256,
spec: &'a ChainSpec,
) -> Result<SignatureSet<'a>>
where
T: EthSpec,
F: Fn(usize) -> Option<Cow<'a, PublicKey>>,
{
let slot = signed_contribution_and_proof.message.contribution.slot;
let domain = spec.get_domain(
slot.epoch(T::slots_per_epoch()),
Domain::SyncCommitteeSelectionProof,
fork,
genesis_validators_root,
);
let selection_data = SyncAggregatorSelectionData {
slot,
subcommittee_index: signed_contribution_and_proof
.message
.contribution
.subcommittee_index,
};
let message = selection_data.signing_root(domain);
let signature = &signed_contribution_and_proof.message.selection_proof;
let validator_index = signed_contribution_and_proof.message.aggregator_index;
Ok(SignatureSet::single_pubkey(
signature,
get_pubkey(validator_index as usize).ok_or(Error::ValidatorUnknown(validator_index))?,
message,
))
}
pub fn signed_sync_aggregate_signature_set<'a, T, F>(
get_pubkey: F,
signed_contribution_and_proof: &'a SignedContributionAndProof<T>,
fork: &Fork,
genesis_validators_root: Hash256,
spec: &'a ChainSpec,
) -> Result<SignatureSet<'a>>
where
T: EthSpec,
F: Fn(usize) -> Option<Cow<'a, PublicKey>>,
{
let epoch = signed_contribution_and_proof
.message
.contribution
.slot
.epoch(T::slots_per_epoch());
let domain = spec.get_domain(
epoch,
Domain::ContributionAndProof,
fork,
genesis_validators_root,
);
let message = signed_contribution_and_proof.message.signing_root(domain);
let signature = &signed_contribution_and_proof.signature;
let validator_index = signed_contribution_and_proof.message.aggregator_index;
Ok(SignatureSet::single_pubkey(
signature,
get_pubkey(validator_index as usize).ok_or(Error::ValidatorUnknown(validator_index))?,
message,
))
}
#[allow(clippy::too_many_arguments)]
pub fn sync_committee_contribution_signature_set_from_pubkeys<'a, T, F>(
get_pubkey: F,
pubkey_bytes: &[PublicKeyBytes],
signature: &'a AggregateSignature,
epoch: Epoch,
beacon_block_root: Hash256,
fork: &Fork,
genesis_validators_root: Hash256,
spec: &'a ChainSpec,
) -> Result<SignatureSet<'a>>
where
T: EthSpec,
F: Fn(&PublicKeyBytes) -> Option<Cow<'a, PublicKey>>,
{
let mut pubkeys = Vec::with_capacity(T::SyncSubcommitteeSize::to_usize());
for pubkey in pubkey_bytes {
pubkeys.push(get_pubkey(pubkey).ok_or_else(|| Error::ValidatorPubkeyUnknown(*pubkey))?);
}
let domain = spec.get_domain(epoch, Domain::SyncCommittee, &fork, genesis_validators_root);
let message = beacon_block_root.signing_root(domain);
Ok(SignatureSet::multiple_pubkeys(signature, pubkeys, message))
}
pub fn sync_committee_message_set_from_pubkeys<'a, T>(
pubkey: Cow<'a, PublicKey>,
signature: &'a AggregateSignature,
epoch: Epoch,
beacon_block_root: Hash256,
fork: &Fork,
genesis_validators_root: Hash256,
spec: &'a ChainSpec,
) -> Result<SignatureSet<'a>>
where
T: EthSpec,
{
let domain = spec.get_domain(epoch, Domain::SyncCommittee, &fork, genesis_validators_root);
let message = beacon_block_root.signing_root(domain);
Ok(SignatureSet::single_pubkey(signature, pubkey, message))
}

View File

@ -1,5 +1,6 @@
use crate::EpochProcessingError;
use safe_arith::SafeArith;
use std::sync::Arc;
use types::beacon_state::BeaconState;
use types::chain_spec::ChainSpec;
use types::eth_spec::EthSpec;
@ -12,7 +13,7 @@ pub fn process_sync_committee_updates<T: EthSpec>(
if next_epoch.safe_rem(spec.epochs_per_sync_committee_period)? == 0 {
*state.current_sync_committee_mut()? = state.next_sync_committee()?.clone();
*state.next_sync_committee_mut()? = state.get_next_sync_committee(spec)?;
*state.next_sync_committee_mut()? = Arc::new(state.get_next_sync_committee(spec)?);
}
Ok(())
}

View File

@ -1,5 +1,6 @@
use crate::common::{get_attestation_participation_flag_indices, get_attesting_indices};
use std::mem;
use std::sync::Arc;
use types::{
BeaconState, BeaconStateAltair, BeaconStateError as Error, ChainSpec, EthSpec, Fork,
ParticipationFlags, PendingAttestation, RelativeEpoch, SyncCommittee, VariableList,
@ -52,6 +53,8 @@ pub fn upgrade_to_altair<E: EthSpec>(
VariableList::new(vec![ParticipationFlags::default(); pre.validators.len()])?;
let inactivity_scores = VariableList::new(vec![0; pre.validators.len()])?;
let temp_sync_committee = Arc::new(SyncCommittee::temporary()?);
// Where possible, use something like `mem::take` to move fields from behind the &mut
// reference. For other fields that don't have a good default value, use `clone`.
//
@ -94,8 +97,8 @@ pub fn upgrade_to_altair<E: EthSpec>(
// Inactivity
inactivity_scores,
// Sync committees
current_sync_committee: SyncCommittee::temporary()?, // not read
next_sync_committee: SyncCommittee::temporary()?, // not read
current_sync_committee: temp_sync_committee.clone(), // not read
next_sync_committee: temp_sync_committee, // not read
// Caches
committee_caches: mem::take(&mut pre.committee_caches),
pubkey_cache: mem::take(&mut pre.pubkey_cache),
@ -109,9 +112,9 @@ pub fn upgrade_to_altair<E: EthSpec>(
// Fill in sync committees
// Note: A duplicate committee is assigned for the current and next committee at the fork
// boundary
let sync_committee = post.get_next_sync_committee(spec)?;
post.as_altair_mut()?.current_sync_committee = sync_committee.clone();
post.as_altair_mut()?.next_sync_committee = sync_committee;
let sync_committee = Arc::new(post.get_next_sync_committee(spec)?);
*post.current_sync_committee_mut()? = sync_committee.clone();
*post.next_sync_committee_mut()? = sync_committee;
*pre_state = post;

View File

@ -22,7 +22,7 @@ merkle_proof = { path = "../merkle_proof" }
rayon = "1.4.1"
rand = "0.7.3"
safe_arith = { path = "../safe_arith" }
serde = "1.0.116"
serde = {version = "1.0.116" , features = ["rc"] }
serde_derive = "1.0.116"
slog = "2.5.2"
eth2_ssz = "0.1.2"

View File

@ -1,14 +1,17 @@
use super::{
AggregateSignature, AttestationData, BitList, ChainSpec, Domain, EthSpec, Fork, SecretKey,
SignedRoot,
};
use crate::{test_utils::TestRandom, Hash256};
use safe_arith::ArithError;
use serde_derive::{Deserialize, Serialize};
use ssz_derive::{Decode, Encode};
use test_random_derive::TestRandom;
use tree_hash_derive::TreeHash;
use crate::slot_data::SlotData;
use crate::{test_utils::TestRandom, Hash256, Slot};
use super::{
AggregateSignature, AttestationData, BitList, ChainSpec, Domain, EthSpec, Fork, SecretKey,
SignedRoot,
};
#[derive(Debug, PartialEq)]
pub enum Error {
SszTypesError(ssz_types::Error),
@ -84,10 +87,17 @@ impl<T: EthSpec> Attestation<T> {
}
}
impl<T: EthSpec> SlotData for Attestation<T> {
fn get_slot(&self) -> Slot {
self.data.slot
}
}
#[cfg(test)]
mod tests {
use super::*;
use crate::*;
use super::*;
ssz_and_tree_hash_tests!(Attestation<MainnetEthSpec>);
}

View File

@ -1,6 +1,7 @@
use crate::test_utils::TestRandom;
use crate::{Checkpoint, Hash256, SignedRoot, Slot};
use crate::slot_data::SlotData;
use serde_derive::{Deserialize, Serialize};
use ssz_derive::{Decode, Encode};
use test_random_derive::TestRandom;
@ -39,6 +40,12 @@ pub struct AttestationData {
impl SignedRoot for AttestationData {}
impl SlotData for AttestationData {
fn get_slot(&self) -> Slot {
self.slot
}
}
#[cfg(test)]
mod tests {
use super::*;

View File

@ -15,7 +15,7 @@ use ssz_derive::{Decode, Encode};
use ssz_types::{typenum::Unsigned, BitVector, FixedVector};
use std::collections::HashSet;
use std::convert::TryInto;
use std::{fmt, mem};
use std::{fmt, mem, sync::Arc};
use superstruct::superstruct;
use swap_or_not_shuffle::compute_shuffled_index;
use test_random_derive::TestRandom;
@ -110,6 +110,10 @@ pub enum Error {
ArithError(ArithError),
MissingBeaconBlock(SignedBeaconBlockHash),
MissingBeaconState(BeaconStateHash),
SyncCommitteeNotKnown {
current_epoch: Epoch,
epoch: Epoch,
},
}
/// Control whether an epoch-indexed field can be indexed at the next epoch or not.
@ -255,9 +259,9 @@ where
// Light-client sync committees
#[superstruct(only(Altair))]
pub current_sync_committee: SyncCommittee<T>,
pub current_sync_committee: Arc<SyncCommittee<T>>,
#[superstruct(only(Altair))]
pub next_sync_committee: SyncCommittee<T>,
pub next_sync_committee: Arc<SyncCommittee<T>>,
// Caching (not in the spec)
#[serde(skip_serializing, skip_deserializing)]
@ -730,6 +734,28 @@ impl<T: EthSpec> BeaconState<T> {
Ok(hash(&preimage))
}
/// Get the already-built current or next sync committee from the state.
pub fn get_built_sync_committee(
&self,
epoch: Epoch,
spec: &ChainSpec,
) -> Result<&Arc<SyncCommittee<T>>, Error> {
let sync_committee_period = epoch.sync_committee_period(spec)?;
let current_sync_committee_period = self.current_epoch().sync_committee_period(spec)?;
let next_sync_committee_period = current_sync_committee_period.safe_add(1)?;
if sync_committee_period == current_sync_committee_period {
self.current_sync_committee()
} else if sync_committee_period == next_sync_committee_period {
self.next_sync_committee()
} else {
Err(Error::SyncCommitteeNotKnown {
current_epoch: self.current_epoch(),
epoch,
})
}
}
/// Get the validator indices of all validators from `sync_committee`.
pub fn get_sync_committee_indices(
&mut self,
@ -1514,6 +1540,28 @@ impl<T: EthSpec> BeaconState<T> {
(self.previous_epoch() - self.finalized_checkpoint().epoch)
> spec.min_epochs_to_inactivity_penalty
}
/// Get the `SyncCommittee` associated with the next slot. Useful because sync committees
/// assigned to `slot` sign for `slot - 1`. This creates the exceptional logic below when
/// transitioning between sync committee periods.
pub fn get_sync_committee_for_next_slot(
&self,
spec: &ChainSpec,
) -> Result<Arc<SyncCommittee<T>>, Error> {
let next_slot_epoch = self
.slot()
.saturating_add(Slot::new(1))
.epoch(T::slots_per_epoch());
let sync_committee = if self.current_epoch().sync_committee_period(spec)
== next_slot_epoch.sync_committee_period(spec)
{
self.current_sync_committee()?.clone()
} else {
self.next_sync_committee()?.clone()
};
Ok(sync_committee)
}
}
impl From<RelativeEpochError> for Error {

View File

@ -17,6 +17,8 @@ pub enum Domain {
SelectionProof,
AggregateAndProof,
SyncCommittee,
ContributionAndProof,
SyncCommitteeSelectionProof,
}
/// Lighthouse's internal configuration struct.
@ -181,6 +183,37 @@ impl ChainSpec {
}
}
/// Returns the fork version for a named fork.
pub fn fork_version_for_name(&self, fork_name: ForkName) -> [u8; 4] {
match fork_name {
ForkName::Base => self.genesis_fork_version,
ForkName::Altair => self.altair_fork_version,
}
}
/// For a given fork name, return the epoch at which it activates.
pub fn fork_epoch(&self, fork_name: ForkName) -> Option<Epoch> {
match fork_name {
ForkName::Base => Some(Epoch::new(0)),
ForkName::Altair => self.altair_fork_epoch,
}
}
/// Returns a full `Fork` struct for a given epoch.
pub fn fork_at_epoch(&self, epoch: Epoch) -> Fork {
let current_fork_name = self.fork_name_at_epoch(epoch);
let previous_fork_name = current_fork_name.previous_fork().unwrap_or(ForkName::Base);
let epoch = self
.fork_epoch(current_fork_name)
.unwrap_or_else(|| Epoch::new(0));
Fork {
previous_version: self.fork_version_for_name(previous_fork_name),
current_version: self.fork_version_for_name(current_fork_name),
epoch,
}
}
/// Get the domain number, unmodified by the fork.
///
/// Spec v0.12.1
@ -194,6 +227,8 @@ impl ChainSpec {
Domain::SelectionProof => self.domain_selection_proof,
Domain::AggregateAndProof => self.domain_aggregate_and_proof,
Domain::SyncCommittee => self.domain_sync_committee,
Domain::ContributionAndProof => self.domain_contribution_and_proof,
Domain::SyncCommitteeSelectionProof => self.domain_sync_committee_selection_proof,
}
}
@ -675,6 +710,18 @@ mod tests {
);
test_domain(Domain::SyncCommittee, spec.domain_sync_committee, &spec);
}
// Test that `fork_name_at_epoch` and `fork_epoch` are consistent.
#[test]
fn fork_name_at_epoch_consistency() {
let spec = ChainSpec::mainnet();
for fork_name in ForkName::list_all() {
if let Some(fork_epoch) = spec.fork_epoch(fork_name) {
assert_eq!(spec.fork_name_at_epoch(fork_epoch), fork_name);
}
}
}
}
#[cfg(test)]

View File

@ -8,6 +8,8 @@ pub mod altair {
pub const SYNC_REWARD_WEIGHT: u64 = 2;
pub const PROPOSER_WEIGHT: u64 = 8;
pub const WEIGHT_DENOMINATOR: u64 = 64;
pub const SYNC_COMMITTEE_SUBNET_COUNT: u64 = 4;
pub const TARGET_AGGREGATORS_PER_SYNC_SUBCOMMITTEE: u64 = 4;
pub const PARTICIPATION_FLAG_WEIGHTS: [u64; NUM_FLAG_INDICES] = [
TIMELY_SOURCE_WEIGHT,

View File

@ -0,0 +1,61 @@
use super::{
ChainSpec, EthSpec, Fork, Hash256, SecretKey, Signature, SignedRoot, SyncCommitteeContribution,
SyncSelectionProof,
};
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;
/// A Validators aggregate sync committee contribution and selection proof.
#[cfg_attr(feature = "arbitrary-fuzz", derive(arbitrary::Arbitrary))]
#[derive(Debug, Clone, PartialEq, Serialize, Deserialize, Encode, Decode, TestRandom, TreeHash)]
#[serde(bound = "T: EthSpec")]
pub struct ContributionAndProof<T: EthSpec> {
/// The index of the validator that created the sync contribution.
#[serde(with = "serde_utils::quoted_u64")]
pub aggregator_index: u64,
/// The aggregate contribution.
pub contribution: SyncCommitteeContribution<T>,
/// A proof provided by the validator that permits them to publish on the
/// `sync_committee_contribution_and_proof` gossipsub topic.
pub selection_proof: Signature,
}
impl<T: EthSpec> ContributionAndProof<T> {
/// Produces a new `ContributionAndProof` with a `selection_proof` generated by signing
/// `SyncAggregatorSelectionData` with `secret_key`.
///
/// If `selection_proof.is_none()` it will be computed locally.
pub fn from_aggregate(
aggregator_index: u64,
contribution: SyncCommitteeContribution<T>,
selection_proof: Option<SyncSelectionProof>,
secret_key: &SecretKey,
fork: &Fork,
genesis_validators_root: Hash256,
spec: &ChainSpec,
) -> Self {
let selection_proof = selection_proof
.unwrap_or_else(|| {
SyncSelectionProof::new::<T>(
contribution.slot,
contribution.subcommittee_index,
secret_key,
fork,
genesis_validators_root,
spec,
)
})
.into();
Self {
aggregator_index,
contribution,
selection_proof,
}
}
}
impl<T: EthSpec> SignedRoot for ContributionAndProof<T> {}

View File

@ -91,6 +91,10 @@ pub trait EthSpec: 'static + Default + Sync + Send + Clone + Debug + PartialEq +
///
/// Must be set to `EpochsPerEth1VotingPeriod * SlotsPerEpoch`
type SlotsPerEth1VotingPeriod: Unsigned + Clone + Sync + Send + Debug + PartialEq;
/// The size of `sync_subcommittees`.
///
/// Must be set to `SyncCommitteeSize / SyncCommitteeSubnetCount`.
type SyncSubcommitteeSize: Unsigned + Clone + Sync + Send + Debug + PartialEq;
fn default_spec() -> ChainSpec;
@ -171,6 +175,16 @@ pub trait EthSpec: 'static + Default + Sync + Send + Clone + Debug + PartialEq +
fn slots_per_eth1_voting_period() -> usize {
Self::SlotsPerEth1VotingPeriod::to_usize()
}
/// Returns the `SYNC_COMMITTEE_SIZE` constant for this specification.
fn sync_committee_size() -> usize {
Self::SyncCommitteeSize::to_usize()
}
/// Returns the `SYNC_COMMITTEE_SIZE / SyncCommitteeSubnetCount`.
fn sync_subcommittee_size() -> usize {
Self::SyncSubcommitteeSize::to_usize()
}
}
/// Macro to inherit some type values from another EthSpec.
@ -204,6 +218,7 @@ impl EthSpec for MainnetEthSpec {
type MaxDeposits = U16;
type MaxVoluntaryExits = U16;
type SyncCommitteeSize = U512;
type SyncSubcommitteeSize = U128; // 512 committee size / 4 sync committee subnet count
type MaxPendingAttestations = U4096; // 128 max attestations * 32 slots per epoch
type SlotsPerEth1VotingPeriod = U2048; // 64 epochs * 32 slots per epoch
@ -228,6 +243,7 @@ impl EthSpec for MinimalEthSpec {
type EpochsPerHistoricalVector = U64;
type EpochsPerSlashingsVector = U64;
type SyncCommitteeSize = U32;
type SyncSubcommitteeSize = U8; // 32 committee size / 4 sync committee subnet count
type MaxPendingAttestations = U1024; // 128 max attestations * 8 slots per epoch
type SlotsPerEth1VotingPeriod = U32; // 4 epochs * 8 slots per epoch

View File

@ -26,6 +26,26 @@ impl ForkName {
}
}
}
/// Return the name of the fork immediately prior to the current one.
///
/// If `self` is `ForkName::Base` then `Base` is returned.
pub fn previous_fork(self) -> Option<ForkName> {
match self {
ForkName::Base => None,
ForkName::Altair => Some(ForkName::Base),
}
}
/// Return the name of the fork immediately after the current one.
///
/// If `self` is the last known fork and has no successor, `None` is returned.
pub fn next_fork(self) -> Option<ForkName> {
match self {
ForkName::Base => Some(ForkName::Altair),
ForkName::Altair => None,
}
}
}
impl std::str::FromStr for ForkName {
@ -45,3 +65,20 @@ pub struct InconsistentFork {
pub fork_at_slot: ForkName,
pub object_fork: ForkName,
}
#[cfg(test)]
mod test {
use super::*;
use itertools::Itertools;
#[test]
fn previous_and_next_fork_consistent() {
assert_eq!(ForkName::Altair.next_fork(), None);
assert_eq!(ForkName::Base.previous_fork(), None);
for (prev_fork, fork) in ForkName::list_all().into_iter().tuple_windows() {
assert_eq!(prev_fork.next_fork(), Some(fork));
assert_eq!(fork.previous_fork(), Some(prev_fork));
}
}
}

View File

@ -30,6 +30,7 @@ pub mod beacon_state;
pub mod chain_spec;
pub mod checkpoint;
pub mod consts;
pub mod contribution_and_proof;
pub mod deposit;
pub mod deposit_data;
pub mod deposit_message;
@ -51,6 +52,7 @@ pub mod shuffling_id;
pub mod signed_aggregate_and_proof;
pub mod signed_beacon_block;
pub mod signed_beacon_block_header;
pub mod signed_contribution_and_proof;
pub mod signed_voluntary_exit;
pub mod signing_data;
pub mod validator;
@ -64,9 +66,15 @@ pub mod preset;
pub mod slot_epoch;
pub mod subnet_id;
pub mod sync_aggregate;
pub mod sync_aggregator_selection_data;
pub mod sync_committee;
pub mod sync_committee_contribution;
pub mod sync_committee_message;
pub mod sync_selection_proof;
pub mod sync_subnet_id;
mod tree_hash_impls;
pub mod slot_data;
#[cfg(feature = "sqlite")]
pub mod sqlite;
@ -90,6 +98,7 @@ pub use crate::beacon_state::{BeaconTreeHashCache, Error as BeaconStateError, *}
pub use crate::chain_spec::{ChainSpec, Config, Domain};
pub use crate::checkpoint::Checkpoint;
pub use crate::config_and_preset::ConfigAndPreset;
pub use crate::contribution_and_proof::ContributionAndProof;
pub use crate::deposit::{Deposit, DEPOSIT_TREE_DEPTH};
pub use crate::deposit_data::DepositData;
pub use crate::deposit_message::DepositMessage;
@ -115,12 +124,18 @@ pub use crate::signed_beacon_block::{
SignedBeaconBlock, SignedBeaconBlockAltair, SignedBeaconBlockBase, SignedBeaconBlockHash,
};
pub use crate::signed_beacon_block_header::SignedBeaconBlockHeader;
pub use crate::signed_contribution_and_proof::SignedContributionAndProof;
pub use crate::signed_voluntary_exit::SignedVoluntaryExit;
pub use crate::signing_data::{SignedRoot, SigningData};
pub use crate::slot_epoch::{Epoch, Slot};
pub use crate::subnet_id::SubnetId;
pub use crate::sync_aggregate::SyncAggregate;
pub use crate::sync_aggregator_selection_data::SyncAggregatorSelectionData;
pub use crate::sync_committee::SyncCommittee;
pub use crate::sync_committee_contribution::SyncCommitteeContribution;
pub use crate::sync_committee_message::SyncCommitteeMessage;
pub use crate::sync_selection_proof::SyncSelectionProof;
pub use crate::sync_subnet_id::SyncSubnetId;
pub use crate::validator::Validator;
pub use crate::validator_subscription::ValidatorSubscription;
pub use crate::voluntary_exit::VoluntaryExit;

View File

@ -1,6 +1,6 @@
use super::{
AggregateAndProof, Attestation, ChainSpec, Domain, EthSpec, Fork, Hash256, PublicKey,
SecretKey, SelectionProof, Signature, SignedRoot,
AggregateAndProof, Attestation, ChainSpec, Domain, EthSpec, Fork, Hash256, SecretKey,
SelectionProof, Signature, SignedRoot,
};
use crate::test_utils::TestRandom;
use serde_derive::{Deserialize, Serialize};
@ -60,41 +60,4 @@ impl<T: EthSpec> SignedAggregateAndProof<T> {
signature: secret_key.sign(signing_message),
}
}
/// Verifies the signature of the `AggregateAndProof`
pub fn is_valid_signature(
&self,
validator_pubkey: &PublicKey,
fork: &Fork,
genesis_validators_root: Hash256,
spec: &ChainSpec,
) -> bool {
let target_epoch = self.message.aggregate.data.slot.epoch(T::slots_per_epoch());
let domain = spec.get_domain(
target_epoch,
Domain::AggregateAndProof,
fork,
genesis_validators_root,
);
let message = self.message.signing_root(domain);
self.signature.verify(validator_pubkey, message)
}
/// Verifies the signature of the `AggregateAndProof` as well the underlying selection_proof in
/// the contained `AggregateAndProof`.
pub fn is_valid(
&self,
validator_pubkey: &PublicKey,
fork: &Fork,
genesis_validators_root: Hash256,
spec: &ChainSpec,
) -> bool {
self.is_valid_signature(validator_pubkey, fork, genesis_validators_root, spec)
&& self.message.is_valid_selection_proof(
validator_pubkey,
fork,
genesis_validators_root,
spec,
)
}
}

View File

@ -0,0 +1,61 @@
use super::{
ChainSpec, ContributionAndProof, Domain, EthSpec, Fork, Hash256, SecretKey, Signature,
SignedRoot, SyncCommitteeContribution, SyncSelectionProof,
};
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;
/// A Validators signed contribution proof to publish on the `sync_committee_contribution_and_proof`
/// gossipsub topic.
#[cfg_attr(feature = "arbitrary-fuzz", derive(arbitrary::Arbitrary))]
#[derive(Debug, Clone, PartialEq, Serialize, Deserialize, Encode, Decode, TestRandom, TreeHash)]
#[serde(bound = "T: EthSpec")]
pub struct SignedContributionAndProof<T: EthSpec> {
/// The `ContributionAndProof` that was signed.
pub message: ContributionAndProof<T>,
/// The validator's signature of `message`.
pub signature: Signature,
}
impl<T: EthSpec> SignedContributionAndProof<T> {
/// Produces a new `SignedContributionAndProof` with a `selection_proof` generated by signing
/// `aggregate.data.slot` with `secret_key`.
///
/// If `selection_proof.is_none()` it will be computed locally.
pub fn from_aggregate(
aggregator_index: u64,
contribution: SyncCommitteeContribution<T>,
selection_proof: Option<SyncSelectionProof>,
secret_key: &SecretKey,
fork: &Fork,
genesis_validators_root: Hash256,
spec: &ChainSpec,
) -> Self {
let message = ContributionAndProof::from_aggregate(
aggregator_index,
contribution,
selection_proof,
secret_key,
fork,
genesis_validators_root,
spec,
);
let epoch = message.contribution.slot.epoch(T::slots_per_epoch());
let domain = spec.get_domain(
epoch,
Domain::ContributionAndProof,
fork,
genesis_validators_root,
);
let signing_message = message.signing_root(domain);
SignedContributionAndProof {
message,
signature: secret_key.sign(signing_message),
}
}
}

View File

@ -0,0 +1,13 @@
use crate::Slot;
/// A trait providing a `Slot` getter for messages that are related to a single slot. Useful in
/// making parts of attestation and sync committee processing generic.
pub trait SlotData {
fn get_slot(&self) -> Slot;
}
impl SlotData for Slot {
fn get_slot(&self) -> Slot {
*self
}
}

View File

@ -11,10 +11,10 @@
//! may lead to programming errors which are not detected by the compiler.
use crate::test_utils::TestRandom;
use crate::SignedRoot;
use crate::{ChainSpec, SignedRoot};
use rand::RngCore;
use safe_arith::SafeArith;
use safe_arith::{ArithError, SafeArith};
use serde_derive::{Deserialize, Serialize};
use ssz::{ssz_encode, Decode, DecodeError, Encode};
use std::fmt;
@ -90,6 +90,13 @@ impl Epoch {
}
}
/// Compute the sync committee period for an epoch.
pub fn sync_committee_period(&self, spec: &ChainSpec) -> Result<u64, ArithError> {
Ok(self
.safe_div(spec.epochs_per_sync_committee_period)?
.as_u64())
}
pub fn slot_iter(&self, slots_per_epoch: u64) -> SlotIter {
SlotIter {
current_iteration: 0,

View File

@ -1,10 +1,24 @@
use crate::consts::altair::SYNC_COMMITTEE_SUBNET_COUNT;
use crate::test_utils::TestRandom;
use crate::{AggregateSignature, BitVector, EthSpec};
use crate::{AggregateSignature, BitVector, EthSpec, SyncCommitteeContribution};
use safe_arith::{ArithError, SafeArith};
use serde_derive::{Deserialize, Serialize};
use ssz_derive::{Decode, Encode};
use test_random_derive::TestRandom;
use tree_hash_derive::TreeHash;
#[derive(Debug, PartialEq)]
pub enum Error {
SszTypesError(ssz_types::Error),
ArithError(ArithError),
}
impl From<ArithError> for Error {
fn from(e: ArithError) -> Error {
Error::ArithError(e)
}
}
#[cfg_attr(feature = "arbitrary-fuzz", derive(arbitrary::Arbitrary))]
#[derive(Debug, PartialEq, Clone, Serialize, Deserialize, Encode, Decode, TreeHash, TestRandom)]
#[serde(bound = "T: EthSpec")]
@ -23,6 +37,34 @@ impl<T: EthSpec> SyncAggregate<T> {
}
}
/// Create a `SyncAggregate` from a slice of `SyncCommitteeContribution`s.
///
/// Equivalent to `process_sync_committee_contributions` from the spec.
pub fn from_contributions(
contributions: &[SyncCommitteeContribution<T>],
) -> Result<SyncAggregate<T>, Error> {
let mut sync_aggregate = Self::new();
let sync_subcommittee_size =
T::sync_committee_size().safe_div(SYNC_COMMITTEE_SUBNET_COUNT as usize)?;
for contribution in contributions {
for (index, participated) in contribution.aggregation_bits.iter().enumerate() {
if participated {
let participant_index = sync_subcommittee_size
.safe_mul(contribution.subcommittee_index as usize)?
.safe_add(index)?;
sync_aggregate
.sync_committee_bits
.set(participant_index, true)
.map_err(Error::SszTypesError)?;
}
}
sync_aggregate
.sync_committee_signature
.add_assign_aggregate(&contribution.signature);
}
Ok(sync_aggregate)
}
/// Empty aggregate to be used at genesis.
///
/// Contains an empty signature and should *not* be used as the starting point for aggregation,

View File

@ -0,0 +1,25 @@
use crate::test_utils::TestRandom;
use crate::{SignedRoot, Slot};
use serde_derive::{Deserialize, Serialize};
use ssz_derive::{Decode, Encode};
use test_random_derive::TestRandom;
use tree_hash_derive::TreeHash;
#[cfg_attr(feature = "arbitrary-fuzz", derive(arbitrary::Arbitrary))]
#[derive(
Debug, PartialEq, Clone, Serialize, Deserialize, Hash, Encode, Decode, TreeHash, TestRandom,
)]
pub struct SyncAggregatorSelectionData {
pub slot: Slot,
pub subcommittee_index: u64,
}
#[cfg(test)]
mod tests {
use super::*;
ssz_and_tree_hash_tests!(SyncAggregatorSelectionData);
}
impl SignedRoot for SyncAggregatorSelectionData {}

View File

@ -1,12 +1,30 @@
use crate::test_utils::TestRandom;
use crate::typenum::Unsigned;
use crate::{EthSpec, FixedVector};
use crate::{EthSpec, FixedVector, SyncSubnetId};
use bls::PublicKeyBytes;
use safe_arith::{ArithError, SafeArith};
use serde_derive::{Deserialize, Serialize};
use ssz_derive::{Decode, Encode};
use std::collections::HashMap;
use test_random_derive::TestRandom;
use tree_hash_derive::TreeHash;
#[derive(Debug, PartialEq)]
pub enum Error {
ArithError(ArithError),
InvalidSubcommitteeRange {
start_subcommittee_index: usize,
end_subcommittee_index: usize,
subcommittee_index: usize,
},
}
impl From<ArithError> for Error {
fn from(e: ArithError) -> Error {
Error::ArithError(e)
}
}
#[cfg_attr(feature = "arbitrary-fuzz", derive(arbitrary::Arbitrary))]
#[derive(Debug, PartialEq, Clone, Serialize, Deserialize, Encode, Decode, TreeHash, TestRandom)]
#[serde(bound = "T: EthSpec")]
@ -26,4 +44,44 @@ impl<T: EthSpec> SyncCommittee<T> {
aggregate_pubkey: PublicKeyBytes::empty(),
})
}
/// Return the pubkeys in this `SyncCommittee` for the given `subcommittee_index`.
pub fn get_subcommittee_pubkeys(
&self,
subcommittee_index: usize,
) -> Result<Vec<PublicKeyBytes>, Error> {
let start_subcommittee_index = subcommittee_index.safe_mul(T::sync_subcommittee_size())?;
let end_subcommittee_index =
start_subcommittee_index.safe_add(T::sync_subcommittee_size())?;
self.pubkeys
.get(start_subcommittee_index..end_subcommittee_index)
.ok_or(Error::InvalidSubcommitteeRange {
start_subcommittee_index,
end_subcommittee_index,
subcommittee_index,
})
.map(|s| s.to_vec())
}
/// For a given `pubkey`, finds all subcommittees that it is included in, and maps the
/// subcommittee index (typed as `SyncSubnetId`) to all positions this `pubkey` is associated
/// with within the subcommittee.
pub fn subcommittee_positions_for_public_key(
&self,
pubkey: &PublicKeyBytes,
) -> Result<HashMap<SyncSubnetId, Vec<usize>>, Error> {
let mut subnet_positions = HashMap::new();
for (committee_index, validator_pubkey) in self.pubkeys.iter().enumerate() {
if pubkey == validator_pubkey {
let subcommittee_index = committee_index.safe_div(T::sync_subcommittee_size())?;
let position_in_subcommittee =
committee_index.safe_rem(T::sync_subcommittee_size())?;
subnet_positions
.entry(SyncSubnetId::new(subcommittee_index as u64))
.or_insert_with(Vec::new)
.push(position_in_subcommittee);
}
}
Ok(subnet_positions)
}
}

View File

@ -0,0 +1,113 @@
use super::{AggregateSignature, EthSpec, SignedRoot};
use crate::slot_data::SlotData;
use crate::{test_utils::TestRandom, BitVector, Hash256, Slot, SyncCommitteeMessage};
use safe_arith::ArithError;
use serde_derive::{Deserialize, Serialize};
use ssz_derive::{Decode, Encode};
use test_random_derive::TestRandom;
use tree_hash_derive::TreeHash;
#[derive(Debug, PartialEq)]
pub enum Error {
SszTypesError(ssz_types::Error),
AlreadySigned(usize),
SubnetCountIsZero(ArithError),
}
/// An aggregation of `SyncCommitteeMessage`s, used in creating a `SignedContributionAndProof`.
#[cfg_attr(feature = "arbitrary-fuzz", derive(arbitrary::Arbitrary))]
#[derive(Debug, Clone, PartialEq, Serialize, Deserialize, Encode, Decode, TreeHash, TestRandom)]
#[serde(bound = "T: EthSpec")]
pub struct SyncCommitteeContribution<T: EthSpec> {
pub slot: Slot,
pub beacon_block_root: Hash256,
pub subcommittee_index: u64,
pub aggregation_bits: BitVector<T::SyncSubcommitteeSize>,
pub signature: AggregateSignature,
}
impl<T: EthSpec> SyncCommitteeContribution<T> {
/// Create a `SyncCommitteeContribution` from:
///
/// - `message`: A single `SyncCommitteeMessage`.
/// - `subcommittee_index`: The subcommittee this contribution pertains to out of the broader
/// sync committee. This can be determined from the `SyncSubnetId` of the gossip subnet
/// this message was seen on.
/// - `validator_sync_committee_index`: The index of the validator **within** the subcommittee.
pub fn from_message(
message: &SyncCommitteeMessage,
subcommittee_index: u64,
validator_sync_committee_index: usize,
) -> Result<Self, Error> {
let mut bits = BitVector::new();
bits.set(validator_sync_committee_index, true)
.map_err(Error::SszTypesError)?;
Ok(Self {
slot: message.slot,
beacon_block_root: message.beacon_block_root,
subcommittee_index,
aggregation_bits: bits,
signature: AggregateSignature::from(&message.signature),
})
}
/// Are the aggregation bitfields of these sync contribution disjoint?
pub fn signers_disjoint_from(&self, other: &Self) -> bool {
self.aggregation_bits
.intersection(&other.aggregation_bits)
.is_zero()
}
/// Aggregate another `SyncCommitteeContribution` into this one.
///
/// The aggregation bitfields must be disjoint, and the data must be the same.
pub fn aggregate(&mut self, other: &Self) {
debug_assert_eq!(self.slot, other.slot);
debug_assert_eq!(self.beacon_block_root, other.beacon_block_root);
debug_assert_eq!(self.subcommittee_index, other.subcommittee_index);
debug_assert!(self.signers_disjoint_from(other));
self.aggregation_bits = self.aggregation_bits.union(&other.aggregation_bits);
self.signature.add_assign_aggregate(&other.signature);
}
}
impl SignedRoot for Hash256 {}
/// This is not in the spec, but useful for determining uniqueness of sync committee contributions
#[derive(Debug, Clone, PartialEq, Serialize, Deserialize, Encode, Decode, TreeHash, TestRandom)]
pub struct SyncContributionData {
slot: Slot,
beacon_block_root: Hash256,
subcommittee_index: u64,
}
impl SyncContributionData {
pub fn from_contribution<T: EthSpec>(signing_data: &SyncCommitteeContribution<T>) -> Self {
Self {
slot: signing_data.slot,
beacon_block_root: signing_data.beacon_block_root,
subcommittee_index: signing_data.subcommittee_index,
}
}
}
impl<T: EthSpec> SlotData for SyncCommitteeContribution<T> {
fn get_slot(&self) -> Slot {
self.slot
}
}
impl SlotData for SyncContributionData {
fn get_slot(&self) -> Slot {
self.slot
}
}
#[cfg(test)]
mod tests {
use super::*;
use crate::*;
ssz_and_tree_hash_tests!(SyncCommitteeContribution<MainnetEthSpec>);
}

View File

@ -0,0 +1,57 @@
use crate::test_utils::TestRandom;
use crate::{ChainSpec, Domain, EthSpec, Fork, Hash256, SecretKey, Signature, SignedRoot, Slot};
use crate::slot_data::SlotData;
use serde_derive::{Deserialize, Serialize};
use ssz_derive::{Decode, Encode};
use test_random_derive::TestRandom;
use tree_hash_derive::TreeHash;
/// The data upon which a `SyncCommitteeContribution` is based.
#[cfg_attr(feature = "arbitrary-fuzz", derive(arbitrary::Arbitrary))]
#[derive(Debug, Clone, PartialEq, Serialize, Deserialize, Encode, Decode, TreeHash, TestRandom)]
pub struct SyncCommitteeMessage {
pub slot: Slot,
pub beacon_block_root: Hash256,
#[serde(with = "serde_utils::quoted_u64")]
pub validator_index: u64,
// Signature by the validator over `beacon_block_root`.
pub signature: Signature,
}
impl SyncCommitteeMessage {
/// Equivalent to `get_sync_committee_message` from the spec.
pub fn new<E: EthSpec>(
slot: Slot,
beacon_block_root: Hash256,
validator_index: u64,
secret_key: &SecretKey,
fork: &Fork,
genesis_validators_root: Hash256,
spec: &ChainSpec,
) -> Self {
let epoch = slot.epoch(E::slots_per_epoch());
let domain = spec.get_domain(epoch, Domain::SyncCommittee, fork, genesis_validators_root);
let message = beacon_block_root.signing_root(domain);
let signature = secret_key.sign(message);
Self {
slot,
beacon_block_root,
validator_index,
signature,
}
}
}
impl SlotData for SyncCommitteeMessage {
fn get_slot(&self) -> Slot {
self.slot
}
}
#[cfg(test)]
mod tests {
use super::*;
ssz_and_tree_hash_tests!(SyncCommitteeMessage);
}

View File

@ -0,0 +1,139 @@
use crate::consts::altair::{
SYNC_COMMITTEE_SUBNET_COUNT, TARGET_AGGREGATORS_PER_SYNC_SUBCOMMITTEE,
};
use crate::{
ChainSpec, Domain, EthSpec, Fork, Hash256, PublicKey, SecretKey, Signature, SignedRoot, Slot,
SyncAggregatorSelectionData,
};
use eth2_hashing::hash;
use safe_arith::{ArithError, SafeArith};
use ssz::Encode;
use ssz_types::typenum::Unsigned;
use std::cmp;
use std::convert::TryInto;
#[cfg_attr(feature = "arbitrary-fuzz", derive(arbitrary::Arbitrary))]
#[derive(PartialEq, Debug, Clone)]
pub struct SyncSelectionProof(Signature);
impl SyncSelectionProof {
pub fn new<T: EthSpec>(
slot: Slot,
subcommittee_index: u64,
secret_key: &SecretKey,
fork: &Fork,
genesis_validators_root: Hash256,
spec: &ChainSpec,
) -> Self {
let domain = spec.get_domain(
slot.epoch(T::slots_per_epoch()),
Domain::SyncCommitteeSelectionProof,
fork,
genesis_validators_root,
);
let message = SyncAggregatorSelectionData {
slot,
subcommittee_index,
}
.signing_root(domain);
Self(secret_key.sign(message))
}
/// Returns the "modulo" used for determining if a `SyncSelectionProof` elects an aggregator.
pub fn modulo<T: EthSpec>() -> Result<u64, ArithError> {
Ok(cmp::max(
1,
(T::SyncCommitteeSize::to_u64())
.safe_div(SYNC_COMMITTEE_SUBNET_COUNT)?
.safe_div(TARGET_AGGREGATORS_PER_SYNC_SUBCOMMITTEE)?,
))
}
pub fn is_aggregator<T: EthSpec>(&self) -> Result<bool, ArithError> {
self.is_aggregator_from_modulo(Self::modulo::<T>()?)
}
pub fn is_aggregator_from_modulo(&self, modulo: u64) -> Result<bool, ArithError> {
let signature_hash = hash(&self.0.as_ssz_bytes());
let signature_hash_int = u64::from_le_bytes(
signature_hash
.get(0..8)
.expect("hash is 32 bytes")
.try_into()
.expect("first 8 bytes of signature should always convert to fixed array"),
);
signature_hash_int.safe_rem(modulo).map(|rem| rem == 0)
}
pub fn verify<T: EthSpec>(
&self,
slot: Slot,
subcommittee_index: u64,
pubkey: &PublicKey,
fork: &Fork,
genesis_validators_root: Hash256,
spec: &ChainSpec,
) -> bool {
let domain = spec.get_domain(
slot.epoch(T::slots_per_epoch()),
Domain::SyncCommitteeSelectionProof,
fork,
genesis_validators_root,
);
let message = SyncAggregatorSelectionData {
slot,
subcommittee_index,
}
.signing_root(domain);
self.0.verify(pubkey, message)
}
}
impl Into<Signature> for SyncSelectionProof {
fn into(self) -> Signature {
self.0
}
}
impl From<Signature> for SyncSelectionProof {
fn from(sig: Signature) -> Self {
Self(sig)
}
}
#[cfg(test)]
mod test {
use super::*;
use crate::MainnetEthSpec;
use eth2_interop_keypairs::keypair;
#[test]
fn proof_sign_and_verify() {
let slot = Slot::new(1000);
let subcommittee_index = 12;
let key = keypair(1);
let fork = &Fork::default();
let genesis_validators_root = Hash256::zero();
let spec = &ChainSpec::mainnet();
let proof = SyncSelectionProof::new::<MainnetEthSpec>(
slot,
subcommittee_index,
&key.sk,
fork,
genesis_validators_root,
spec,
);
assert!(proof.verify::<MainnetEthSpec>(
slot,
subcommittee_index,
&key.pk,
fork,
genesis_validators_root,
spec
));
}
}

View File

@ -0,0 +1,74 @@
//! Identifies each sync committee subnet by an integer identifier.
use crate::consts::altair::SYNC_COMMITTEE_SUBNET_COUNT;
use serde_derive::{Deserialize, Serialize};
use std::ops::{Deref, DerefMut};
lazy_static! {
static ref SYNC_SUBNET_ID_TO_STRING: Vec<String> = {
let mut v = Vec::with_capacity(SYNC_COMMITTEE_SUBNET_COUNT as usize);
for i in 0..SYNC_COMMITTEE_SUBNET_COUNT {
v.push(i.to_string());
}
v
};
}
#[cfg_attr(feature = "arbitrary-fuzz", derive(arbitrary::Arbitrary))]
#[derive(Clone, Copy, Debug, PartialEq, Eq, Hash, Serialize, Deserialize)]
#[serde(transparent)]
pub struct SyncSubnetId(#[serde(with = "serde_utils::quoted_u64")] u64);
pub fn sync_subnet_id_to_string(i: u64) -> &'static str {
if i < SYNC_COMMITTEE_SUBNET_COUNT {
&SYNC_SUBNET_ID_TO_STRING
.get(i as usize)
.expect("index below SYNC_COMMITTEE_SUBNET_COUNT")
} else {
"sync subnet id out of range"
}
}
impl SyncSubnetId {
pub fn new(id: u64) -> Self {
id.into()
}
}
impl Deref for SyncSubnetId {
type Target = u64;
fn deref(&self) -> &Self::Target {
&self.0
}
}
impl DerefMut for SyncSubnetId {
fn deref_mut(&mut self) -> &mut Self::Target {
&mut self.0
}
}
impl From<u64> for SyncSubnetId {
fn from(x: u64) -> Self {
Self(x)
}
}
impl Into<u64> for SyncSubnetId {
fn into(self) -> u64 {
self.0
}
}
impl Into<u64> for &SyncSubnetId {
fn into(self) -> u64 {
self.0
}
}
impl AsRef<str> for SyncSubnetId {
fn as_ref(&self) -> &str {
sync_subnet_id_to_string(self.0)
}
}

View File

@ -3,6 +3,7 @@ use rand::RngCore;
use rand::SeedableRng;
use rand_xorshift::XorShiftRng;
use ssz_types::typenum::Unsigned;
use std::sync::Arc;
mod address;
mod aggregate_signature;
@ -68,6 +69,15 @@ where
}
}
impl<U> TestRandom for Arc<U>
where
U: TestRandom,
{
fn random_for_test(rng: &mut impl RngCore) -> Self {
Arc::new(U::random_for_test(rng))
}
}
impl<T, N: Unsigned> TestRandom for FixedVector<T, N>
where
T: TestRandom,

View File

@ -219,6 +219,20 @@ where
}
}
/// Allow aggregate signatures to be created from single signatures.
impl<Pub, AggPub, Sig, AggSig> From<&GenericSignature<Pub, Sig>>
for GenericAggregateSignature<Pub, AggPub, Sig, AggSig>
where
Sig: TSignature<Pub>,
AggSig: TAggregateSignature<Pub, AggPub, Sig>,
{
fn from(sig: &GenericSignature<Pub, Sig>) -> Self {
let mut agg = Self::infinity();
agg.add_assign(&sig);
agg
}
}
impl<Pub, AggPub, Sig, AggSig> Encode for GenericAggregateSignature<Pub, AggPub, Sig, AggSig>
where
Sig: TSignature<Pub>,

View File

@ -1,4 +1,4 @@
TESTS_TAG := v1.1.0-alpha.7
TESTS_TAG := v1.1.0-alpha.8
TESTS = general minimal mainnet
TARBALLS = $(patsubst %,%-$(TESTS_TAG).tar.gz,$(TESTS))

View File

@ -45,24 +45,6 @@ excluded_paths = [
# LightClientSnapshot
"tests/minimal/altair/ssz_static/LightClientSnapshot",
"tests/mainnet/altair/ssz_static/LightClientSnapshot",
# ContributionAndProof
"tests/minimal/altair/ssz_static/ContributionAndProof",
"tests/mainnet/altair/ssz_static/ContributionAndProof",
# SignedContributionAndProof
"tests/minimal/altair/ssz_static/SignedContributionAndProof",
"tests/mainnet/altair/ssz_static/SignedContributionAndProof",
# SyncCommitteeContribution
"tests/minimal/altair/ssz_static/SyncCommitteeContribution",
"tests/mainnet/altair/ssz_static/SyncCommitteeContribution",
# SyncCommitteeMessage
"tests/minimal/altair/ssz_static/SyncCommitteeMessage",
"tests/mainnet/altair/ssz_static/SyncCommitteeMessage",
# SyncCommitteeSigningData
"tests/minimal/altair/ssz_static/SyncCommitteeSigningData",
"tests/mainnet/altair/ssz_static/SyncCommitteeSigningData",
# SyncAggregatorSelectionData
"tests/minimal/altair/ssz_static/SyncAggregatorSelectionData",
"tests/mainnet/altair/ssz_static/SyncAggregatorSelectionData",
# Fork choice
"tests/mainnet/phase0/fork_choice",
"tests/minimal/phase0/fork_choice",

View File

@ -51,6 +51,7 @@ type_name_generic!(BeaconState);
type_name_generic!(BeaconStateBase, "BeaconState");
type_name_generic!(BeaconStateAltair, "BeaconState");
type_name!(Checkpoint);
type_name_generic!(ContributionAndProof);
type_name!(Deposit);
type_name!(DepositData);
type_name!(DepositMessage);
@ -64,8 +65,12 @@ type_name!(ProposerSlashing);
type_name_generic!(SignedAggregateAndProof);
type_name_generic!(SignedBeaconBlock);
type_name!(SignedBeaconBlockHeader);
type_name_generic!(SignedContributionAndProof);
type_name!(SignedVoluntaryExit);
type_name!(SigningData);
type_name_generic!(SyncCommitteeContribution);
type_name!(SyncCommitteeMessage);
type_name!(SyncAggregatorSelectionData);
type_name_generic!(SyncAggregate);
type_name_generic!(SyncCommittee);
type_name!(Validator);

View File

@ -216,6 +216,20 @@ mod ssz_static {
}
// Altair-only
#[test]
fn contribution_and_proof() {
SszStaticHandler::<ContributionAndProof<MinimalEthSpec>, MinimalEthSpec>::altair_only()
.run();
SszStaticHandler::<ContributionAndProof<MainnetEthSpec>, MainnetEthSpec>::altair_only()
.run();
}
#[test]
fn signed_contribution_and_proof() {
SszStaticHandler::<SignedContributionAndProof<MinimalEthSpec>, MinimalEthSpec>::altair_only().run();
SszStaticHandler::<SignedContributionAndProof<MainnetEthSpec>, MainnetEthSpec>::altair_only().run();
}
#[test]
fn sync_aggregate() {
SszStaticHandler::<SyncAggregate<MinimalEthSpec>, MinimalEthSpec>::altair_only().run();
@ -227,6 +241,28 @@ mod ssz_static {
SszStaticHandler::<SyncCommittee<MinimalEthSpec>, MinimalEthSpec>::altair_only().run();
SszStaticHandler::<SyncCommittee<MainnetEthSpec>, MainnetEthSpec>::altair_only().run();
}
#[test]
fn sync_committee_contribution() {
SszStaticHandler::<SyncCommitteeContribution<MinimalEthSpec>, MinimalEthSpec>::altair_only(
)
.run();
SszStaticHandler::<SyncCommitteeContribution<MainnetEthSpec>, MainnetEthSpec>::altair_only(
)
.run();
}
#[test]
fn sync_committee_message() {
SszStaticHandler::<SyncCommitteeMessage, MinimalEthSpec>::altair_only().run();
SszStaticHandler::<SyncCommitteeMessage, MainnetEthSpec>::altair_only().run();
}
#[test]
fn sync_aggregator_selection_data() {
SszStaticHandler::<SyncAggregatorSelectionData, MinimalEthSpec>::altair_only().run();
SszStaticHandler::<SyncAggregatorSelectionData, MainnetEthSpec>::altair_only().run();
}
}
#[test]