diff --git a/beacon_node/beacon_chain/src/attestation_verification.rs b/beacon_node/beacon_chain/src/attestation_verification.rs index f2e3565ae66..6044cafd6e0 100644 --- a/beacon_node/beacon_chain/src/attestation_verification.rs +++ b/beacon_node/beacon_chain/src/attestation_verification.rs @@ -61,9 +61,10 @@ use std::borrow::Cow; use strum::AsRefStr; use tree_hash::TreeHash; use types::{ - Attestation, AttestationRef, BeaconCommittee, BeaconStateError, - BeaconStateError::NoCommitteeFound, ChainSpec, CommitteeIndex, Epoch, EthSpec, ForkName, - Hash256, IndexedAttestation, SelectionProof, SignedAggregateAndProof, Slot, SubnetId, + Attestation, AttestationRef, BeaconCommittee, + BeaconStateError::{self, NoCommitteeFound}, + ChainSpec, CommitteeIndex, Epoch, EthSpec, ForkName, Hash256, IndexedAttestation, + SelectionProof, SignedAggregateAndProof, Slot, SubnetId, }; pub use batch::{batch_verify_aggregated_attestations, batch_verify_unaggregated_attestations}; @@ -519,7 +520,7 @@ impl<'a, T: BeaconChainTypes> IndexedAggregatedAttestation<'a, T> { .tree_hash_root(); // [New in Electra:EIP7549] - verify_committee_index(attestation, &chain.spec)?; + verify_committee_index(attestation)?; if chain .observed_attestations @@ -596,59 +597,62 @@ impl<'a, T: BeaconChainTypes> IndexedAggregatedAttestation<'a, T> { )) } }; + + // Committees must be sorted by ascending index order 0..committees_per_slot let get_indexed_attestation_with_committee = |(committees, _): (Vec, CommitteesPerSlot)| { - 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) - .at_most_one() - .map_err(|_| Error::NoCommitteeForSlotAndIndex { - slot: att.data.slot, - index: att.data.index, - })?; - - // TODO(electra): + let (index, aggregator_index, selection_proof, data) = match signed_aggregate { + SignedAggregateAndProof::Base(signed_aggregate) => ( + signed_aggregate.message.aggregate.data.index, + signed_aggregate.message.aggregator_index, // 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 { - let selection_proof = SelectionProof::from( - signed_aggregate.message.selection_proof.clone(), - ); - - if !selection_proof - .is_aggregator(committee.committee.len(), &chain.spec) - .map_err(|e| Error::BeaconChainError(e.into()))? - { - return Err(Error::InvalidSelectionProof { aggregator_index }); - } + signed_aggregate.message.selection_proof.clone(), + signed_aggregate.message.aggregate.data.clone(), + ), + SignedAggregateAndProof::Electra(signed_aggregate) => ( + signed_aggregate + .message + .aggregate + .committee_index() + .ok_or(Error::NotExactlyOneCommitteeBitSet(0))?, + signed_aggregate.message.aggregator_index, + signed_aggregate.message.selection_proof.clone(), + signed_aggregate.message.aggregate.data.clone(), + ), + }; + let slot = data.slot; - // Ensure the aggregator is a member of the committee for which it is aggregating. - if !committee.committee.contains(&(aggregator_index as usize)) { - return Err(Error::AggregatorNotInCommittee { aggregator_index }); - } + let committee = committees + .get(index as usize) + .ok_or(Error::NoCommitteeForSlotAndIndex { slot, index })?; - attesting_indices_base::get_indexed_attestation( - committee.committee, - att, - ) - .map_err(|e| BeaconChainError::from(e).into()) - } else { - Err(Error::NoCommitteeForSlotAndIndex { - slot: att.data.slot, - index: att.data.index, - }) - } + if !SelectionProof::from(selection_proof) + .is_aggregator(committee.committee.len(), &chain.spec) + .map_err(|e| Error::BeaconChainError(e.into()))? + { + return Err(Error::InvalidSelectionProof { aggregator_index }); + } + + // Ensure the aggregator is a member of the committee for which it is aggregating. + if !committee.committee.contains(&(aggregator_index as usize)) { + return Err(Error::AggregatorNotInCommittee { aggregator_index }); + } + + // p2p aggregates have a single committee, we can assert that aggregation_bits is always + // less then MaxValidatorsPerCommittee + match signed_aggregate { + SignedAggregateAndProof::Base(signed_aggregate) => { + attesting_indices_base::get_indexed_attestation( + committee.committee, + &signed_aggregate.message.aggregate, + ) + .map_err(|e| BeaconChainError::from(e).into()) } SignedAggregateAndProof::Electra(signed_aggregate) => { - attesting_indices_electra::get_indexed_attestation_from_signed_aggregate( + attesting_indices_electra::get_indexed_attestation( &committees, - signed_aggregate, - &chain.spec, + &signed_aggregate.message.aggregate, ) .map_err(|e| BeaconChainError::from(e).into()) } @@ -837,7 +841,7 @@ impl<'a, T: BeaconChainTypes> IndexedUnaggregatedAttestation<'a, T> { } // [New in Electra:EIP7549] - verify_committee_index(attestation, &chain.spec)?; + verify_committee_index(attestation)?; // Attestations must be for a known block. If the block is unknown, we simply drop the // attestation and do not delay consideration for later. @@ -1337,17 +1341,10 @@ pub fn verify_signed_aggregate_signatures( /// 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 { +pub fn verify_committee_index(attestation: AttestationRef) -> Result<(), Error> { + if let Ok(committee_bits) = attestation.committee_bits() { // 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()))?, - ); + let num_committee_bits = get_committee_indices::(committee_bits); if num_committee_bits.len() != 1 { return Err(Error::NotExactlyOneCommitteeBitSet( num_committee_bits.len(), @@ -1424,6 +1421,8 @@ pub fn obtain_indexed_attestation_and_committees_per_slot( /// /// 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`. +/// +/// Committees are sorted by ascending index order 0..committees_per_slot fn map_attestation_committees( chain: &BeaconChain, attestation: AttestationRef, diff --git a/beacon_node/beacon_chain/src/attester_cache.rs b/beacon_node/beacon_chain/src/attester_cache.rs index 2e07cd32ed9..b5012e8e4e4 100644 --- a/beacon_node/beacon_chain/src/attester_cache.rs +++ b/beacon_node/beacon_chain/src/attester_cache.rs @@ -15,6 +15,7 @@ use state_processing::state_advance::{partial_state_advance, Error as StateAdvan use std::collections::HashMap; use std::ops::Range; use types::{ + attestation::Error as AttestationError, beacon_state::{ compute_committee_index_in_epoch, compute_committee_range_in_epoch, epoch_committee_count, }, @@ -59,6 +60,7 @@ pub enum Error { InverseRange { range: Range, }, + AttestationError(AttestationError), } impl From for Error { diff --git a/beacon_node/beacon_chain/src/beacon_chain.rs b/beacon_node/beacon_chain/src/beacon_chain.rs index abe14796ce7..f1d9ce791e0 100644 --- a/beacon_node/beacon_chain/src/beacon_chain.rs +++ b/beacon_node/beacon_chain/src/beacon_chain.rs @@ -122,7 +122,6 @@ use store::{ use task_executor::{ShutdownReason, TaskExecutor}; use tokio_stream::Stream; use tree_hash::TreeHash; -use types::attestation::AttestationBase; use types::blob_sidecar::FixedBlobSidecarList; use types::payload::BlockProductionVersion; use types::*; @@ -1994,36 +1993,15 @@ impl BeaconChain { }; drop(cache_timer); - if self.spec.fork_name_at_slot::(request_slot) >= ForkName::Electra { - let mut committee_bits = BitVector::default(); - if committee_len > 0 { - committee_bits.set(request_index as usize, true)?; - } - Ok(Attestation::Electra(AttestationElectra { - aggregation_bits: BitList::with_capacity(committee_len)?, - data: AttestationData { - slot: request_slot, - index: 0u64, - beacon_block_root, - source: justified_checkpoint, - target, - }, - committee_bits, - signature: AggregateSignature::empty(), - })) - } else { - Ok(Attestation::Base(AttestationBase { - aggregation_bits: BitList::with_capacity(committee_len)?, - data: AttestationData { - slot: request_slot, - index: request_index, - beacon_block_root, - source: justified_checkpoint, - target, - }, - signature: AggregateSignature::empty(), - })) - } + Ok(Attestation::::empty_for_signing( + request_index, + committee_len, + request_slot, + beacon_block_root, + justified_checkpoint, + target, + &self.spec, + )?) } /// Performs the same validation as `Self::verify_unaggregated_attestation_for_gossip`, but for diff --git a/beacon_node/beacon_chain/src/early_attester_cache.rs b/beacon_node/beacon_chain/src/early_attester_cache.rs index 936f4de3ee1..dda699cc6c1 100644 --- a/beacon_node/beacon_chain/src/early_attester_cache.rs +++ b/beacon_node/beacon_chain/src/early_attester_cache.rs @@ -6,7 +6,6 @@ use crate::{ use parking_lot::RwLock; use proto_array::Block as ProtoBlock; use std::sync::Arc; -use types::attestation::AttestationBase; use types::*; pub struct CacheItem { @@ -123,40 +122,16 @@ impl EarlyAttesterCache { item.committee_lengths .get_committee_length::(request_slot, request_index, spec)?; - let attestation = if spec.fork_name_at_slot::(request_slot) >= ForkName::Electra { - let mut committee_bits = BitVector::default(); - if committee_len > 0 { - committee_bits - .set(request_index as usize, true) - .map_err(BeaconStateError::from)?; - } - Attestation::Electra(AttestationElectra { - aggregation_bits: BitList::with_capacity(committee_len) - .map_err(BeaconStateError::from)?, - committee_bits, - data: AttestationData { - slot: request_slot, - index: 0u64, - beacon_block_root: item.beacon_block_root, - source: item.source, - target: item.target, - }, - signature: AggregateSignature::empty(), - }) - } else { - Attestation::Base(AttestationBase { - aggregation_bits: BitList::with_capacity(committee_len) - .map_err(BeaconStateError::from)?, - data: AttestationData { - slot: request_slot, - index: request_index, - beacon_block_root: item.beacon_block_root, - source: item.source, - target: item.target, - }, - signature: AggregateSignature::empty(), - }) - }; + let attestation = Attestation::empty_for_signing( + request_index, + committee_len, + request_slot, + item.beacon_block_root, + item.source, + item.target, + spec, + ) + .map_err(Error::AttestationError)?; metrics::inc_counter(&metrics::BEACON_EARLY_ATTESTER_CACHE_HITS); diff --git a/beacon_node/beacon_chain/src/observed_aggregates.rs b/beacon_node/beacon_chain/src/observed_aggregates.rs index b02bff96a2d..00476bfe7af 100644 --- a/beacon_node/beacon_chain/src/observed_aggregates.rs +++ b/beacon_node/beacon_chain/src/observed_aggregates.rs @@ -473,12 +473,13 @@ where #[cfg(not(debug_assertions))] mod tests { use super::*; - use types::{test_utils::test_random_instance, Hash256}; + use types::{test_utils::test_random_instance, AttestationBase, Hash256}; type E = types::MainnetEthSpec; fn get_attestation(slot: Slot, beacon_block_root: u64) -> Attestation { - let mut a: Attestation = test_random_instance(); + let a: AttestationBase = test_random_instance(); + let mut a = Attestation::Base(a); a.data_mut().slot = slot; a.data_mut().beacon_block_root = Hash256::from_low_u64_be(beacon_block_root); a diff --git a/beacon_node/beacon_chain/src/test_utils.rs b/beacon_node/beacon_chain/src/test_utils.rs index 34ad6d43244..bd98f19af6f 100644 --- a/beacon_node/beacon_chain/src/test_utils.rs +++ b/beacon_node/beacon_chain/src/test_utils.rs @@ -58,7 +58,6 @@ use store::{config::StoreConfig, HotColdDB, ItemStore, LevelDB, MemoryStore}; use task_executor::TaskExecutor; use task_executor::{test_utils::TestRuntime, ShutdownReason}; use tree_hash::TreeHash; -use types::attestation::AttestationBase; use types::indexed_attestation::IndexedAttestationBase; use types::payload::BlockProductionVersion; pub use types::test_utils::generate_deterministic_keypairs; @@ -1033,40 +1032,18 @@ where *state.get_block_root(target_slot)? }; - if self.spec.fork_name_at_slot::(slot) >= ForkName::Electra { - let mut committee_bits = BitVector::default(); - committee_bits.set(index as usize, true)?; - Ok(Attestation::Electra(AttestationElectra { - aggregation_bits: BitList::with_capacity(committee_len)?, - committee_bits, - data: AttestationData { - slot, - index: 0u64, - beacon_block_root, - source: state.current_justified_checkpoint(), - target: Checkpoint { - epoch, - root: target_root, - }, - }, - signature: AggregateSignature::empty(), - })) - } else { - Ok(Attestation::Base(AttestationBase { - aggregation_bits: BitList::with_capacity(committee_len)?, - data: AttestationData { - slot, - index, - beacon_block_root, - source: state.current_justified_checkpoint(), - target: Checkpoint { - epoch, - root: target_root, - }, - }, - signature: AggregateSignature::empty(), - })) - } + Ok(Attestation::empty_for_signing( + index, + committee_len, + slot, + beacon_block_root, + state.current_justified_checkpoint(), + Checkpoint { + epoch, + root: target_root, + }, + &self.spec, + )?) } /// A list of attestations for each committee for the given slot. @@ -1376,37 +1353,26 @@ where 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 - }, - ) - }) + let aggregate = if fork_name.electra_enabled() { + self.chain.get_aggregated_attestation_electra( + slot, + &attestation.data().tree_hash_root(), + bc.index, + ) } 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 - }, - ) - }) - }; + } + .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. @@ -1541,40 +1507,29 @@ where let fork_name = self.spec.fork_name_at_slot::(Slot::new(0)); - let mut attestation_1 = if fork_name >= ForkName::Electra { + let 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)), + }, + }; + let mut attestation_1 = if fork_name.electra_enabled() { 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)), - }, - }, + data, 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)), - }, - }, + data, signature: AggregateSignature::infinity(), }) }; @@ -1623,7 +1578,7 @@ where } } - if fork_name >= ForkName::Electra { + if fork_name.electra_enabled() { AttesterSlashing::Electra(AttesterSlashingElectra { attestation_1: attestation_1.as_electra().unwrap().clone(), attestation_2: attestation_2.as_electra().unwrap().clone(), @@ -1657,7 +1612,7 @@ where }, }; - let (mut attestation_1, mut attestation_2) = if fork_name >= ForkName::Electra { + let (mut attestation_1, mut attestation_2) = if fork_name.electra_enabled() { let attestation_1 = IndexedAttestationElectra { attesting_indices: VariableList::new(validator_indices_1).unwrap(), data: data.clone(), @@ -1735,7 +1690,7 @@ where } } - if fork_name >= ForkName::Electra { + if fork_name.electra_enabled() { AttesterSlashing::Electra(AttesterSlashingElectra { attestation_1: attestation_1.as_electra().unwrap().clone(), attestation_2: attestation_2.as_electra().unwrap().clone(), diff --git a/beacon_node/beacon_chain/tests/attestation_production.rs b/beacon_node/beacon_chain/tests/attestation_production.rs index a58cfd9f2d1..697e449dc6e 100644 --- a/beacon_node/beacon_chain/tests/attestation_production.rs +++ b/beacon_node/beacon_chain/tests/attestation_production.rs @@ -190,35 +190,22 @@ async fn produces_attestations() { .produce_unaggregated_attestation(slot, index) .expect("should produce attestation"); - match &attestation { + let (aggregation_bits_len, aggregation_bits_zero) = match &attestation { Attestation::Base(att) => { - assert_eq!( - att.aggregation_bits.len(), - committee_len, - "bad committee len" - ); - assert!( - att.aggregation_bits.is_zero(), - "some committee bits are set" - ); + (att.aggregation_bits.len(), att.aggregation_bits.is_zero()) } Attestation::Electra(att) => { - assert_eq!( - att.aggregation_bits.len(), - committee_len, - "bad committee len" - ); - assert!( - att.aggregation_bits.is_zero(), - "some committee bits are set" - ); + (att.aggregation_bits.len(), att.aggregation_bits.is_zero()) } - } + }; + assert_eq!(aggregation_bits_len, committee_len, "bad committee len"); + assert!(aggregation_bits_zero, "some committee bits are set"); + let data = attestation.data(); assert_eq!( attestation.signature(), - &AggregateSignature::empty(), + &AggregateSignature::infinity(), "bad signature" ); assert_eq!(data.index, index, "bad index"); diff --git a/beacon_node/beacon_chain/tests/block_verification.rs b/beacon_node/beacon_chain/tests/block_verification.rs index 0fe406a0172..4c7a499697d 100644 --- a/beacon_node/beacon_chain/tests/block_verification.rs +++ b/beacon_node/beacon_chain/tests/block_verification.rs @@ -668,7 +668,7 @@ async fn invalid_signature_attester_slashing() { let mut snapshots = chain_segment.clone(); let fork_name = harness.chain.spec.fork_name_at_slot::(Slot::new(0)); - let attester_slashing = if fork_name >= ForkName::Electra { + let attester_slashing = if fork_name.electra_enabled() { let indexed_attestation = IndexedAttestationElectra { attesting_indices: vec![0].into(), data: AttestationData { @@ -1308,62 +1308,34 @@ async fn verify_block_for_gossip_doppelganger_detection() { } }; - match indexed_attestation { - IndexedAttestation::Base(indexed_attestation) => { - for &index in &indexed_attestation.attesting_indices { - let index = index as usize; - - assert!(harness.chain.validator_seen_at_epoch(index, epoch)); - - // Check the correct beacon cache is populated - assert!(harness - .chain - .observed_block_attesters - .read() - .validator_has_been_observed(epoch, index) - .expect("should check if block attester was observed")); - assert!(!harness - .chain - .observed_gossip_attesters - .read() - .validator_has_been_observed(epoch, index) - .expect("should check if gossip attester was observed")); - assert!(!harness - .chain - .observed_aggregators - .read() - .validator_has_been_observed(epoch, index) - .expect("should check if gossip aggregator was observed")); - } - } - IndexedAttestation::Electra(indexed_attestation) => { - for &index in &indexed_attestation.attesting_indices { - let index = index as usize; - - assert!(harness.chain.validator_seen_at_epoch(index, epoch)); - - // Check the correct beacon cache is populated - assert!(harness - .chain - .observed_block_attesters - .read() - .validator_has_been_observed(epoch, index) - .expect("should check if block attester was observed")); - assert!(!harness - .chain - .observed_gossip_attesters - .read() - .validator_has_been_observed(epoch, index) - .expect("should check if gossip attester was observed")); - assert!(!harness - .chain - .observed_aggregators - .read() - .validator_has_been_observed(epoch, index) - .expect("should check if gossip aggregator was observed")); - } - } - }; + for index in match indexed_attestation { + IndexedAttestation::Base(att) => att.attesting_indices.into_iter(), + IndexedAttestation::Electra(att) => att.attesting_indices.into_iter(), + } { + let index = index as usize; + + assert!(harness.chain.validator_seen_at_epoch(index, epoch)); + + // Check the correct beacon cache is populated + assert!(harness + .chain + .observed_block_attesters + .read() + .validator_has_been_observed(epoch, index) + .expect("should check if block attester was observed")); + assert!(!harness + .chain + .observed_gossip_attesters + .read() + .validator_has_been_observed(epoch, index) + .expect("should check if gossip attester was observed")); + assert!(!harness + .chain + .observed_aggregators + .read() + .validator_has_been_observed(epoch, index) + .expect("should check if gossip aggregator was observed")); + } } } diff --git a/beacon_node/beacon_chain/tests/store_tests.rs b/beacon_node/beacon_chain/tests/store_tests.rs index ef873909309..c2468102eed 100644 --- a/beacon_node/beacon_chain/tests/store_tests.rs +++ b/beacon_node/beacon_chain/tests/store_tests.rs @@ -1025,7 +1025,7 @@ async fn multiple_attestations_per_block() { let slot = snapshot.beacon_block.slot(); let fork_name = harness.chain.spec.fork_name_at_slot::(slot); - if fork_name >= ForkName::Electra { + if fork_name.electra_enabled() { assert_eq!( snapshot .beacon_block diff --git a/beacon_node/client/src/notifier.rs b/beacon_node/client/src/notifier.rs index 7c224671ff3..632188014eb 100644 --- a/beacon_node/client/src/notifier.rs +++ b/beacon_node/client/src/notifier.rs @@ -551,7 +551,7 @@ async fn electra_readiness_logging( .snapshot .beacon_state .fork_name_unchecked() - >= ForkName::Electra; + .electra_enabled(); let has_execution_layer = beacon_chain.execution_layer.is_some(); diff --git a/beacon_node/lighthouse_network/src/types/pubsub.rs b/beacon_node/lighthouse_network/src/types/pubsub.rs index 7be68f879fe..b443ecd1b9b 100644 --- a/beacon_node/lighthouse_network/src/types/pubsub.rs +++ b/beacon_node/lighthouse_network/src/types/pubsub.rs @@ -158,18 +158,19 @@ impl PubsubMessage { GossipKind::BeaconAggregateAndProof => { let signed_aggregate_and_proof = match fork_context.from_context_bytes(gossip_topic.fork_digest) { - Some(ForkName::Base) - | Some(ForkName::Altair) - | Some(ForkName::Bellatrix) - | Some(ForkName::Capella) - | Some(ForkName::Deneb) => SignedAggregateAndProof::Base( - SignedAggregateAndProofBase::from_ssz_bytes(data) - .map_err(|e| format!("{:?}", e))?, - ), - Some(ForkName::Electra) => SignedAggregateAndProof::Electra( - SignedAggregateAndProofElectra::from_ssz_bytes(data) - .map_err(|e| format!("{:?}", e))?, - ), + Some(&fork_name) => { + if fork_name.electra_enabled() { + SignedAggregateAndProof::Electra( + SignedAggregateAndProofElectra::from_ssz_bytes(data) + .map_err(|e| format!("{:?}", e))?, + ) + } else { + SignedAggregateAndProof::Base( + SignedAggregateAndProofBase::from_ssz_bytes(data) + .map_err(|e| format!("{:?}", e))?, + ) + } + } None => { return Err(format!( "Unknown gossipsub fork digest: {:?}", @@ -184,18 +185,19 @@ impl PubsubMessage { GossipKind::Attestation(subnet_id) => { let attestation = match fork_context.from_context_bytes(gossip_topic.fork_digest) { - Some(ForkName::Base) - | Some(ForkName::Altair) - | Some(ForkName::Bellatrix) - | Some(ForkName::Capella) - | Some(ForkName::Deneb) => Attestation::Base( - AttestationBase::from_ssz_bytes(data) - .map_err(|e| format!("{:?}", e))?, - ), - Some(ForkName::Electra) => Attestation::Electra( - AttestationElectra::from_ssz_bytes(data) - .map_err(|e| format!("{:?}", e))?, - ), + Some(&fork_name) => { + if fork_name.electra_enabled() { + Attestation::Electra( + AttestationElectra::from_ssz_bytes(data) + .map_err(|e| format!("{:?}", e))?, + ) + } else { + Attestation::Base( + AttestationBase::from_ssz_bytes(data) + .map_err(|e| format!("{:?}", e))?, + ) + } + } None => { return Err(format!( "Unknown gossipsub fork digest: {:?}", @@ -281,18 +283,19 @@ impl PubsubMessage { GossipKind::AttesterSlashing => { let attester_slashing = match fork_context.from_context_bytes(gossip_topic.fork_digest) { - Some(ForkName::Base) - | Some(ForkName::Altair) - | Some(ForkName::Bellatrix) - | Some(ForkName::Capella) - | Some(ForkName::Deneb) => AttesterSlashing::Base( - AttesterSlashingBase::from_ssz_bytes(data) - .map_err(|e| format!("{:?}", e))?, - ), - Some(ForkName::Electra) => AttesterSlashing::Electra( - AttesterSlashingElectra::from_ssz_bytes(data) - .map_err(|e| format!("{:?}", e))?, - ), + Some(&fork_name) => { + if fork_name.electra_enabled() { + AttesterSlashing::Electra( + AttesterSlashingElectra::from_ssz_bytes(data) + .map_err(|e| format!("{:?}", e))?, + ) + } else { + AttesterSlashing::Base( + AttesterSlashingBase::from_ssz_bytes(data) + .map_err(|e| format!("{:?}", e))?, + ) + } + } None => { return Err(format!( "Unknown gossipsub fork digest: {:?}", 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 34eb8b26b13..8cbc7782188 100644 --- a/beacon_node/network/src/network_beacon_processor/gossip_methods.rs +++ b/beacon_node/network/src/network_beacon_processor/gossip_methods.rs @@ -105,12 +105,7 @@ impl VerifiedAttestation for VerifiedAggregate { /// Efficient clone-free implementation that moves out of the `Box`. fn into_attestation_and_indices(self) -> (Attestation, Vec) { - // TODO(electra): technically we shouldn't have to clone.. - let attestation = self - .signed_aggregate - .message() - .aggregate() - .clone_as_attestation(); + let attestation = self.signed_aggregate.into_attestation(); let attesting_indices = self.indexed_attestation.attesting_indices_to_vec(); (attestation, attesting_indices) } diff --git a/beacon_node/operation_pool/src/attestation_storage.rs b/beacon_node/operation_pool/src/attestation_storage.rs index 43b1c3abbb3..ad4ffe6d3c1 100644 --- a/beacon_node/operation_pool/src/attestation_storage.rs +++ b/beacon_node/operation_pool/src/attestation_storage.rs @@ -217,7 +217,6 @@ impl CompactIndexedAttestationElectra { } pub fn aggregate_same_committee(&mut self, other: &Self) -> Option<()> { - // TODO(electra): remove assert in favour of Result if self.committee_bits != other.committee_bits { return None; } diff --git a/beacon_node/operation_pool/src/lib.rs b/beacon_node/operation_pool/src/lib.rs index f9021bb258d..49fbed56869 100644 --- a/beacon_node/operation_pool/src/lib.rs +++ b/beacon_node/operation_pool/src/lib.rs @@ -286,7 +286,7 @@ impl OperationPool { // TODO(electra): Work out how to do this more elegantly. This is a bit of a hack. let mut all_attestations = self.attestations.write(); - if fork_name >= ForkName::Electra { + if fork_name.electra_enabled() { all_attestations.aggregate_across_committees(prev_epoch_key); all_attestations.aggregate_across_committees(curr_epoch_key); } diff --git a/common/eth2/src/types.rs b/common/eth2/src/types.rs index 2bb749af9f6..c945e09619f 100644 --- a/common/eth2/src/types.rs +++ b/common/eth2/src/types.rs @@ -716,6 +716,21 @@ pub struct AttesterData { pub slot: Slot, } +impl AttesterData { + pub fn match_attestation_data( + &self, + attestation_data: &AttestationData, + spec: &ChainSpec, + ) -> bool { + if spec.fork_name_at_slot::(attestation_data.slot) < ForkName::Electra { + self.slot == attestation_data.slot && self.committee_index == attestation_data.index + } else { + // After electra `attestation_data.index` is set to 0 and does not match the duties + self.slot == attestation_data.slot + } + } +} + #[derive(Debug, Clone, PartialEq, Serialize, Deserialize)] pub struct ProposerData { pub pubkey: PublicKeyBytes, diff --git a/consensus/state_processing/src/common/get_attesting_indices.rs b/consensus/state_processing/src/common/get_attesting_indices.rs index a79604003ee..457a5261041 100644 --- a/consensus/state_processing/src/common/get_attesting_indices.rs +++ b/consensus/state_processing/src/common/get_attesting_indices.rs @@ -44,107 +44,16 @@ pub mod attesting_indices_base { } pub mod attesting_indices_electra { - use std::collections::{HashMap, HashSet}; + use std::collections::HashSet; use crate::per_block_processing::errors::{AttestationInvalid as Invalid, BlockOperationError}; use itertools::Itertools; 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(), - })) - } - + /// Compute an Electra IndexedAttestation given a list of committees. + /// + /// Committees must be sorted by ascending order 0..committees_per_slot pub fn get_indexed_attestation( committees: &[BeaconCommittee], attestation: &AttestationElectra, @@ -167,17 +76,7 @@ pub mod attesting_indices_electra { attestation: &AttestationElectra, ) -> Result, BlockOperationError> { let committees = beacon_state.get_beacon_committees_at_slot(attestation.data.slot)?; - let attesting_indices = get_attesting_indices::( - &committees, - &attestation.aggregation_bits, - &attestation.committee_bits, - )?; - - Ok(IndexedAttestation::Electra(IndexedAttestationElectra { - attesting_indices: VariableList::new(attesting_indices)?, - data: attestation.data.clone(), - signature: attestation.signature.clone(), - })) + get_indexed_attestation(&committees, attestation) } /// Shortcut for getting the attesting indices while fetching the committee from the state's cache. @@ -190,6 +89,8 @@ pub mod attesting_indices_electra { } /// Returns validator indices which participated in the attestation, sorted by increasing index. + /// + /// Committees must be sorted by ascending order 0..committees_per_slot pub fn get_attesting_indices( committees: &[BeaconCommittee], aggregation_bits: &BitList, @@ -201,40 +102,35 @@ pub mod attesting_indices_electra { 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; for index in committee_indices { - if let Some(&beacon_committee) = committees_map.get(&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); - } + let beacon_committee = committees + .get(index as usize) + .ok_or(Error::NoCommitteeFound(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::>(); + } + None + }) + .collect::>(); - output.extend(committee_attesters); + output.extend(committee_attesters); - committee_offset.safe_add_assign(beacon_committee.committee.len())?; - } else { - return Err(Error::NoCommitteeFound(index)); - } + committee_offset.safe_add_assign(beacon_committee.committee.len())?; } // This check is new to the spec's `process_attestation` in Electra. diff --git a/consensus/state_processing/src/per_block_processing/errors.rs b/consensus/state_processing/src/per_block_processing/errors.rs index 336895514f9..71a284dea49 100644 --- a/consensus/state_processing/src/per_block_processing/errors.rs +++ b/consensus/state_processing/src/per_block_processing/errors.rs @@ -89,6 +89,7 @@ pub enum BlockProcessingError { found: Hash256, }, WithdrawalCredentialsInvalid, + PendingAttestationInElectra, } impl From for BlockProcessingError { diff --git a/consensus/state_processing/src/per_block_processing/process_operations.rs b/consensus/state_processing/src/per_block_processing/process_operations.rs index 9b4e9d71394..bd354901a8b 100644 --- a/consensus/state_processing/src/per_block_processing/process_operations.rs +++ b/consensus/state_processing/src/per_block_processing/process_operations.rs @@ -76,33 +76,30 @@ pub mod base { ) .map_err(|e| e.into_with_index(i))?; - match attestation { - AttestationRef::Base(att) => { - let pending_attestation = PendingAttestation { - aggregation_bits: att.aggregation_bits.clone(), - data: att.data.clone(), - inclusion_delay: state.slot().safe_sub(att.data.slot)?.as_u64(), - proposer_index, - }; - - if attestation.data().target.epoch == state.current_epoch() { - state - .as_base_mut()? - .current_epoch_attestations - .push(pending_attestation)?; - } else { - state - .as_base_mut()? - .previous_epoch_attestations - .push(pending_attestation)?; - } - } - AttestationRef::Electra(_) => { - // TODO(electra) pending attestations are only phase 0 - // so we should just raise a relevant error here - todo!() - } + let AttestationRef::Base(attestation) = attestation else { + // Pending attestations have been deprecated in a altair, this branch should + // never happen + return Err(BlockProcessingError::PendingAttestationInElectra); }; + + let pending_attestation = PendingAttestation { + aggregation_bits: attestation.aggregation_bits.clone(), + data: attestation.data.clone(), + inclusion_delay: state.slot().safe_sub(attestation.data.slot)?.as_u64(), + proposer_index, + }; + + if attestation.data.target.epoch == state.current_epoch() { + state + .as_base_mut()? + .current_epoch_attestations + .push(pending_attestation)?; + } else { + state + .as_base_mut()? + .previous_epoch_attestations + .push(pending_attestation)?; + } } Ok(()) diff --git a/consensus/types/src/aggregate_and_proof.rs b/consensus/types/src/aggregate_and_proof.rs index e8fa01c143e..ae80369130a 100644 --- a/consensus/types/src/aggregate_and_proof.rs +++ b/consensus/types/src/aggregate_and_proof.rs @@ -4,6 +4,7 @@ use super::{ SignedRoot, }; use crate::test_utils::TestRandom; +use crate::Attestation; use serde::{Deserialize, Serialize}; use ssz_derive::{Decode, Encode}; use superstruct::superstruct; @@ -88,28 +89,39 @@ impl AggregateAndProof { genesis_validators_root: Hash256, spec: &ChainSpec, ) -> Self { - let selection_proof = selection_proof - .unwrap_or_else(|| { - SelectionProof::new::( - aggregate.data().slot, - secret_key, - fork, - genesis_validators_root, - spec, - ) - }) - .into(); + let selection_proof = selection_proof.unwrap_or_else(|| { + SelectionProof::new::( + aggregate.data().slot, + secret_key, + fork, + genesis_validators_root, + spec, + ) + }); + Self::from_attestation( + aggregator_index, + aggregate.clone_as_attestation(), + selection_proof, + ) + } + + /// Produces a new `AggregateAndProof` given a `selection_proof` + pub fn from_attestation( + aggregator_index: u64, + aggregate: Attestation, + selection_proof: SelectionProof, + ) -> Self { match aggregate { - AttestationRef::Base(attestation) => Self::Base(AggregateAndProofBase { + Attestation::Base(aggregate) => Self::Base(AggregateAndProofBase { aggregator_index, - aggregate: attestation.clone(), - selection_proof, + aggregate, + selection_proof: selection_proof.into(), }), - AttestationRef::Electra(attestation) => Self::Electra(AggregateAndProofElectra { + Attestation::Electra(aggregate) => Self::Electra(AggregateAndProofElectra { aggregator_index, - aggregate: attestation.clone(), - selection_proof, + aggregate, + selection_proof: selection_proof.into(), }), } } diff --git a/consensus/types/src/attestation.rs b/consensus/types/src/attestation.rs index 3df14feadef..c8d1c3fb9b7 100644 --- a/consensus/types/src/attestation.rs +++ b/consensus/types/src/attestation.rs @@ -1,7 +1,7 @@ use crate::slot_data::SlotData; use crate::{test_utils::TestRandom, Hash256, Slot}; +use crate::{Checkpoint, ForkName}; use derivative::Derivative; -use rand::RngCore; use safe_arith::ArithError; use serde::{Deserialize, Serialize}; use ssz_derive::{Decode, Encode}; @@ -22,6 +22,8 @@ pub enum Error { AlreadySigned(usize), SubnetCountIsZero(ArithError), IncorrectStateVariant, + InvalidCommitteeLength, + InvalidCommitteeIndex, } #[superstruct( @@ -74,23 +76,6 @@ pub struct Attestation { pub signature: AggregateSignature, } -// 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::random_for_test(rng); - let data = AttestationData::random_for_test(rng); - let signature = AggregateSignature::random_for_test(rng); - let committee_bits = BitVector::random_for_test(rng); - - Self::Electra(AttestationElectra { - aggregation_bits, - committee_bits, - data, - signature, - }) - } -} - impl Hash for Attestation { fn hash(&self, state: &mut H) where @@ -104,6 +89,50 @@ impl Hash for Attestation { } impl Attestation { + /// Produces an attestation with empty signature. + pub fn empty_for_signing( + committee_index: u64, + committee_length: usize, + slot: Slot, + beacon_block_root: Hash256, + source: Checkpoint, + target: Checkpoint, + spec: &ChainSpec, + ) -> Result { + if spec.fork_name_at_slot::(slot) >= ForkName::Electra { + let mut committee_bits: BitVector = BitVector::default(); + committee_bits + .set(committee_index as usize, true) + .map_err(|_| Error::InvalidCommitteeIndex)?; + Ok(Attestation::Electra(AttestationElectra { + aggregation_bits: BitList::with_capacity(committee_length) + .map_err(|_| Error::InvalidCommitteeLength)?, + data: AttestationData { + slot, + index: 0u64, + beacon_block_root, + source, + target, + }, + committee_bits, + signature: AggregateSignature::infinity(), + })) + } else { + Ok(Attestation::Base(AttestationBase { + aggregation_bits: BitList::with_capacity(committee_length) + .map_err(|_| Error::InvalidCommitteeLength)?, + data: AttestationData { + slot, + index: committee_index, + beacon_block_root, + source, + target, + }, + signature: AggregateSignature::infinity(), + })) + } + } + /// Aggregate another Attestation into this one. /// /// The aggregation bitfields must be disjoint, and the data must be the same. diff --git a/consensus/types/src/beacon_block_body.rs b/consensus/types/src/beacon_block_body.rs index 76a2fc1872e..9cf66184f8d 100644 --- a/consensus/types/src/beacon_block_body.rs +++ b/consensus/types/src/beacon_block_body.rs @@ -820,6 +820,11 @@ impl From>> } impl BeaconBlockBody { + /// Returns the name of the fork pertaining to `self`. + pub fn fork_name(&self) -> ForkName { + self.to_ref().fork_name() + } + pub fn block_body_merkle_proof(&self, generalized_index: usize) -> Result, Error> { let field_index = match generalized_index { light_client_update::EXECUTION_PAYLOAD_INDEX => { @@ -834,22 +839,16 @@ impl BeaconBlockBody { _ => return Err(Error::IndexNotSupported(generalized_index)), }; - let attestations_root = match self { - BeaconBlockBody::Base(_) - | BeaconBlockBody::Altair(_) - | BeaconBlockBody::Bellatrix(_) - | BeaconBlockBody::Capella(_) - | BeaconBlockBody::Deneb(_) => self.attestations_base()?.tree_hash_root(), - BeaconBlockBody::Electra(_) => self.attestations_electra()?.tree_hash_root(), + let attestations_root = if self.fork_name() > ForkName::Electra { + self.attestations_electra()?.tree_hash_root() + } else { + self.attestations_base()?.tree_hash_root() }; - let attester_slashings_root = match self { - BeaconBlockBody::Base(_) - | BeaconBlockBody::Altair(_) - | BeaconBlockBody::Bellatrix(_) - | BeaconBlockBody::Capella(_) - | BeaconBlockBody::Deneb(_) => self.attester_slashings_base()?.tree_hash_root(), - BeaconBlockBody::Electra(_) => self.attester_slashings_electra()?.tree_hash_root(), + let attester_slashings_root = if self.fork_name() > ForkName::Electra { + self.attester_slashings_electra()?.tree_hash_root() + } else { + self.attester_slashings_base()?.tree_hash_root() }; let mut leaves = vec![ diff --git a/consensus/types/src/beacon_state/committee_cache.rs b/consensus/types/src/beacon_state/committee_cache.rs index 209659ea889..161f8541573 100644 --- a/consensus/types/src/beacon_state/committee_cache.rs +++ b/consensus/types/src/beacon_state/committee_cache.rs @@ -183,6 +183,8 @@ impl CommitteeCache { } /// Get all the Beacon committees at a given `slot`. + /// + /// Committees are sorted by ascending index order 0..committees_per_slot pub fn get_beacon_committees_at_slot(&self, slot: Slot) -> Result, Error> { if self.initialized_epoch.is_none() { return Err(Error::CommitteeCacheUninitialized(None)); diff --git a/consensus/types/src/chain_spec.rs b/consensus/types/src/chain_spec.rs index b0346a14ef8..7609e360355 100644 --- a/consensus/types/src/chain_spec.rs +++ b/consensus/types/src/chain_spec.rs @@ -376,7 +376,7 @@ impl ChainSpec { state: &BeaconState, ) -> u64 { let fork_name = state.fork_name_unchecked(); - if fork_name >= ForkName::Electra { + if fork_name.electra_enabled() { self.min_slashing_penalty_quotient_electra } else if fork_name >= ForkName::Bellatrix { self.min_slashing_penalty_quotient_bellatrix diff --git a/consensus/types/src/eth_spec.rs b/consensus/types/src/eth_spec.rs index cdbab2d5406..1c379f5de42 100644 --- a/consensus/types/src/eth_spec.rs +++ b/consensus/types/src/eth_spec.rs @@ -545,10 +545,12 @@ impl EthSpec for GnosisEthSpec { #[cfg(test)] mod test { use crate::{EthSpec, GnosisEthSpec, MainnetEthSpec, MinimalEthSpec}; + use ssz_types::typenum::Unsigned; fn assert_valid_spec() { E::kzg_commitments_tree_depth(); E::block_body_tree_depth(); + assert!(E::MaxValidatorsPerSlot::to_i32() >= E::MaxValidatorsPerCommittee::to_i32()); } #[test] diff --git a/consensus/types/src/fork_name.rs b/consensus/types/src/fork_name.rs index 5cc66214733..f0810e2bdbe 100644 --- a/consensus/types/src/fork_name.rs +++ b/consensus/types/src/fork_name.rs @@ -119,6 +119,10 @@ impl ForkName { ForkName::Electra => None, } } + + pub fn electra_enabled(self) -> bool { + self >= ForkName::Electra + } } /// Map a fork name into a fork-versioned superstruct type like `BeaconBlock`. diff --git a/consensus/types/src/indexed_attestation.rs b/consensus/types/src/indexed_attestation.rs index d282e2f259c..900c9059389 100644 --- a/consensus/types/src/indexed_attestation.rs +++ b/consensus/types/src/indexed_attestation.rs @@ -1,9 +1,7 @@ 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; use ssz_derive::{Decode, Encode}; use std::hash::{Hash, Hasher}; @@ -116,11 +114,14 @@ impl IndexedAttestation { } } - pub fn to_electra(self) -> Result, ssz_types::Error> { - Ok(match self { + pub fn to_electra(self) -> IndexedAttestationElectra { + match self { Self::Base(att) => { let extended_attesting_indices: VariableList = - VariableList::new(att.attesting_indices.to_vec())?; + VariableList::new(att.attesting_indices.to_vec()) + .expect("MaxValidatorsPerSlot must be >= MaxValidatorsPerCommittee"); + // Note a unit test in consensus/types/src/eth_spec.rs asserts this invariant for + // all known specs IndexedAttestationElectra { attesting_indices: extended_attesting_indices, @@ -129,7 +130,7 @@ impl IndexedAttestation { } } Self::Electra(att) => att, - }) + } } } @@ -186,43 +187,6 @@ impl<'a, E: EthSpec> IndexedAttestationRef<'a, E> { } } -impl Decode for IndexedAttestation { - fn is_ssz_fixed_len() -> bool { - false - } - - fn from_ssz_bytes(bytes: &[u8]) -> Result { - if let Ok(result) = IndexedAttestationBase::from_ssz_bytes(bytes) { - return Ok(IndexedAttestation::Base(result)); - } - - if let Ok(result) = IndexedAttestationElectra::from_ssz_bytes(bytes) { - return Ok(IndexedAttestation::Electra(result)); - } - - Err(ssz::DecodeError::BytesInvalid(String::from( - "bytes not valid for any fork variant", - ))) - } -} - -// TODO(electra): think about how to handle fork variants here -impl TestRandom for IndexedAttestation { - fn random_for_test(rng: &mut impl RngCore) -> Self { - let attesting_indices = VariableList::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(IndexedAttestationBase { - attesting_indices, - // committee_bits, - data, - signature, - }) - } -} - /// Implementation of non-crypto-secure `Hash`, for use with `HashMap` and `HashSet`. /// /// Guarantees `att1 == att2 -> hash(att1) == hash(att2)`. @@ -276,6 +240,38 @@ mod quoted_variable_list_u64 { } } +#[derive(Debug, Clone, Encode, Decode, PartialEq)] +#[ssz(enum_behaviour = "union")] +pub enum IndexedAttestationOnDisk { + Base(IndexedAttestationBase), + Electra(IndexedAttestationElectra), +} + +#[derive(Debug, Clone, Encode, PartialEq)] +#[ssz(enum_behaviour = "union")] +pub enum IndexedAttestationRefOnDisk<'a, E: EthSpec> { + Base(&'a IndexedAttestationBase), + Electra(&'a IndexedAttestationElectra), +} + +impl<'a, E: EthSpec> From<&'a IndexedAttestation> for IndexedAttestationRefOnDisk<'a, E> { + fn from(attestation: &'a IndexedAttestation) -> Self { + match attestation { + IndexedAttestation::Base(attestation) => Self::Base(attestation), + IndexedAttestation::Electra(attestation) => Self::Electra(attestation), + } + } +} + +impl From> for IndexedAttestation { + fn from(attestation: IndexedAttestationOnDisk) -> Self { + match attestation { + IndexedAttestationOnDisk::Base(attestation) => Self::Base(attestation), + IndexedAttestationOnDisk::Electra(attestation) => Self::Electra(attestation), + } + } +} + #[cfg(test)] mod tests { use super::*; @@ -331,14 +327,22 @@ mod tests { assert!(!indexed_vote_first.is_surround_vote(&indexed_vote_second)); } - ssz_and_tree_hash_tests!(IndexedAttestation); + mod base { + use super::*; + ssz_and_tree_hash_tests!(IndexedAttestationBase); + } + mod electra { + use super::*; + ssz_and_tree_hash_tests!(IndexedAttestationElectra); + } fn create_indexed_attestation( target_epoch: u64, source_epoch: u64, ) -> IndexedAttestation { let mut rng = XorShiftRng::from_seed([42; 16]); - let mut indexed_vote = IndexedAttestation::random_for_test(&mut rng); + let mut indexed_vote = + IndexedAttestation::Base(IndexedAttestationBase::random_for_test(&mut rng)); indexed_vote.data_mut().source.epoch = Epoch::new(source_epoch); indexed_vote.data_mut().target.epoch = Epoch::new(target_epoch); diff --git a/consensus/types/src/lib.rs b/consensus/types/src/lib.rs index 75456446bf2..2a73ff0f37a 100644 --- a/consensus/types/src/lib.rs +++ b/consensus/types/src/lib.rs @@ -176,7 +176,8 @@ pub use crate::fork_versioned_response::{ForkVersionDeserialize, ForkVersionedRe pub use crate::graffiti::{Graffiti, GRAFFITI_BYTES_LEN}; pub use crate::historical_batch::HistoricalBatch; pub use crate::indexed_attestation::{ - IndexedAttestation, IndexedAttestationBase, IndexedAttestationElectra, IndexedAttestationRef, + IndexedAttestation, IndexedAttestationBase, IndexedAttestationElectra, + IndexedAttestationOnDisk, IndexedAttestationRef, IndexedAttestationRefOnDisk, }; pub use crate::light_client_bootstrap::{ LightClientBootstrap, LightClientBootstrapAltair, LightClientBootstrapCapella, diff --git a/consensus/types/src/signed_aggregate_and_proof.rs b/consensus/types/src/signed_aggregate_and_proof.rs index ddf1dedb040..d339cecaae0 100644 --- a/consensus/types/src/signed_aggregate_and_proof.rs +++ b/consensus/types/src/signed_aggregate_and_proof.rs @@ -6,6 +6,7 @@ use super::{ Signature, SignedRoot, }; use crate::test_utils::TestRandom; +use crate::Attestation; use serde::{Deserialize, Serialize}; use ssz_derive::{Decode, Encode}; use superstruct::superstruct; @@ -83,17 +84,19 @@ impl SignedAggregateAndProof { ); let signing_message = message.signing_root(domain); - match message { + Self::from_aggregate_and_proof(message, secret_key.sign(signing_message)) + } + + /// Produces a new `SignedAggregateAndProof` given a `signature` of `aggregate` + pub fn from_aggregate_and_proof(aggregate: AggregateAndProof, signature: Signature) -> Self { + match aggregate { AggregateAndProof::Base(message) => { - SignedAggregateAndProof::Base(SignedAggregateAndProofBase { - message, - signature: secret_key.sign(signing_message), - }) + SignedAggregateAndProof::Base(SignedAggregateAndProofBase { message, signature }) } AggregateAndProof::Electra(message) => { SignedAggregateAndProof::Electra(SignedAggregateAndProofElectra { message, - signature: secret_key.sign(signing_message), + signature, }) } } @@ -107,4 +110,11 @@ impl SignedAggregateAndProof { } } } + + pub fn into_attestation(self) -> Attestation { + match self { + Self::Base(att) => Attestation::Base(att.message.aggregate), + Self::Electra(att) => Attestation::Electra(att.message.aggregate), + } + } } diff --git a/consensus/types/src/validator.rs b/consensus/types/src/validator.rs index ba99a0a68d8..f2b36ee1533 100644 --- a/consensus/types/src/validator.rs +++ b/consensus/types/src/validator.rs @@ -63,7 +63,7 @@ impl Validator { spec: &ChainSpec, current_fork: ForkName, ) -> bool { - if current_fork >= ForkName::Electra { + if current_fork.electra_enabled() { self.is_eligible_for_activation_queue_electra(spec) } else { self.is_eligible_for_activation_queue_base(spec) @@ -162,7 +162,7 @@ impl Validator { spec: &ChainSpec, current_fork: ForkName, ) -> bool { - if current_fork >= ForkName::Electra { + if current_fork.electra_enabled() { self.is_fully_withdrawable_at_electra(balance, epoch, spec) } else { self.is_fully_withdrawable_at_capella(balance, epoch, spec) @@ -202,7 +202,7 @@ impl Validator { spec: &ChainSpec, current_fork: ForkName, ) -> bool { - if current_fork >= ForkName::Electra { + if current_fork.electra_enabled() { self.is_partially_withdrawable_validator_electra(balance, spec) } else { self.is_partially_withdrawable_validator_capella(balance, spec) diff --git a/slasher/src/array.rs b/slasher/src/array.rs index 714ccf4e77b..77ddceb85fe 100644 --- a/slasher/src/array.rs +++ b/slasher/src/array.rs @@ -5,7 +5,6 @@ use crate::{ }; use flate2::bufread::{ZlibDecoder, ZlibEncoder}; use serde::{Deserialize, Serialize}; -use slog::Logger; use std::borrow::Borrow; use std::collections::{btree_map::Entry, BTreeMap, HashSet}; use std::io::Read; @@ -486,7 +485,6 @@ pub fn update( batch: Vec>>, current_epoch: Epoch, config: &Config, - log: &Logger, ) -> Result>, Error> { // Split the batch up into horizontal segments. // Map chunk indexes in the range `0..self.config.chunk_size` to attestations @@ -506,7 +504,6 @@ pub fn update( &chunk_attestations, current_epoch, config, - log, )?; slashings.extend(update_array::<_, MaxTargetChunk>( db, @@ -515,7 +512,6 @@ pub fn update( &chunk_attestations, current_epoch, config, - log, )?); // Update all current epochs. @@ -574,7 +570,6 @@ pub fn update_array( chunk_attestations: &BTreeMap>>>, current_epoch: Epoch, config: &Config, - log: &Logger, ) -> Result>, Error> { let mut slashings = HashSet::new(); // Map from chunk index to updated chunk at that index. @@ -608,19 +603,8 @@ pub fn update_array( current_epoch, config, )?; - match slashing_status.into_slashing(&attestation.indexed) { - Ok(Some(slashing)) => { - slashings.insert(slashing); - } - Err(e) => { - slog::error!( - log, - "Invalid slashing conversion"; - "validator_index" => validator_index, - "error" => e - ); - } - Ok(None) => {} + if let Some(slashing) = slashing_status.into_slashing(&attestation.indexed) { + slashings.insert(slashing); } } } diff --git a/slasher/src/database.rs b/slasher/src/database.rs index d80112c553d..16089706a00 100644 --- a/slasher/src/database.rs +++ b/slasher/src/database.rs @@ -18,11 +18,12 @@ use std::marker::PhantomData; use std::sync::Arc; use tree_hash::TreeHash; use types::{ - Epoch, EthSpec, Hash256, IndexedAttestation, ProposerSlashing, SignedBeaconBlockHeader, Slot, + Epoch, EthSpec, Hash256, IndexedAttestation, IndexedAttestationOnDisk, + IndexedAttestationRefOnDisk, ProposerSlashing, SignedBeaconBlockHeader, Slot, }; /// Current database schema version, to check compatibility of on-disk DB with software. -pub const CURRENT_SCHEMA_VERSION: u64 = 3; +pub const CURRENT_SCHEMA_VERSION: u64 = 4; /// Metadata about the slashing database itself. const METADATA_DB: &str = "metadata"; @@ -457,7 +458,9 @@ impl SlasherDB { }; let attestation_key = IndexedAttestationId::new(indexed_att_id); - let data = indexed_attestation.as_ssz_bytes(); + let indexed_attestation_on_disk: IndexedAttestationRefOnDisk = + indexed_attestation.into(); + let data = indexed_attestation_on_disk.as_ssz_bytes(); cursor.put(attestation_key.as_ref(), &data)?; drop(cursor); @@ -481,7 +484,8 @@ impl SlasherDB { .ok_or(Error::MissingIndexedAttestation { id: indexed_attestation_id.as_u64(), })?; - ssz_decode(bytes) + let indexed_attestation: IndexedAttestationOnDisk = ssz_decode(bytes)?; + Ok(indexed_attestation.into()) } fn get_attestation_data_root( diff --git a/slasher/src/lib.rs b/slasher/src/lib.rs index 0c097ac77fb..4d58fa77029 100644 --- a/slasher/src/lib.rs +++ b/slasher/src/lib.rs @@ -53,11 +53,11 @@ impl AttesterSlashingStatus { pub fn into_slashing( self, new_attestation: &IndexedAttestation, - ) -> Result>, String> { + ) -> Option> { use AttesterSlashingStatus::*; // The surrounding attestation must be in `attestation_1` to be valid. - Ok(match self { + match self { NotSlashable => None, AlreadyDoubleVoted => None, DoubleVote(existing) | SurroundedByExisting(existing) => { @@ -70,14 +70,8 @@ impl AttesterSlashingStatus { } // A slashing involving an electra attestation type must return an `AttesterSlashingElectra` type (_, _) => Some(AttesterSlashing::Electra(AttesterSlashingElectra { - attestation_1: existing - .clone() - .to_electra() - .map_err(|e| format!("{e:?}"))?, - attestation_2: new_attestation - .clone() - .to_electra() - .map_err(|e| format!("{e:?}"))?, + attestation_1: existing.clone().to_electra(), + attestation_2: new_attestation.clone().to_electra(), })), } } @@ -90,16 +84,10 @@ impl AttesterSlashingStatus { } // A slashing involving an electra attestation type must return an `AttesterSlashingElectra` type (_, _) => Some(AttesterSlashing::Electra(AttesterSlashingElectra { - attestation_1: new_attestation - .clone() - .to_electra() - .map_err(|e| format!("{e:?}"))?, - attestation_2: existing - .clone() - .to_electra() - .map_err(|e| format!("{e:?}"))?, + attestation_1: new_attestation.clone().to_electra(), + attestation_2: existing.clone().to_electra(), })), }, - }) + } } } diff --git a/slasher/src/migrate.rs b/slasher/src/migrate.rs index 674ab9c132d..ce298caaf2c 100644 --- a/slasher/src/migrate.rs +++ b/slasher/src/migrate.rs @@ -17,6 +17,10 @@ impl SlasherDB { software_schema_version: CURRENT_SCHEMA_VERSION, }), (x, y) if x == y => Ok(self), + (3, 4) => { + // TODO(electra): db migration due to `IndexedAttestationOnDisk` + Ok(self) + } (_, _) => Err(Error::IncompatibleSchemaVersion { database_schema_version: schema_version, software_schema_version: CURRENT_SCHEMA_VERSION, diff --git a/slasher/src/slasher.rs b/slasher/src/slasher.rs index ef6dcce9b1b..fc8e8453c89 100644 --- a/slasher/src/slasher.rs +++ b/slasher/src/slasher.rs @@ -247,7 +247,6 @@ impl Slasher { batch, current_epoch, &self.config, - &self.log, ) { Ok(slashings) => { if !slashings.is_empty() { @@ -295,25 +294,14 @@ impl Slasher { indexed_attestation_id, )?; - match slashing_status.into_slashing(attestation) { - Ok(Some(slashing)) => { - debug!( - self.log, - "Found double-vote slashing"; - "validator_index" => validator_index, - "epoch" => slashing.attestation_1().data().target.epoch, - ); - slashings.insert(slashing); - } - Err(e) => { - error!( - self.log, - "Invalid slashing conversion"; - "validator_index" => validator_index, - "error" => e - ); - } - Ok(None) => {} + if let Some(slashing) = slashing_status.into_slashing(attestation) { + debug!( + self.log, + "Found double-vote slashing"; + "validator_index" => validator_index, + "epoch" => slashing.attestation_1().data().target.epoch, + ); + slashings.insert(slashing); } } diff --git a/slasher/src/test_utils.rs b/slasher/src/test_utils.rs index ef623911bfa..7019a8aed76 100644 --- a/slasher/src/test_utils.rs +++ b/slasher/src/test_utils.rs @@ -71,8 +71,8 @@ pub fn att_slashing( } // A slashing involving an electra attestation type must return an electra AttesterSlashing type (_, _) => AttesterSlashing::Electra(AttesterSlashingElectra { - attestation_1: attestation_1.clone().to_electra().unwrap(), - attestation_2: attestation_2.clone().to_electra().unwrap(), + attestation_1: attestation_1.clone().to_electra(), + attestation_2: attestation_2.clone().to_electra(), }), } } diff --git a/testing/ef_tests/src/cases/fork_choice.rs b/testing/ef_tests/src/cases/fork_choice.rs index b3b20d6efe1..2a2cc067e58 100644 --- a/testing/ef_tests/src/cases/fork_choice.rs +++ b/testing/ef_tests/src/cases/fork_choice.rs @@ -131,7 +131,6 @@ pub struct ForkChoiceTest { pub anchor_state: BeaconState, pub anchor_block: BeaconBlock, #[allow(clippy::type_complexity)] - // TODO(electra): these tests will need to be updated to use new types pub steps: Vec< Step, BlobsList, Attestation, AttesterSlashing, PowBlock>, >, @@ -180,42 +179,34 @@ impl LoadCase for ForkChoiceTest { valid, }) } - Step::Attestation { attestation } => match fork_name { - ForkName::Base - | ForkName::Altair - | ForkName::Bellatrix - | ForkName::Capella - | ForkName::Deneb => ssz_decode_file( - &path.join(format!("{}.ssz_snappy", attestation)), - ) - .map(|attestation| Step::Attestation { - attestation: Attestation::Base(attestation), - }), - ForkName::Electra => ssz_decode_file( - &path.join(format!("{}.ssz_snappy", attestation)), - ) - .map(|attestation| Step::Attestation { - attestation: Attestation::Electra(attestation), - }), - }, - Step::AttesterSlashing { attester_slashing } => match fork_name { - ForkName::Base - | ForkName::Altair - | ForkName::Bellatrix - | ForkName::Capella - | ForkName::Deneb => { + Step::Attestation { attestation } => { + if fork_name.electra_enabled() { + ssz_decode_file(&path.join(format!("{}.ssz_snappy", attestation))).map( + |attestation| Step::Attestation { + attestation: Attestation::Electra(attestation), + }, + ) + } else { + ssz_decode_file(&path.join(format!("{}.ssz_snappy", attestation))).map( + |attestation| Step::Attestation { + attestation: Attestation::Base(attestation), + }, + ) + } + } + Step::AttesterSlashing { attester_slashing } => { + if fork_name.electra_enabled() { ssz_decode_file(&path.join(format!("{}.ssz_snappy", attester_slashing))) .map(|attester_slashing| Step::AttesterSlashing { - attester_slashing: AttesterSlashing::Base(attester_slashing), + attester_slashing: AttesterSlashing::Electra(attester_slashing), }) - } - ForkName::Electra => { + } else { ssz_decode_file(&path.join(format!("{}.ssz_snappy", attester_slashing))) .map(|attester_slashing| Step::AttesterSlashing { - attester_slashing: AttesterSlashing::Electra(attester_slashing), + attester_slashing: AttesterSlashing::Base(attester_slashing), }) } - }, + } Step::PowBlock { pow_block } => { ssz_decode_file(&path.join(format!("{}.ssz_snappy", pow_block))) .map(|pow_block| Step::PowBlock { pow_block }) diff --git a/validator_client/src/attestation_service.rs b/validator_client/src/attestation_service.rs index 404fe41249b..0cff39546dd 100644 --- a/validator_client/src/attestation_service.rs +++ b/validator_client/src/attestation_service.rs @@ -14,11 +14,7 @@ use std::ops::Deref; use std::sync::Arc; use tokio::time::{sleep, sleep_until, Duration, Instant}; use tree_hash::TreeHash; -use types::ForkName; -use types::{ - attestation::AttestationBase, AggregateSignature, Attestation, AttestationData, - AttestationElectra, BitList, BitVector, ChainSpec, CommitteeIndex, EthSpec, Slot, -}; +use types::{Attestation, AttestationData, ChainSpec, CommitteeIndex, EthSpec, Slot}; /// Builds an `AttestationService`. pub struct AttestationServiceBuilder { @@ -363,17 +359,8 @@ 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 - || (fork_name < ForkName::Electra && duty.committee_index != attestation_data.index) - { + if !duty.match_attestation_data::(attestation_data, &self.context.eth2_config.spec) { crit!( log, "Inconsistent validator duties during signing"; @@ -386,25 +373,26 @@ impl AttestationService { return None; } - 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(), - }) + let mut attestation = match Attestation::::empty_for_signing( + duty.committee_index, + duty.committee_length as usize, + attestation_data.slot, + attestation_data.beacon_block_root, + attestation_data.source, + attestation_data.target, + &self.context.eth2_config.spec, + ) { + Ok(attestation) => attestation, + Err(err) => { + crit!( + log, + "Invalid validator duties during signing"; + "validator" => ?duty.pubkey, + "duty" => ?duty, + "err" => ?err, + ); + return None; + } }; match self @@ -554,23 +542,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; let selection_proof = duty_and_proof.selection_proof.as_ref()?; - let slot = attestation_data.slot; - let committee_index = attestation_data.index; - - if duty.slot != slot - || (fork_name < ForkName::Electra && duty.committee_index != committee_index) - { + if !duty.match_attestation_data::(attestation_data, &self.context.eth2_config.spec) { crit!(log, "Inconsistent validator duties during signing"); return None; } diff --git a/validator_client/src/validator_store.rs b/validator_client/src/validator_store.rs index a173e5da12b..e6e19b6e06c 100644 --- a/validator_client/src/validator_store.rs +++ b/validator_client/src/validator_store.rs @@ -18,16 +18,12 @@ use std::sync::Arc; use task_executor::TaskExecutor; use types::{ attestation::Error as AttestationError, graffiti::GraffitiString, AbstractExecPayload, Address, - Attestation, BeaconBlock, BlindedPayload, ChainSpec, ContributionAndProof, Domain, Epoch, - EthSpec, Fork, ForkName, Graffiti, Hash256, PublicKeyBytes, SelectionProof, Signature, - SignedBeaconBlock, SignedContributionAndProof, SignedRoot, SignedValidatorRegistrationData, - SignedVoluntaryExit, Slot, SyncAggregatorSelectionData, SyncCommitteeContribution, - SyncCommitteeMessage, SyncSelectionProof, SyncSubnetId, ValidatorRegistrationData, - VoluntaryExit, -}; -use types::{ - AggregateAndProof, AggregateAndProofBase, AggregateAndProofElectra, SignedAggregateAndProof, - SignedAggregateAndProofBase, SignedAggregateAndProofElectra, + AggregateAndProof, Attestation, BeaconBlock, BlindedPayload, ChainSpec, ContributionAndProof, + Domain, Epoch, EthSpec, Fork, ForkName, Graffiti, Hash256, PublicKeyBytes, SelectionProof, + Signature, SignedAggregateAndProof, SignedBeaconBlock, SignedContributionAndProof, SignedRoot, + SignedValidatorRegistrationData, SignedVoluntaryExit, Slot, SyncAggregatorSelectionData, + SyncCommitteeContribution, SyncCommitteeMessage, SyncSelectionProof, SyncSubnetId, + ValidatorRegistrationData, VoluntaryExit, }; pub use crate::doppelganger_service::DoppelgangerStatus; @@ -805,18 +801,8 @@ impl ValidatorStore { let signing_epoch = aggregate.data().target.epoch; let signing_context = self.signing_context(Domain::AggregateAndProof, signing_epoch); - let message = match aggregate { - Attestation::Base(att) => AggregateAndProof::Base(AggregateAndProofBase { - aggregator_index, - aggregate: att, - selection_proof: selection_proof.into(), - }), - Attestation::Electra(att) => AggregateAndProof::Electra(AggregateAndProofElectra { - aggregator_index, - aggregate: att, - selection_proof: selection_proof.into(), - }), - }; + let message = + AggregateAndProof::from_attestation(aggregator_index, aggregate, selection_proof); let signing_method = self.doppelganger_checked_signing_method(validator_pubkey)?; let signature = signing_method @@ -830,17 +816,9 @@ impl ValidatorStore { metrics::inc_counter_vec(&metrics::SIGNED_AGGREGATES_TOTAL, &[metrics::SUCCESS]); - match message { - AggregateAndProof::Base(message) => { - Ok(SignedAggregateAndProof::Base(SignedAggregateAndProofBase { - message, - signature, - })) - } - AggregateAndProof::Electra(message) => Ok(SignedAggregateAndProof::Electra( - SignedAggregateAndProofElectra { message, signature }, - )), - } + Ok(SignedAggregateAndProof::from_aggregate_and_proof( + message, signature, + )) } /// Produces a `SelectionProof` for the `slot`, signed by with corresponding secret key to