From ef7db9acdedfc29bd02043d85ad4c3e13acf2777 Mon Sep 17 00:00:00 2001 From: Eitan Seri-Levi Date: Fri, 17 May 2024 09:47:38 +0300 Subject: [PATCH] - get attestation related beacon chain tests to pass - observed attestations are now keyed off of data + committee index - rename op pool attestationref to compactattestationref - remove unwraps in agg pool and use options instead - cherry pick some changes from ef-tests-electra --- Dockerfile | 2 +- .../src/attestation_verification.rs | 184 +++++++++--- .../src/attestation_verification/batch.rs | 1 - beacon_node/beacon_chain/src/beacon_chain.rs | 28 +- .../beacon_chain/src/early_attester_cache.rs | 1 - beacon_node/beacon_chain/src/errors.rs | 2 + .../src/naive_aggregation_pool.rs | 6 +- beacon_node/beacon_chain/src/test_utils.rs | 279 +++++++++++++----- .../tests/attestation_verification.rs | 27 +- .../beacon_chain/tests/block_verification.rs | 83 ++++-- .../tests/payload_invalidation.rs | 5 +- beacon_node/beacon_chain/tests/store_tests.rs | 33 ++- .../tests/sync_committee_verification.rs | 3 +- beacon_node/http_api/src/lib.rs | 4 +- .../http_api/src/publish_attestations.rs | 2 +- .../lighthouse_network/src/types/pubsub.rs | 4 +- .../gossip_methods.rs | 34 +++ beacon_node/operation_pool/src/attestation.rs | 20 +- .../operation_pool/src/attestation_storage.rs | 118 ++++---- beacon_node/operation_pool/src/lib.rs | 12 +- consensus/fork_choice/tests/tests.rs | 2 +- .../src/common/get_attesting_indices.rs | 97 +++++- .../per_block_processing/signature_sets.rs | 2 +- consensus/types/src/attestation.rs | 42 +-- consensus/types/src/beacon_state.rs | 6 + consensus/types/src/indexed_attestation.rs | 15 - .../types/src/signed_aggregate_and_proof.rs | 1 - .../src/test_utils/test_random/bitfield.rs | 3 + validator_client/src/attestation_service.rs | 48 ++- 29 files changed, 759 insertions(+), 305 deletions(-) diff --git a/Dockerfile b/Dockerfile index 901c1b83d63..8479e850ef5 100644 --- a/Dockerfile +++ b/Dockerfile @@ -1,7 +1,7 @@ FROM rust:1.75.0-bullseye AS builder RUN apt-get update && apt-get -y upgrade && apt-get install -y cmake libclang-dev COPY . lighthouse -ARG FEATURES +ARG FEATURES=spec-minimal ARG PROFILE=release ARG CARGO_USE_GIT_CLI=true ENV FEATURES $FEATURES diff --git a/beacon_node/beacon_chain/src/attestation_verification.rs b/beacon_node/beacon_chain/src/attestation_verification.rs index 9c44c5529b3..a8460caa322 100644 --- a/beacon_node/beacon_chain/src/attestation_verification.rs +++ b/beacon_node/beacon_chain/src/attestation_verification.rs @@ -45,7 +45,10 @@ use proto_array::Block as ProtoBlock; use slog::debug; use slot_clock::SlotClock; use state_processing::{ - common::{attesting_indices_base, attesting_indices_electra}, + common::{ + attesting_indices_base, + attesting_indices_electra::{self, get_committee_indices}, + }, per_block_processing::errors::{AttestationValidationError, BlockOperationError}, signature_sets::{ indexed_attestation_signature_set_from_pubkeys, @@ -55,10 +58,11 @@ use state_processing::{ use std::borrow::Cow; use strum::AsRefStr; use tree_hash::TreeHash; +use tree_hash_derive::TreeHash; use types::{ - Attestation, AttestationRef, BeaconCommittee, BeaconStateError::NoCommitteeFound, ChainSpec, - CommitteeIndex, Epoch, EthSpec, ForkName, Hash256, IndexedAttestation, SelectionProof, - SignedAggregateAndProof, Slot, SubnetId, + Attestation, AttestationData, AttestationRef, BeaconCommittee, BeaconStateError, + BeaconStateError::NoCommitteeFound, ChainSpec, CommitteeIndex, Epoch, EthSpec, ForkName, + Hash256, IndexedAttestation, SelectionProof, SignedAggregateAndProof, Slot, SubnetId, }; pub use batch::{batch_verify_aggregated_attestations, batch_verify_unaggregated_attestations}; @@ -139,6 +143,12 @@ pub enum Error { /// /// The peer has sent an invalid message. ValidatorIndexTooHigh(usize), + /// The validator index is not set to zero after Electra. + /// + /// ## Peer scoring + /// + /// The peer has sent an invalid message. + CommitteeIndexNonZero(usize), /// The `attestation.data.beacon_block_root` block is unknown. /// /// ## Peer scoring @@ -187,6 +197,12 @@ pub enum Error { /// /// The peer has sent an invalid message. NotExactlyOneAggregationBitSet(usize), + /// The attestation doesn't have only one aggregation bit set. + /// + /// ## Peer scoring + /// + /// The peer has sent an invalid message. + NotExactlyOneCommitteeBitSet(usize), /// We have already observed an attestation for the `validator_index` and refuse to process /// another. /// @@ -248,9 +264,30 @@ pub enum Error { BeaconChainError(BeaconChainError), } +// TODO(electra) the error conversion changes here are to get a test case to pass +// this could easily be cleaned up impl From for Error { fn from(e: BeaconChainError) -> Self { - Error::BeaconChainError(e) + match &e { + BeaconChainError::BeaconStateError(beacon_state_error) => { + if let BeaconStateError::AggregatorNotInCommittee { aggregator_index } = + beacon_state_error + { + Self::AggregatorNotInCommittee { + aggregator_index: *aggregator_index, + } + } else if let BeaconStateError::InvalidSelectionProof { aggregator_index } = + beacon_state_error + { + Self::InvalidSelectionProof { + aggregator_index: *aggregator_index, + } + } else { + Error::BeaconChainError(e) + } + } + _ => Error::BeaconChainError(e), + } } } @@ -265,10 +302,17 @@ enum CheckAttestationSignature { /// `IndexedAttestation` can be derived. /// /// These attestations have *not* undergone signature verification. +/// The `observed_attestation_key_root` is the hashed value of an `ObservedAttestationKey`. struct IndexedAggregatedAttestation<'a, T: BeaconChainTypes> { signed_aggregate: &'a SignedAggregateAndProof, indexed_attestation: IndexedAttestation, - attestation_data_root: Hash256, + observed_attestation_key_root: Hash256, +} + +#[derive(TreeHash)] +pub struct ObservedAttestationKey { + pub committee_index: u64, + pub attestation_data: AttestationData, } /// Wraps a `Attestation` that has been verified up until the point that an `IndexedAttestation` can @@ -466,18 +510,34 @@ impl<'a, T: BeaconChainTypes> IndexedAggregatedAttestation<'a, T> { }); } +<<<<<<< HEAD + let observed_attestation_key_root = ObservedAttestationKey { + committee_index: attestation.committee_index(), + attestation_data: attestation.data().clone(), + } + .tree_hash_root(); + + // [New in Electra:EIP7549] + verify_committee_index(attestation, &chain.spec)?; +======= + // [New in Electra:EIP7549] + verify_committee_index(attestation, &chain.spec)?; + // Ensure the valid aggregated attestation has not already been seen locally. let attestation_data = attestation.data(); let attestation_data_root = attestation_data.tree_hash_root(); +>>>>>>> f5eb33d54 (add new gossip conditions) if chain .observed_attestations .write() - .is_known_subset(attestation, attestation_data_root) + .is_known_subset(attestation, observed_attestation_key_root) .map_err(|e| Error::BeaconChainError(e.into()))? { metrics::inc_counter(&metrics::AGGREGATED_ATTESTATION_SUBSETS); - return Err(Error::AttestationSupersetKnown(attestation_data_root)); + return Err(Error::AttestationSupersetKnown( + observed_attestation_key_root, + )); } let aggregator_index = signed_aggregate.message().aggregator_index(); @@ -523,7 +583,7 @@ impl<'a, T: BeaconChainTypes> IndexedAggregatedAttestation<'a, T> { if attestation.is_aggregation_bits_zero() { Err(Error::EmptyAggregationBitfield) } else { - Ok(attestation_data_root) + Ok(observed_attestation_key_root) } } @@ -533,10 +593,8 @@ impl<'a, T: BeaconChainTypes> IndexedAggregatedAttestation<'a, T> { chain: &BeaconChain, ) -> Result> { use AttestationSlashInfo::*; - - let attestation = signed_aggregate.message().aggregate(); - let aggregator_index = signed_aggregate.message().aggregator_index(); - let attestation_data_root = match Self::verify_early_checks(signed_aggregate, chain) { + let observed_attestation_key_root = match Self::verify_early_checks(signed_aggregate, chain) + { Ok(root) => root, Err(e) => { return Err(SignatureNotChecked( @@ -545,11 +603,12 @@ impl<'a, T: BeaconChainTypes> IndexedAggregatedAttestation<'a, T> { )) } }; - let get_indexed_attestation_with_committee = |(committees, _): (Vec, CommitteesPerSlot)| { - match attestation { - AttestationRef::Base(att) => { + match signed_aggregate { + SignedAggregateAndProof::Base(signed_aggregate) => { + let att = &signed_aggregate.message.aggregate; + let aggregator_index = signed_aggregate.message.aggregator_index; let committee = committees .iter() .filter(|&committee| committee.index == att.data.index) @@ -559,13 +618,13 @@ impl<'a, T: BeaconChainTypes> IndexedAggregatedAttestation<'a, T> { index: att.data.index, })?; + // TODO(electra): + // Note: this clones the signature which is known to be a relatively slow operation. + // + // Future optimizations should remove this clone. if let Some(committee) = committee { - // TODO(electra): - // Note: this clones the signature which is known to be a relatively slow operation. - // - // Future optimizations should remove this clone. let selection_proof = SelectionProof::from( - signed_aggregate.message().selection_proof().clone(), + signed_aggregate.message.selection_proof.clone(), ); if !selection_proof @@ -579,6 +638,7 @@ impl<'a, T: BeaconChainTypes> IndexedAggregatedAttestation<'a, T> { if !committee.committee.contains(&(aggregator_index as usize)) { return Err(Error::AggregatorNotInCommittee { aggregator_index }); } + attesting_indices_base::get_indexed_attestation( committee.committee, att, @@ -591,13 +651,18 @@ impl<'a, T: BeaconChainTypes> IndexedAggregatedAttestation<'a, T> { }) } } - AttestationRef::Electra(att) => { - attesting_indices_electra::get_indexed_attestation(&committees, att) - .map_err(|e| BeaconChainError::from(e).into()) + SignedAggregateAndProof::Electra(signed_aggregate) => { + attesting_indices_electra::get_indexed_attestation_from_signed_aggregate( + &committees, + signed_aggregate, + &chain.spec, + ) + .map_err(|e| BeaconChainError::from(e).into()) } } }; + let attestation = signed_aggregate.message().aggregate(); let indexed_attestation = match map_attestation_committees( chain, attestation, @@ -611,11 +676,10 @@ impl<'a, T: BeaconChainTypes> IndexedAggregatedAttestation<'a, T> { )) } }; - Ok(IndexedAggregatedAttestation { signed_aggregate, indexed_attestation, - attestation_data_root, + observed_attestation_key_root, }) } } @@ -624,7 +688,7 @@ impl<'a, T: BeaconChainTypes> VerifiedAggregatedAttestation<'a, T> { /// Run the checks that happen after the indexed attestation and signature have been checked. fn verify_late_checks( signed_aggregate: &SignedAggregateAndProof, - attestation_data_root: Hash256, + observed_attestation_key_root: Hash256, chain: &BeaconChain, ) -> Result<(), Error> { let attestation = signed_aggregate.message().aggregate(); @@ -637,11 +701,13 @@ impl<'a, T: BeaconChainTypes> VerifiedAggregatedAttestation<'a, T> { if let ObserveOutcome::Subset = chain .observed_attestations .write() - .observe_item(attestation, Some(attestation_data_root)) + .observe_item(attestation, Some(observed_attestation_key_root)) .map_err(|e| Error::BeaconChainError(e.into()))? { metrics::inc_counter(&metrics::AGGREGATED_ATTESTATION_SUBSETS); - return Err(Error::AttestationSupersetKnown(attestation_data_root)); + return Err(Error::AttestationSupersetKnown( + observed_attestation_key_root, + )); } // Observe the aggregator so we don't process another aggregate from them. @@ -701,7 +767,7 @@ impl<'a, T: BeaconChainTypes> VerifiedAggregatedAttestation<'a, T> { let IndexedAggregatedAttestation { signed_aggregate, indexed_attestation, - attestation_data_root, + observed_attestation_key_root, } = signed_aggregate; match check_signature { @@ -725,7 +791,9 @@ impl<'a, T: BeaconChainTypes> VerifiedAggregatedAttestation<'a, T> { CheckAttestationSignature::No => (), }; - if let Err(e) = Self::verify_late_checks(signed_aggregate, attestation_data_root, chain) { + if let Err(e) = + Self::verify_late_checks(signed_aggregate, observed_attestation_key_root, chain) + { return Err(SignatureValid(indexed_attestation, e)); } @@ -775,6 +843,9 @@ impl<'a, T: BeaconChainTypes> IndexedUnaggregatedAttestation<'a, T> { return Err(Error::NotExactlyOneAggregationBitSet(num_aggregation_bits)); } + // [New in Electra:EIP7549] + verify_committee_index(attestation, &chain.spec)?; + // Attestations must be for a known block. If the block is unknown, we simply drop the // attestation and do not delay consideration for later. // @@ -1101,14 +1172,13 @@ pub fn verify_propagation_slot_range( let current_fork = spec.fork_name_at_slot::(slot_clock.now().ok_or(BeaconChainError::UnableToReadSlot)?); - let earliest_permissible_slot = match current_fork { - ForkName::Base | ForkName::Altair | ForkName::Bellatrix | ForkName::Capella => { - one_epoch_prior - } - // EIP-7045 - ForkName::Deneb | ForkName::Electra => one_epoch_prior + let earliest_permissible_slot = if current_fork < ForkName::Deneb { + one_epoch_prior + // EIP-7045 + } else { + one_epoch_prior .epoch(E::slots_per_epoch()) - .start_slot(E::slots_per_epoch()), + .start_slot(E::slots_per_epoch()) }; if attestation_slot < earliest_permissible_slot { @@ -1147,7 +1217,6 @@ pub fn verify_attestation_signature( &chain.spec, ) .map_err(BeaconChainError::SignatureSetError)?; - metrics::stop_timer(signature_setup_timer); let _signature_verification_timer = @@ -1273,6 +1342,35 @@ pub fn verify_signed_aggregate_signatures( Ok(verify_signature_sets(signature_sets.iter())) } +/// Verify that the `attestation` committee index is properly set for the attestation's fork. +/// This function will only apply verification post-Electra. +pub fn verify_committee_index( + attestation: AttestationRef, + spec: &ChainSpec, +) -> Result<(), Error> { + if spec.fork_name_at_slot::(attestation.data().slot) >= ForkName::Electra { + // Check to ensure that the attestation is for a single committee. + let num_committee_bits = get_committee_indices::( + attestation + .committee_bits() + .map_err(|e| Error::BeaconChainError(e.into()))?, + ); + if num_committee_bits.len() != 1 { + return Err(Error::NotExactlyOneCommitteeBitSet( + num_committee_bits.len(), + )); + } + + // Ensure the attestation index is set to zero post Electra. + if attestation.data().index != 0 { + return Err(Error::CommitteeIndexNonZero( + attestation.data().index as usize, + )); + } + } + Ok(()) +} + /// Assists in readability. type CommitteesPerSlot = u64; @@ -1324,16 +1422,14 @@ pub fn obtain_indexed_attestation_and_committees_per_slot( }) } -// TODO(electra) update comments below to reflect logic changes -// i.e. this now runs the map_fn on a list of committees for the slot of the provided attestation /// Runs the `map_fn` with the committee and committee count per slot for the given `attestation`. /// -/// This function exists in this odd "map" pattern because efficiently obtaining the committee for -/// an attestation can be complex. It might involve reading straight from the +/// This function exists in this odd "map" pattern because efficiently obtaining the committees for +/// an attestations slot can be complex. It might involve reading straight from the /// `beacon_chain.shuffling_cache` or it might involve reading it from a state from the DB. Due to /// the complexities of `RwLock`s on the shuffling cache, a simple `Cow` isn't suitable here. /// -/// If the committee for `attestation` isn't found in the `shuffling_cache`, we will read a state +/// If the committees for an `attestation`'s slot isn't found in the `shuffling_cache`, we will read a state /// from disk and then update the `shuffling_cache`. fn map_attestation_committees( chain: &BeaconChain, diff --git a/beacon_node/beacon_chain/src/attestation_verification/batch.rs b/beacon_node/beacon_chain/src/attestation_verification/batch.rs index 1ec752ff5c5..07fad1bd4a8 100644 --- a/beacon_node/beacon_chain/src/attestation_verification/batch.rs +++ b/beacon_node/beacon_chain/src/attestation_verification/batch.rs @@ -66,7 +66,6 @@ where .ok_or(BeaconChainError::ValidatorPubkeyCacheLockTimeout)?; let mut signature_sets = Vec::with_capacity(num_indexed * 3); - // Iterate, flattening to get only the `Ok` values. for indexed in indexing_results.iter().flatten() { let signed_aggregate = &indexed.signed_aggregate; diff --git a/beacon_node/beacon_chain/src/beacon_chain.rs b/beacon_node/beacon_chain/src/beacon_chain.rs index 9cbb46afe93..ffae19205a7 100644 --- a/beacon_node/beacon_chain/src/beacon_chain.rs +++ b/beacon_node/beacon_chain/src/beacon_chain.rs @@ -85,7 +85,9 @@ use futures::channel::mpsc::Sender; use itertools::process_results; use itertools::Itertools; use kzg::Kzg; -use operation_pool::{AttestationRef, OperationPool, PersistedOperationPool, ReceivedPreCapella}; +use operation_pool::{ + CompactAttestationRef, OperationPool, PersistedOperationPool, ReceivedPreCapella, +}; use parking_lot::{Mutex, RwLock}; use proto_array::{DoNotReOrg, ProposerHeadError}; use safe_arith::SafeArith; @@ -1609,6 +1611,20 @@ impl BeaconChain { Ok((duties, dependent_root, execution_status)) } + pub fn get_aggregated_attestation( + &self, + attestation: &AttestationRef, + ) -> Result>, Error> { + match attestation { + AttestationRef::Base(att) => self.get_aggregated_attestation_base(&att.data), + AttestationRef::Electra(att) => self.get_aggregated_attestation_electra( + att.data.slot, + &att.data.tree_hash_root(), + att.committee_index(), + ), + } + } + /// Returns an aggregated `Attestation`, if any, that has a matching `attestation.data`. /// /// The attestation will be obtained from `self.naive_aggregation_pool`. @@ -2185,7 +2201,7 @@ impl BeaconChain { self.log, "Stored unaggregated attestation"; "outcome" => ?outcome, - "index" => attestation.data().index, + "index" => attestation.committee_index(), "slot" => attestation.data().slot.as_u64(), ), Err(NaiveAggregationError::SlotTooLow { @@ -2204,7 +2220,7 @@ impl BeaconChain { self.log, "Failed to store unaggregated attestation"; "error" => ?e, - "index" => attestation.data().index, + "index" => attestation.committee_index(), "slot" => attestation.data().slot.as_u64(), ); return Err(Error::from(e).into()); @@ -2329,7 +2345,7 @@ impl BeaconChain { pub fn filter_op_pool_attestation( &self, filter_cache: &mut HashMap<(Hash256, Epoch), bool>, - att: &AttestationRef, + att: &CompactAttestationRef, state: &BeaconState, ) -> bool { *filter_cache @@ -4940,11 +4956,11 @@ impl BeaconChain { initialize_epoch_cache(&mut state, &self.spec)?; let mut prev_filter_cache = HashMap::new(); - let prev_attestation_filter = |att: &AttestationRef| { + let prev_attestation_filter = |att: &CompactAttestationRef| { self.filter_op_pool_attestation(&mut prev_filter_cache, att, &state) }; let mut curr_filter_cache = HashMap::new(); - let curr_attestation_filter = |att: &AttestationRef| { + let curr_attestation_filter = |att: &CompactAttestationRef| { self.filter_op_pool_attestation(&mut curr_filter_cache, att, &state) }; diff --git a/beacon_node/beacon_chain/src/early_attester_cache.rs b/beacon_node/beacon_chain/src/early_attester_cache.rs index 8ed4e5db40b..936f4de3ee1 100644 --- a/beacon_node/beacon_chain/src/early_attester_cache.rs +++ b/beacon_node/beacon_chain/src/early_attester_cache.rs @@ -123,7 +123,6 @@ impl EarlyAttesterCache { item.committee_lengths .get_committee_length::(request_slot, request_index, spec)?; - // TODO(electra) make fork-agnostic let attestation = if spec.fork_name_at_slot::(request_slot) >= ForkName::Electra { let mut committee_bits = BitVector::default(); if committee_len > 0 { diff --git a/beacon_node/beacon_chain/src/errors.rs b/beacon_node/beacon_chain/src/errors.rs index 340f1f9f797..b210536a628 100644 --- a/beacon_node/beacon_chain/src/errors.rs +++ b/beacon_node/beacon_chain/src/errors.rs @@ -226,6 +226,7 @@ pub enum BeaconChainError { LightClientError(LightClientError), UnsupportedFork, MilhouseError(MilhouseError), + AttestationError(AttestationError), } easy_from_to!(SlotProcessingError, BeaconChainError); @@ -256,6 +257,7 @@ easy_from_to!(AvailabilityCheckError, BeaconChainError); easy_from_to!(EpochCacheError, BeaconChainError); easy_from_to!(LightClientError, BeaconChainError); easy_from_to!(MilhouseError, BeaconChainError); +easy_from_to!(AttestationError, BeaconChainError); #[derive(Debug)] pub enum BlockProductionError { diff --git a/beacon_node/beacon_chain/src/naive_aggregation_pool.rs b/beacon_node/beacon_chain/src/naive_aggregation_pool.rs index a1c736cd0ef..b16ced789ff 100644 --- a/beacon_node/beacon_chain/src/naive_aggregation_pool.rs +++ b/beacon_node/beacon_chain/src/naive_aggregation_pool.rs @@ -261,9 +261,9 @@ impl AggregateMap for AggregatedAttestationMap { }; let attestation_key = AttestationKey::from_attestation_ref(a)?; - let attestation_data_root = attestation_key.tree_hash_root(); + let attestation_key_root = attestation_key.tree_hash_root(); - if let Some(existing_attestation) = self.map.get_mut(&attestation_data_root) { + if let Some(existing_attestation) = self.map.get_mut(&attestation_key_root) { if existing_attestation .get_aggregation_bit(aggregation_bit) .map_err(|_| Error::InconsistentBitfieldLengths)? @@ -285,7 +285,7 @@ impl AggregateMap for AggregatedAttestationMap { } self.map - .insert(attestation_data_root, a.clone_as_attestation()); + .insert(attestation_key_root, a.clone_as_attestation()); Ok(InsertOutcome::NewItemInserted { committee_index: aggregation_bit, }) diff --git a/beacon_node/beacon_chain/src/test_utils.rs b/beacon_node/beacon_chain/src/test_utils.rs index 77bdf7bcc2c..783aab99d42 100644 --- a/beacon_node/beacon_chain/src/test_utils.rs +++ b/beacon_node/beacon_chain/src/test_utils.rs @@ -1343,7 +1343,10 @@ where // If there are any attestations in this committee, create an aggregate. if let Some((attestation, _)) = committee_attestations.first() { let bc = state - .get_beacon_committee(attestation.data().slot, attestation.data().index) + .get_beacon_committee( + attestation.data().slot, + attestation.committee_index(), + ) .unwrap(); // Find an aggregator if one exists. Return `None` if there are no @@ -1370,21 +1373,42 @@ where }) .copied()?; + let fork_name = self.spec.fork_name_at_slot::(slot); + + let aggregate = if fork_name >= ForkName::Electra { + self.chain + .get_aggregated_attestation_electra( + slot, + &attestation.data().tree_hash_root(), + bc.index, + ) + .unwrap() + .unwrap_or_else(|| { + committee_attestations.iter().skip(1).fold( + attestation.clone(), + |mut agg, (att, _)| { + agg.aggregate(att.to_ref()); + agg + }, + ) + }) + } else { + self.chain + .get_aggregated_attestation_base(attestation.data()) + .unwrap() + .unwrap_or_else(|| { + committee_attestations.iter().skip(1).fold( + attestation.clone(), + |mut agg, (att, _)| { + agg.aggregate(att.to_ref()); + agg + }, + ) + }) + }; + // If the chain is able to produce an aggregate, use that. Otherwise, build an // aggregate locally. - let aggregate = self - .chain - .get_aggregated_attestation_base(attestation.data()) - .unwrap() - .unwrap_or_else(|| { - committee_attestations.iter().skip(1).fold( - attestation.clone(), - |mut agg, (att, _)| { - agg.aggregate(att.to_ref()); - agg - }, - ) - }); let signed_aggregate = SignedAggregateAndProof::from_aggregate( aggregator_index as u64, @@ -1514,54 +1538,101 @@ where ) -> AttesterSlashing { let fork = self.chain.canonical_head.cached_head().head_fork(); - // TODO(electra): consider making this test fork-agnostic - let mut attestation_1 = IndexedAttestationBase { - attesting_indices: VariableList::new(validator_indices).unwrap(), - data: AttestationData { - slot: Slot::new(0), - index: 0, - beacon_block_root: Hash256::zero(), - target: Checkpoint { - root: Hash256::zero(), - epoch: target1.unwrap_or(fork.epoch), + let fork_name = self.spec.fork_name_at_slot::(Slot::new(0)); + + let mut attestation_1 = if fork_name >= ForkName::Electra { + IndexedAttestation::Electra(IndexedAttestationElectra { + attesting_indices: VariableList::new(validator_indices).unwrap(), + data: AttestationData { + slot: Slot::new(0), + index: 0, + beacon_block_root: Hash256::zero(), + target: Checkpoint { + root: Hash256::zero(), + epoch: target1.unwrap_or(fork.epoch), + }, + source: Checkpoint { + root: Hash256::zero(), + epoch: source1.unwrap_or(Epoch::new(0)), + }, }, - source: Checkpoint { - root: Hash256::zero(), - epoch: source1.unwrap_or(Epoch::new(0)), + signature: AggregateSignature::infinity(), + }) + } else { + IndexedAttestation::Base(IndexedAttestationBase { + attesting_indices: VariableList::new(validator_indices).unwrap(), + data: AttestationData { + slot: Slot::new(0), + index: 0, + beacon_block_root: Hash256::zero(), + target: Checkpoint { + root: Hash256::zero(), + epoch: target1.unwrap_or(fork.epoch), + }, + source: Checkpoint { + root: Hash256::zero(), + epoch: source1.unwrap_or(Epoch::new(0)), + }, }, - }, - signature: AggregateSignature::infinity(), + signature: AggregateSignature::infinity(), + }) }; let mut attestation_2 = attestation_1.clone(); - attestation_2.data.index += 1; - attestation_2.data.source.epoch = source2.unwrap_or(Epoch::new(0)); - attestation_2.data.target.epoch = target2.unwrap_or(fork.epoch); + attestation_2.data_mut().index += 1; + attestation_2.data_mut().source.epoch = source2.unwrap_or(Epoch::new(0)); + attestation_2.data_mut().target.epoch = target2.unwrap_or(fork.epoch); for attestation in &mut [&mut attestation_1, &mut attestation_2] { - // TODO(electra) we could explore iter mut here - for i in attestation.attesting_indices.iter() { - let sk = &self.validator_keypairs[*i as usize].sk; + match attestation { + IndexedAttestation::Base(attestation) => { + for i in attestation.attesting_indices.iter() { + let sk = &self.validator_keypairs[*i as usize].sk; - let genesis_validators_root = self.chain.genesis_validators_root; + let genesis_validators_root = self.chain.genesis_validators_root; - let domain = self.chain.spec.get_domain( - attestation.data.target.epoch, - Domain::BeaconAttester, - &fork, - genesis_validators_root, - ); - let message = attestation.data.signing_root(domain); + let domain = self.chain.spec.get_domain( + attestation.data.target.epoch, + Domain::BeaconAttester, + &fork, + genesis_validators_root, + ); + let message = attestation.data.signing_root(domain); + + attestation.signature.add_assign(&sk.sign(message)); + } + } + IndexedAttestation::Electra(attestation) => { + for i in attestation.attesting_indices.iter() { + let sk = &self.validator_keypairs[*i as usize].sk; - attestation.signature.add_assign(&sk.sign(message)); + let genesis_validators_root = self.chain.genesis_validators_root; + + let domain = self.chain.spec.get_domain( + attestation.data.target.epoch, + Domain::BeaconAttester, + &fork, + genesis_validators_root, + ); + let message = attestation.data.signing_root(domain); + + attestation.signature.add_assign(&sk.sign(message)); + } + } } } - // TODO(electra): fix this test - AttesterSlashing::Base(AttesterSlashingBase { - attestation_1, - attestation_2, - }) + if fork_name >= ForkName::Electra { + AttesterSlashing::Electra(AttesterSlashingElectra { + attestation_1: attestation_1.as_electra().unwrap().clone(), + attestation_2: attestation_2.as_electra().unwrap().clone(), + }) + } else { + AttesterSlashing::Base(AttesterSlashingBase { + attestation_1: attestation_1.as_base().unwrap().clone(), + attestation_2: attestation_2.as_base().unwrap().clone(), + }) + } } pub fn make_attester_slashing_different_indices( @@ -1569,6 +1640,8 @@ where validator_indices_1: Vec, validator_indices_2: Vec, ) -> AttesterSlashing { + let fork_name = self.spec.fork_name_at_slot::(Slot::new(0)); + let data = AttestationData { slot: Slot::new(0), index: 0, @@ -1583,45 +1656,95 @@ where }, }; - // TODO(electra): make this test fork-agnostic - let mut attestation_1 = IndexedAttestationBase { - attesting_indices: VariableList::new(validator_indices_1).unwrap(), - data: data.clone(), - signature: AggregateSignature::infinity(), - }; + let (mut attestation_1, mut attestation_2) = if fork_name >= ForkName::Electra { + let attestation_1 = IndexedAttestationElectra { + attesting_indices: VariableList::new(validator_indices_1).unwrap(), + data: data.clone(), + signature: AggregateSignature::infinity(), + }; + + let attestation_2 = IndexedAttestationElectra { + attesting_indices: VariableList::new(validator_indices_2).unwrap(), + data, + signature: AggregateSignature::infinity(), + }; - let mut attestation_2 = IndexedAttestationBase { - attesting_indices: VariableList::new(validator_indices_2).unwrap(), - data, - signature: AggregateSignature::infinity(), + ( + IndexedAttestation::Electra(attestation_1), + IndexedAttestation::Electra(attestation_2), + ) + } else { + let attestation_1 = IndexedAttestationBase { + attesting_indices: VariableList::new(validator_indices_1).unwrap(), + data: data.clone(), + signature: AggregateSignature::infinity(), + }; + + let attestation_2 = IndexedAttestationBase { + attesting_indices: VariableList::new(validator_indices_2).unwrap(), + data, + signature: AggregateSignature::infinity(), + }; + + ( + IndexedAttestation::Base(attestation_1), + IndexedAttestation::Base(attestation_2), + ) }; - attestation_2.data.index += 1; + attestation_2.data_mut().index += 1; let fork = self.chain.canonical_head.cached_head().head_fork(); for attestation in &mut [&mut attestation_1, &mut attestation_2] { - for i in attestation.attesting_indices.iter() { - let sk = &self.validator_keypairs[*i as usize].sk; + match attestation { + IndexedAttestation::Base(attestation) => { + for i in attestation.attesting_indices.iter() { + let sk = &self.validator_keypairs[*i as usize].sk; - let genesis_validators_root = self.chain.genesis_validators_root; + let genesis_validators_root = self.chain.genesis_validators_root; - let domain = self.chain.spec.get_domain( - attestation.data.target.epoch, - Domain::BeaconAttester, - &fork, - genesis_validators_root, - ); - let message = attestation.data.signing_root(domain); + let domain = self.chain.spec.get_domain( + attestation.data.target.epoch, + Domain::BeaconAttester, + &fork, + genesis_validators_root, + ); + let message = attestation.data.signing_root(domain); - attestation.signature.add_assign(&sk.sign(message)); + attestation.signature.add_assign(&sk.sign(message)); + } + } + IndexedAttestation::Electra(attestation) => { + for i in attestation.attesting_indices.iter() { + let sk = &self.validator_keypairs[*i as usize].sk; + + let genesis_validators_root = self.chain.genesis_validators_root; + + let domain = self.chain.spec.get_domain( + attestation.data.target.epoch, + Domain::BeaconAttester, + &fork, + genesis_validators_root, + ); + let message = attestation.data.signing_root(domain); + + attestation.signature.add_assign(&sk.sign(message)); + } + } } } - // TODO(electra): fix this test - AttesterSlashing::Base(AttesterSlashingBase { - attestation_1, - attestation_2, - }) + if fork_name >= ForkName::Electra { + AttesterSlashing::Electra(AttesterSlashingElectra { + attestation_1: attestation_1.as_electra().unwrap().clone(), + attestation_2: attestation_2.as_electra().unwrap().clone(), + }) + } else { + AttesterSlashing::Base(AttesterSlashingBase { + attestation_1: attestation_1.as_base().unwrap().clone(), + attestation_2: attestation_2.as_base().unwrap().clone(), + }) + } } pub fn make_proposer_slashing(&self, validator_index: u64) -> ProposerSlashing { @@ -2411,6 +2534,7 @@ where AttestationStrategy::AllValidators => self.get_all_validators(), AttestationStrategy::SomeValidators(vals) => vals, }; + let state_root = state.update_tree_hash_cache().unwrap(); let (_, _, last_produced_block_hash, _) = self .add_attested_blocks_at_slots_with_sync( @@ -2536,7 +2660,9 @@ pub fn generate_rand_block_and_blobs( num_blobs: NumBlobs, rng: &mut impl Rng, ) -> (SignedBeaconBlock>, Vec>) { + println!("randomly generating the beacon block is the trickt part. probably beacuse attestations im guessing?"); let inner = map_fork_name!(fork_name, BeaconBlock, <_>::random_for_test(rng)); + let mut block = SignedBeaconBlock::from_block(inner, types::Signature::random_for_test(rng)); let mut blob_sidecars = vec![]; @@ -2573,7 +2699,6 @@ pub fn generate_rand_block_and_blobs( }; let (bundle, transactions) = execution_layer::test_utils::generate_blobs::(num_blobs).unwrap(); - payload.execution_payload.transactions = <_>::default(); for tx in Vec::from(transactions) { payload.execution_payload.transactions.push(tx).unwrap(); diff --git a/beacon_node/beacon_chain/tests/attestation_verification.rs b/beacon_node/beacon_chain/tests/attestation_verification.rs index e91e8c77a3c..b63c07a25cd 100644 --- a/beacon_node/beacon_chain/tests/attestation_verification.rs +++ b/beacon_node/beacon_chain/tests/attestation_verification.rs @@ -2,6 +2,7 @@ use beacon_chain::attestation_verification::{ batch_verify_aggregated_attestations, batch_verify_unaggregated_attestations, Error, + ObservedAttestationKey, }; use beacon_chain::test_utils::{MakeAttestationOptions, HARNESS_GENESIS_TIME}; use beacon_chain::{ @@ -128,7 +129,7 @@ fn get_valid_unaggregated_attestation( let validator_committee_index = 0; let validator_index = *head .beacon_state - .get_beacon_committee(current_slot, valid_attestation.data().index) + .get_beacon_committee(current_slot, valid_attestation.committee_index()) .expect("should get committees") .committee .get(validator_committee_index) @@ -146,7 +147,7 @@ fn get_valid_unaggregated_attestation( ) .expect("should sign attestation"); - let subnet_id = SubnetId::compute_subnet_for_attestation::( + let subnet_id = SubnetId::compute_subnet_for_attestation::( &valid_attestation.to_ref(), head.beacon_state .get_committee_count_at_slot(current_slot) @@ -173,7 +174,7 @@ fn get_valid_aggregated_attestation( let current_slot = chain.slot().expect("should get slot"); let committee = state - .get_beacon_committee(current_slot, aggregate.data().index) + .get_beacon_committee(current_slot, aggregate.committee_index()) .expect("should get committees"); let committee_len = committee.committee.len(); @@ -224,7 +225,7 @@ fn get_non_aggregator( // TODO(electra) make fork-agnostic let committee = state - .get_beacon_committee(current_slot, aggregate.data().index) + .get_beacon_committee(current_slot, aggregate.committee_index()) .expect("should get committees"); let committee_len = committee.committee.len(); @@ -429,6 +430,7 @@ impl GossipTester { vec![&self.invalid_aggregate, &aggregate].into_iter(), ) .unwrap(); + assert_eq!(results.len(), 2); let batch_err = results.pop().unwrap().err().expect(&format!( "{} should error during batch_verify_aggregated_attestations_for_gossip", @@ -662,7 +664,7 @@ async fn aggregated_gossip_verification() { .chain .head_snapshot() .beacon_state - .get_beacon_committee(tester.slot(), a.message().aggregate().data().index) + .get_beacon_committee(tester.slot(), a.message().aggregate().committee_index()) .expect("should get committees") .committee .len(); @@ -778,12 +780,7 @@ async fn aggregated_gossip_verification() { // However, the following error is triggered first: AttnError::AggregatorNotInCommittee { aggregator_index - } | - // unless were working with electra attestations - // in which case this error is triggered instead: - AttnError::AggregatorPubkeyUnknown( - aggregator_index - ) + } if aggregator_index == VALIDATOR_COUNT as u64 )) }, @@ -792,7 +789,7 @@ async fn aggregated_gossip_verification() { * The following test ensures: * * aggregate_and_proof.selection_proof selects the validator as an aggregator for the slot -- - * i.e. is_aggregator(state, aggregate.data.slot, aggregate.data.index, + * i.e. is_aggregator(state, aggregate.data.slot, aggregate.committee_index(), * aggregate_and_proof.selection_proof) returns True. */ .inspect_aggregate_err( @@ -812,6 +809,7 @@ async fn aggregated_gossip_verification() { }, |tester, err| { let (val_index, _) = tester.non_aggregator(); + assert!(matches!( err, AttnError::InvalidSelectionProof { @@ -838,7 +836,10 @@ async fn aggregated_gossip_verification() { assert!(matches!( err, AttnError::AttestationSupersetKnown(hash) - if hash == tester.valid_aggregate.message().aggregate().data().tree_hash_root() + if hash == ObservedAttestationKey { + committee_index: tester.valid_aggregate.message().aggregate().committee_index(), + attestation_data: tester.valid_aggregate.message().aggregate().data().clone(), + }.tree_hash_root() )) }, ) diff --git a/beacon_node/beacon_chain/tests/block_verification.rs b/beacon_node/beacon_chain/tests/block_verification.rs index 257b6d53463..e3930cdad70 100644 --- a/beacon_node/beacon_chain/tests/block_verification.rs +++ b/beacon_node/beacon_chain/tests/block_verification.rs @@ -664,27 +664,58 @@ async fn invalid_signature_attester_slashing() { for &block_index in BLOCK_INDICES { let harness = get_invalid_sigs_harness(&chain_segment).await; let mut snapshots = chain_segment.clone(); - let indexed_attestation = IndexedAttestationBase { - attesting_indices: vec![0].into(), - data: AttestationData { - slot: Slot::new(0), - index: 0, - beacon_block_root: Hash256::zero(), - source: Checkpoint { - epoch: Epoch::new(0), - root: Hash256::zero(), + let fork_name = harness.chain.spec.fork_name_at_slot::(Slot::new(0)); + + let attester_slashing = if fork_name >= ForkName::Electra { + let indexed_attestation = IndexedAttestationElectra { + attesting_indices: vec![0].into(), + data: AttestationData { + slot: Slot::new(0), + index: 0, + beacon_block_root: Hash256::zero(), + source: Checkpoint { + epoch: Epoch::new(0), + root: Hash256::zero(), + }, + target: Checkpoint { + epoch: Epoch::new(0), + root: Hash256::zero(), + }, }, - target: Checkpoint { - epoch: Epoch::new(0), - root: Hash256::zero(), + signature: junk_aggregate_signature(), + }; + let attester_slashing = AttesterSlashingElectra { + attestation_1: indexed_attestation.clone(), + attestation_2: indexed_attestation, + }; + + AttesterSlashing::Electra(attester_slashing) + } else { + let indexed_attestation = IndexedAttestationBase { + attesting_indices: vec![0].into(), + data: AttestationData { + slot: Slot::new(0), + index: 0, + beacon_block_root: Hash256::zero(), + source: Checkpoint { + epoch: Epoch::new(0), + root: Hash256::zero(), + }, + target: Checkpoint { + epoch: Epoch::new(0), + root: Hash256::zero(), + }, }, - }, - signature: junk_aggregate_signature(), - }; - let attester_slashing = AttesterSlashingBase { - attestation_1: indexed_attestation.clone(), - attestation_2: indexed_attestation, + signature: junk_aggregate_signature(), + }; + let attester_slashing = AttesterSlashingBase { + attestation_1: indexed_attestation.clone(), + attestation_2: indexed_attestation, + }; + + AttesterSlashing::Base(attester_slashing) }; + let (mut block, signature) = snapshots[block_index] .beacon_block .as_ref() @@ -693,31 +724,33 @@ async fn invalid_signature_attester_slashing() { match &mut block.body_mut() { BeaconBlockBodyRefMut::Base(ref mut blk) => { blk.attester_slashings - .push(attester_slashing) + .push(attester_slashing.as_base().unwrap().clone()) .expect("should update attester slashing"); } BeaconBlockBodyRefMut::Altair(ref mut blk) => { blk.attester_slashings - .push(attester_slashing) + .push(attester_slashing.as_base().unwrap().clone()) .expect("should update attester slashing"); } BeaconBlockBodyRefMut::Bellatrix(ref mut blk) => { blk.attester_slashings - .push(attester_slashing) + .push(attester_slashing.as_base().unwrap().clone()) .expect("should update attester slashing"); } BeaconBlockBodyRefMut::Capella(ref mut blk) => { blk.attester_slashings - .push(attester_slashing) + .push(attester_slashing.as_base().unwrap().clone()) .expect("should update attester slashing"); } BeaconBlockBodyRefMut::Deneb(ref mut blk) => { blk.attester_slashings - .push(attester_slashing) + .push(attester_slashing.as_base().unwrap().clone()) .expect("should update attester slashing"); } - BeaconBlockBodyRefMut::Electra(_) => { - panic!("electra test not implemented!"); + BeaconBlockBodyRefMut::Electra(ref mut blk) => { + blk.attester_slashings + .push(attester_slashing.as_electra().unwrap().clone()) + .expect("should update attester slashing"); } } snapshots[block_index].beacon_block = diff --git a/beacon_node/beacon_chain/tests/payload_invalidation.rs b/beacon_node/beacon_chain/tests/payload_invalidation.rs index 594872e2fff..517c7f00398 100644 --- a/beacon_node/beacon_chain/tests/payload_invalidation.rs +++ b/beacon_node/beacon_chain/tests/payload_invalidation.rs @@ -25,7 +25,6 @@ use std::collections::HashMap; use std::sync::Arc; use std::time::Duration; use task_executor::ShutdownReason; -use tree_hash::TreeHash; use types::*; const VALIDATOR_COUNT: usize = 32; @@ -1222,13 +1221,13 @@ async fn attesting_to_optimistic_head() { let get_aggregated = || { rig.harness .chain - .get_aggregated_attestation(attestation.data()) + .get_aggregated_attestation(&attestation.to_ref()) }; let get_aggregated_by_slot_and_root = || { rig.harness .chain - .get_aggregated_attestation_base(attestation.data()) + .get_aggregated_attestation(&attestation.to_ref()) }; /* diff --git a/beacon_node/beacon_chain/tests/store_tests.rs b/beacon_node/beacon_chain/tests/store_tests.rs index 8c0464187f3..340f9617aea 100644 --- a/beacon_node/beacon_chain/tests/store_tests.rs +++ b/beacon_node/beacon_chain/tests/store_tests.rs @@ -1014,6 +1014,7 @@ async fn multiple_attestations_per_block() { .await; let head = harness.chain.head_snapshot(); + let committees_per_slot = head .beacon_state .get_committee_count_at_slot(head.beacon_state.slot()) @@ -1022,15 +1023,29 @@ async fn multiple_attestations_per_block() { for snapshot in harness.chain.chain_dump().unwrap() { let slot = snapshot.beacon_block.slot(); - assert_eq!( - snapshot - .beacon_block - .as_ref() - .message() - .body() - .attestations_len() as u64, - if slot <= 1 { 0 } else { committees_per_slot } - ); + let fork_name = harness.chain.spec.fork_name_at_slot::(slot); + + if fork_name >= ForkName::Electra { + assert_eq!( + snapshot + .beacon_block + .as_ref() + .message() + .body() + .attestations_len() as u64, + if slot <= 1 { 0 } else { 1 } + ); + } else { + assert_eq!( + snapshot + .beacon_block + .as_ref() + .message() + .body() + .attestations_len() as u64, + if slot <= 1 { 0 } else { committees_per_slot } + ); + } } } diff --git a/beacon_node/beacon_chain/tests/sync_committee_verification.rs b/beacon_node/beacon_chain/tests/sync_committee_verification.rs index 0e4745ff6b8..242ed558475 100644 --- a/beacon_node/beacon_chain/tests/sync_committee_verification.rs +++ b/beacon_node/beacon_chain/tests/sync_committee_verification.rs @@ -318,7 +318,6 @@ async fn aggregated_gossip_verification() { * 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", { @@ -354,7 +353,6 @@ async fn aggregated_gossip_verification() { * derived from the participation info in `aggregation_bits` for the subcommittee specified by * the `contribution.subcommittee_index`. */ - assert_invalid!( "aggregate with bad aggregate signature", { @@ -450,6 +448,7 @@ async fn aggregated_gossip_verification() { root: contribution.beacon_block_root, subcommittee_index: contribution.subcommittee_index, }; + assert_invalid!( "aggregate that has already been seen", valid_aggregate.clone(), diff --git a/beacon_node/http_api/src/lib.rs b/beacon_node/http_api/src/lib.rs index 838f7233052..494e3cf363e 100644 --- a/beacon_node/http_api/src/lib.rs +++ b/beacon_node/http_api/src/lib.rs @@ -3369,7 +3369,7 @@ pub fn serve( "error" => format!("{:?}", e), "request_index" => index, "aggregator_index" => aggregate.message().aggregator_index(), - "attestation_index" => aggregate.message().aggregate().data().index, + "attestation_index" => aggregate.message().aggregate().committee_index(), "attestation_slot" => aggregate.message().aggregate().data().slot, ); failures.push(api_types::Failure::new(index, format!("Verification: {:?}", e))); @@ -3390,7 +3390,7 @@ pub fn serve( "error" => format!("{:?}", e), "request_index" => index, "aggregator_index" => verified_aggregate.aggregate().message().aggregator_index(), - "attestation_index" => verified_aggregate.attestation().data().index, + "attestation_index" => verified_aggregate.attestation().committee_index(), "attestation_slot" => verified_aggregate.attestation().data().slot, ); failures.push(api_types::Failure::new(index, format!("Fork choice: {:?}", e))); diff --git a/beacon_node/http_api/src/publish_attestations.rs b/beacon_node/http_api/src/publish_attestations.rs index 541ba8b7871..00654765325 100644 --- a/beacon_node/http_api/src/publish_attestations.rs +++ b/beacon_node/http_api/src/publish_attestations.rs @@ -141,7 +141,7 @@ pub async fn publish_attestations( // move the `attestations` vec into the blocking task, so this small overhead is unavoidable. let attestation_metadata = attestations .iter() - .map(|att| (att.data().slot, att.data().index)) + .map(|att| (att.data().slot, att.committee_index())) .collect::>(); // Gossip validate and publish attestations that can be immediately processed. diff --git a/beacon_node/lighthouse_network/src/types/pubsub.rs b/beacon_node/lighthouse_network/src/types/pubsub.rs index 8945c23a549..6dd6efa9822 100644 --- a/beacon_node/lighthouse_network/src/types/pubsub.rs +++ b/beacon_node/lighthouse_network/src/types/pubsub.rs @@ -383,7 +383,7 @@ impl std::fmt::Display for PubsubMessage { f, "Aggregate and Proof: slot: {}, index: {}, aggregator_index: {}", att.message().aggregate().data().slot, - att.message().aggregate().data().index, + att.message().aggregate().committee_index(), att.message().aggregator_index(), ), PubsubMessage::Attestation(data) => write!( @@ -391,7 +391,7 @@ impl std::fmt::Display for PubsubMessage { "Attestation: subnet_id: {}, attestation_slot: {}, attestation_index: {}", *data.0, data.1.data().slot, - data.1.data().index, + data.1.committee_index(), ), PubsubMessage::VoluntaryExit(_data) => write!(f, "Voluntary Exit"), PubsubMessage::ProposerSlashing(_data) => write!(f, "Proposer Slashing"), diff --git a/beacon_node/network/src/network_beacon_processor/gossip_methods.rs b/beacon_node/network/src/network_beacon_processor/gossip_methods.rs index d41233c903b..d1e7f54abf0 100644 --- a/beacon_node/network/src/network_beacon_processor/gossip_methods.rs +++ b/beacon_node/network/src/network_beacon_processor/gossip_methods.rs @@ -2038,6 +2038,27 @@ impl NetworkBeaconProcessor { "attn_val_index_too_high", ); } + AttnError::CommitteeIndexNonZero(index) => { + /* + * The validator index is not set to zero after Electra. + * + * The peer has published an invalid consensus message. + */ + debug!( + self.log, + "Committee index non zero"; + "peer_id" => %peer_id, + "block" => ?beacon_block_root, + "type" => ?attestation_type, + "committee_index" => index, + ); + self.propagate_validation_result(message_id, peer_id, MessageAcceptance::Reject); + self.gossip_penalize_peer( + peer_id, + PeerAction::LowToleranceError, + "attn_comm_index_non_zero", + ); + } AttnError::UnknownHeadBlock { beacon_block_root } => { trace!( self.log, @@ -2193,6 +2214,19 @@ impl NetworkBeaconProcessor { "attn_too_many_agg_bits", ); } + AttnError::NotExactlyOneCommitteeBitSet(_) => { + /* + * The attestation doesn't have only one committee bit set. + * + * The peer has published an invalid consensus message. + */ + self.propagate_validation_result(message_id, peer_id, MessageAcceptance::Reject); + self.gossip_penalize_peer( + peer_id, + PeerAction::LowToleranceError, + "attn_too_many_comm_bits", + ); + } AttnError::AttestsToFutureBlock { .. } => { /* * The beacon_block_root is from a higher slot than the attestation. diff --git a/beacon_node/operation_pool/src/attestation.rs b/beacon_node/operation_pool/src/attestation.rs index 91fd00a3979..c6ed6eb7f6e 100644 --- a/beacon_node/operation_pool/src/attestation.rs +++ b/beacon_node/operation_pool/src/attestation.rs @@ -1,4 +1,4 @@ -use crate::attestation_storage::{AttestationRef, CompactIndexedAttestation}; +use crate::attestation_storage::{CompactAttestationRef, CompactIndexedAttestation}; use crate::max_cover::MaxCover; use crate::reward_cache::RewardCache; use state_processing::common::{ @@ -14,14 +14,14 @@ use types::{ #[derive(Debug, Clone)] pub struct AttMaxCover<'a, E: EthSpec> { /// Underlying attestation. - pub att: AttestationRef<'a, E>, + pub att: CompactAttestationRef<'a, E>, /// Mapping of validator indices and their rewards. pub fresh_validators_rewards: HashMap, } impl<'a, E: EthSpec> AttMaxCover<'a, E> { pub fn new( - att: AttestationRef<'a, E>, + att: CompactAttestationRef<'a, E>, state: &BeaconState, reward_cache: &'a RewardCache, total_active_balance: u64, @@ -36,7 +36,7 @@ impl<'a, E: EthSpec> AttMaxCover<'a, E> { /// Initialise an attestation cover object for base/phase0 hard fork. pub fn new_for_base( - att: AttestationRef<'a, E>, + att: CompactAttestationRef<'a, E>, state: &BeaconState, base_state: &BeaconStateBase, total_active_balance: u64, @@ -69,7 +69,7 @@ impl<'a, E: EthSpec> AttMaxCover<'a, E> { /// Initialise an attestation cover object for Altair or later. pub fn new_for_altair_deneb( - att: AttestationRef<'a, E>, + att: CompactAttestationRef<'a, E>, state: &BeaconState, reward_cache: &'a RewardCache, spec: &ChainSpec, @@ -119,14 +119,14 @@ impl<'a, E: EthSpec> AttMaxCover<'a, E> { impl<'a, E: EthSpec> MaxCover for AttMaxCover<'a, E> { type Object = Attestation; - type Intermediate = AttestationRef<'a, E>; + type Intermediate = CompactAttestationRef<'a, E>; type Set = HashMap; - fn intermediate(&self) -> &AttestationRef<'a, E> { + fn intermediate(&self) -> &CompactAttestationRef<'a, E> { &self.att } - fn convert_to_object(att_ref: &AttestationRef<'a, E>) -> Attestation { + fn convert_to_object(att_ref: &CompactAttestationRef<'a, E>) -> Attestation { att_ref.clone_as_attestation() } @@ -153,7 +153,7 @@ impl<'a, E: EthSpec> MaxCover for AttMaxCover<'a, E> { /// executing the `retain` when the `committee_bits` of the two attestations intersect. fn update_covering_set( &mut self, - best_att: &AttestationRef<'a, E>, + best_att: &CompactAttestationRef<'a, E>, covered_validators: &HashMap, ) { if self.att.data.slot == best_att.data.slot && self.att.data.index == best_att.data.index { @@ -177,7 +177,7 @@ impl<'a, E: EthSpec> MaxCover for AttMaxCover<'a, E> { /// /// This isn't optimal, but with the Altair fork this code is obsolete and not worth upgrading. pub fn earliest_attestation_validators( - attestation: &AttestationRef, + attestation: &CompactAttestationRef, state: &BeaconState, base_state: &BeaconStateBase, ) -> BitList { diff --git a/beacon_node/operation_pool/src/attestation_storage.rs b/beacon_node/operation_pool/src/attestation_storage.rs index f06da2afb17..43b1c3abbb3 100644 --- a/beacon_node/operation_pool/src/attestation_storage.rs +++ b/beacon_node/operation_pool/src/attestation_storage.rs @@ -41,9 +41,8 @@ pub struct SplitAttestation { pub indexed: CompactIndexedAttestation, } -// TODO(electra): rename this type #[derive(Debug, Clone)] -pub struct AttestationRef<'a, E: EthSpec> { +pub struct CompactAttestationRef<'a, E: EthSpec> { pub checkpoint: &'a CheckpointKey, pub data: &'a CompactAttestationData, pub indexed: &'a CompactIndexedAttestation, @@ -97,8 +96,8 @@ impl SplitAttestation { } } - pub fn as_ref(&self) -> AttestationRef { - AttestationRef { + pub fn as_ref(&self) -> CompactAttestationRef { + CompactAttestationRef { checkpoint: &self.checkpoint, data: &self.data, indexed: &self.indexed, @@ -106,7 +105,7 @@ impl SplitAttestation { } } -impl<'a, E: EthSpec> AttestationRef<'a, E> { +impl<'a, E: EthSpec> CompactAttestationRef<'a, E> { pub fn attestation_data(&self) -> AttestationData { AttestationData { slot: self.data.slot, @@ -171,7 +170,7 @@ impl CompactIndexedAttestation { } } - pub fn aggregate(&mut self, other: &Self) { + pub fn aggregate(&mut self, other: &Self) -> Option<()> { match (self, other) { (CompactIndexedAttestation::Base(this), CompactIndexedAttestation::Base(other)) => { this.aggregate(other) @@ -181,7 +180,7 @@ impl CompactIndexedAttestation { CompactIndexedAttestation::Electra(other), ) => this.aggregate_same_committee(other), // TODO(electra) is a mix of electra and base compact indexed attestations an edge case we need to deal with? - _ => (), + _ => None, } } } @@ -193,7 +192,7 @@ impl CompactIndexedAttestationBase { .is_zero() } - pub fn aggregate(&mut self, other: &Self) { + pub fn aggregate(&mut self, other: &Self) -> Option<()> { self.attesting_indices = self .attesting_indices .drain(..) @@ -202,6 +201,8 @@ impl CompactIndexedAttestationBase { .collect(); self.aggregation_bits = self.aggregation_bits.union(&other.aggregation_bits); self.signature.add_assign_aggregate(&other.signature); + + Some(()) } } @@ -215,9 +216,11 @@ impl CompactIndexedAttestationElectra { .is_zero() } - pub fn aggregate_same_committee(&mut self, other: &Self) { + pub fn aggregate_same_committee(&mut self, other: &Self) -> Option<()> { // TODO(electra): remove assert in favour of Result - assert_eq!(self.committee_bits, other.committee_bits); + if self.committee_bits != other.committee_bits { + return None; + } self.aggregation_bits = self.aggregation_bits.union(&other.aggregation_bits); self.attesting_indices = self .attesting_indices @@ -226,34 +229,48 @@ impl CompactIndexedAttestationElectra { .dedup() .collect(); self.signature.add_assign_aggregate(&other.signature); + Some(()) } - pub fn aggregate_with_disjoint_committees(&mut self, other: &Self) { - // TODO(electra): remove asserts or use Result - assert!(self + pub fn aggregate_with_disjoint_committees(&mut self, other: &Self) -> Option<()> { + if !self .committee_bits .intersection(&other.committee_bits) - .is_zero(),); + .is_zero() + { + return None; + } // The attestation being aggregated in must only have 1 committee bit set. - assert_eq!(other.committee_bits.num_set_bits(), 1); + if other.committee_bits.num_set_bits() != 1 { + return None; + } + // Check we are aggregating in increasing committee index order (so we can append // aggregation bits). - assert!(self.committee_bits.highest_set_bit() < other.committee_bits.highest_set_bit()); + if self.committee_bits.highest_set_bit() >= other.committee_bits.highest_set_bit() { + return None; + } self.committee_bits = self.committee_bits.union(&other.committee_bits); - self.aggregation_bits = - bitlist_extend(&self.aggregation_bits, &other.aggregation_bits).unwrap(); - self.attesting_indices = self - .attesting_indices - .drain(..) - .merge(other.attesting_indices.iter().copied()) - .dedup() - .collect(); - self.signature.add_assign_aggregate(&other.signature); + if let Some(agg_bits) = bitlist_extend(&self.aggregation_bits, &other.aggregation_bits) { + self.aggregation_bits = agg_bits; + + self.attesting_indices = self + .attesting_indices + .drain(..) + .merge(other.attesting_indices.iter().copied()) + .dedup() + .collect(); + self.signature.add_assign_aggregate(&other.signature); + + return Some(()); + } + + None } - pub fn committee_index(&self) -> u64 { - *self.get_committee_indices().first().unwrap_or(&0u64) + pub fn committee_index(&self) -> Option { + self.get_committee_indices().first().copied() } pub fn get_committee_indices(&self) -> Vec { @@ -350,27 +367,28 @@ impl AttestationMap { continue; } }; - let committee_index = electra_attestation.committee_index(); - if let Some(existing_attestation) = - best_attestations_by_committee.get_mut(&committee_index) - { - // Search for the best (most aggregation bits) attestation for this committee - // index. - if electra_attestation.aggregation_bits.num_set_bits() - > existing_attestation.aggregation_bits.num_set_bits() + if let Some(committee_index) = electra_attestation.committee_index() { + if let Some(existing_attestation) = + best_attestations_by_committee.get_mut(&committee_index) { - // New attestation is better than the previously known one for this - // committee. Replace it. - std::mem::swap(existing_attestation, &mut electra_attestation); + // Search for the best (most aggregation bits) attestation for this committee + // index. + if electra_attestation.aggregation_bits.num_set_bits() + > existing_attestation.aggregation_bits.num_set_bits() + { + // New attestation is better than the previously known one for this + // committee. Replace it. + std::mem::swap(existing_attestation, &mut electra_attestation); + } + // Put the inferior attestation into the list of aggregated attestations + // without performing any cross-committee aggregation. + aggregated_attestations + .push(CompactIndexedAttestation::Electra(electra_attestation)); + } else { + // First attestation seen for this committee. Place it in the map + // provisionally. + best_attestations_by_committee.insert(committee_index, electra_attestation); } - // Put the inferior attestation into the list of aggregated attestations - // without performing any cross-committee aggregation. - aggregated_attestations - .push(CompactIndexedAttestation::Electra(electra_attestation)); - } else { - // First attestation seen for this committee. Place it in the map - // provisionally. - best_attestations_by_committee.insert(committee_index, electra_attestation); } } @@ -399,7 +417,7 @@ impl AttestationMap { pub fn get_attestations<'a>( &'a self, checkpoint_key: &'a CheckpointKey, - ) -> impl Iterator> + 'a { + ) -> impl Iterator> + 'a { self.checkpoint_map .get(checkpoint_key) .into_iter() @@ -407,7 +425,7 @@ impl AttestationMap { } /// Iterate all attestations in the map. - pub fn iter(&self) -> impl Iterator> { + pub fn iter(&self) -> impl Iterator> { self.checkpoint_map .iter() .flat_map(|(checkpoint_key, attestation_map)| attestation_map.iter(checkpoint_key)) @@ -438,9 +456,9 @@ impl AttestationDataMap { pub fn iter<'a>( &'a self, checkpoint_key: &'a CheckpointKey, - ) -> impl Iterator> + 'a { + ) -> impl Iterator> + 'a { self.attestations.iter().flat_map(|(data, vec_indexed)| { - vec_indexed.iter().map(|indexed| AttestationRef { + vec_indexed.iter().map(|indexed| CompactAttestationRef { checkpoint: checkpoint_key, data, indexed, diff --git a/beacon_node/operation_pool/src/lib.rs b/beacon_node/operation_pool/src/lib.rs index daddbf76652..e3eb64f00d8 100644 --- a/beacon_node/operation_pool/src/lib.rs +++ b/beacon_node/operation_pool/src/lib.rs @@ -11,7 +11,7 @@ mod sync_aggregate_id; pub use crate::bls_to_execution_changes::ReceivedPreCapella; pub use attestation::{earliest_attestation_validators, AttMaxCover}; -pub use attestation_storage::{AttestationRef, SplitAttestation}; +pub use attestation_storage::{CompactAttestationRef, SplitAttestation}; pub use max_cover::MaxCover; pub use persistence::{ PersistedOperationPool, PersistedOperationPoolV12, PersistedOperationPoolV14, @@ -228,7 +228,7 @@ impl OperationPool { state: &'a BeaconState, reward_cache: &'a RewardCache, total_active_balance: u64, - validity_filter: impl FnMut(&AttestationRef<'a, E>) -> bool + Send, + validity_filter: impl FnMut(&CompactAttestationRef<'a, E>) -> bool + Send, spec: &'a ChainSpec, ) -> impl Iterator> + Send { all_attestations @@ -252,8 +252,8 @@ impl OperationPool { pub fn get_attestations( &self, state: &BeaconState, - prev_epoch_validity_filter: impl for<'a> FnMut(&AttestationRef<'a, E>) -> bool + Send, - curr_epoch_validity_filter: impl for<'a> FnMut(&AttestationRef<'a, E>) -> bool + Send, + prev_epoch_validity_filter: impl for<'a> FnMut(&CompactAttestationRef<'a, E>) -> bool + Send, + curr_epoch_validity_filter: impl for<'a> FnMut(&CompactAttestationRef<'a, E>) -> bool + Send, spec: &ChainSpec, ) -> Result>, OpPoolError> { let fork_name = state.fork_name_unchecked(); @@ -1282,9 +1282,7 @@ mod release_tests { for att in &best_attestations { match fork_name { ForkName::Electra => { - // TODO(electra) some attestations only have 2 or 3 agg bits set - // others have 5 - assert!(att.num_set_aggregation_bits() >= 2); + assert!(att.num_set_aggregation_bits() >= small_step_size); } _ => { assert!(att.num_set_aggregation_bits() >= big_step_size); diff --git a/consensus/fork_choice/tests/tests.rs b/consensus/fork_choice/tests/tests.rs index 3b05cfab1fd..ed2cfd8a60f 100644 --- a/consensus/fork_choice/tests/tests.rs +++ b/consensus/fork_choice/tests/tests.rs @@ -435,7 +435,7 @@ impl ForkChoiceTest { let validator_committee_index = 0; let validator_index = *head .beacon_state - .get_beacon_committee(current_slot, attestation.data().index) + .get_beacon_committee(current_slot, attestation.committee_index()) .expect("should get committees") .committee .get(validator_committee_index) diff --git a/consensus/state_processing/src/common/get_attesting_indices.rs b/consensus/state_processing/src/common/get_attesting_indices.rs index 33249c49417..8a9e60e2fab 100644 --- a/consensus/state_processing/src/common/get_attesting_indices.rs +++ b/consensus/state_processing/src/common/get_attesting_indices.rs @@ -13,7 +13,6 @@ pub mod attesting_indices_base { ) -> Result, BlockOperationError> { let attesting_indices = get_attesting_indices::(committee, &attestation.aggregation_bits)?; - Ok(IndexedAttestation::Base(IndexedAttestationBase { attesting_indices: VariableList::new(attesting_indices)?, data: attestation.data.clone(), @@ -52,6 +51,100 @@ pub mod attesting_indices_electra { use safe_arith::SafeArith; use types::*; + // TODO(electra) remove duplicate code + // get_indexed_attestation is almost an exact duplicate + // the only differences are the invalid selection proof + // and aggregator not in committee checks + pub fn get_indexed_attestation_from_signed_aggregate( + committees: &[BeaconCommittee], + signed_aggregate: &SignedAggregateAndProofElectra, + spec: &ChainSpec, + ) -> Result, BeaconStateError> { + let mut output: HashSet = HashSet::new(); + + let committee_bits = &signed_aggregate.message.aggregate.committee_bits; + let aggregation_bits = &signed_aggregate.message.aggregate.aggregation_bits; + let aggregator_index = signed_aggregate.message.aggregator_index; + let attestation = &signed_aggregate.message.aggregate; + + let committee_indices = get_committee_indices::(committee_bits); + + let mut committee_offset = 0; + + let committees_map: HashMap = committees + .iter() + .map(|committee| (committee.index, committee)) + .collect(); + + let committee_count_per_slot = committees.len() as u64; + let mut participant_count = 0; + + // TODO(electra): + // Note: this clones the signature which is known to be a relatively slow operation. + // + // Future optimizations should remove this clone. + let selection_proof = + SelectionProof::from(signed_aggregate.message.selection_proof.clone()); + + for index in committee_indices { + if let Some(&beacon_committee) = committees_map.get(&index) { + if !selection_proof + .is_aggregator(beacon_committee.committee.len(), spec) + .map_err(BeaconStateError::ArithError)? + { + return Err(BeaconStateError::InvalidSelectionProof { aggregator_index }); + } + + if !beacon_committee + .committee + .contains(&(aggregator_index as usize)) + { + return Err(BeaconStateError::AggregatorNotInCommittee { aggregator_index }); + } + + // This check is new to the spec's `process_attestation` in Electra. + if index >= committee_count_per_slot { + return Err(BeaconStateError::InvalidCommitteeIndex(index)); + } + + participant_count.safe_add_assign(beacon_committee.committee.len() as u64)?; + let committee_attesters = beacon_committee + .committee + .iter() + .enumerate() + .filter_map(|(i, &index)| { + if let Ok(aggregation_bit_index) = committee_offset.safe_add(i) { + if aggregation_bits.get(aggregation_bit_index).unwrap_or(false) { + return Some(index as u64); + } + } + None + }) + .collect::>(); + + output.extend(committee_attesters); + + committee_offset.safe_add_assign(beacon_committee.committee.len())?; + } else { + return Err(Error::NoCommitteeFound(index)); + } + } + + // This check is new to the spec's `process_attestation` in Electra. + if participant_count as usize != aggregation_bits.len() { + return Err(Error::InvalidBitfield); + } + + let mut indices = output.into_iter().collect_vec(); + indices.sort_unstable(); + + Ok(IndexedAttestation::Electra(IndexedAttestationElectra { + attesting_indices: VariableList::new(indices)?, + data: attestation.data.clone(), + signature: attestation.signature.clone(), + })) + } + pub fn get_indexed_attestation( committees: &[BeaconCommittee], attestation: &AttestationElectra, @@ -155,7 +248,7 @@ pub mod attesting_indices_electra { Ok(indices) } - fn get_committee_indices( + pub fn get_committee_indices( committee_bits: &BitVector, ) -> Vec { committee_bits diff --git a/consensus/state_processing/src/per_block_processing/signature_sets.rs b/consensus/state_processing/src/per_block_processing/signature_sets.rs index a179583556f..8cf39abf437 100644 --- a/consensus/state_processing/src/per_block_processing/signature_sets.rs +++ b/consensus/state_processing/src/per_block_processing/signature_sets.rs @@ -326,6 +326,7 @@ where genesis_validators_root, ); + // TODO(electra), signing root isnt unique in the case of electra let message = indexed_attestation.data().signing_root(domain); Ok(SignatureSet::multiple_pubkeys(signature, pubkeys, message)) @@ -436,7 +437,6 @@ where let message = slot.signing_root(domain); let signature = signed_aggregate_and_proof.message().selection_proof(); let validator_index = signed_aggregate_and_proof.message().aggregator_index(); - Ok(SignatureSet::single_pubkey( signature, get_pubkey(validator_index as usize).ok_or(Error::ValidatorUnknown(validator_index))?, diff --git a/consensus/types/src/attestation.rs b/consensus/types/src/attestation.rs index 8c8a81b90f2..81107be86df 100644 --- a/consensus/types/src/attestation.rs +++ b/consensus/types/src/attestation.rs @@ -1,7 +1,6 @@ use crate::slot_data::SlotData; use crate::{test_utils::TestRandom, Hash256, Slot}; use derivative::Derivative; -use rand::RngCore; use safe_arith::ArithError; use serde::{Deserialize, Serialize}; use ssz::Decode; @@ -22,6 +21,7 @@ pub enum Error { SszTypesError(ssz_types::Error), AlreadySigned(usize), SubnetCountIsZero(ArithError), + IncorrectStateVariant, } #[superstruct( @@ -43,7 +43,9 @@ pub enum Error { serde(bound = "E: EthSpec", deny_unknown_fields), arbitrary(bound = "E: EthSpec"), ), - ref_attributes(derive(TreeHash), tree_hash(enum_behaviour = "transparent")) + ref_attributes(derive(TreeHash), tree_hash(enum_behaviour = "transparent")), + cast_error(ty = "Error", expr = "Error::IncorrectStateVariant"), + partial_getter_error(ty = "Error", expr = "Error::IncorrectStateVariant") )] #[derive( Debug, @@ -92,23 +94,6 @@ impl Decode for Attestation { } } -// TODO(electra): think about how to handle fork variants here -impl TestRandom for Attestation { - fn random_for_test(rng: &mut impl RngCore) -> Self { - let aggregation_bits: BitList = BitList::random_for_test(rng); - // let committee_bits: BitList = BitList::random_for_test(rng); - let data = AttestationData::random_for_test(rng); - let signature = AggregateSignature::random_for_test(rng); - - Self::Base(AttestationBase { - aggregation_bits, - // committee_bits, - data, - signature, - }) - } -} - impl Hash for Attestation { fn hash(&self, state: &mut H) where @@ -244,6 +229,25 @@ impl<'a, E: EthSpec> AttestationRef<'a, E> { AttestationRef::Electra(att) => att.committee_index(), } } + + pub fn set_aggregation_bits(&self) -> Vec { + match self { + Self::Base(att) => att + .aggregation_bits + .iter() + .enumerate() + .filter(|(_i, bit)| *bit) + .map(|(i, _bit)| i) + .collect::>(), + Self::Electra(att) => att + .aggregation_bits + .iter() + .enumerate() + .filter(|(_i, bit)| *bit) + .map(|(i, _bit)| i) + .collect::>(), + } + } } impl AttestationElectra { diff --git a/consensus/types/src/beacon_state.rs b/consensus/types/src/beacon_state.rs index d9c7a78537a..bc20f3aa7bb 100644 --- a/consensus/types/src/beacon_state.rs +++ b/consensus/types/src/beacon_state.rs @@ -161,6 +161,12 @@ pub enum Error { MerkleTreeError(merkle_proof::MerkleTreeError), NoCommitteeFound(CommitteeIndex), InvalidCommitteeIndex(CommitteeIndex), + InvalidSelectionProof { + aggregator_index: u64, + }, + AggregatorNotInCommittee { + aggregator_index: u64, + }, } /// Control whether an epoch-indexed field can be indexed at the next epoch or not. diff --git a/consensus/types/src/indexed_attestation.rs b/consensus/types/src/indexed_attestation.rs index 4e7e061b45f..a2265cdd71b 100644 --- a/consensus/types/src/indexed_attestation.rs +++ b/consensus/types/src/indexed_attestation.rs @@ -1,7 +1,6 @@ use crate::{test_utils::TestRandom, AggregateSignature, AttestationData, EthSpec, VariableList}; use core::slice::Iter; use derivative::Derivative; -use rand::RngCore; use serde::{Deserialize, Serialize}; use ssz::Decode; use ssz::Encode; @@ -190,20 +189,6 @@ impl Decode for IndexedAttestation { } } -impl TestRandom for IndexedAttestation { - fn random_for_test(rng: &mut impl RngCore) -> Self { - let attesting_indices = VariableList::random_for_test(rng); - let data = AttestationData::random_for_test(rng); - let signature = AggregateSignature::random_for_test(rng); - - Self::Base(IndexedAttestationBase { - attesting_indices, - data, - signature, - }) - } -} - /// Implementation of non-crypto-secure `Hash`, for use with `HashMap` and `HashSet`. /// /// Guarantees `att1 == att2 -> hash(att1) == hash(att2)`. diff --git a/consensus/types/src/signed_aggregate_and_proof.rs b/consensus/types/src/signed_aggregate_and_proof.rs index 57a2ce5babe..ddf1dedb040 100644 --- a/consensus/types/src/signed_aggregate_and_proof.rs +++ b/consensus/types/src/signed_aggregate_and_proof.rs @@ -74,7 +74,6 @@ impl SignedAggregateAndProof { genesis_validators_root, spec, ); - let target_epoch = message.aggregate().data().slot.epoch(E::slots_per_epoch()); let domain = spec.get_domain( target_epoch, diff --git a/consensus/types/src/test_utils/test_random/bitfield.rs b/consensus/types/src/test_utils/test_random/bitfield.rs index f73f7c18c5a..ab6ab259deb 100644 --- a/consensus/types/src/test_utils/test_random/bitfield.rs +++ b/consensus/types/src/test_utils/test_random/bitfield.rs @@ -24,8 +24,11 @@ impl TestRandom for BitList { impl TestRandom for BitVector { fn random_for_test(rng: &mut impl RngCore) -> Self { + println!("check {}", N::to_usize()); let mut raw_bytes = smallvec![0; std::cmp::max(1, (N::to_usize() + 7) / 8)]; rng.fill_bytes(&mut raw_bytes); + println!("raw bytes {:?}", raw_bytes); + println!("{:?}", raw_bytes); Self::from_bytes(raw_bytes).expect("we generate a valid BitVector") } } diff --git a/validator_client/src/attestation_service.rs b/validator_client/src/attestation_service.rs index 5b7f31867bb..58883800763 100644 --- a/validator_client/src/attestation_service.rs +++ b/validator_client/src/attestation_service.rs @@ -362,9 +362,16 @@ impl AttestationService { let duty = &duty_and_proof.duty; let attestation_data = attestation_data_ref; + let fork_name = self + .context + .eth2_config + .spec + .fork_name_at_slot::(attestation_data.slot); + // Ensure that the attestation matches the duties. #[allow(clippy::suspicious_operation_groupings)] - if duty.slot != attestation_data.slot || duty.committee_index != attestation_data.index + if duty.slot != attestation_data.slot + || (fork_name < ForkName::Electra && duty.committee_index != attestation_data.index) { crit!( log, @@ -378,11 +385,26 @@ impl AttestationService { return None; } - let mut attestation = Attestation::Base(AttestationBase { - aggregation_bits: BitList::with_capacity(duty.committee_length as usize).unwrap(), - data: attestation_data.clone(), - signature: AggregateSignature::infinity(), - }); + let mut attestation = if fork_name >= ForkName::Electra { + let mut committee_bits: BitVector = BitVector::default(); + committee_bits + .set(duty.committee_index as usize, true) + .unwrap(); + Attestation::Electra(AttestationElectra { + aggregation_bits: BitList::with_capacity(duty.committee_length as usize) + .unwrap(), + data: attestation_data.clone(), + committee_bits, + signature: AggregateSignature::infinity(), + }) + } else { + Attestation::Base(AttestationBase { + aggregation_bits: BitList::with_capacity(duty.committee_length as usize) + .unwrap(), + data: attestation_data.clone(), + signature: AggregateSignature::infinity(), + }) + }; match self .validator_store @@ -531,6 +553,12 @@ impl AttestationService { .await .map_err(|e| e.to_string())?; + let fork_name = self + .context + .eth2_config + .spec + .fork_name_at_slot::(attestation_data.slot); + // Create futures to produce the signed aggregated attestations. let signing_futures = validator_duties.iter().map(|duty_and_proof| async move { let duty = &duty_and_proof.duty; @@ -539,7 +567,9 @@ impl AttestationService { let slot = attestation_data.slot; let committee_index = attestation_data.index; - if duty.slot != slot || duty.committee_index != committee_index { + if duty.slot != slot + || (fork_name < ForkName::Electra && duty.committee_index != committee_index) + { crit!(log, "Inconsistent validator duties during signing"); return None; } @@ -612,7 +642,7 @@ impl AttestationService { "aggregator" => signed_aggregate_and_proof.message().aggregator_index(), "signatures" => attestation.num_set_aggregation_bits(), "head_block" => format!("{:?}", attestation.data().beacon_block_root), - "committee_index" => attestation.data().index, + "committee_index" => attestation.committee_index(), "slot" => attestation.data().slot.as_u64(), "type" => "aggregated", ); @@ -626,7 +656,7 @@ impl AttestationService { "Failed to publish attestation"; "error" => %e, "aggregator" => signed_aggregate_and_proof.message().aggregator_index(), - "committee_index" => attestation.data().index, + "committee_index" => attestation.committee_index(), "slot" => attestation.data().slot.as_u64(), "type" => "aggregated", );