From 5166a3a778c62592667da58803a26f155abd67ce Mon Sep 17 00:00:00 2001 From: Diva M Date: Mon, 5 Apr 2021 14:02:10 -0500 Subject: [PATCH 01/43] wip --- .../src/beacon_processor/block_delay_queue.rs | 224 ++++++++++-------- .../network/src/beacon_processor/mod.rs | 26 +- .../network/src/beacon_processor/tests.rs | 6 +- .../src/beacon_processor/worker/mod.rs | 3 +- 4 files changed, 146 insertions(+), 113 deletions(-) diff --git a/beacon_node/network/src/beacon_processor/block_delay_queue.rs b/beacon_node/network/src/beacon_processor/block_delay_queue.rs index c259d95fdc7..ed72133f89a 100644 --- a/beacon_node/network/src/beacon_processor/block_delay_queue.rs +++ b/beacon_node/network/src/beacon_processor/block_delay_queue.rs @@ -8,13 +8,14 @@ //! There is the edge-case where the slot arrives before this queue manages to process it. In that //! case, the block will be sent off for immediate processing (skipping the `DelayQueue`). use super::MAX_DELAYED_BLOCK_QUEUE_LEN; +use beacon_chain::types::Hash256; use beacon_chain::{BeaconChainTypes, GossipVerifiedBlock}; use eth2_libp2p::PeerId; use futures::stream::{Stream, StreamExt}; use futures::task::Poll; use slog::{crit, debug, error, Logger}; use slot_clock::SlotClock; -use std::collections::HashSet; +use std::collections::{HashSet, VecDeque}; use std::pin::Pin; use std::task::Context; use std::time::Duration; @@ -34,6 +35,23 @@ const ADDITIONAL_DELAY: Duration = Duration::from_millis(5); /// it's nice to have extra protection. const MAXIMUM_QUEUED_BLOCKS: usize = 16; +/// Messages that the scheduler can receive. +pub enum ReprocessSchedulerMessage { + /// A block that has been received early that we should queue for later processing. + EarlyBlock(QueuedBlock), + /// A block that was succesfully processed. We use this to handle attestations for unknown + /// blocks. + BlockProcessed(Hash256), + UnknownBlockAttestation, + UnknownBlockAggregate, +} + +pub enum ReadyWork { + Block(QueuedBlock), + Attestation, + Aggregate, +} + /// A block that arrived early and has been queued for later import. pub struct QueuedBlock { pub peer_id: PeerId, @@ -53,11 +71,14 @@ enum InboundEvent { /// Combines the `DelayQueue` and `Receiver` streams into a single stream. /// -/// This struct has a similar purpose to `tokio::select!`, however it allows for more fine-grained +//struct has a similar purpose to `tokio::select!`, however it allows for more fine-grained /// control (specifically in the ordering of event processing). struct InboundEvents { pub delay_queue: DelayQueue>, - early_blocks_rx: Receiver>, + /// When several works are ready simultaneously, they are queued to be sent back. + ready_works: VecDeque>, + /// Receiver of messages relevant to schedule works for reprocessing. + work_reprocessing_rx: Receiver>, } impl Stream for InboundEvents { @@ -80,7 +101,7 @@ impl Stream for InboundEvents { Poll::Ready(None) | Poll::Pending => (), } - match self.early_blocks_rx.poll_recv(cx) { + match self.work_reprocessing_rx.poll_recv(cx) { Poll::Ready(Some(queued_block)) => { return Poll::Ready(Some(InboundEvent::EarlyBlock(queued_block))); } @@ -94,117 +115,126 @@ impl Stream for InboundEvents { } } -/// Spawn a queue which will accept blocks via the returned `Sender`, potentially queue them until -/// their slot arrives, then send them back out via `ready_blocks_tx`. -pub fn spawn_block_delay_queue( - ready_blocks_tx: Sender>, +/// Starts the job that manages scheduling works that need re-processing. The returned `Sender` +/// gives the communicating channel to receive those works. Once a work is ready, send them back +/// out via `ready_work_tx`. +pub fn spawn_reprocess_scheduler( + ready_work_tx: Sender>, executor: &TaskExecutor, slot_clock: T::SlotClock, log: Logger, -) -> Sender> { - let (early_blocks_tx, early_blocks_rx): (_, Receiver>) = - mpsc::channel(MAX_DELAYED_BLOCK_QUEUE_LEN); - - let queue_future = async move { - let mut queued_block_roots = HashSet::new(); - - let mut inbound_events = InboundEvents { - early_blocks_rx, - delay_queue: DelayQueue::new(), - }; - - loop { - match inbound_events.next().await { - // Some block has been indicated as "early" and should be processed when the - // appropriate slot arrives. - Some(InboundEvent::EarlyBlock(early_block)) => { - let block_slot = early_block.block.block.slot(); - let block_root = early_block.block.block_root; - - // Don't add the same block to the queue twice. This prevents DoS attacks. - if queued_block_roots.contains(&block_root) { +) -> Sender> { + let (work_reprocessing_tx, work_reprocessing_rx) = mpsc::channel(MAX_DELAYED_BLOCK_QUEUE_LEN); + + let queue_future = + work_reprocessing_scheduler(work_reprocessing_rx, ready_work_tx, slot_clock, log); + + executor.spawn(queue_future, TASK_NAME); + + work_reprocessing_tx +} + +async fn work_reprocessing_scheduler( + work_reprocessing_rx: Receiver>, + ready_work_tx: Sender>, + slot_clock: T::SlotClock, + log: Logger, +) { + let mut queued_block_roots = HashSet::new(); + + let mut inbound_events = InboundEvents { + work_reprocessing_rx, + ready_works: VecDeque::new(), // TODO: bound this + delay_queue: DelayQueue::new(), + }; + + loop { + match inbound_events.next().await { + // Some block has been indicated as "early" and should be processed when the + // appropriate slot arrives. + Some(InboundEvent::EarlyBlock(early_block)) => { + let block_slot = early_block.block.block.slot(); + let block_root = early_block.block.block_root; + + // Don't add the same block to the queue twice. This prevents DoS attacks. + if queued_block_roots.contains(&block_root) { + continue; + } + + if let Some(duration_till_slot) = slot_clock.duration_to_slot(block_slot) { + // Check to ensure this won't over-fill the queue. + if queued_block_roots.len() >= MAXIMUM_QUEUED_BLOCKS { + error!( + log, + "Early blocks queue is full"; + "queue_size" => MAXIMUM_QUEUED_BLOCKS, + "msg" => "check system clock" + ); + // Drop the block. continue; } - if let Some(duration_till_slot) = slot_clock.duration_to_slot(block_slot) { - // Check to ensure this won't over-fill the queue. - if queued_block_roots.len() >= MAXIMUM_QUEUED_BLOCKS { + queued_block_roots.insert(block_root); + // Queue the block until the start of the appropriate slot, plus + // `ADDITIONAL_DELAY`. + inbound_events + .delay_queue + .insert(early_block, duration_till_slot + ADDITIONAL_DELAY); + } else { + // If there is no duration till the next slot, check to see if the slot + // has already arrived. If it has already arrived, send it out for + // immediate processing. + // + // If we can't read the slot or the slot hasn't arrived, simply drop the + // block. + // + // This logic is slightly awkward since `SlotClock::duration_to_slot` + // doesn't distinguish between a slot that has already arrived and an + // error reading the slot clock. + if let Some(now) = slot_clock.now() { + if block_slot <= now && ready_work_tx.try_send(early_block).is_err() { error!( log, - "Early blocks queue is full"; - "queue_size" => MAXIMUM_QUEUED_BLOCKS, - "msg" => "check system clock" - ); - // Drop the block. - continue; - } - - queued_block_roots.insert(block_root); - // Queue the block until the start of the appropriate slot, plus - // `ADDITIONAL_DELAY`. - inbound_events - .delay_queue - .insert(early_block, duration_till_slot + ADDITIONAL_DELAY); - } else { - // If there is no duration till the next slot, check to see if the slot - // has already arrived. If it has already arrived, send it out for - // immediate processing. - // - // If we can't read the slot or the slot hasn't arrived, simply drop the - // block. - // - // This logic is slightly awkward since `SlotClock::duration_to_slot` - // doesn't distinguish between a slot that has already arrived and an - // error reading the slot clock. - if let Some(now) = slot_clock.now() { - if block_slot <= now && ready_blocks_tx.try_send(early_block).is_err() { - error!( - log, - "Failed to send block"; + "Failed to send block"; ); - } } } } - // A block that was queued for later processing is now ready to be processed. - Some(InboundEvent::ReadyBlock(ready_block)) => { - let block_root = ready_block.block.block_root; - - if !queued_block_roots.remove(&block_root) { - // Log an error to alert that we've made a bad assumption about how this - // program works, but still process the block anyway. - error!( - log, - "Unknown block in delay queue"; - "block_root" => ?block_root + } + // A block that was queued for later processing is now ready to be processed. + Some(InboundEvent::ReadyBlock(ready_block)) => { + let block_root = ready_block.block.block_root; + + if !queued_block_roots.remove(&block_root) { + // Log an error to alert that we've made a bad assumption about how this + // program works, but still process the block anyway. + error!( + log, + "Unknown block in delay queue"; + "block_root" => ?block_root ); - } + } - if ready_blocks_tx.try_send(ready_block).is_err() { - error!( - log, - "Failed to pop queued block"; + if ready_work_tx.try_send(ready_block).is_err() { + error!( + log, + "Failed to pop queued block"; ); - } } - Some(InboundEvent::DelayQueueError(e)) => crit!( - log, - "Failed to poll block delay queue"; - "e" => ?e + } + Some(InboundEvent::DelayQueueError(e)) => crit!( + log, + "Failed to poll block delay queue"; + "e" => ?e ), - None => { - debug!( - log, - "Block delay queue stopped"; - "msg" => "shutting down" + None => { + debug!( + log, + "Block delay queue stopped"; + "msg" => "shutting down" ); - break; - } + break; } } - }; - - executor.spawn(queue_future, TASK_NAME); - - early_blocks_tx + } } diff --git a/beacon_node/network/src/beacon_processor/mod.rs b/beacon_node/network/src/beacon_processor/mod.rs index 04c17242696..991bc4cbbf1 100644 --- a/beacon_node/network/src/beacon_processor/mod.rs +++ b/beacon_node/network/src/beacon_processor/mod.rs @@ -37,7 +37,7 @@ use crate::{metrics, service::NetworkMessage, sync::SyncMessage}; use beacon_chain::{BeaconChain, BeaconChainTypes, BlockError, GossipVerifiedBlock}; -use block_delay_queue::{spawn_block_delay_queue, QueuedBlock}; +use block_delay_queue::{spawn_reprocess_scheduler, QueuedBlock, ReadyWork}; use eth2_libp2p::{ rpc::{BlocksByRangeRequest, BlocksByRootRequest, StatusMessage}, MessageId, NetworkGlobals, PeerId, PeerRequestId, @@ -555,7 +555,7 @@ enum InboundEvent { /// There is new work to be done. WorkEvent(WorkEvent), /// A block that was delayed for import at a later slot has become ready. - QueuedBlock(Box>), + ReprocessingWork(WorkEvent), } /// Combines the various incoming event streams for the `BeaconProcessor` into a single stream. @@ -567,8 +567,8 @@ struct InboundEvents { idle_rx: mpsc::Receiver<()>, /// Used by upstream processes to send new work to the `BeaconProcessor`. event_rx: mpsc::Receiver>, - /// Used internally for queuing blocks for processing once their slot arrives. - post_delay_block_queue_rx: mpsc::Receiver>, + /// Used internally for queuing work ready to be reprocessed. + reprocess_work_rx: mpsc::Receiver>, } impl Stream for InboundEvents { @@ -589,9 +589,9 @@ impl Stream for InboundEvents { // Poll for delayed blocks before polling for new work. It might be the case that a delayed // block is required to successfully process some new work. - match self.post_delay_block_queue_rx.poll_recv(cx) { - Poll::Ready(Some(queued_block)) => { - return Poll::Ready(Some(InboundEvent::QueuedBlock(Box::new(queued_block)))); + match self.reprocess_work_rx.poll_recv(cx) { + Poll::Ready(Some(ready_work)) => { + return Poll::Ready(Some(InboundEvent::ReprocessingWork(ready_work.into()))); } Poll::Ready(None) => { return Poll::Ready(None); @@ -683,7 +683,7 @@ impl BeaconProcessor { mpsc::channel(MAX_DELAYED_BLOCK_QUEUE_LEN); let pre_delay_block_queue_tx = { if let Some(chain) = self.beacon_chain.upgrade() { - spawn_block_delay_queue( + spawn_reprocess_scheduler( post_delay_block_queue_tx, &self.executor, chain.slot_clock.clone(), @@ -704,7 +704,7 @@ impl BeaconProcessor { let mut inbound_events = InboundEvents { idle_rx, event_rx, - post_delay_block_queue_rx, + reprocess_work_rx: post_delay_block_queue_rx, }; loop { @@ -714,7 +714,7 @@ impl BeaconProcessor { None } Some(InboundEvent::WorkEvent(event)) => Some(event), - Some(InboundEvent::QueuedBlock(queued_block)) => { + Some(InboundEvent::ReprocessingWork(queued_block)) => { Some(WorkEvent::delayed_import_beacon_block( queued_block.peer_id, Box::new(queued_block.block), @@ -766,7 +766,7 @@ impl BeaconProcessor { None if can_spawn => { let toolbox = Toolbox { idle_tx: idle_tx.clone(), - delayed_block_tx: pre_delay_block_queue_tx.clone(), + work_reprocessing_tx: pre_delay_block_queue_tx.clone(), }; // Check for chain segments first, they're the most efficient way to get @@ -857,7 +857,7 @@ impl BeaconProcessor { let work_id = work.str_id(); let toolbox = Toolbox { idle_tx: idle_tx.clone(), - delayed_block_tx: pre_delay_block_queue_tx.clone(), + work_reprocessing_tx: pre_delay_block_queue_tx.clone(), }; match work { @@ -960,7 +960,7 @@ impl BeaconProcessor { /// Sends an message on `idle_tx` when the work is complete and the task is stopping. fn spawn_worker(&mut self, work: Work, toolbox: Toolbox) { let idle_tx = toolbox.idle_tx; - let delayed_block_tx = toolbox.delayed_block_tx; + let delayed_block_tx = toolbox.work_reprocessing_tx; // Wrap the `idle_tx` in a struct that will fire the idle message whenever it is dropped. // diff --git a/beacon_node/network/src/beacon_processor/tests.rs b/beacon_node/network/src/beacon_processor/tests.rs index 6549a0263ec..d6f07cb7310 100644 --- a/beacon_node/network/src/beacon_processor/tests.rs +++ b/beacon_node/network/src/beacon_processor/tests.rs @@ -1,4 +1,4 @@ -#![cfg(not(debug_assertions))] // Tests are too slow in debug. +// #![cfg(not(debug_assertions))] // Tests are too slow in debug. #![cfg(test)] use crate::beacon_processor::*; @@ -46,6 +46,7 @@ struct TestRig { proposer_slashing: ProposerSlashing, voluntary_exit: SignedVoluntaryExit, beacon_processor_tx: mpsc::Sender>, + /// TODO: document this guy work_journal_rx: mpsc::Receiver, _network_rx: mpsc::UnboundedReceiver>, _sync_rx: mpsc::UnboundedReceiver>, @@ -64,6 +65,7 @@ impl Drop for TestRig { impl TestRig { pub fn new(chain_length: u64) -> Self { + let mut harness = BeaconChainHarness::new( MainnetEthSpec, generate_deterministic_keypairs(VALIDATOR_COUNT), @@ -272,7 +274,7 @@ impl TestRig { /// to use the `NOTHING_TO_DO` event to ensure that execution has completed. pub fn assert_event_journal(&mut self, expected: &[&str]) { let events = self.runtime().block_on(async { - let mut events = vec![]; + let mut events = Vec::with_capacity(expected.len()); let drain_future = async { loop { diff --git a/beacon_node/network/src/beacon_processor/worker/mod.rs b/beacon_node/network/src/beacon_processor/worker/mod.rs index 1ac5a863c50..573e53e8351 100644 --- a/beacon_node/network/src/beacon_processor/worker/mod.rs +++ b/beacon_node/network/src/beacon_processor/worker/mod.rs @@ -1,4 +1,5 @@ use super::QueuedBlock; +use super::block_delay_queue::ReprocessSchedulerMessage; use crate::{service::NetworkMessage, sync::SyncMessage}; use beacon_chain::{BeaconChain, BeaconChainTypes}; use slog::{error, Logger}; @@ -46,5 +47,5 @@ impl Worker { /// Contains the necessary items for a worker to do their job. pub struct Toolbox { pub idle_tx: mpsc::Sender<()>, - pub delayed_block_tx: mpsc::Sender>, + pub work_reprocessing_tx: mpsc::Sender>, } From c48fafa5deb895ed923c314563f4edc8c1a49f6b Mon Sep 17 00:00:00 2001 From: Diva M Date: Mon, 5 Apr 2021 18:31:31 -0500 Subject: [PATCH 02/43] wip compiles --- .../src/beacon_processor/block_delay_queue.rs | 35 ++++++++++++++----- .../network/src/beacon_processor/mod.rs | 25 +++++++------ .../beacon_processor/worker/gossip_methods.rs | 10 +++--- 3 files changed, 45 insertions(+), 25 deletions(-) diff --git a/beacon_node/network/src/beacon_processor/block_delay_queue.rs b/beacon_node/network/src/beacon_processor/block_delay_queue.rs index ed72133f89a..155780c6faf 100644 --- a/beacon_node/network/src/beacon_processor/block_delay_queue.rs +++ b/beacon_node/network/src/beacon_processor/block_delay_queue.rs @@ -37,7 +37,7 @@ const MAXIMUM_QUEUED_BLOCKS: usize = 16; /// Messages that the scheduler can receive. pub enum ReprocessSchedulerMessage { - /// A block that has been received early that we should queue for later processing. + /// A block that has been received early and we should queue for later processing. EarlyBlock(QueuedBlock), /// A block that was succesfully processed. We use this to handle attestations for unknown /// blocks. @@ -67,6 +67,18 @@ enum InboundEvent { ReadyBlock(QueuedBlock), /// The `DelayQueue` returned an error. DelayQueueError(TimeError), + BlockProcessed(Hash256), +} + +impl From> for InboundEvent { + fn from(msg: ReprocessSchedulerMessage) -> Self { + match msg { + ReprocessSchedulerMessage::EarlyBlock(block) => InboundEvent::EarlyBlock(block), + ReprocessSchedulerMessage::BlockProcessed(hash) => InboundEvent::BlockProcessed(hash), + ReprocessSchedulerMessage::UnknownBlockAttestation => todo!(), + ReprocessSchedulerMessage::UnknownBlockAggregate => todo!(), + } + } } /// Combines the `DelayQueue` and `Receiver` streams into a single stream. @@ -76,7 +88,7 @@ enum InboundEvent { struct InboundEvents { pub delay_queue: DelayQueue>, /// When several works are ready simultaneously, they are queued to be sent back. - ready_works: VecDeque>, + // ready_works: VecDeque>, /// Receiver of messages relevant to schedule works for reprocessing. work_reprocessing_rx: Receiver>, } @@ -102,8 +114,8 @@ impl Stream for InboundEvents { } match self.work_reprocessing_rx.poll_recv(cx) { - Poll::Ready(Some(queued_block)) => { - return Poll::Ready(Some(InboundEvent::EarlyBlock(queued_block))); + Poll::Ready(Some(message)) => { + return Poll::Ready(Some(message.into())); } Poll::Ready(None) => { return Poll::Ready(None); @@ -144,7 +156,7 @@ async fn work_reprocessing_scheduler( let mut inbound_events = InboundEvents { work_reprocessing_rx, - ready_works: VecDeque::new(), // TODO: bound this + // ready_works: VecDeque::new(), // TODO: bound this delay_queue: DelayQueue::new(), }; @@ -192,7 +204,11 @@ async fn work_reprocessing_scheduler( // doesn't distinguish between a slot that has already arrived and an // error reading the slot clock. if let Some(now) = slot_clock.now() { - if block_slot <= now && ready_work_tx.try_send(early_block).is_err() { + if block_slot <= now + && ready_work_tx + .try_send(ReadyWork::Block(early_block)) + .is_err() + { error!( log, "Failed to send block"; @@ -215,18 +231,19 @@ async fn work_reprocessing_scheduler( ); } - if ready_work_tx.try_send(ready_block).is_err() { + if ready_work_tx.try_send(ReadyWork::Block(ready_block)).is_err() { error!( log, "Failed to pop queued block"; - ); + ); } } + Some(InboundEvent::BlockProcessed(hash)) => todo!(), Some(InboundEvent::DelayQueueError(e)) => crit!( log, "Failed to poll block delay queue"; "e" => ?e - ), + ), None => { debug!( log, diff --git a/beacon_node/network/src/beacon_processor/mod.rs b/beacon_node/network/src/beacon_processor/mod.rs index 991bc4cbbf1..f9089c6bd55 100644 --- a/beacon_node/network/src/beacon_processor/mod.rs +++ b/beacon_node/network/src/beacon_processor/mod.rs @@ -591,7 +591,16 @@ impl Stream for InboundEvents { // block is required to successfully process some new work. match self.reprocess_work_rx.poll_recv(cx) { Poll::Ready(Some(ready_work)) => { - return Poll::Ready(Some(InboundEvent::ReprocessingWork(ready_work.into()))); + let event = match ready_work { + ReadyWork::Block(queued_block) => WorkEvent::delayed_import_beacon_block( + queued_block.peer_id, + Box::new(queued_block.block), + queued_block.seen_timestamp, + ), + ReadyWork::Attestation => todo!(), + ReadyWork::Aggregate => todo!(), + }; + return Poll::Ready(Some(InboundEvent::ReprocessingWork(event))); } Poll::Ready(None) => { return Poll::Ready(None); @@ -713,14 +722,8 @@ impl BeaconProcessor { self.current_workers = self.current_workers.saturating_sub(1); None } - Some(InboundEvent::WorkEvent(event)) => Some(event), - Some(InboundEvent::ReprocessingWork(queued_block)) => { - Some(WorkEvent::delayed_import_beacon_block( - queued_block.peer_id, - Box::new(queued_block.block), - queued_block.seen_timestamp, - )) - } + Some(InboundEvent::WorkEvent(event)) + | Some(InboundEvent::ReprocessingWork(event)) => Some(event), None => { debug!( self.log, @@ -960,7 +963,7 @@ impl BeaconProcessor { /// Sends an message on `idle_tx` when the work is complete and the task is stopping. fn spawn_worker(&mut self, work: Work, toolbox: Toolbox) { let idle_tx = toolbox.idle_tx; - let delayed_block_tx = toolbox.work_reprocessing_tx; + let reprocess_tx = toolbox.work_reprocessing_tx; // Wrap the `idle_tx` in a struct that will fire the idle message whenever it is dropped. // @@ -1059,7 +1062,7 @@ impl BeaconProcessor { message_id, peer_id, *block, - delayed_block_tx, + reprocess_tx, seen_timestamp, ), /* diff --git a/beacon_node/network/src/beacon_processor/worker/gossip_methods.rs b/beacon_node/network/src/beacon_processor/worker/gossip_methods.rs index 4b3fd716b5a..a3e91ac9550 100644 --- a/beacon_node/network/src/beacon_processor/worker/gossip_methods.rs +++ b/beacon_node/network/src/beacon_processor/worker/gossip_methods.rs @@ -17,7 +17,7 @@ use types::{ SignedBeaconBlock, SignedVoluntaryExit, SubnetId, }; -use super::{super::block_delay_queue::QueuedBlock, Worker}; +use super::{super::block_delay_queue::QueuedBlock, ReprocessSchedulerMessage, Worker}; impl Worker { /* Auxiliary functions */ @@ -238,7 +238,7 @@ impl Worker { message_id: MessageId, peer_id: PeerId, block: SignedBeaconBlock, - delayed_import_tx: mpsc::Sender>, + reprocess_tx: mpsc::Sender>, seen_duration: Duration, ) { // Log metrics to track delay from other nodes on the network. @@ -360,12 +360,12 @@ impl Worker { metrics::inc_counter(&metrics::BEACON_PROCESSOR_GOSSIP_BLOCK_REQUEUED_TOTAL); - if delayed_import_tx - .try_send(QueuedBlock { + if reprocess_tx + .try_send(ReprocessSchedulerMessage::EarlyBlock(QueuedBlock { peer_id, block: verified_block, seen_timestamp: seen_duration, - }) + })) .is_err() { error!( From 157e5db07e7d66348064949a09a735ad89763008 Mon Sep 17 00:00:00 2001 From: Diva M Date: Tue, 6 Apr 2021 10:37:55 -0500 Subject: [PATCH 03/43] move file names --- beacon_node/network/src/beacon_processor/mod.rs | 4 ++-- .../{block_delay_queue.rs => work_reprocessing_queue.rs} | 2 -- 2 files changed, 2 insertions(+), 4 deletions(-) rename beacon_node/network/src/beacon_processor/{block_delay_queue.rs => work_reprocessing_queue.rs} (98%) diff --git a/beacon_node/network/src/beacon_processor/mod.rs b/beacon_node/network/src/beacon_processor/mod.rs index f9089c6bd55..6405eebc89d 100644 --- a/beacon_node/network/src/beacon_processor/mod.rs +++ b/beacon_node/network/src/beacon_processor/mod.rs @@ -37,7 +37,7 @@ use crate::{metrics, service::NetworkMessage, sync::SyncMessage}; use beacon_chain::{BeaconChain, BeaconChainTypes, BlockError, GossipVerifiedBlock}; -use block_delay_queue::{spawn_reprocess_scheduler, QueuedBlock, ReadyWork}; +use work_reprocessing_queue::{spawn_reprocess_scheduler, QueuedBlock, ReadyWork}; use eth2_libp2p::{ rpc::{BlocksByRangeRequest, BlocksByRootRequest, StatusMessage}, MessageId, NetworkGlobals, PeerId, PeerRequestId, @@ -60,7 +60,7 @@ use types::{ use worker::{Toolbox, Worker}; -mod block_delay_queue; +mod work_reprocessing_queue; mod tests; mod worker; diff --git a/beacon_node/network/src/beacon_processor/block_delay_queue.rs b/beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs similarity index 98% rename from beacon_node/network/src/beacon_processor/block_delay_queue.rs rename to beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs index 155780c6faf..814fc0ce8fe 100644 --- a/beacon_node/network/src/beacon_processor/block_delay_queue.rs +++ b/beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs @@ -87,8 +87,6 @@ impl From> for InboundEvent /// control (specifically in the ordering of event processing). struct InboundEvents { pub delay_queue: DelayQueue>, - /// When several works are ready simultaneously, they are queued to be sent back. - // ready_works: VecDeque>, /// Receiver of messages relevant to schedule works for reprocessing. work_reprocessing_rx: Receiver>, } From ced9b1a16974995899a648cb317b288277921843 Mon Sep 17 00:00:00 2001 From: Diva M Date: Fri, 9 Apr 2021 13:08:02 -0500 Subject: [PATCH 04/43] make ethspec unpin --- consensus/types/src/eth_spec.rs | 36 ++++++++++++++++----------------- 1 file changed, 18 insertions(+), 18 deletions(-) diff --git a/consensus/types/src/eth_spec.rs b/consensus/types/src/eth_spec.rs index 3cd5555a2c2..6156b40204b 100644 --- a/consensus/types/src/eth_spec.rs +++ b/consensus/types/src/eth_spec.rs @@ -6,7 +6,7 @@ use ssz_types::typenum::{ Unsigned, U0, U1024, U1099511627776, U128, U16, U16777216, U2, U2048, U32, U4, U4096, U64, U65536, U8, U8192, }; -use std::fmt::{self, Debug}; +use std::{fmt::{self, Debug}, marker::Unpin}; use std::str::FromStr; const MAINNET: &str = "mainnet"; @@ -50,9 +50,9 @@ pub trait EthSpec: 'static + Default + Sync + Send + Clone + Debug + PartialEq + /* * Constants */ - type GenesisEpoch: Unsigned + Clone + Sync + Send + Debug + PartialEq; - type JustificationBitsLength: Unsigned + Clone + Sync + Send + Debug + PartialEq + Default; - type SubnetBitfieldLength: Unsigned + Clone + Sync + Send + Debug + PartialEq + Default; + type GenesisEpoch: Unsigned + Clone + Sync + Send + Debug + PartialEq + Unpin; + type JustificationBitsLength: Unsigned + Clone + Sync + Send + Debug + PartialEq + Default + Unpin; + type SubnetBitfieldLength: Unsigned + Clone + Sync + Send + Debug + PartialEq + Default + Unpin; /* * Misc */ @@ -60,24 +60,24 @@ pub trait EthSpec: 'static + Default + Sync + Send + Clone + Debug + PartialEq + /* * Time parameters */ - type SlotsPerEpoch: Unsigned + Clone + Sync + Send + Debug + PartialEq; - type EpochsPerEth1VotingPeriod: Unsigned + Clone + Sync + Send + Debug + PartialEq; - type SlotsPerHistoricalRoot: Unsigned + Clone + Sync + Send + Debug + PartialEq; + type SlotsPerEpoch: Unsigned + Clone + Sync + Send + Debug + PartialEq + Unpin; + type EpochsPerEth1VotingPeriod: Unsigned + Clone + Sync + Send + Debug + PartialEq + Unpin; + type SlotsPerHistoricalRoot: Unsigned + Clone + Sync + Send + Debug + PartialEq + Unpin; /* * State list lengths */ - type EpochsPerHistoricalVector: Unsigned + Clone + Sync + Send + Debug + PartialEq; - type EpochsPerSlashingsVector: Unsigned + Clone + Sync + Send + Debug + PartialEq; - type HistoricalRootsLimit: Unsigned + Clone + Sync + Send + Debug + PartialEq; - type ValidatorRegistryLimit: Unsigned + Clone + Sync + Send + Debug + PartialEq; + type EpochsPerHistoricalVector: Unsigned + Clone + Sync + Send + Debug + PartialEq + Unpin; + type EpochsPerSlashingsVector: Unsigned + Clone + Sync + Send + Debug + PartialEq + Unpin; + type HistoricalRootsLimit: Unsigned + Clone + Sync + Send + Debug + PartialEq + Unpin; + type ValidatorRegistryLimit: Unsigned + Clone + Sync + Send + Debug + PartialEq + Unpin; /* * Max operations per block */ - type MaxProposerSlashings: Unsigned + Clone + Sync + Send + Debug + PartialEq; - type MaxAttesterSlashings: Unsigned + Clone + Sync + Send + Debug + PartialEq; - type MaxAttestations: Unsigned + Clone + Sync + Send + Debug + PartialEq; - type MaxDeposits: Unsigned + Clone + Sync + Send + Debug + PartialEq; - type MaxVoluntaryExits: Unsigned + Clone + Sync + Send + Debug + PartialEq; + type MaxProposerSlashings: Unsigned + Clone + Sync + Send + Debug + PartialEq + Unpin; + type MaxAttesterSlashings: Unsigned + Clone + Sync + Send + Debug + PartialEq + Unpin; + type MaxAttestations: Unsigned + Clone + Sync + Send + Debug + PartialEq + Unpin; + type MaxDeposits: Unsigned + Clone + Sync + Send + Debug + PartialEq + Unpin; + type MaxVoluntaryExits: Unsigned + Clone + Sync + Send + Debug + PartialEq + Unpin; /* * Derived values (set these CAREFULLY) */ @@ -86,11 +86,11 @@ pub trait EthSpec: 'static + Default + Sync + Send + Clone + Debug + PartialEq + /// Must be set to `MaxAttestations * SlotsPerEpoch` // NOTE: we could safely instantiate these by using type-level arithmetic, but doing // so adds ~25s to the time required to type-check this crate - type MaxPendingAttestations: Unsigned + Clone + Sync + Send + Debug + PartialEq; + type MaxPendingAttestations: Unsigned + Clone + Sync + Send + Debug + PartialEq + Unpin; /// The length of `eth1_data_votes`. /// /// Must be set to `EpochsPerEth1VotingPeriod * SlotsPerEpoch` - type SlotsPerEth1VotingPeriod: Unsigned + Clone + Sync + Send + Debug + PartialEq; + type SlotsPerEth1VotingPeriod: Unsigned + Clone + Sync + Send + Debug + PartialEq + Unpin; fn default_spec() -> ChainSpec; From bbd8a87cf04c8b915d5687e436926e0d1be1508e Mon Sep 17 00:00:00 2001 From: Diva M Date: Mon, 12 Apr 2021 11:15:37 -0500 Subject: [PATCH 05/43] add queued types --- .../network/src/beacon_processor/mod.rs | 6 +- .../work_reprocessing_queue.rs | 126 +++++++++++------- .../beacon_processor/worker/gossip_methods.rs | 16 +-- .../src/beacon_processor/worker/mod.rs | 4 +- 4 files changed, 92 insertions(+), 60 deletions(-) diff --git a/beacon_node/network/src/beacon_processor/mod.rs b/beacon_node/network/src/beacon_processor/mod.rs index 6405eebc89d..b0bda5f7b53 100644 --- a/beacon_node/network/src/beacon_processor/mod.rs +++ b/beacon_node/network/src/beacon_processor/mod.rs @@ -597,8 +597,8 @@ impl Stream for InboundEvents { Box::new(queued_block.block), queued_block.seen_timestamp, ), - ReadyWork::Attestation => todo!(), - ReadyWork::Aggregate => todo!(), + ReadyWork::Attestation(queued_attestation) => todo!(), + ReadyWork::Aggregate(queued_aggregate) => todo!(), }; return Poll::Ready(Some(InboundEvent::ReprocessingWork(event))); } @@ -1072,7 +1072,7 @@ impl BeaconProcessor { peer_id, block, seen_timestamp, - } => worker.process_gossip_verified_block(peer_id, *block, seen_timestamp), + } => worker.process_gossip_verified_block(peer_id, *block, reprocess_tx, seen_timestamp), /* * Voluntary exits received on gossip. */ diff --git a/beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs b/beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs index 814fc0ce8fe..bf7539bcaac 100644 --- a/beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs +++ b/beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs @@ -8,14 +8,13 @@ //! There is the edge-case where the slot arrives before this queue manages to process it. In that //! case, the block will be sent off for immediate processing (skipping the `DelayQueue`). use super::MAX_DELAYED_BLOCK_QUEUE_LEN; -use beacon_chain::types::Hash256; use beacon_chain::{BeaconChainTypes, GossipVerifiedBlock}; use eth2_libp2p::PeerId; use futures::stream::{Stream, StreamExt}; use futures::task::Poll; use slog::{crit, debug, error, Logger}; use slot_clock::SlotClock; -use std::collections::{HashSet, VecDeque}; +use std::collections::{HashMap, HashSet}; use std::pin::Pin; use std::task::Context; use std::time::Duration; @@ -23,6 +22,10 @@ use task_executor::TaskExecutor; use tokio::sync::mpsc::{self, Receiver, Sender}; use tokio::time::error::Error as TimeError; use tokio_util::time::DelayQueue; +use types::{ + Attestation, AttesterSlashing, Hash256, ProposerSlashing, SignedAggregateAndProof, + SignedBeaconBlock, SignedVoluntaryExit, SubnetId, EthSpec +}; const TASK_NAME: &str = "beacon_processor_block_delay_queue"; @@ -36,7 +39,7 @@ const ADDITIONAL_DELAY: Duration = Duration::from_millis(5); const MAXIMUM_QUEUED_BLOCKS: usize = 16; /// Messages that the scheduler can receive. -pub enum ReprocessSchedulerMessage { +pub enum ReprocessQueueMessage { /// A block that has been received early and we should queue for later processing. EarlyBlock(QueuedBlock), /// A block that was succesfully processed. We use this to handle attestations for unknown @@ -48,8 +51,20 @@ pub enum ReprocessSchedulerMessage { pub enum ReadyWork { Block(QueuedBlock), - Attestation, - Aggregate, + Attestation(QueuedAttestation), + Aggregate(QueuedAggregate), +} + +pub struct QueuedAttestation { + peer_id: PeerId, + attestation: Attestation, + should_import: bool, +} + +pub struct QueuedAggregate { + peer_id: PeerId, + attestation: SignedAggregateAndProof, + should_import: bool, } /// A block that arrived early and has been queued for later import. @@ -67,39 +82,38 @@ enum InboundEvent { ReadyBlock(QueuedBlock), /// The `DelayQueue` returned an error. DelayQueueError(TimeError), - BlockProcessed(Hash256), -} - -impl From> for InboundEvent { - fn from(msg: ReprocessSchedulerMessage) -> Self { - match msg { - ReprocessSchedulerMessage::EarlyBlock(block) => InboundEvent::EarlyBlock(block), - ReprocessSchedulerMessage::BlockProcessed(hash) => InboundEvent::BlockProcessed(hash), - ReprocessSchedulerMessage::UnknownBlockAttestation => todo!(), - ReprocessSchedulerMessage::UnknownBlockAggregate => todo!(), - } - } } /// Combines the `DelayQueue` and `Receiver` streams into a single stream. /// //struct has a similar purpose to `tokio::select!`, however it allows for more fine-grained /// control (specifically in the ordering of event processing). -struct InboundEvents { - pub delay_queue: DelayQueue>, +// TODO: rename and update docs +struct ReprocessQueue { /// Receiver of messages relevant to schedule works for reprocessing. - work_reprocessing_rx: Receiver>, + work_reprocessing_rx: Receiver>, + /// Queue to manage scheduled early blocks. + block_delay_queue: DelayQueue>, + /// Queue to manage scheduled aggreated attestations. + aggregate_delay_queue: DelayQueue>, + /// Queue to manage scheduled attestations. + attestations_delay_queue: DelayQueue>, + /// Last processed root with works to unqueue. + current_processing_root: Option, + awaiting_attestations: HashMap>>, + // work: ReadyWork, } -impl Stream for InboundEvents { +impl Stream for ReprocessQueue { type Item = InboundEvent; fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + let work = ReadyWork::Aggregate::; // Poll for expired blocks *before* we try to process new blocks. // // The sequential nature of blockchains means it is generally better to try and import all // existing blocks before new ones. - match self.delay_queue.poll_expired(cx) { + match self.block_delay_queue.poll_expired(cx) { Poll::Ready(Some(Ok(queued_block))) => { return Poll::Ready(Some(InboundEvent::ReadyBlock(queued_block.into_inner()))); } @@ -112,9 +126,14 @@ impl Stream for InboundEvents { } match self.work_reprocessing_rx.poll_recv(cx) { - Poll::Ready(Some(message)) => { - return Poll::Ready(Some(message.into())); - } + Poll::Ready(Some(message)) => match message { + ReprocessQueueMessage::EarlyBlock(block) => { + return Poll::Ready(Some(InboundEvent::EarlyBlock(block))) + } + ReprocessQueueMessage::BlockProcessed(hash) => todo!(), + ReprocessQueueMessage::UnknownBlockAttestation => todo!(), + ReprocessQueueMessage::UnknownBlockAggregate => todo!(), + }, Poll::Ready(None) => { return Poll::Ready(None); } @@ -133,7 +152,7 @@ pub fn spawn_reprocess_scheduler( executor: &TaskExecutor, slot_clock: T::SlotClock, log: Logger, -) -> Sender> { +) -> Sender> { let (work_reprocessing_tx, work_reprocessing_rx) = mpsc::channel(MAX_DELAYED_BLOCK_QUEUE_LEN); let queue_future = @@ -145,20 +164,31 @@ pub fn spawn_reprocess_scheduler( } async fn work_reprocessing_scheduler( - work_reprocessing_rx: Receiver>, + work_reprocessing_rx: Receiver>, ready_work_tx: Sender>, slot_clock: T::SlotClock, log: Logger, -) { + ) { let mut queued_block_roots = HashSet::new(); - - let mut inbound_events = InboundEvents { + // + // let delay_queue = DelayQueue::new(); + // + let mut inbound_events = ReprocessQueue { work_reprocessing_rx, - // ready_works: VecDeque::new(), // TODO: bound this - delay_queue: DelayQueue::new(), + block_delay_queue: DelayQueue::new(), + current_processing_root: None, + awaiting_attestations: HashMap::new(), }; loop { + // Poll for expired blocks *before* we try to process new blocks. + // + // The sequential nature of blockchains means it is generally better to try and import all + // existing blocks before new ones. + // match delay_queue.next().await { + // + // } + match inbound_events.next().await { // Some block has been indicated as "early" and should be processed when the // appropriate slot arrives. @@ -188,7 +218,7 @@ async fn work_reprocessing_scheduler( // Queue the block until the start of the appropriate slot, plus // `ADDITIONAL_DELAY`. inbound_events - .delay_queue + .block_delay_queue .insert(early_block, duration_till_slot + ADDITIONAL_DELAY); } else { // If there is no duration till the next slot, check to see if the slot @@ -206,12 +236,12 @@ async fn work_reprocessing_scheduler( && ready_work_tx .try_send(ReadyWork::Block(early_block)) .is_err() - { - error!( - log, - "Failed to send block"; - ); - } + { + error!( + log, + "Failed to send block"; + ); + } } } } @@ -229,19 +259,21 @@ async fn work_reprocessing_scheduler( ); } - if ready_work_tx.try_send(ReadyWork::Block(ready_block)).is_err() { - error!( - log, - "Failed to pop queued block"; - ); - } + if ready_work_tx + .try_send(ReadyWork::Block(ready_block)) + .is_err() + { + error!( + log, + "Failed to pop queued block"; + ); + } } - Some(InboundEvent::BlockProcessed(hash)) => todo!(), Some(InboundEvent::DelayQueueError(e)) => crit!( log, "Failed to poll block delay queue"; "e" => ?e - ), + ), None => { debug!( log, diff --git a/beacon_node/network/src/beacon_processor/worker/gossip_methods.rs b/beacon_node/network/src/beacon_processor/worker/gossip_methods.rs index a3e91ac9550..6d05a874f87 100644 --- a/beacon_node/network/src/beacon_processor/worker/gossip_methods.rs +++ b/beacon_node/network/src/beacon_processor/worker/gossip_methods.rs @@ -17,7 +17,7 @@ use types::{ SignedBeaconBlock, SignedVoluntaryExit, SubnetId, }; -use super::{super::block_delay_queue::QueuedBlock, ReprocessSchedulerMessage, Worker}; +use super::{super::work_reprocessing_queue::{QueuedBlock, ReprocessQueueMessage}, Worker}; impl Worker { /* Auxiliary functions */ @@ -238,7 +238,7 @@ impl Worker { message_id: MessageId, peer_id: PeerId, block: SignedBeaconBlock, - reprocess_tx: mpsc::Sender>, + reprocess_tx: mpsc::Sender>, seen_duration: Duration, ) { // Log metrics to track delay from other nodes on the network. @@ -361,7 +361,7 @@ impl Worker { metrics::inc_counter(&metrics::BEACON_PROCESSOR_GOSSIP_BLOCK_REQUEUED_TOTAL); if reprocess_tx - .try_send(ReprocessSchedulerMessage::EarlyBlock(QueuedBlock { + .try_send(ReprocessQueueMessage::EarlyBlock(QueuedBlock { peer_id, block: verified_block, seen_timestamp: seen_duration, @@ -377,7 +377,7 @@ impl Worker { ) } } - Ok(_) => self.process_gossip_verified_block(peer_id, verified_block, seen_duration), + Ok(_) => self.process_gossip_verified_block(peer_id, verified_block, reprocess_tx, seen_duration), Err(e) => { error!( self.log, @@ -398,24 +398,24 @@ impl Worker { self, peer_id: PeerId, verified_block: GossipVerifiedBlock, + reprocess_tx: mpsc::Sender>, // This value is not used presently, but it might come in handy for debugging. _seen_duration: Duration, ) { let block = Box::new(verified_block.block.clone()); match self.chain.process_block(verified_block) { - Ok(_block_root) => { + Ok(block_root) => { metrics::inc_counter(&metrics::BEACON_PROCESSOR_GOSSIP_BLOCK_IMPORTED_TOTAL); + reprocess_tx.try_send(ReprocessQueueMessage::BlockProcessed(block_root)); + trace!( self.log, "Gossipsub block processed"; "peer_id" => %peer_id ); - // The `MessageHandler` would be the place to put this, however it doesn't seem - // to have a reference to the `BeaconChain`. I will leave this for future - // works. match self.chain.fork_choice() { Ok(()) => trace!( self.log, diff --git a/beacon_node/network/src/beacon_processor/worker/mod.rs b/beacon_node/network/src/beacon_processor/worker/mod.rs index 573e53e8351..36503944576 100644 --- a/beacon_node/network/src/beacon_processor/worker/mod.rs +++ b/beacon_node/network/src/beacon_processor/worker/mod.rs @@ -1,5 +1,5 @@ use super::QueuedBlock; -use super::block_delay_queue::ReprocessSchedulerMessage; +use super::work_reprocessing_queue::ReprocessQueueMessage; use crate::{service::NetworkMessage, sync::SyncMessage}; use beacon_chain::{BeaconChain, BeaconChainTypes}; use slog::{error, Logger}; @@ -47,5 +47,5 @@ impl Worker { /// Contains the necessary items for a worker to do their job. pub struct Toolbox { pub idle_tx: mpsc::Sender<()>, - pub work_reprocessing_tx: mpsc::Sender>, + pub work_reprocessing_tx: mpsc::Sender>, } From 316a0e28beaa5eec86ee5cc991b3719698d24323 Mon Sep 17 00:00:00 2001 From: Diva M Date: Tue, 13 Apr 2021 14:38:13 -0500 Subject: [PATCH 06/43] wip --- .../work_reprocessing_queue.rs | 380 ++++++++++++------ 1 file changed, 263 insertions(+), 117 deletions(-) diff --git a/beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs b/beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs index bf7539bcaac..35626d43e0e 100644 --- a/beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs +++ b/beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs @@ -10,11 +10,12 @@ use super::MAX_DELAYED_BLOCK_QUEUE_LEN; use beacon_chain::{BeaconChainTypes, GossipVerifiedBlock}; use eth2_libp2p::PeerId; -use futures::stream::{Stream, StreamExt}; +use fnv::FnvHashMap; +use futures::stream::Stream; use futures::task::Poll; use slog::{crit, debug, error, Logger}; use slot_clock::SlotClock; -use std::collections::{HashMap, HashSet}; +use std::collections::{HashMap, HashSet, VecDeque}; use std::pin::Pin; use std::task::Context; use std::time::Duration; @@ -22,10 +23,7 @@ use task_executor::TaskExecutor; use tokio::sync::mpsc::{self, Receiver, Sender}; use tokio::time::error::Error as TimeError; use tokio_util::time::DelayQueue; -use types::{ - Attestation, AttesterSlashing, Hash256, ProposerSlashing, SignedAggregateAndProof, - SignedBeaconBlock, SignedVoluntaryExit, SubnetId, EthSpec -}; +use types::{Attestation, EthSpec, Hash256, SignedAggregateAndProof}; const TASK_NAME: &str = "beacon_processor_block_delay_queue"; @@ -49,18 +47,23 @@ pub enum ReprocessQueueMessage { UnknownBlockAggregate, } +/// Events sent by the scheduler once they are ready for re-processing. pub enum ReadyWork { Block(QueuedBlock), Attestation(QueuedAttestation), Aggregate(QueuedAggregate), } +/// An Attestation for which the corresponding block was not seen while processing, queued for +/// later. pub struct QueuedAttestation { peer_id: PeerId, attestation: Attestation, should_import: bool, } +/// An aggregated attestation for which the corresponding block was not seen while processing, queued for +/// later. pub struct QueuedAggregate { peer_id: PeerId, attestation: SignedAggregateAndProof, @@ -85,60 +88,83 @@ enum InboundEvent { } /// Combines the `DelayQueue` and `Receiver` streams into a single stream. -/// -//struct has a similar purpose to `tokio::select!`, however it allows for more fine-grained +/// struct has a similar purpose to `tokio::select!`, however it allows for more fine-grained /// control (specifically in the ordering of event processing). // TODO: rename and update docs -struct ReprocessQueue { +struct ReprocessQueue { /// Receiver of messages relevant to schedule works for reprocessing. work_reprocessing_rx: Receiver>, + + /* Queues */ /// Queue to manage scheduled early blocks. block_delay_queue: DelayQueue>, /// Queue to manage scheduled aggreated attestations. - aggregate_delay_queue: DelayQueue>, + aggregate_delay_queue: DelayQueue, /// Queue to manage scheduled attestations. - attestations_delay_queue: DelayQueue>, + attestations_delay_queue: DelayQueue, + + /* Queued items */ + /// Queued blocks. + queued_block_roots: HashSet, + /// Queued aggreated attestations. + queued_aggregates: FnvHashMap>, + /// Queued attestations. + queued_attestations: FnvHashMap>, + /// Attestations (aggreated and unaggreated) per root. + awaiting_attestations_per_root: HashMap>, + + /* Aux */ + /// Next attestation id, used for both aggreated and unaggreated attestations + next_attestation: usize, /// Last processed root with works to unqueue. current_processing_root: Option, - awaiting_attestations: HashMap>>, - // work: ReadyWork, + + slot_clock: T::SlotClock, + + log: Logger, +} + +enum QueuedAttestationId { + Aggregated(usize), + Unaggreated(usize), } -impl Stream for ReprocessQueue { +impl Stream for ReprocessQueue { type Item = InboundEvent; - fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { - let work = ReadyWork::Aggregate::; + fn poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { // Poll for expired blocks *before* we try to process new blocks. // // The sequential nature of blockchains means it is generally better to try and import all // existing blocks before new ones. - match self.block_delay_queue.poll_expired(cx) { - Poll::Ready(Some(Ok(queued_block))) => { - return Poll::Ready(Some(InboundEvent::ReadyBlock(queued_block.into_inner()))); - } - Poll::Ready(Some(Err(e))) => { - return Poll::Ready(Some(InboundEvent::DelayQueueError(e))); - } - // `Poll::Ready(None)` means that there are no more entries in the delay queue and we - // will continue to get this result until something else is added into the queue. - Poll::Ready(None) | Poll::Pending => (), - } - - match self.work_reprocessing_rx.poll_recv(cx) { - Poll::Ready(Some(message)) => match message { - ReprocessQueueMessage::EarlyBlock(block) => { - return Poll::Ready(Some(InboundEvent::EarlyBlock(block))) - } - ReprocessQueueMessage::BlockProcessed(hash) => todo!(), - ReprocessQueueMessage::UnknownBlockAttestation => todo!(), - ReprocessQueueMessage::UnknownBlockAggregate => todo!(), - }, - Poll::Ready(None) => { - return Poll::Ready(None); - } - Poll::Pending => {} - } + // { + // match self.block_delay_queue.poll_expired(cx) { + // Poll::Ready(Some(Ok(queued_block))) => { + // return Poll::Ready(Some(InboundEvent::ReadyBlock(queued_block.into_inner()))); + // } + // Poll::Ready(Some(Err(e))) => { + // return Poll::Ready(Some(InboundEvent::DelayQueueError(e))); + // } + // // `Poll::Ready(None)` means that there are no more entries in the delay queue and we + // // will continue to get this result until something else is added into the queue. + // Poll::Ready(None) | Poll::Pending => (), + // } + // } + // + // match self.work_reprocessing_rx.poll_recv(cx) { + // Poll::Ready(Some(message)) => match message { + // ReprocessQueueMessage::EarlyBlock(block) => { + // return Poll::Ready(Some(InboundEvent::EarlyBlock(block))) + // } + // ReprocessQueueMessage::BlockProcessed(_hash) => todo!(), + // ReprocessQueueMessage::UnknownBlockAttestation => todo!(), + // ReprocessQueueMessage::UnknownBlockAggregate => todo!(), + // }, + // Poll::Ready(None) => { + // return Poll::Ready(None); + // } + // Poll::Pending => {} + // } Poll::Pending } @@ -155,41 +181,28 @@ pub fn spawn_reprocess_scheduler( ) -> Sender> { let (work_reprocessing_tx, work_reprocessing_rx) = mpsc::channel(MAX_DELAYED_BLOCK_QUEUE_LEN); - let queue_future = - work_reprocessing_scheduler(work_reprocessing_rx, ready_work_tx, slot_clock, log); + // spawn handler task and move the message handler instance into the spawned thread + // TODO: check here + // executor.spawn( + // async move { + // debug!(log, "Network message router started"); + // UnboundedReceiverStream::new(handler_recv) + // .for_each(move |msg| future::ready(handler.handle_message(msg))) + // .await; + // }, + // "router", + // ); + // let queue_future = + // work_reprocessing_scheduler(work_reprocessing_rx, ready_work_tx, slot_clock, log); - executor.spawn(queue_future, TASK_NAME); + // executor.spawn(queue_future, TASK_NAME); work_reprocessing_tx } -async fn work_reprocessing_scheduler( - work_reprocessing_rx: Receiver>, - ready_work_tx: Sender>, - slot_clock: T::SlotClock, - log: Logger, - ) { - let mut queued_block_roots = HashSet::new(); - // - // let delay_queue = DelayQueue::new(); - // - let mut inbound_events = ReprocessQueue { - work_reprocessing_rx, - block_delay_queue: DelayQueue::new(), - current_processing_root: None, - awaiting_attestations: HashMap::new(), - }; - - loop { - // Poll for expired blocks *before* we try to process new blocks. - // - // The sequential nature of blockchains means it is generally better to try and import all - // existing blocks before new ones. - // match delay_queue.next().await { - // - // } - - match inbound_events.next().await { +impl ReprocessQueue { + fn handle_message(&mut self, msg: Option>) { + match msg { // Some block has been indicated as "early" and should be processed when the // appropriate slot arrives. Some(InboundEvent::EarlyBlock(early_block)) => { @@ -197,28 +210,27 @@ async fn work_reprocessing_scheduler( let block_root = early_block.block.block_root; // Don't add the same block to the queue twice. This prevents DoS attacks. - if queued_block_roots.contains(&block_root) { - continue; + if self.queued_block_roots.contains(&block_root) { + return; } - if let Some(duration_till_slot) = slot_clock.duration_to_slot(block_slot) { + if let Some(duration_till_slot) = self.slot_clock.duration_to_slot(block_slot) { // Check to ensure this won't over-fill the queue. - if queued_block_roots.len() >= MAXIMUM_QUEUED_BLOCKS { + if self.queued_block_roots.len() >= MAXIMUM_QUEUED_BLOCKS { error!( - log, - "Early blocks queue is full"; - "queue_size" => MAXIMUM_QUEUED_BLOCKS, - "msg" => "check system clock" - ); + self.log, + "Early blocks queue is full"; + "queue_size" => MAXIMUM_QUEUED_BLOCKS, + "msg" => "check system clock" + ); // Drop the block. - continue; + return; } - queued_block_roots.insert(block_root); + self.queued_block_roots.insert(block_root); // Queue the block until the start of the appropriate slot, plus // `ADDITIONAL_DELAY`. - inbound_events - .block_delay_queue + self.block_delay_queue .insert(early_block, duration_till_slot + ADDITIONAL_DELAY); } else { // If there is no duration till the next slot, check to see if the slot @@ -231,17 +243,17 @@ async fn work_reprocessing_scheduler( // This logic is slightly awkward since `SlotClock::duration_to_slot` // doesn't distinguish between a slot that has already arrived and an // error reading the slot clock. - if let Some(now) = slot_clock.now() { + if let Some(now) = self.slot_clock.now() { if block_slot <= now - && ready_work_tx - .try_send(ReadyWork::Block(early_block)) - .is_err() - { - error!( - log, - "Failed to send block"; - ); - } + // && ready_work_tx + // .try_send(ReadyWork::Block(early_block)) + // .is_err() + { + error!( + self.log, + "Failed to send block"; + ); + } } } } @@ -249,39 +261,173 @@ async fn work_reprocessing_scheduler( Some(InboundEvent::ReadyBlock(ready_block)) => { let block_root = ready_block.block.block_root; - if !queued_block_roots.remove(&block_root) { + if !self.queued_block_roots.remove(&block_root) { // Log an error to alert that we've made a bad assumption about how this // program works, but still process the block anyway. error!( - log, - "Unknown block in delay queue"; - "block_root" => ?block_root - ); + self.log, + "Unknown block in delay queue"; + "block_root" => ?block_root + ); } - if ready_work_tx - .try_send(ReadyWork::Block(ready_block)) - .is_err() - { - error!( - log, - "Failed to pop queued block"; - ); - } + // if ready_work_tx + // .try_send(ReadyWork::Block(ready_block)) + // .is_err() + // { + // error!( + // log, + // "Failed to pop queued block"; + // ); + // } } - Some(InboundEvent::DelayQueueError(e)) => crit!( - log, + Some(InboundEvent::DelayQueueError(e)) => { + crit!( + self.log, "Failed to poll block delay queue"; "e" => ?e - ), + ) + } None => { debug!( - log, - "Block delay queue stopped"; - "msg" => "shutting down" - ); - break; + self.log, + "Block delay queue stopped"; + "msg" => "shutting down" + ); + return; } } } } +// async fn handle_message() + +async fn work_reprocessing_scheduler( + work_reprocessing_rx: Receiver>, + _ready_work_tx: Sender>, + slot_clock: T::SlotClock, + log: Logger, +) { + // let mut queued_block_roots = HashSet::new(); + // + // let delay_queue = DelayQueue::new(); + // + let _inbound_events = ReprocessQueue { + work_reprocessing_rx, + block_delay_queue: DelayQueue::new(), + aggregate_delay_queue: DelayQueue::new(), + attestations_delay_queue: DelayQueue::new(), + queued_block_roots: HashSet::new(), + queued_aggregates: FnvHashMap::default(), + queued_attestations: FnvHashMap::default(), + awaiting_attestations_per_root: HashMap::new(), + next_attestation: 0, + current_processing_root: None, + slot_clock, + log, + }; + + loop { + // Poll for expired blocks *before* we try to process new blocks. + // + // The sequential nature of blockchains means it is generally better to try and import all + // existing blocks before new ones. + // match delay_queue.next().await { + // + // } + + // match inbound_events.next().await { + // // Some block has been indicated as "early" and should be processed when the + // // appropriate slot arrives. + // Some(InboundEvent::EarlyBlock(early_block)) => { + // let block_slot = early_block.block.block.slot(); + // let block_root = early_block.block.block_root; + // + // // Don't add the same block to the queue twice. This prevents DoS attacks. + // if queued_block_roots.contains(&block_root) { + // continue; + // } + // + // if let Some(duration_till_slot) = slot_clock.duration_to_slot(block_slot) { + // // Check to ensure this won't over-fill the queue. + // if queued_block_roots.len() >= MAXIMUM_QUEUED_BLOCKS { + // error!( + // log, + // "Early blocks queue is full"; + // "queue_size" => MAXIMUM_QUEUED_BLOCKS, + // "msg" => "check system clock" + // ); + // // Drop the block. + // continue; + // } + // + // queued_block_roots.insert(block_root); + // // Queue the block until the start of the appropriate slot, plus + // // `ADDITIONAL_DELAY`. + // inbound_events + // .block_delay_queue + // .insert(early_block, duration_till_slot + ADDITIONAL_DELAY); + // } else { + // // If there is no duration till the next slot, check to see if the slot + // // has already arrived. If it has already arrived, send it out for + // // immediate processing. + // // + // // If we can't read the slot or the slot hasn't arrived, simply drop the + // // block. + // // + // // This logic is slightly awkward since `SlotClock::duration_to_slot` + // // doesn't distinguish between a slot that has already arrived and an + // // error reading the slot clock. + // if let Some(now) = slot_clock.now() { + // if block_slot <= now + // && ready_work_tx + // .try_send(ReadyWork::Block(early_block)) + // .is_err() + // { + // error!( + // log, + // "Failed to send block"; + // ); + // } + // } + // } + // } + // // A block that was queued for later processing is now ready to be processed. + // Some(InboundEvent::ReadyBlock(ready_block)) => { + // let block_root = ready_block.block.block_root; + // + // if !queued_block_roots.remove(&block_root) { + // // Log an error to alert that we've made a bad assumption about how this + // // program works, but still process the block anyway. + // error!( + // log, + // "Unknown block in delay queue"; + // "block_root" => ?block_root + // ); + // } + // + // if ready_work_tx + // .try_send(ReadyWork::Block(ready_block)) + // .is_err() + // { + // error!( + // log, + // "Failed to pop queued block"; + // ); + // } + // } + // Some(InboundEvent::DelayQueueError(e)) => crit!( + // log, + // "Failed to poll block delay queue"; + // "e" => ?e + // ), + // None => { + // debug!( + // log, + // "Block delay queue stopped"; + // "msg" => "shutting down" + // ); + // break; + // } + // } + } +} From c5b986177f3c5055ddf2ba6810513088939f5de1 Mon Sep 17 00:00:00 2001 From: Diva M Date: Wed, 14 Apr 2021 12:09:14 -0500 Subject: [PATCH 07/43] solving unpin problems --- .../network/src/beacon_processor/mod.rs | 59 +++--- .../work_reprocessing_queue.rs | 168 ++++++++++++------ common/slot_clock/src/lib.rs | 4 +- 3 files changed, 150 insertions(+), 81 deletions(-) diff --git a/beacon_node/network/src/beacon_processor/mod.rs b/beacon_node/network/src/beacon_processor/mod.rs index b0bda5f7b53..b8ff9f65192 100644 --- a/beacon_node/network/src/beacon_processor/mod.rs +++ b/beacon_node/network/src/beacon_processor/mod.rs @@ -37,7 +37,6 @@ use crate::{metrics, service::NetworkMessage, sync::SyncMessage}; use beacon_chain::{BeaconChain, BeaconChainTypes, BlockError, GossipVerifiedBlock}; -use work_reprocessing_queue::{spawn_reprocess_scheduler, QueuedBlock, ReadyWork}; use eth2_libp2p::{ rpc::{BlocksByRangeRequest, BlocksByRootRequest, StatusMessage}, MessageId, NetworkGlobals, PeerId, PeerRequestId, @@ -57,11 +56,12 @@ use types::{ Attestation, AttesterSlashing, Hash256, ProposerSlashing, SignedAggregateAndProof, SignedBeaconBlock, SignedVoluntaryExit, SubnetId, }; +use work_reprocessing_queue::{spawn_reprocess_scheduler, QueuedBlock, ReadyWork}; use worker::{Toolbox, Worker}; -mod work_reprocessing_queue; mod tests; +mod work_reprocessing_queue; mod worker; pub use worker::ProcessId; @@ -308,22 +308,6 @@ impl WorkEvent { } } - /// Create a new `Work` event for some block that was delayed for later processing. - pub fn delayed_import_beacon_block( - peer_id: PeerId, - block: Box>, - seen_timestamp: Duration, - ) -> Self { - Self { - drop_during_sync: false, - work: Work::DelayedImportBlock { - peer_id, - block, - seen_timestamp, - }, - } - } - /// Create a new `Work` event for some exit. pub fn gossip_voluntary_exit( message_id: MessageId, @@ -442,6 +426,27 @@ impl WorkEvent { } } +impl std::convert::From> for WorkEvent { + fn from(ready_work: ReadyWork) -> Self { + match ready_work { + ReadyWork::Block(QueuedBlock { + peer_id, + block, + seen_timestamp, + }) => Self { + drop_during_sync: false, + work: Work::DelayedImportBlock { + peer_id, + block: Box::new(block), + seen_timestamp, + }, + }, + ReadyWork::Attestation(_) => {todo!()} + ReadyWork::Aggregate(_) => {todo!()} + } + } +} + /// A consensus message (or multiple) from the network that requires processing. #[derive(Debug)] pub enum Work { @@ -591,16 +596,7 @@ impl Stream for InboundEvents { // block is required to successfully process some new work. match self.reprocess_work_rx.poll_recv(cx) { Poll::Ready(Some(ready_work)) => { - let event = match ready_work { - ReadyWork::Block(queued_block) => WorkEvent::delayed_import_beacon_block( - queued_block.peer_id, - Box::new(queued_block.block), - queued_block.seen_timestamp, - ), - ReadyWork::Attestation(queued_attestation) => todo!(), - ReadyWork::Aggregate(queued_aggregate) => todo!(), - }; - return Poll::Ready(Some(InboundEvent::ReprocessingWork(event))); + return Poll::Ready(Some(InboundEvent::ReprocessingWork(ready_work.into()))); } Poll::Ready(None) => { return Poll::Ready(None); @@ -1072,7 +1068,12 @@ impl BeaconProcessor { peer_id, block, seen_timestamp, - } => worker.process_gossip_verified_block(peer_id, *block, reprocess_tx, seen_timestamp), + } => worker.process_gossip_verified_block( + peer_id, + *block, + reprocess_tx, + seen_timestamp, + ), /* * Voluntary exits received on gossip. */ diff --git a/beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs b/beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs index 35626d43e0e..f41b1b78f91 100644 --- a/beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs +++ b/beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs @@ -11,8 +11,9 @@ use super::MAX_DELAYED_BLOCK_QUEUE_LEN; use beacon_chain::{BeaconChainTypes, GossipVerifiedBlock}; use eth2_libp2p::PeerId; use fnv::FnvHashMap; -use futures::stream::Stream; +use futures::future::ready; use futures::task::Poll; +use futures::{Stream, StreamExt}; use slog::{crit, debug, error, Logger}; use slot_clock::SlotClock; use std::collections::{HashMap, HashSet, VecDeque}; @@ -83,15 +84,17 @@ enum InboundEvent { EarlyBlock(QueuedBlock), /// A block that was queued for later processing and is ready for import. ReadyBlock(QueuedBlock), - /// The `DelayQueue` returned an error. - DelayQueueError(TimeError), + /// An aggregated or unaggreated attestation is ready for re-processing. + AttestationReady(QueuedAttestationId), + /// Some `DelayQueue` returned an error. + DelayQueueError(TimeError, &'static str), } /// Combines the `DelayQueue` and `Receiver` streams into a single stream. /// struct has a similar purpose to `tokio::select!`, however it allows for more fine-grained /// control (specifically in the ordering of event processing). // TODO: rename and update docs -struct ReprocessQueue { +struct ReprocessQueue { /// Receiver of messages relevant to schedule works for reprocessing. work_reprocessing_rx: Receiver>, @@ -125,46 +128,95 @@ struct ReprocessQueue { } enum QueuedAttestationId { - Aggregated(usize), - Unaggreated(usize), + Aggregate(usize), + Unaggregate(usize), } -impl Stream for ReprocessQueue { +impl Stream for ReprocessQueue { type Item = InboundEvent; - fn poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { // Poll for expired blocks *before* we try to process new blocks. // // The sequential nature of blockchains means it is generally better to try and import all // existing blocks before new ones. - // { - // match self.block_delay_queue.poll_expired(cx) { - // Poll::Ready(Some(Ok(queued_block))) => { - // return Poll::Ready(Some(InboundEvent::ReadyBlock(queued_block.into_inner()))); - // } - // Poll::Ready(Some(Err(e))) => { - // return Poll::Ready(Some(InboundEvent::DelayQueueError(e))); - // } - // // `Poll::Ready(None)` means that there are no more entries in the delay queue and we - // // will continue to get this result until something else is added into the queue. - // Poll::Ready(None) | Poll::Pending => (), - // } - // } - // - // match self.work_reprocessing_rx.poll_recv(cx) { - // Poll::Ready(Some(message)) => match message { - // ReprocessQueueMessage::EarlyBlock(block) => { - // return Poll::Ready(Some(InboundEvent::EarlyBlock(block))) - // } - // ReprocessQueueMessage::BlockProcessed(_hash) => todo!(), - // ReprocessQueueMessage::UnknownBlockAttestation => todo!(), - // ReprocessQueueMessage::UnknownBlockAggregate => todo!(), - // }, - // Poll::Ready(None) => { - // return Poll::Ready(None); - // } - // Poll::Pending => {} - // } + match self.block_delay_queue.poll_expired(cx) { + Poll::Ready(Some(Ok(queued_block))) => { + return Poll::Ready(Some(InboundEvent::ReadyBlock(queued_block.into_inner()))); + } + Poll::Ready(Some(Err(e))) => { + return Poll::Ready(Some(InboundEvent::DelayQueueError(e, "block_queue"))); + } + // `Poll::Ready(None)` means that there are no more entries in the delay queue and we + // will continue to get this result until something else is added into the queue. + Poll::Ready(None) | Poll::Pending => (), + } + + // Next get the aggregates, since these should be more useful. + match self.aggregate_delay_queue.poll_expired(cx) { + Poll::Ready(Some(Ok(aggregate_id))) => { + return Poll::Ready(Some(InboundEvent::AttestationReady( + QueuedAttestationId::Aggregate(aggregate_id.into_inner()), + ))); + } + Poll::Ready(Some(Err(e))) => { + return Poll::Ready(Some(InboundEvent::DelayQueueError(e, "aggregates_queue"))); + } + // `Poll::Ready(None)` means that there are no more entries in the delay queue and we + // will continue to get this result until something else is added into the queue. + Poll::Ready(None) | Poll::Pending => (), + } + + // Next get the unaggregates. + match self.attestations_delay_queue.poll_expired(cx) { + Poll::Ready(Some(Ok(attestation_id))) => { + return Poll::Ready(Some(InboundEvent::AttestationReady( + QueuedAttestationId::Unaggregate(attestation_id.into_inner()), + ))); + } + Poll::Ready(Some(Err(e))) => { + return Poll::Ready(Some(InboundEvent::DelayQueueError(e, "unaggregates_queue"))); + } + // `Poll::Ready(None)` means that there are no more entries in the delay queue and we + // will continue to get this result until something else is added into the queue. + Poll::Ready(None) | Poll::Pending => (), + } + + // First empty the messages channel to ensure that when we start unqueuing attestations for + // each root, we have as many as temporarily possible. This mitigates excesive hashing and + // heap allocs. + match self.work_reprocessing_rx.poll_recv(cx) { + Poll::Ready(Some(message)) => match message { + ReprocessQueueMessage::EarlyBlock(block) => { + return Poll::Ready(Some(InboundEvent::EarlyBlock(block))) + } + ReprocessQueueMessage::BlockProcessed(_hash) => todo!(), + ReprocessQueueMessage::UnknownBlockAttestation => todo!(), + ReprocessQueueMessage::UnknownBlockAggregate => todo!(), + }, + Poll::Ready(None) => { + return Poll::Ready(None); + } + Poll::Pending => {} + } + + // Next unqueue attestations for which we know their block is already processed. + if let Some(root) = self.current_processing_root { + match self.awaiting_attestations_per_root.get_mut(&root) { + Some(queued_attestations) => match queued_attestations.pop_front() { + Some(id) => { + if queued_attestations.is_empty() { + self.awaiting_attestations_per_root.remove(&root); + } + return Poll::Ready(Some(InboundEvent::AttestationReady(id))); + } + None => todo!(), + }, + None => { + todo!("check what happens on remove") + } + } + } Poll::Pending } @@ -181,26 +233,40 @@ pub fn spawn_reprocess_scheduler( ) -> Sender> { let (work_reprocessing_tx, work_reprocessing_rx) = mpsc::channel(MAX_DELAYED_BLOCK_QUEUE_LEN); + let mut queue = ReprocessQueue { + work_reprocessing_rx, + block_delay_queue: DelayQueue::new(), + aggregate_delay_queue: DelayQueue::new(), + attestations_delay_queue: DelayQueue::new(), + queued_block_roots: HashSet::new(), + queued_aggregates: FnvHashMap::default(), + queued_attestations: FnvHashMap::default(), + awaiting_attestations_per_root: HashMap::new(), + next_attestation: 0, + current_processing_root: None, + slot_clock, + log, + }; // spawn handler task and move the message handler instance into the spawned thread // TODO: check here - // executor.spawn( - // async move { - // debug!(log, "Network message router started"); - // UnboundedReceiverStream::new(handler_recv) - // .for_each(move |msg| future::ready(handler.handle_message(msg))) - // .await; - // }, - // "router", - // ); + executor.spawn( + async move { + loop { + let msg = queue.next().await; + queue.handle_message(msg); + } + }, + TASK_NAME, + ); // let queue_future = - // work_reprocessing_scheduler(work_reprocessing_rx, ready_work_tx, slot_clock, log); + // work_reprocessing_scheduler(work_reprocessing_rx, ready_work_tx, slot_clock, log); // executor.spawn(queue_future, TASK_NAME); work_reprocessing_tx } -impl ReprocessQueue { +impl ReprocessQueue { fn handle_message(&mut self, msg: Option>) { match msg { // Some block has been indicated as "early" and should be processed when the @@ -281,13 +347,15 @@ impl ReprocessQueue { // ); // } } - Some(InboundEvent::DelayQueueError(e)) => { + Some(InboundEvent::DelayQueueError(e, queue_name)) => { crit!( self.log, - "Failed to poll block delay queue"; + "Failed to poll queue"; + "queue" => queue_name, "e" => ?e ) } + Some(InboundEvent::AttestationReady(_id)) => {} None => { debug!( self.log, @@ -301,7 +369,7 @@ impl ReprocessQueue { } // async fn handle_message() -async fn work_reprocessing_scheduler( +async fn work_reprocessing_scheduler( work_reprocessing_rx: Receiver>, _ready_work_tx: Sender>, slot_clock: T::SlotClock, diff --git a/common/slot_clock/src/lib.rs b/common/slot_clock/src/lib.rs index 53f00b0313c..7ba08ccde55 100644 --- a/common/slot_clock/src/lib.rs +++ b/common/slot_clock/src/lib.rs @@ -5,7 +5,7 @@ mod manual_slot_clock; mod metrics; mod system_time_slot_clock; -use std::time::Duration; +use std::{marker::Unpin, time::Duration}; pub use crate::manual_slot_clock::ManualSlotClock; pub use crate::manual_slot_clock::ManualSlotClock as TestingSlotClock; @@ -16,7 +16,7 @@ pub use types::Slot; /// A clock that reports the current slot. /// /// The clock is not required to be monotonically increasing and may go backwards. -pub trait SlotClock: Send + Sync + Sized + Clone { +pub trait SlotClock: Send + Sync + Sized + Clone + Unpin { /// Creates a new slot clock where the first slot is `genesis_slot`, genesis occurred /// `genesis_duration` after the `UNIX_EPOCH` and each slot is `slot_duration` apart. fn new(genesis_slot: Slot, genesis_duration: Duration, slot_duration: Duration) -> Self; From 9c0c982c666abfd7ea40628d5d988053b9418953 Mon Sep 17 00:00:00 2001 From: Diva M Date: Wed, 14 Apr 2021 15:43:13 -0500 Subject: [PATCH 08/43] wip --- .../src/beacon_processor/work_reprocessing_queue.rs | 11 +++++------ .../src/beacon_processor/worker/gossip_methods.rs | 4 ---- 2 files changed, 5 insertions(+), 10 deletions(-) diff --git a/beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs b/beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs index f41b1b78f91..87049a06fe1 100644 --- a/beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs +++ b/beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs @@ -11,7 +11,6 @@ use super::MAX_DELAYED_BLOCK_QUEUE_LEN; use beacon_chain::{BeaconChainTypes, GossipVerifiedBlock}; use eth2_libp2p::PeerId; use fnv::FnvHashMap; -use futures::future::ready; use futures::task::Poll; use futures::{Stream, StreamExt}; use slog::{crit, debug, error, Logger}; @@ -44,20 +43,20 @@ pub enum ReprocessQueueMessage { /// A block that was succesfully processed. We use this to handle attestations for unknown /// blocks. BlockProcessed(Hash256), - UnknownBlockAttestation, + UnknownBlockUnaggregate(QueuedUnaggregate), UnknownBlockAggregate, } /// Events sent by the scheduler once they are ready for re-processing. pub enum ReadyWork { Block(QueuedBlock), - Attestation(QueuedAttestation), + Attestation(QueuedUnaggregate), Aggregate(QueuedAggregate), } /// An Attestation for which the corresponding block was not seen while processing, queued for /// later. -pub struct QueuedAttestation { +pub struct QueuedUnaggregate { peer_id: PeerId, attestation: Attestation, should_import: bool, @@ -112,7 +111,7 @@ struct ReprocessQueue { /// Queued aggreated attestations. queued_aggregates: FnvHashMap>, /// Queued attestations. - queued_attestations: FnvHashMap>, + queued_attestations: FnvHashMap>, /// Attestations (aggreated and unaggreated) per root. awaiting_attestations_per_root: HashMap>, @@ -191,7 +190,7 @@ impl Stream for ReprocessQueue { return Poll::Ready(Some(InboundEvent::EarlyBlock(block))) } ReprocessQueueMessage::BlockProcessed(_hash) => todo!(), - ReprocessQueueMessage::UnknownBlockAttestation => todo!(), + ReprocessQueueMessage::UnknownBlockUnaggregate(_) => todo!(), ReprocessQueueMessage::UnknownBlockAggregate => todo!(), }, Poll::Ready(None) => { diff --git a/beacon_node/network/src/beacon_processor/worker/gossip_methods.rs b/beacon_node/network/src/beacon_processor/worker/gossip_methods.rs index 6d05a874f87..7de465592dd 100644 --- a/beacon_node/network/src/beacon_processor/worker/gossip_methods.rs +++ b/beacon_node/network/src/beacon_processor/worker/gossip_methods.rs @@ -790,10 +790,6 @@ impl Worker { self.gossip_penalize_peer(peer_id, PeerAction::LowToleranceError); } AttnError::UnknownHeadBlock { beacon_block_root } => { - // Note: its a little bit unclear as to whether or not this block is unknown or - // just old. See: - // - // https://github.com/sigp/lighthouse/issues/1039 // TODO: Maintain this attestation and re-process once sync completes // TODO: We then score based on whether we can download the block and re-process. From c6ed5536aa42deaf6011dade00a6aa73fe29d9dd Mon Sep 17 00:00:00 2001 From: Diva M Date: Thu, 15 Apr 2021 11:30:58 -0500 Subject: [PATCH 09/43] do not lose the attestation info on error --- .../src/attestation_verification.rs | 55 ++++++++++++------- beacon_node/beacon_chain/src/beacon_chain.rs | 4 +- beacon_node/http_api/src/lib.rs | 8 +-- .../network/src/beacon_processor/mod.rs | 1 + .../beacon_processor/worker/gossip_methods.rs | 13 +++-- 5 files changed, 51 insertions(+), 30 deletions(-) diff --git a/beacon_node/beacon_chain/src/attestation_verification.rs b/beacon_node/beacon_chain/src/attestation_verification.rs index 11eb4bbdb67..545a36bc729 100644 --- a/beacon_node/beacon_chain/src/attestation_verification.rs +++ b/beacon_node/beacon_chain/src/attestation_verification.rs @@ -137,7 +137,7 @@ pub enum Error { /// /// The attestation points to a block we have not yet imported. It's unclear if the attestation /// is valid or not. - UnknownHeadBlock { beacon_block_root: Hash256 }, + UnknownHeadBlock, /// The `attestation.data.slot` is not from the same epoch as `data.target.epoch`. /// /// ## Peer scoring @@ -375,7 +375,7 @@ impl VerifiedAggregatedAttestation { pub fn verify( signed_aggregate: SignedAggregateAndProof, chain: &BeaconChain, - ) -> Result { + ) -> Result)> { Self::verify_slashable(signed_aggregate, chain) .map(|verified_aggregate| { if let Some(slasher) = chain.slasher.as_ref() { @@ -383,7 +383,7 @@ impl VerifiedAggregatedAttestation { } verified_aggregate }) - .map_err(|slash_info| process_slash_info(slash_info, chain)) + .map_err(|(slash_info, original_aggregate)| (process_slash_info(slash_info, chain), original_aggregate)) } /// Run the checks that happen before an indexed attestation is constructed. @@ -502,14 +502,25 @@ impl VerifiedAggregatedAttestation { pub fn verify_slashable( signed_aggregate: SignedAggregateAndProof, chain: &BeaconChain, - ) -> Result> { + ) -> Result< + Self, + ( + AttestationSlashInfo, + SignedAggregateAndProof, + ), + > { use AttestationSlashInfo::*; let attestation = &signed_aggregate.message.aggregate; let aggregator_index = signed_aggregate.message.aggregator_index; let attestation_root = match Self::verify_early_checks(&signed_aggregate, chain) { Ok(root) => root, - Err(e) => return Err(SignatureNotChecked(signed_aggregate.message.aggregate, e)), + Err(e) => { + return Err(( + SignatureNotChecked(signed_aggregate.message.aggregate.clone(), e), + signed_aggregate, + )) + } }; let indexed_attestation = @@ -536,7 +547,7 @@ impl VerifiedAggregatedAttestation { .map_err(|e| BeaconChainError::from(e).into()) }) { Ok(indexed_attestation) => indexed_attestation, - Err(e) => return Err(SignatureNotChecked(signed_aggregate.message.aggregate, e)), + Err(e) => return Err((SignatureNotChecked(signed_aggregate.message.aggregate.clone(), e), signed_aggregate)), }; // Ensure that all signatures are valid. @@ -550,11 +561,11 @@ impl VerifiedAggregatedAttestation { } }) { - return Err(SignatureInvalid(e)); + return Err((SignatureInvalid(e), signed_aggregate)); } if let Err(e) = Self::verify_late_checks(&signed_aggregate, attestation_root, chain) { - return Err(SignatureValid(indexed_attestation, e)); + return Err((SignatureValid(indexed_attestation, e), signed_aggregate)); } Ok(VerifiedAggregatedAttestation { @@ -705,7 +716,7 @@ impl VerifiedUnaggregatedAttestation { attestation: Attestation, subnet_id: Option, chain: &BeaconChain, - ) -> Result { + ) -> Result)> { Self::verify_slashable(attestation, subnet_id, chain) .map(|verified_unaggregated| { if let Some(slasher) = chain.slasher.as_ref() { @@ -713,7 +724,9 @@ impl VerifiedUnaggregatedAttestation { } verified_unaggregated }) - .map_err(|slash_info| process_slash_info(slash_info, chain)) + .map_err(|(slash_info, original_attestation)| { + (process_slash_info(slash_info, chain), original_attestation) + }) } /// Verify the attestation, producing extra information about whether it might be slashable. @@ -721,18 +734,19 @@ impl VerifiedUnaggregatedAttestation { attestation: Attestation, subnet_id: Option, chain: &BeaconChain, - ) -> Result> { + ) -> Result, Attestation)> { use AttestationSlashInfo::*; if let Err(e) = Self::verify_early_checks(&attestation, chain) { - return Err(SignatureNotChecked(attestation, e)); + // TODO: check how to avoid copying this + return Err((SignatureNotChecked(attestation.clone(), e), attestation)); } let (indexed_attestation, committees_per_slot) = match obtain_indexed_attestation_and_committees_per_slot(chain, &attestation) { Ok(x) => x, Err(e) => { - return Err(SignatureNotChecked(attestation, e)); + return Err((SignatureNotChecked(attestation.clone(), e), attestation)); } }; @@ -744,16 +758,21 @@ impl VerifiedUnaggregatedAttestation { chain, ) { Ok(t) => t, - Err(e) => return Err(SignatureNotCheckedIndexed(indexed_attestation, e)), + Err(e) => { + return Err(( + SignatureNotCheckedIndexed(indexed_attestation, e), + attestation, + )) + } }; // The aggregate signature of the attestation is valid. if let Err(e) = verify_attestation_signature(chain, &indexed_attestation) { - return Err(SignatureInvalid(e)); + return Err((SignatureInvalid(e), attestation)); } if let Err(e) = Self::verify_late_checks(&attestation, validator_index, chain) { - return Err(SignatureValid(indexed_attestation, e)); + return Err((SignatureValid(indexed_attestation, e), attestation)); } Ok(Self { @@ -825,9 +844,7 @@ fn verify_head_block_is_known( Ok(block) } else { - Err(Error::UnknownHeadBlock { - beacon_block_root: attestation.data.beacon_block_root, - }) + Err(Error::UnknownHeadBlock) } } diff --git a/beacon_node/beacon_chain/src/beacon_chain.rs b/beacon_node/beacon_chain/src/beacon_chain.rs index 144f08550d8..2d5997228ff 100644 --- a/beacon_node/beacon_chain/src/beacon_chain.rs +++ b/beacon_node/beacon_chain/src/beacon_chain.rs @@ -1017,7 +1017,7 @@ impl BeaconChain { &self, unaggregated_attestation: Attestation, subnet_id: Option, - ) -> Result, AttestationError> { + ) -> Result, (AttestationError, Attestation)> { metrics::inc_counter(&metrics::UNAGGREGATED_ATTESTATION_PROCESSING_REQUESTS); let _timer = metrics::start_timer(&metrics::UNAGGREGATED_ATTESTATION_GOSSIP_VERIFICATION_TIMES); @@ -1041,7 +1041,7 @@ impl BeaconChain { pub fn verify_aggregated_attestation_for_gossip( &self, signed_aggregate: SignedAggregateAndProof, - ) -> Result, AttestationError> { + ) -> Result, (AttestationError, SignedAggregateAndProof)> { metrics::inc_counter(&metrics::AGGREGATED_ATTESTATION_PROCESSING_REQUESTS); let _timer = metrics::start_timer(&metrics::AGGREGATED_ATTESTATION_GOSSIP_VERIFICATION_TIMES); diff --git a/beacon_node/http_api/src/lib.rs b/beacon_node/http_api/src/lib.rs index 653820367c1..cbe3d7ffc8d 100644 --- a/beacon_node/http_api/src/lib.rs +++ b/beacon_node/http_api/src/lib.rs @@ -1785,8 +1785,8 @@ pub fn serve( let mut failures = Vec::new(); // Verify that all messages in the post are valid before processing further - for (index, aggregate) in aggregates.as_slice().iter().enumerate() { - match chain.verify_aggregated_attestation_for_gossip(aggregate.clone()) { + for (index, aggregate) in aggregates.into_iter().enumerate() { + match chain.verify_aggregated_attestation_for_gossip(aggregate) { Ok(verified_aggregate) => { messages.push(PubsubMessage::AggregateAndProofAttestation(Box::new( verified_aggregate.aggregate().clone(), @@ -1811,8 +1811,8 @@ pub fn serve( // It's reasonably likely that two different validators produce // identical aggregates, especially if they're using the same beacon // node. - Err(AttnError::AttestationAlreadyKnown(_)) => continue, - Err(e) => { + Err((AttnError::AttestationAlreadyKnown(_), _)) => continue, + Err((e, aggregate)) => { error!(log, "Failure verifying aggregate and proofs"; "error" => format!("{:?}", e), diff --git a/beacon_node/network/src/beacon_processor/mod.rs b/beacon_node/network/src/beacon_processor/mod.rs index b8ff9f65192..f890e8ed571 100644 --- a/beacon_node/network/src/beacon_processor/mod.rs +++ b/beacon_node/network/src/beacon_processor/mod.rs @@ -1031,6 +1031,7 @@ impl BeaconProcessor { subnet_id, should_import, seen_timestamp, + reprocess_tx ), /* * Aggregated attestation verification. diff --git a/beacon_node/network/src/beacon_processor/worker/gossip_methods.rs b/beacon_node/network/src/beacon_processor/worker/gossip_methods.rs index 7de465592dd..3d3f9913977 100644 --- a/beacon_node/network/src/beacon_processor/worker/gossip_methods.rs +++ b/beacon_node/network/src/beacon_processor/worker/gossip_methods.rs @@ -67,6 +67,7 @@ impl Worker { subnet_id: SubnetId, should_import: bool, seen_timestamp: Duration, + reprocess_tx: mpsc::Sender>, ) { let beacon_block_root = attestation.data.beacon_block_root; @@ -75,7 +76,7 @@ impl Worker { .verify_unaggregated_attestation_for_gossip(attestation, Some(subnet_id)) { Ok(attestation) => attestation, - Err(e) => { + Err((e, attestation)) => { self.handle_attestation_verification_failure( peer_id, message_id, @@ -83,6 +84,7 @@ impl Worker { "unaggregated", e, ); + // TODO: diff between first and second return; } }; @@ -162,15 +164,16 @@ impl Worker { .verify_aggregated_attestation_for_gossip(aggregate) { Ok(aggregate) => aggregate, - Err(e) => { + Err((e, original_aggregate)) => { // Report the failure to gossipsub self.handle_attestation_verification_failure( peer_id, message_id, - beacon_block_root, + original_aggregate.message.aggregate.data.beacon_block_root, "aggregated", e, ); + // TODO: do something here with this return; } }; @@ -789,7 +792,7 @@ impl Worker { self.propagate_validation_result(message_id, peer_id, MessageAcceptance::Reject); self.gossip_penalize_peer(peer_id, PeerAction::LowToleranceError); } - AttnError::UnknownHeadBlock { beacon_block_root } => { + AttnError::UnknownHeadBlock => { // TODO: Maintain this attestation and re-process once sync completes // TODO: We then score based on whether we can download the block and re-process. @@ -801,7 +804,7 @@ impl Worker { ); // we don't know the block, get the sync manager to handle the block lookup self.sync_tx - .send(SyncMessage::UnknownBlockHash(peer_id, *beacon_block_root)) + .send(SyncMessage::UnknownBlockHash(peer_id, beacon_block_root)) .unwrap_or_else(|_| { warn!( self.log, From 32bd1498648a09ff35d3d3acd9d437a815c0c06c Mon Sep 17 00:00:00 2001 From: Diva M Date: Thu, 15 Apr 2021 20:20:31 -0500 Subject: [PATCH 10/43] add glue --- .../network/src/beacon_processor/mod.rs | 11 +- .../work_reprocessing_queue.rs | 231 +++--------------- .../beacon_processor/worker/gossip_methods.rs | 129 ++++++++-- 3 files changed, 147 insertions(+), 224 deletions(-) diff --git a/beacon_node/network/src/beacon_processor/mod.rs b/beacon_node/network/src/beacon_processor/mod.rs index f890e8ed571..ee9aa4b09b7 100644 --- a/beacon_node/network/src/beacon_processor/mod.rs +++ b/beacon_node/network/src/beacon_processor/mod.rs @@ -441,8 +441,12 @@ impl std::convert::From> for WorkEvent { seen_timestamp, }, }, - ReadyWork::Attestation(_) => {todo!()} - ReadyWork::Aggregate(_) => {todo!()} + ReadyWork::Attestation(_) => { + todo!() + } + ReadyWork::Aggregate(_) => { + todo!() + } } } } @@ -1030,8 +1034,8 @@ impl BeaconProcessor { *attestation, subnet_id, should_import, + Some(reprocess_tx), seen_timestamp, - reprocess_tx ), /* * Aggregated attestation verification. @@ -1045,6 +1049,7 @@ impl BeaconProcessor { message_id, peer_id, *aggregate, + Some(reprocess_tx), seen_timestamp, ), /* diff --git a/beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs b/beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs index 87049a06fe1..e3394ad9860 100644 --- a/beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs +++ b/beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs @@ -42,9 +42,9 @@ pub enum ReprocessQueueMessage { EarlyBlock(QueuedBlock), /// A block that was succesfully processed. We use this to handle attestations for unknown /// blocks. - BlockProcessed(Hash256), + BlockImported(Hash256), UnknownBlockUnaggregate(QueuedUnaggregate), - UnknownBlockAggregate, + UnknownBlockAggregate(QueuedAggregate), } /// Events sent by the scheduler once they are ready for re-processing. @@ -59,6 +59,7 @@ pub enum ReadyWork { pub struct QueuedUnaggregate { peer_id: PeerId, attestation: Attestation, + subnet_id: SubnetId, should_import: bool, } @@ -79,28 +80,30 @@ pub struct QueuedBlock { /// Unifies the different messages processed by the block delay queue. enum InboundEvent { - /// A block that has been received early that we should queue for later processing. - EarlyBlock(QueuedBlock), /// A block that was queued for later processing and is ready for import. ReadyBlock(QueuedBlock), /// An aggregated or unaggreated attestation is ready for re-processing. - AttestationReady(QueuedAttestationId), - /// Some `DelayQueue` returned an error. + ReadyAttestation(QueuedAttestationId), + /// A `DelayQueue` returned an error. DelayQueueError(TimeError, &'static str), + /// A message sent to the `ReprocessQueue` + Msg(ReprocessQueueMessage), } /// Combines the `DelayQueue` and `Receiver` streams into a single stream. /// struct has a similar purpose to `tokio::select!`, however it allows for more fine-grained /// control (specifically in the ordering of event processing). -// TODO: rename and update docs +// TODO: update docs struct ReprocessQueue { /// Receiver of messages relevant to schedule works for reprocessing. work_reprocessing_rx: Receiver>, + /// Sender of works once they become ready + ready_work_tx: Sender>, /* Queues */ /// Queue to manage scheduled early blocks. block_delay_queue: DelayQueue>, - /// Queue to manage scheduled aggreated attestations. + /// Queue to manage scheduled aggregated attestations. aggregate_delay_queue: DelayQueue, /// Queue to manage scheduled attestations. attestations_delay_queue: DelayQueue, @@ -118,8 +121,6 @@ struct ReprocessQueue { /* Aux */ /// Next attestation id, used for both aggreated and unaggreated attestations next_attestation: usize, - /// Last processed root with works to unqueue. - current_processing_root: Option, slot_clock: T::SlotClock, @@ -154,7 +155,7 @@ impl Stream for ReprocessQueue { // Next get the aggregates, since these should be more useful. match self.aggregate_delay_queue.poll_expired(cx) { Poll::Ready(Some(Ok(aggregate_id))) => { - return Poll::Ready(Some(InboundEvent::AttestationReady( + return Poll::Ready(Some(InboundEvent::ReadyAttestation( QueuedAttestationId::Aggregate(aggregate_id.into_inner()), ))); } @@ -169,7 +170,7 @@ impl Stream for ReprocessQueue { // Next get the unaggregates. match self.attestations_delay_queue.poll_expired(cx) { Poll::Ready(Some(Ok(attestation_id))) => { - return Poll::Ready(Some(InboundEvent::AttestationReady( + return Poll::Ready(Some(InboundEvent::ReadyAttestation( QueuedAttestationId::Unaggregate(attestation_id.into_inner()), ))); } @@ -185,38 +186,11 @@ impl Stream for ReprocessQueue { // each root, we have as many as temporarily possible. This mitigates excesive hashing and // heap allocs. match self.work_reprocessing_rx.poll_recv(cx) { - Poll::Ready(Some(message)) => match message { - ReprocessQueueMessage::EarlyBlock(block) => { - return Poll::Ready(Some(InboundEvent::EarlyBlock(block))) - } - ReprocessQueueMessage::BlockProcessed(_hash) => todo!(), - ReprocessQueueMessage::UnknownBlockUnaggregate(_) => todo!(), - ReprocessQueueMessage::UnknownBlockAggregate => todo!(), - }, - Poll::Ready(None) => { - return Poll::Ready(None); - } + Poll::Ready(Some(message)) => return Poll::Ready(Some(InboundEvent::Msg(message))), + Poll::Ready(None) => return Poll::Ready(None), Poll::Pending => {} } - // Next unqueue attestations for which we know their block is already processed. - if let Some(root) = self.current_processing_root { - match self.awaiting_attestations_per_root.get_mut(&root) { - Some(queued_attestations) => match queued_attestations.pop_front() { - Some(id) => { - if queued_attestations.is_empty() { - self.awaiting_attestations_per_root.remove(&root); - } - return Poll::Ready(Some(InboundEvent::AttestationReady(id))); - } - None => todo!(), - }, - None => { - todo!("check what happens on remove") - } - } - } - Poll::Pending } } @@ -234,6 +208,7 @@ pub fn spawn_reprocess_scheduler( let mut queue = ReprocessQueue { work_reprocessing_rx, + ready_work_tx, block_delay_queue: DelayQueue::new(), aggregate_delay_queue: DelayQueue::new(), attestations_delay_queue: DelayQueue::new(), @@ -242,12 +217,10 @@ pub fn spawn_reprocess_scheduler( queued_attestations: FnvHashMap::default(), awaiting_attestations_per_root: HashMap::new(), next_attestation: 0, - current_processing_root: None, slot_clock, log, }; - // spawn handler task and move the message handler instance into the spawned thread - // TODO: check here + executor.spawn( async move { loop { @@ -257,20 +230,17 @@ pub fn spawn_reprocess_scheduler( }, TASK_NAME, ); - // let queue_future = - // work_reprocessing_scheduler(work_reprocessing_rx, ready_work_tx, slot_clock, log); - - // executor.spawn(queue_future, TASK_NAME); work_reprocessing_tx } impl ReprocessQueue { fn handle_message(&mut self, msg: Option>) { + use ReprocessQueueMessage::*; match msg { // Some block has been indicated as "early" and should be processed when the // appropriate slot arrives. - Some(InboundEvent::EarlyBlock(early_block)) => { + Some(InboundEvent::Msg(EarlyBlock(early_block))) => { let block_slot = early_block.block.block.slot(); let block_root = early_block.block.block_root; @@ -310,9 +280,10 @@ impl ReprocessQueue { // error reading the slot clock. if let Some(now) = self.slot_clock.now() { if block_slot <= now - // && ready_work_tx - // .try_send(ReadyWork::Block(early_block)) - // .is_err() + && self + .ready_work_tx + .try_send(ReadyWork::Block(early_block)) + .is_err() { error!( self.log, @@ -322,6 +293,9 @@ impl ReprocessQueue { } } } + Some(InboundEvent::Msg(UnknownBlockAggregate(queued_aggregate))) => {} + Some(InboundEvent::Msg(UnknownBlockUnaggregate(queued_unaggregate))) => {} + Some(InboundEvent::Msg(BlockImported(root))) => {} // A block that was queued for later processing is now ready to be processed. Some(InboundEvent::ReadyBlock(ready_block)) => { let block_root = ready_block.block.block_root; @@ -336,15 +310,16 @@ impl ReprocessQueue { ); } - // if ready_work_tx - // .try_send(ReadyWork::Block(ready_block)) - // .is_err() - // { - // error!( - // log, - // "Failed to pop queued block"; - // ); - // } + if self + .ready_work_tx + .try_send(ReadyWork::Block(ready_block)) + .is_err() + { + error!( + self.log, + "Failed to pop queued block"; + ); + } } Some(InboundEvent::DelayQueueError(e, queue_name)) => { crit!( @@ -354,7 +329,7 @@ impl ReprocessQueue { "e" => ?e ) } - Some(InboundEvent::AttestationReady(_id)) => {} + Some(InboundEvent::ReadyAttestation(_id)) => {} None => { debug!( self.log, @@ -366,135 +341,3 @@ impl ReprocessQueue { } } } -// async fn handle_message() - -async fn work_reprocessing_scheduler( - work_reprocessing_rx: Receiver>, - _ready_work_tx: Sender>, - slot_clock: T::SlotClock, - log: Logger, -) { - // let mut queued_block_roots = HashSet::new(); - // - // let delay_queue = DelayQueue::new(); - // - let _inbound_events = ReprocessQueue { - work_reprocessing_rx, - block_delay_queue: DelayQueue::new(), - aggregate_delay_queue: DelayQueue::new(), - attestations_delay_queue: DelayQueue::new(), - queued_block_roots: HashSet::new(), - queued_aggregates: FnvHashMap::default(), - queued_attestations: FnvHashMap::default(), - awaiting_attestations_per_root: HashMap::new(), - next_attestation: 0, - current_processing_root: None, - slot_clock, - log, - }; - - loop { - // Poll for expired blocks *before* we try to process new blocks. - // - // The sequential nature of blockchains means it is generally better to try and import all - // existing blocks before new ones. - // match delay_queue.next().await { - // - // } - - // match inbound_events.next().await { - // // Some block has been indicated as "early" and should be processed when the - // // appropriate slot arrives. - // Some(InboundEvent::EarlyBlock(early_block)) => { - // let block_slot = early_block.block.block.slot(); - // let block_root = early_block.block.block_root; - // - // // Don't add the same block to the queue twice. This prevents DoS attacks. - // if queued_block_roots.contains(&block_root) { - // continue; - // } - // - // if let Some(duration_till_slot) = slot_clock.duration_to_slot(block_slot) { - // // Check to ensure this won't over-fill the queue. - // if queued_block_roots.len() >= MAXIMUM_QUEUED_BLOCKS { - // error!( - // log, - // "Early blocks queue is full"; - // "queue_size" => MAXIMUM_QUEUED_BLOCKS, - // "msg" => "check system clock" - // ); - // // Drop the block. - // continue; - // } - // - // queued_block_roots.insert(block_root); - // // Queue the block until the start of the appropriate slot, plus - // // `ADDITIONAL_DELAY`. - // inbound_events - // .block_delay_queue - // .insert(early_block, duration_till_slot + ADDITIONAL_DELAY); - // } else { - // // If there is no duration till the next slot, check to see if the slot - // // has already arrived. If it has already arrived, send it out for - // // immediate processing. - // // - // // If we can't read the slot or the slot hasn't arrived, simply drop the - // // block. - // // - // // This logic is slightly awkward since `SlotClock::duration_to_slot` - // // doesn't distinguish between a slot that has already arrived and an - // // error reading the slot clock. - // if let Some(now) = slot_clock.now() { - // if block_slot <= now - // && ready_work_tx - // .try_send(ReadyWork::Block(early_block)) - // .is_err() - // { - // error!( - // log, - // "Failed to send block"; - // ); - // } - // } - // } - // } - // // A block that was queued for later processing is now ready to be processed. - // Some(InboundEvent::ReadyBlock(ready_block)) => { - // let block_root = ready_block.block.block_root; - // - // if !queued_block_roots.remove(&block_root) { - // // Log an error to alert that we've made a bad assumption about how this - // // program works, but still process the block anyway. - // error!( - // log, - // "Unknown block in delay queue"; - // "block_root" => ?block_root - // ); - // } - // - // if ready_work_tx - // .try_send(ReadyWork::Block(ready_block)) - // .is_err() - // { - // error!( - // log, - // "Failed to pop queued block"; - // ); - // } - // } - // Some(InboundEvent::DelayQueueError(e)) => crit!( - // log, - // "Failed to poll block delay queue"; - // "e" => ?e - // ), - // None => { - // debug!( - // log, - // "Block delay queue stopped"; - // "msg" => "shutting down" - // ); - // break; - // } - // } - } -} diff --git a/beacon_node/network/src/beacon_processor/worker/gossip_methods.rs b/beacon_node/network/src/beacon_processor/worker/gossip_methods.rs index 3d3f9913977..9e978a5c71d 100644 --- a/beacon_node/network/src/beacon_processor/worker/gossip_methods.rs +++ b/beacon_node/network/src/beacon_processor/worker/gossip_methods.rs @@ -13,11 +13,38 @@ use ssz::Encode; use std::time::{Duration, SystemTime, UNIX_EPOCH}; use tokio::sync::mpsc; use types::{ - Attestation, AttesterSlashing, Hash256, ProposerSlashing, SignedAggregateAndProof, + Attestation, AttesterSlashing, EthSpec, Hash256, ProposerSlashing, SignedAggregateAndProof, SignedBeaconBlock, SignedVoluntaryExit, SubnetId, }; -use super::{super::work_reprocessing_queue::{QueuedBlock, ReprocessQueueMessage}, Worker}; +use super::{ + super::work_reprocessing_queue::{ + QueuedAggregate, QueuedBlock, QueuedUnaggregate, ReprocessQueueMessage, + }, + Worker, +}; + +/// Data for an aggregated or unaggregated attestation that failed verification. +enum FailedAtt { + Unaggregate(Attestation), + Aggregate(SignedAggregateAndProof), +} + +impl FailedAtt { + pub fn root(&self) -> &Hash256 { + match self { + FailedAtt::Unaggregate(unaggregate) => &unaggregate.data.beacon_block_root, + FailedAtt::Aggregate(aggregate) => &aggregate.message.aggregate.data.beacon_block_root, + } + } + + pub fn kind(&self) -> &'static str { + match self { + FailedAtt::Unaggregate { .. } => "unaggregated", + FailedAtt::Aggregate { .. } => "aggregated", + } + } +} impl Worker { /* Auxiliary functions */ @@ -66,8 +93,8 @@ impl Worker { attestation: Attestation, subnet_id: SubnetId, should_import: bool, + reprocess_tx: Option>>, seen_timestamp: Duration, - reprocess_tx: mpsc::Sender>, ) { let beacon_block_root = attestation.data.beacon_block_root; @@ -76,12 +103,12 @@ impl Worker { .verify_unaggregated_attestation_for_gossip(attestation, Some(subnet_id)) { Ok(attestation) => attestation, - Err((e, attestation)) => { + Err((e, unaggregate)) => { self.handle_attestation_verification_failure( peer_id, message_id, - beacon_block_root, - "unaggregated", + FailedAtt::Unaggregate(unaggregate), + reprocess_tx, e, ); // TODO: diff between first and second @@ -155,6 +182,7 @@ impl Worker { message_id: MessageId, peer_id: PeerId, aggregate: SignedAggregateAndProof, + reprocess_tx: Option>>, seen_timestamp: Duration, ) { let beacon_block_root = aggregate.message.aggregate.data.beacon_block_root; @@ -164,13 +192,13 @@ impl Worker { .verify_aggregated_attestation_for_gossip(aggregate) { Ok(aggregate) => aggregate, - Err((e, original_aggregate)) => { + Err((e, aggregate)) => { // Report the failure to gossipsub self.handle_attestation_verification_failure( peer_id, message_id, - original_aggregate.message.aggregate.data.beacon_block_root, - "aggregated", + FailedAtt::Aggregate(aggregate), + reprocess_tx, e, ); // TODO: do something here with this @@ -380,7 +408,12 @@ impl Worker { ) } } - Ok(_) => self.process_gossip_verified_block(peer_id, verified_block, reprocess_tx, seen_duration), + Ok(_) => self.process_gossip_verified_block( + peer_id, + verified_block, + reprocess_tx, + seen_duration, + ), Err(e) => { error!( self.log, @@ -411,7 +444,16 @@ impl Worker { Ok(block_root) => { metrics::inc_counter(&metrics::BEACON_PROCESSOR_GOSSIP_BLOCK_IMPORTED_TOTAL); - reprocess_tx.try_send(ReprocessQueueMessage::BlockProcessed(block_root)); + if reprocess_tx + .try_send(ReprocessQueueMessage::BlockImported(block_root)) + .is_err() + { + error!( + self.log, + "Failed to inform block import"; + "block_root" => %block_root, + ) + }; trace!( self.log, @@ -629,14 +671,16 @@ impl Worker { /// Handle an error whilst verifying an `Attestation` or `SignedAggregateAndProof` from the /// network. - pub fn handle_attestation_verification_failure( + fn handle_attestation_verification_failure( &self, peer_id: PeerId, message_id: MessageId, - beacon_block_root: Hash256, - attestation_type: &str, + failed_att: FailedAtt, + reprocess_tx: Option>>, error: AttnError, ) { + let beacon_block_root = failed_att.root(); + let attestation_type = failed_att.kind(); metrics::register_attestation_error(&error); match &error { AttnError::FutureEpoch { .. } @@ -793,7 +837,6 @@ impl Worker { self.gossip_penalize_peer(peer_id, PeerAction::LowToleranceError); } AttnError::UnknownHeadBlock => { - // TODO: Maintain this attestation and re-process once sync completes // TODO: We then score based on whether we can download the block and re-process. debug!( @@ -802,18 +845,50 @@ impl Worker { "peer_id" => %peer_id, "block" => %beacon_block_root ); - // we don't know the block, get the sync manager to handle the block lookup - self.sync_tx - .send(SyncMessage::UnknownBlockHash(peer_id, beacon_block_root)) - .unwrap_or_else(|_| { - warn!( - self.log, - "Failed to send to sync service"; - "msg" => "UnknownBlockHash" - ) - }); - self.propagate_validation_result(message_id, peer_id, MessageAcceptance::Ignore); - return; + if let Some(sender) = reprocess_tx { + // we don't know the block, get the sync manager to handle the block lookup, and + // send the attestation to be scheduled for re-processing., and send the + // attestation to be scheduled for re-processing. + self.sync_tx + .send(SyncMessage::UnknownBlockHash( + peer_id, + beacon_block_root.clone(), + )) + .unwrap_or_else(|_| { + warn!( + self.log, + "Failed to send to sync service"; + "msg" => "UnknownBlockHash" + ) + }); + let msg = match failed_att { + FailedAtt::Aggregate(aggregate) => { + ReprocessQueueMessage::UnknownBlockAggregate(QueuedAggregate { + peer_id, + attestation, + should_import, + }) + } + FailedAtt::Unaggregate(unaggregate) => { + ReprocessQueueMessage::UnknownBlockUnaggregate(QueuedUnaggregate { + peer_id, + attestation, + should_import, + }) + } + }; + if sender.try_send(msg).is_err() {} + self.propagate_validation_result( + message_id, + peer_id, + MessageAcceptance::Ignore, + ); + return; + } else { + // We shouldn't make any further attempts to process this attestation. + // Downscore the peer. + self.gossip_penalize_peer(peer_id, PeerAction::LowToleranceError); + } } AttnError::UnknownTargetRoot(_) => { /* From 587c634be58c222e715d675743019a3fc79bd25d Mon Sep 17 00:00:00 2001 From: Diva M Date: Fri, 16 Apr 2021 20:02:54 -0500 Subject: [PATCH 11/43] add more glue --- .../work_reprocessing_queue.rs | 21 +++--- .../beacon_processor/worker/gossip_methods.rs | 72 ++++++++++++------- 2 files changed, 60 insertions(+), 33 deletions(-) diff --git a/beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs b/beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs index e3394ad9860..8cd6ce8d03e 100644 --- a/beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs +++ b/beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs @@ -9,7 +9,7 @@ //! case, the block will be sent off for immediate processing (skipping the `DelayQueue`). use super::MAX_DELAYED_BLOCK_QUEUE_LEN; use beacon_chain::{BeaconChainTypes, GossipVerifiedBlock}; -use eth2_libp2p::PeerId; +use eth2_libp2p::{MessageId, PeerId}; use fnv::FnvHashMap; use futures::task::Poll; use futures::{Stream, StreamExt}; @@ -23,7 +23,7 @@ use task_executor::TaskExecutor; use tokio::sync::mpsc::{self, Receiver, Sender}; use tokio::time::error::Error as TimeError; use tokio_util::time::DelayQueue; -use types::{Attestation, EthSpec, Hash256, SignedAggregateAndProof}; +use types::{Attestation, EthSpec, Hash256, SignedAggregateAndProof, SubnetId}; const TASK_NAME: &str = "beacon_processor_block_delay_queue"; @@ -57,18 +57,21 @@ pub enum ReadyWork { /// An Attestation for which the corresponding block was not seen while processing, queued for /// later. pub struct QueuedUnaggregate { - peer_id: PeerId, - attestation: Attestation, - subnet_id: SubnetId, - should_import: bool, + pub peer_id: PeerId, + pub message_id: MessageId, + pub attestation: Attestation, + pub subnet_id: SubnetId, + pub should_import: bool, + pub seen_timestamp: Duration, } /// An aggregated attestation for which the corresponding block was not seen while processing, queued for /// later. pub struct QueuedAggregate { - peer_id: PeerId, - attestation: SignedAggregateAndProof, - should_import: bool, + pub peer_id: PeerId, + pub message_id: MessageId, + pub attestation: SignedAggregateAndProof, + pub seen_timestamp: Duration, } /// A block that arrived early and has been queued for later import. diff --git a/beacon_node/network/src/beacon_processor/worker/gossip_methods.rs b/beacon_node/network/src/beacon_processor/worker/gossip_methods.rs index 9e978a5c71d..bda30d68e53 100644 --- a/beacon_node/network/src/beacon_processor/worker/gossip_methods.rs +++ b/beacon_node/network/src/beacon_processor/worker/gossip_methods.rs @@ -26,15 +26,25 @@ use super::{ /// Data for an aggregated or unaggregated attestation that failed verification. enum FailedAtt { - Unaggregate(Attestation), - Aggregate(SignedAggregateAndProof), + Unaggregate { + attestation: Attestation, + subnet_id: SubnetId, + should_import: bool, + seen_timestamp: Duration, + }, + Aggregate { + attestation: SignedAggregateAndProof, + seen_timestamp: Duration, + }, } impl FailedAtt { pub fn root(&self) -> &Hash256 { match self { - FailedAtt::Unaggregate(unaggregate) => &unaggregate.data.beacon_block_root, - FailedAtt::Aggregate(aggregate) => &aggregate.message.aggregate.data.beacon_block_root, + FailedAtt::Unaggregate { attestation, .. } => &attestation.data.beacon_block_root, + FailedAtt::Aggregate { attestation, .. } => { + &attestation.message.aggregate.data.beacon_block_root + } } } @@ -103,15 +113,19 @@ impl Worker { .verify_unaggregated_attestation_for_gossip(attestation, Some(subnet_id)) { Ok(attestation) => attestation, - Err((e, unaggregate)) => { + Err((e, attestation)) => { self.handle_attestation_verification_failure( peer_id, message_id, - FailedAtt::Unaggregate(unaggregate), + FailedAtt::Unaggregate { + attestation, + subnet_id, + should_import, + seen_timestamp, + }, reprocess_tx, e, ); - // TODO: diff between first and second return; } }; @@ -192,16 +206,18 @@ impl Worker { .verify_aggregated_attestation_for_gossip(aggregate) { Ok(aggregate) => aggregate, - Err((e, aggregate)) => { + Err((e, attestation)) => { // Report the failure to gossipsub self.handle_attestation_verification_failure( peer_id, message_id, - FailedAtt::Aggregate(aggregate), + FailedAtt::Aggregate { + attestation, + seen_timestamp, + }, reprocess_tx, e, ); - // TODO: do something here with this return; } }; @@ -862,20 +878,28 @@ impl Worker { ) }); let msg = match failed_att { - FailedAtt::Aggregate(aggregate) => { - ReprocessQueueMessage::UnknownBlockAggregate(QueuedAggregate { - peer_id, - attestation, - should_import, - }) - } - FailedAtt::Unaggregate(unaggregate) => { - ReprocessQueueMessage::UnknownBlockUnaggregate(QueuedUnaggregate { - peer_id, - attestation, - should_import, - }) - } + FailedAtt::Aggregate { + attestation, + seen_timestamp, + } => ReprocessQueueMessage::UnknownBlockAggregate(QueuedAggregate { + peer_id, + attestation, + message_id: message_id.clone(), + seen_timestamp, + }), + FailedAtt::Unaggregate { + attestation, + subnet_id, + should_import, + seen_timestamp, + } => ReprocessQueueMessage::UnknownBlockUnaggregate(QueuedUnaggregate { + peer_id, + should_import, + message_id: message_id.clone(), + attestation, + subnet_id, + seen_timestamp, + }), }; if sender.try_send(msg).is_err() {} self.propagate_validation_result( From e8597250d40e0fc0fac135340f81291692e209fd Mon Sep 17 00:00:00 2001 From: Diva M Date: Tue, 20 Apr 2021 14:32:12 -0500 Subject: [PATCH 12/43] almost there --- .../network/src/beacon_processor/mod.rs | 98 +++++++- .../work_reprocessing_queue.rs | 210 +++++++++++++----- .../beacon_processor/worker/gossip_methods.rs | 21 +- .../src/beacon_processor/worker/mod.rs | 1 - 4 files changed, 258 insertions(+), 72 deletions(-) diff --git a/beacon_node/network/src/beacon_processor/mod.rs b/beacon_node/network/src/beacon_processor/mod.rs index ee9aa4b09b7..56df1568204 100644 --- a/beacon_node/network/src/beacon_processor/mod.rs +++ b/beacon_node/network/src/beacon_processor/mod.rs @@ -56,7 +56,9 @@ use types::{ Attestation, AttesterSlashing, Hash256, ProposerSlashing, SignedAggregateAndProof, SignedBeaconBlock, SignedVoluntaryExit, SubnetId, }; -use work_reprocessing_queue::{spawn_reprocess_scheduler, QueuedBlock, ReadyWork}; +use work_reprocessing_queue::{ + spawn_reprocess_scheduler, QueuedAggregate, QueuedBlock, QueuedUnaggregate, ReadyWork, +}; use worker::{Toolbox, Worker}; @@ -148,6 +150,8 @@ pub const CHAIN_SEGMENT: &str = "chain_segment"; pub const STATUS_PROCESSING: &str = "status_processing"; pub const BLOCKS_BY_RANGE_REQUEST: &str = "blocks_by_range_request"; pub const BLOCKS_BY_ROOTS_REQUEST: &str = "blocks_by_roots_request"; +pub const UNKNOWN_BLOCK_ATTESTATION: &str = "unknown_block_attestation"; +pub const UNKNOWN_BLOCK_AGGREGATE: &str = "unknown_block_aggregate"; /// Used to send/receive results from a rpc block import in a blocking task. pub type BlockResultSender = oneshot::Sender>>; @@ -441,12 +445,38 @@ impl std::convert::From> for WorkEvent { seen_timestamp, }, }, - ReadyWork::Attestation(_) => { - todo!() - } - ReadyWork::Aggregate(_) => { - todo!() - } + ReadyWork::Unaggregate(QueuedUnaggregate { + peer_id, + message_id, + attestation, + subnet_id, + should_import, + seen_timestamp, + }) => Self { + drop_during_sync: true, + work: Work::UnknownBlockAttestation { + message_id, + peer_id, + attestation: Box::new(attestation), + subnet_id, + should_import, + seen_timestamp, + }, + }, + ReadyWork::Aggregate(QueuedAggregate { + peer_id, + message_id, + attestation, + seen_timestamp, + }) => Self { + drop_during_sync: true, + work: Work::UnkonwnBlockAggregate { + message_id, + peer_id, + aggregate: Box::new(attestation), + seen_timestamp, + }, + }, } } } @@ -468,6 +498,20 @@ pub enum Work { aggregate: Box>, seen_timestamp: Duration, }, + UnknownBlockAttestation { + message_id: MessageId, + peer_id: PeerId, + attestation: Box>, + subnet_id: SubnetId, + should_import: bool, + seen_timestamp: Duration, + }, + UnkonwnBlockAggregate { + message_id: MessageId, + peer_id: PeerId, + aggregate: Box>, + seen_timestamp: Duration, + }, GossipBlock { message_id: MessageId, peer_id: PeerId, @@ -534,6 +578,8 @@ impl Work { Work::Status { .. } => STATUS_PROCESSING, Work::BlocksByRangeRequest { .. } => BLOCKS_BY_RANGE_REQUEST, Work::BlocksByRootsRequest { .. } => BLOCKS_BY_ROOTS_REQUEST, + Work::UnknownBlockAttestation { .. } => UNKNOWN_BLOCK_ATTESTATION, + Work::UnkonwnBlockAggregate { .. } => UNKNOWN_BLOCK_AGGREGATE, } } } @@ -664,6 +710,10 @@ impl BeaconProcessor { let mut aggregate_debounce = TimeLatch::default(); let mut attestation_queue = LifoQueue::new(MAX_UNAGGREGATED_ATTESTATION_QUEUE_LEN); let mut attestation_debounce = TimeLatch::default(); + let mut unknown_block_aggregate_queue = + LifoQueue::new(MAX_AGGREGATED_ATTESTATION_QUEUE_LEN); + let mut unknown_block_attestation_queue = + LifoQueue::new(MAX_UNAGGREGATED_ATTESTATION_QUEUE_LEN); // Using a FIFO queue for voluntary exits since it prevents exit censoring. I don't have // a strong feeling about queue type for exits. @@ -893,6 +943,12 @@ impl BeaconProcessor { Work::BlocksByRootsRequest { .. } => { bbroots_queue.push(work, work_id, &self.log) } + Work::UnknownBlockAttestation { .. } => { + unknown_block_attestation_queue.push(work) + } + Work::UnkonwnBlockAggregate { .. } => { + unknown_block_aggregate_queue.push(work) + } } } } @@ -1144,6 +1200,34 @@ impl BeaconProcessor { request_id, request, } => worker.handle_blocks_by_root_request(peer_id, request_id, request), + Work::UnknownBlockAttestation { + message_id, + peer_id, + attestation, + subnet_id, + should_import, + seen_timestamp, + } => worker.process_gossip_attestation( + message_id, + peer_id, + *attestation, + subnet_id, + should_import, + None, // do not allow this attestation to be re processed beyond this point + seen_timestamp, + ), + Work::UnkonwnBlockAggregate { + message_id, + peer_id, + aggregate, + seen_timestamp, + } => worker.process_gossip_aggregate( + message_id, + peer_id, + *aggregate, + None, + seen_timestamp, + ), }; trace!( diff --git a/beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs b/beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs index 8cd6ce8d03e..fc82b908358 100644 --- a/beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs +++ b/beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs @@ -22,7 +22,7 @@ use std::time::Duration; use task_executor::TaskExecutor; use tokio::sync::mpsc::{self, Receiver, Sender}; use tokio::time::error::Error as TimeError; -use tokio_util::time::DelayQueue; +use tokio_util::time::delay_queue::{DelayQueue, Key as DelayKey}; use types::{Attestation, EthSpec, Hash256, SignedAggregateAndProof, SubnetId}; const TASK_NAME: &str = "beacon_processor_block_delay_queue"; @@ -50,7 +50,7 @@ pub enum ReprocessQueueMessage { /// Events sent by the scheduler once they are ready for re-processing. pub enum ReadyWork { Block(QueuedBlock), - Attestation(QueuedUnaggregate), + Unaggregate(QueuedUnaggregate), Aggregate(QueuedAggregate), } @@ -106,18 +106,16 @@ struct ReprocessQueue { /* Queues */ /// Queue to manage scheduled early blocks. block_delay_queue: DelayQueue>, - /// Queue to manage scheduled aggregated attestations. - aggregate_delay_queue: DelayQueue, /// Queue to manage scheduled attestations. - attestations_delay_queue: DelayQueue, + attestations_delay_queue: DelayQueue, /* Queued items */ /// Queued blocks. queued_block_roots: HashSet, /// Queued aggreated attestations. - queued_aggregates: FnvHashMap>, + queued_aggregates: FnvHashMap, DelayKey)>, /// Queued attestations. - queued_attestations: FnvHashMap>, + queued_unaggregates: FnvHashMap, DelayKey)>, /// Attestations (aggreated and unaggreated) per root. awaiting_attestations_per_root: HashMap>, @@ -130,15 +128,31 @@ struct ReprocessQueue { log: Logger, } +#[derive(Debug, Clone, Copy)] enum QueuedAttestationId { Aggregate(usize), Unaggregate(usize), } +impl QueuedAggregate { + pub fn root(&self) -> &Hash256 { + &self.attestation.message.aggregate.data.beacon_block_root + } +} + +impl QueuedUnaggregate { + pub fn root(&self) -> &Hash256 { + &self.attestation.data.beacon_block_root + } +} + impl Stream for ReprocessQueue { type Item = InboundEvent; fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + // NOTE: implementing `Stream` is not necessary but allows to maintain the future selection + // order fine-grained and separate from the logic of handling each message, which is nice. + // Poll for expired blocks *before* we try to process new blocks. // // The sequential nature of blockchains means it is generally better to try and import all @@ -155,26 +169,10 @@ impl Stream for ReprocessQueue { Poll::Ready(None) | Poll::Pending => (), } - // Next get the aggregates, since these should be more useful. - match self.aggregate_delay_queue.poll_expired(cx) { - Poll::Ready(Some(Ok(aggregate_id))) => { - return Poll::Ready(Some(InboundEvent::ReadyAttestation( - QueuedAttestationId::Aggregate(aggregate_id.into_inner()), - ))); - } - Poll::Ready(Some(Err(e))) => { - return Poll::Ready(Some(InboundEvent::DelayQueueError(e, "aggregates_queue"))); - } - // `Poll::Ready(None)` means that there are no more entries in the delay queue and we - // will continue to get this result until something else is added into the queue. - Poll::Ready(None) | Poll::Pending => (), - } - - // Next get the unaggregates. match self.attestations_delay_queue.poll_expired(cx) { Poll::Ready(Some(Ok(attestation_id))) => { return Poll::Ready(Some(InboundEvent::ReadyAttestation( - QueuedAttestationId::Unaggregate(attestation_id.into_inner()), + attestation_id.into_inner(), ))); } Poll::Ready(Some(Err(e))) => { @@ -185,13 +183,10 @@ impl Stream for ReprocessQueue { Poll::Ready(None) | Poll::Pending => (), } - // First empty the messages channel to ensure that when we start unqueuing attestations for - // each root, we have as many as temporarily possible. This mitigates excesive hashing and - // heap allocs. + // Last empty the messages channel. match self.work_reprocessing_rx.poll_recv(cx) { Poll::Ready(Some(message)) => return Poll::Ready(Some(InboundEvent::Msg(message))), - Poll::Ready(None) => return Poll::Ready(None), - Poll::Pending => {} + Poll::Ready(None) | Poll::Pending => {} } Poll::Pending @@ -213,11 +208,10 @@ pub fn spawn_reprocess_scheduler( work_reprocessing_rx, ready_work_tx, block_delay_queue: DelayQueue::new(), - aggregate_delay_queue: DelayQueue::new(), attestations_delay_queue: DelayQueue::new(), queued_block_roots: HashSet::new(), queued_aggregates: FnvHashMap::default(), - queued_attestations: FnvHashMap::default(), + queued_unaggregates: FnvHashMap::default(), awaiting_attestations_per_root: HashMap::new(), next_attestation: 0, slot_clock, @@ -256,10 +250,10 @@ impl ReprocessQueue { // Check to ensure this won't over-fill the queue. if self.queued_block_roots.len() >= MAXIMUM_QUEUED_BLOCKS { error!( - self.log, - "Early blocks queue is full"; - "queue_size" => MAXIMUM_QUEUED_BLOCKS, - "msg" => "check system clock" + self.log, + "Early blocks queue is full"; + "queue_size" => MAXIMUM_QUEUED_BLOCKS, + "msg" => "check system clock" ); // Drop the block. return; @@ -289,16 +283,107 @@ impl ReprocessQueue { .is_err() { error!( - self.log, - "Failed to send block"; + self.log, + "Failed to send block"; ); } } } } - Some(InboundEvent::Msg(UnknownBlockAggregate(queued_aggregate))) => {} - Some(InboundEvent::Msg(UnknownBlockUnaggregate(queued_unaggregate))) => {} - Some(InboundEvent::Msg(BlockImported(root))) => {} + Some(InboundEvent::Msg(UnknownBlockAggregate(queued_aggregate))) => { + if self.attestations_delay_queue.len() >= MAXIMUM_QUEUED_BLOCKS { + error!( + self.log, + "Attestation delay queue is full"; + "queue_size" => MAXIMUM_QUEUED_BLOCKS, + "msg" => "check system clock" + ); + // Drop the attestation. + return; + } + + let att_id = QueuedAttestationId::Aggregate(self.next_attestation); + + // Register the delay. + let delay_key = self + .attestations_delay_queue + .insert(att_id, ADDITIONAL_DELAY); + + // Register this attestation for the corresponding root. + self.awaiting_attestations_per_root + .entry(*queued_aggregate.root()) + .or_default() + .push_back(att_id); + + // Store the attestation and its info. + self.queued_aggregates + .insert(self.next_attestation, (queued_aggregate, delay_key)); + + self.next_attestation += 1; + } + Some(InboundEvent::Msg(UnknownBlockUnaggregate(queued_unaggregate))) => { + if self.attestations_delay_queue.len() >= MAXIMUM_QUEUED_BLOCKS { + error!( + self.log, + "Attestation delay queue is full"; + "queue_size" => MAXIMUM_QUEUED_BLOCKS, + "msg" => "check system clock" + ); + // Drop the attestation. + return; + } + + let att_id = QueuedAttestationId::Unaggregate(self.next_attestation); + + // Register the delay. + let delay_key = self + .attestations_delay_queue + .insert(att_id, ADDITIONAL_DELAY); + + // Register this attestation for the corresponding root. + self.awaiting_attestations_per_root + .entry(*queued_unaggregate.root()) + .or_default() + .push_back(att_id); + + // Store the attestation and its info. + self.queued_unaggregates + .insert(self.next_attestation, (queued_unaggregate, delay_key)); + + self.next_attestation += 1; + } + Some(InboundEvent::Msg(BlockImported(root))) => { + // Unqueue the attestations we have for this root, if any. + if let Some(queued_ids) = self.awaiting_attestations_per_root.remove(&root) { + for id in queued_ids { + if let Some((work, delay_key)) = match id { + QueuedAttestationId::Aggregate(id) => self + .queued_aggregates + .remove(&id) + .map(|(aggregate, delay_key)| { + (ReadyWork::Aggregate(aggregate), delay_key) + }), + QueuedAttestationId::Unaggregate(id) => self + .queued_unaggregates + .remove(&id) + .map(|(unaggregate, delay_key)| { + (ReadyWork::Unaggregate(unaggregate), delay_key) + }), + } { + // remove the delay + self.attestations_delay_queue.remove(&delay_key); + + // send the work + if self.ready_work_tx.try_send(work).is_err() { + error!( + self.log, + "Failed to send scheduled attestation"; + ); + } + } + } + } + } // A block that was queued for later processing is now ready to be processed. Some(InboundEvent::ReadyBlock(ready_block)) => { let block_root = ready_block.block.block_root; @@ -307,9 +392,9 @@ impl ReprocessQueue { // Log an error to alert that we've made a bad assumption about how this // program works, but still process the block anyway. error!( - self.log, - "Unknown block in delay queue"; - "block_root" => ?block_root + self.log, + "Unknown block in delay queue"; + "block_root" => ?block_root ); } @@ -319,25 +404,44 @@ impl ReprocessQueue { .is_err() { error!( - self.log, - "Failed to pop queued block"; + self.log, + "Failed to pop queued block"; ); } } Some(InboundEvent::DelayQueueError(e, queue_name)) => { crit!( - self.log, - "Failed to poll queue"; - "queue" => queue_name, - "e" => ?e + self.log, + "Failed to poll queue"; + "queue" => queue_name, + "e" => ?e ) } - Some(InboundEvent::ReadyAttestation(_id)) => {} + Some(InboundEvent::ReadyAttestation(queued_id)) => { + if let Some(work) = match queued_id { + QueuedAttestationId::Aggregate(id) => self + .queued_aggregates + .remove(&id) + .map(|(aggregate, _delay_key)| ReadyWork::Aggregate(aggregate)), + QueuedAttestationId::Unaggregate(id) => self + .queued_unaggregates + .remove(&id) + .map(|(unaggregate, _delay_key)| ReadyWork::Unaggregate(unaggregate)), + } { + if self.ready_work_tx.try_send(work).is_err() { + error!( + self.log, + "Failed to send scheduled attestation"; + ); + } + // TODO: remove from awaiting_attestations_per_root + } + } None => { debug!( - self.log, - "Block delay queue stopped"; - "msg" => "shutting down" + self.log, + "Block delay queue stopped"; + "msg" => "shutting down" ); return; } diff --git a/beacon_node/network/src/beacon_processor/worker/gossip_methods.rs b/beacon_node/network/src/beacon_processor/worker/gossip_methods.rs index bda30d68e53..140f8835cc6 100644 --- a/beacon_node/network/src/beacon_processor/worker/gossip_methods.rs +++ b/beacon_node/network/src/beacon_processor/worker/gossip_methods.rs @@ -853,8 +853,6 @@ impl Worker { self.gossip_penalize_peer(peer_id, PeerAction::LowToleranceError); } AttnError::UnknownHeadBlock => { - // TODO: Maintain this attestation and re-process once sync completes - // TODO: We then score based on whether we can download the block and re-process. debug!( self.log, "Attestation for unknown block"; @@ -863,8 +861,7 @@ impl Worker { ); if let Some(sender) = reprocess_tx { // we don't know the block, get the sync manager to handle the block lookup, and - // send the attestation to be scheduled for re-processing., and send the - // attestation to be scheduled for re-processing. + // send the attestation to be scheduled for re-processing. self.sync_tx self.sync_tx .send(SyncMessage::UnknownBlockHash( peer_id, @@ -901,18 +898,20 @@ impl Worker { seen_timestamp, }), }; - if sender.try_send(msg).is_err() {} - self.propagate_validation_result( - message_id, - peer_id, - MessageAcceptance::Ignore, - ); - return; + if sender.try_send(msg).is_err() { + error!( + self.log, + "Failed to send attestation for re-processing"; + ) + } } else { // We shouldn't make any further attempts to process this attestation. // Downscore the peer. self.gossip_penalize_peer(peer_id, PeerAction::LowToleranceError); } + + self.propagate_validation_result(message_id, peer_id, MessageAcceptance::Ignore); + return; } AttnError::UnknownTargetRoot(_) => { /* diff --git a/beacon_node/network/src/beacon_processor/worker/mod.rs b/beacon_node/network/src/beacon_processor/worker/mod.rs index 36503944576..e79889274fd 100644 --- a/beacon_node/network/src/beacon_processor/worker/mod.rs +++ b/beacon_node/network/src/beacon_processor/worker/mod.rs @@ -1,4 +1,3 @@ -use super::QueuedBlock; use super::work_reprocessing_queue::ReprocessQueueMessage; use crate::{service::NetworkMessage, sync::SyncMessage}; use beacon_chain::{BeaconChain, BeaconChainTypes}; From d366868af8e867d0902adabeef73172dc7f1220f Mon Sep 17 00:00:00 2001 From: Diva M Date: Wed, 21 Apr 2021 10:22:38 -0500 Subject: [PATCH 13/43] wip --- .../work_reprocessing_queue.rs | 39 +++++++++++-------- 1 file changed, 23 insertions(+), 16 deletions(-) diff --git a/beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs b/beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs index fc82b908358..b9cc01873bd 100644 --- a/beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs +++ b/beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs @@ -15,7 +15,7 @@ use futures::task::Poll; use futures::{Stream, StreamExt}; use slog::{crit, debug, error, Logger}; use slot_clock::SlotClock; -use std::collections::{HashMap, HashSet, VecDeque}; +use std::collections::{HashMap, HashSet}; use std::pin::Pin; use std::task::Context; use std::time::Duration; @@ -93,10 +93,7 @@ enum InboundEvent { Msg(ReprocessQueueMessage), } -/// Combines the `DelayQueue` and `Receiver` streams into a single stream. -/// struct has a similar purpose to `tokio::select!`, however it allows for more fine-grained -/// control (specifically in the ordering of event processing). -// TODO: update docs +/// Manages scheduling works that need to be later re-processed. struct ReprocessQueue { /// Receiver of messages relevant to schedule works for reprocessing. work_reprocessing_rx: Receiver>, @@ -117,7 +114,7 @@ struct ReprocessQueue { /// Queued attestations. queued_unaggregates: FnvHashMap, DelayKey)>, /// Attestations (aggreated and unaggreated) per root. - awaiting_attestations_per_root: HashMap>, + awaiting_attestations_per_root: HashMap>, /* Aux */ /// Next attestation id, used for both aggreated and unaggreated attestations @@ -128,7 +125,7 @@ struct ReprocessQueue { log: Logger, } -#[derive(Debug, Clone, Copy)] +#[derive(Debug, Clone, Copy, PartialEq, Eq)] enum QueuedAttestationId { Aggregate(usize), Unaggregate(usize), @@ -313,7 +310,7 @@ impl ReprocessQueue { self.awaiting_attestations_per_root .entry(*queued_aggregate.root()) .or_default() - .push_back(att_id); + .push(att_id); // Store the attestation and its info. self.queued_aggregates @@ -344,7 +341,7 @@ impl ReprocessQueue { self.awaiting_attestations_per_root .entry(*queued_unaggregate.root()) .or_default() - .push_back(att_id); + .push(att_id); // Store the attestation and its info. self.queued_unaggregates @@ -418,15 +415,20 @@ impl ReprocessQueue { ) } Some(InboundEvent::ReadyAttestation(queued_id)) => { - if let Some(work) = match queued_id { - QueuedAttestationId::Aggregate(id) => self - .queued_aggregates - .remove(&id) - .map(|(aggregate, _delay_key)| ReadyWork::Aggregate(aggregate)), + if let Some((root, work)) = match queued_id { + QueuedAttestationId::Aggregate(id) => { + self.queued_aggregates + .remove(&id) + .map(|(aggregate, _delay_key)| { + (*aggregate.root(), ReadyWork::Aggregate(aggregate)) + }) + } QueuedAttestationId::Unaggregate(id) => self .queued_unaggregates .remove(&id) - .map(|(unaggregate, _delay_key)| ReadyWork::Unaggregate(unaggregate)), + .map(|(unaggregate, _delay_key)| { + (*unaggregate.root(), ReadyWork::Unaggregate(unaggregate)) + }), } { if self.ready_work_tx.try_send(work).is_err() { error!( @@ -434,7 +436,12 @@ impl ReprocessQueue { "Failed to send scheduled attestation"; ); } - // TODO: remove from awaiting_attestations_per_root + + if let Some(queued_atts) = self.awaiting_attestations_per_root.get_mut(&root) { + if let Some(index) = queued_atts.iter().position(|&id| id == queued_id) { + queued_atts.swap_remove(index); + } + } } } None => { From 1bad0642a4c85e700475da009105b9a70f3f57f5 Mon Sep 17 00:00:00 2001 From: Diva M Date: Thu, 22 Apr 2021 11:28:32 -0500 Subject: [PATCH 14/43] update docs and constants --- .../network/src/beacon_processor/mod.rs | 74 ++++++++++++------- .../work_reprocessing_queue.rs | 53 ++++++++----- 2 files changed, 80 insertions(+), 47 deletions(-) diff --git a/beacon_node/network/src/beacon_processor/mod.rs b/beacon_node/network/src/beacon_processor/mod.rs index 56df1568204..431558c9a75 100644 --- a/beacon_node/network/src/beacon_processor/mod.rs +++ b/beacon_node/network/src/beacon_processor/mod.rs @@ -5,6 +5,7 @@ //! //! - A "manager" task, which either spawns worker tasks or enqueues work. //! - One or more "worker" tasks which perform time-intensive work on the `BeaconChain`. +//! - A task managing the scheduling of work that needs to be re-processed. //! //! ## Purpose //! @@ -19,10 +20,12 @@ //! //! ## Detail //! -//! There is a single "manager" thread who listens to two event channels. These events are either: +//! There is a single "manager" thread who listens to three event channels. These events are +//! either: //! //! - A new parcel of work (work event). //! - Indication that a worker has finished a parcel of work (worker idle). +//! - A work ready for reprocessing (work event). //! //! Then, there is a maximum of `n` "worker" blocking threads, where `n` is the CPU count. //! @@ -79,14 +82,25 @@ pub const MAX_WORK_EVENT_QUEUE_LEN: usize = 16_384; /// set to the CPU count, but we set it high to be safe. const MAX_IDLE_QUEUE_LEN: usize = 16_384; +/// The maximum size of the channel for re-processing work events. +const MAX_SCHEDULED_WORK_QUEUE_LEN: usize = 16_384; + /// The maximum number of queued `Attestation` objects that will be stored before we start dropping /// them. const MAX_UNAGGREGATED_ATTESTATION_QUEUE_LEN: usize = 16_384; +/// The maximum number of queued `Attestation` objects that will be stored before we start dropping +/// them. +const MAX_UNAGGREGATED_ATTESTATION_REPROCESS_QUEUE_LEN: usize = 8_192; + /// The maximum number of queued `SignedAggregateAndProof` objects that will be stored before we /// start dropping them. const MAX_AGGREGATED_ATTESTATION_QUEUE_LEN: usize = 1_024; +/// The maximum number of queued `SignedAggregateAndProof` objects that will be stored before we +/// start dropping them. +const MAX_AGGREGATED_ATTESTATION_REPROCESS_QUEUE_LEN: usize = 1_024; + /// The maximum number of queued `SignedBeaconBlock` objects received on gossip that will be stored /// before we start dropping them. const MAX_GOSSIP_BLOCK_QUEUE_LEN: usize = 1_024; @@ -129,6 +143,7 @@ const MAX_BLOCKS_BY_ROOTS_QUEUE_LEN: usize = 1_024; /// The name of the manager tokio task. const MANAGER_TASK_NAME: &str = "beacon_processor_manager"; + /// The name of the worker tokio tasks. const WORKER_TASK_NAME: &str = "beacon_processor_worker"; @@ -492,12 +507,6 @@ pub enum Work { should_import: bool, seen_timestamp: Duration, }, - GossipAggregate { - message_id: MessageId, - peer_id: PeerId, - aggregate: Box>, - seen_timestamp: Duration, - }, UnknownBlockAttestation { message_id: MessageId, peer_id: PeerId, @@ -506,6 +515,12 @@ pub enum Work { should_import: bool, seen_timestamp: Duration, }, + GossipAggregate { + message_id: MessageId, + peer_id: PeerId, + aggregate: Box>, + seen_timestamp: Duration, + }, UnkonwnBlockAggregate { message_id: MessageId, peer_id: PeerId, @@ -609,7 +624,7 @@ enum InboundEvent { WorkerIdle, /// There is new work to be done. WorkEvent(WorkEvent), - /// A block that was delayed for import at a later slot has become ready. + /// A work that was queued for re-processing has become ready. ReprocessingWork(WorkEvent), } @@ -698,7 +713,7 @@ impl BeaconProcessor { pub fn spawn_manager( mut self, event_rx: mpsc::Receiver>, - work_journal_tx: Option>, + work_journal_tx: Option>, ) { // Used by workers to communicate that they are finished a task. let (idle_tx, idle_rx) = mpsc::channel::<()>(MAX_IDLE_QUEUE_LEN); @@ -711,9 +726,9 @@ impl BeaconProcessor { let mut attestation_queue = LifoQueue::new(MAX_UNAGGREGATED_ATTESTATION_QUEUE_LEN); let mut attestation_debounce = TimeLatch::default(); let mut unknown_block_aggregate_queue = - LifoQueue::new(MAX_AGGREGATED_ATTESTATION_QUEUE_LEN); + LifoQueue::new(MAX_AGGREGATED_ATTESTATION_REPROCESS_QUEUE_LEN); let mut unknown_block_attestation_queue = - LifoQueue::new(MAX_UNAGGREGATED_ATTESTATION_QUEUE_LEN); + LifoQueue::new(MAX_UNAGGREGATED_ATTESTATION_REPROCESS_QUEUE_LEN); // Using a FIFO queue for voluntary exits since it prevents exit censoring. I don't have // a strong feeling about queue type for exits. @@ -736,14 +751,13 @@ impl BeaconProcessor { let mut bbrange_queue = FifoQueue::new(MAX_BLOCKS_BY_RANGE_QUEUE_LEN); let mut bbroots_queue = FifoQueue::new(MAX_BLOCKS_BY_ROOTS_QUEUE_LEN); - // The delayed block queues are used to re-queue blocks for processing at a later time if - // they're received early. - let (post_delay_block_queue_tx, post_delay_block_queue_rx) = - mpsc::channel(MAX_DELAYED_BLOCK_QUEUE_LEN); - let pre_delay_block_queue_tx = { + // Channels for sending work to the re-process scheduler (`work_reprocessing_tx`) and to + // receive them back once they are ready (`ready_work_rx`). + let (ready_work_tx, ready_work_rx) = mpsc::channel(MAX_SCHEDULED_WORK_QUEUE_LEN); + let work_reprocessing_tx = { if let Some(chain) = self.beacon_chain.upgrade() { spawn_reprocess_scheduler( - post_delay_block_queue_tx, + ready_work_tx, &self.executor, chain.slot_clock.clone(), self.log.clone(), @@ -763,7 +777,7 @@ impl BeaconProcessor { let mut inbound_events = InboundEvents { idle_rx, event_rx, - reprocess_work_rx: post_delay_block_queue_rx, + reprocess_work_rx: ready_work_rx, }; loop { @@ -803,7 +817,7 @@ impl BeaconProcessor { // We don't care if this message was successfully sent, we only use the journal // during testing. - let _ = work_journal_tx.try_send(id.to_string()); + let _ = work_journal_tx.try_send(id); } let can_spawn = self.current_workers < self.max_workers; @@ -819,7 +833,7 @@ impl BeaconProcessor { None if can_spawn => { let toolbox = Toolbox { idle_tx: idle_tx.clone(), - work_reprocessing_tx: pre_delay_block_queue_tx.clone(), + work_reprocessing_tx: work_reprocessing_tx.clone(), }; // Check for chain segments first, they're the most efficient way to get @@ -845,6 +859,12 @@ impl BeaconProcessor { self.spawn_worker(item, toolbox); } else if let Some(item) = attestation_queue.pop() { self.spawn_worker(item, toolbox); + // Aggregates and unaggregates queued for re-processing are older and we + // care about fresher ones, so check those first. + } else if let Some(item) = unknown_block_aggregate_queue.pop() { + self.spawn_worker(item, toolbox); + } else if let Some(item) = unknown_block_attestation_queue.pop() { + self.spawn_worker(item, toolbox); // Check RPC methods next. Status messages are needed for sync so // prioritize them over syncing requests from other peers (BlocksByRange // and BlocksByRoot) @@ -873,7 +893,7 @@ impl BeaconProcessor { if let Some(work_journal_tx) = &work_journal_tx { // We don't care if this message was successfully sent, we only use the journal // during testing. - let _ = work_journal_tx.try_send(NOTHING_TO_DO.to_string()); + let _ = work_journal_tx.try_send(NOTHING_TO_DO); } } } @@ -910,7 +930,7 @@ impl BeaconProcessor { let work_id = work.str_id(); let toolbox = Toolbox { idle_tx: idle_tx.clone(), - work_reprocessing_tx: pre_delay_block_queue_tx.clone(), + work_reprocessing_tx: work_reprocessing_tx.clone(), }; match work { @@ -1019,7 +1039,7 @@ impl BeaconProcessor { /// Sends an message on `idle_tx` when the work is complete and the task is stopping. fn spawn_worker(&mut self, work: Work, toolbox: Toolbox) { let idle_tx = toolbox.idle_tx; - let reprocess_tx = toolbox.work_reprocessing_tx; + let work_reprocessing_tx = toolbox.work_reprocessing_tx; // Wrap the `idle_tx` in a struct that will fire the idle message whenever it is dropped. // @@ -1090,7 +1110,7 @@ impl BeaconProcessor { *attestation, subnet_id, should_import, - Some(reprocess_tx), + Some(work_reprocessing_tx), seen_timestamp, ), /* @@ -1105,7 +1125,7 @@ impl BeaconProcessor { message_id, peer_id, *aggregate, - Some(reprocess_tx), + Some(work_reprocessing_tx), seen_timestamp, ), /* @@ -1120,7 +1140,7 @@ impl BeaconProcessor { message_id, peer_id, *block, - reprocess_tx, + work_reprocessing_tx, seen_timestamp, ), /* @@ -1133,7 +1153,7 @@ impl BeaconProcessor { } => worker.process_gossip_verified_block( peer_id, *block, - reprocess_tx, + work_reprocessing_tx, seen_timestamp, ), /* diff --git a/beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs b/beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs index b9cc01873bd..c749fb80d8c 100644 --- a/beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs +++ b/beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs @@ -1,4 +1,4 @@ -//! Provides a mechanism which queues blocks for later processing when they arrive too early. +//! Provides a mechanism which queues work for later processing. //! //! When the `beacon_processor::Worker` imports a block that is acceptably early (i.e., within the //! gossip propagation tolerance) it will send it to this queue where it will be placed in a @@ -7,8 +7,11 @@ //! //! There is the edge-case where the slot arrives before this queue manages to process it. In that //! case, the block will be sent off for immediate processing (skipping the `DelayQueue`). -use super::MAX_DELAYED_BLOCK_QUEUE_LEN; -use beacon_chain::{BeaconChainTypes, GossipVerifiedBlock}; +//! +//! Aggregated and unaggreated attestations that failed verification due to referencing an unknown +//! block will be delayed for `` +use super::MAX_SCHEDULED_WORK_QUEUE_LEN; +use beacon_chain::{BeaconChainTypes, GossipVerifiedBlock, MAXIMUM_GOSSIP_CLOCK_DISPARITY}; use eth2_libp2p::{MessageId, PeerId}; use fnv::FnvHashMap; use futures::task::Poll; @@ -25,17 +28,23 @@ use tokio::time::error::Error as TimeError; use tokio_util::time::delay_queue::{DelayQueue, Key as DelayKey}; use types::{Attestation, EthSpec, Hash256, SignedAggregateAndProof, SubnetId}; -const TASK_NAME: &str = "beacon_processor_block_delay_queue"; +const TASK_NAME: &str = "beacon_processor_reprocess_queue"; + +/// Queue blocks for re-processing with an `ADDITIONAL_QUEUED_BLOCK_DELAY` after the slot starts. +/// This is to account for any slight drift in the system clock. +const ADDITIONAL_QUEUED_BLOCK_DELAY: Duration = Duration::from_millis(5); -/// Queue blocks for re-processing with an `ADDITIONAL_DELAY` after the slot starts. This is to -/// account for any slight drift in the system clock. -const ADDITIONAL_DELAY: Duration = Duration::from_millis(5); +/// For how long attestations will be kept before sending them back for reprocessing. +const QUEUED_ATTESTATION_DELAY: Duration = Duration::from_secs(20); /// Set an arbitrary upper-bound on the number of queued blocks to avoid DoS attacks. The fact that /// we signature-verify blocks before putting them in the queue *should* protect against this, but /// it's nice to have extra protection. const MAXIMUM_QUEUED_BLOCKS: usize = 16; +/// How many attestations we keep before new ones get dropped. +const MAXIMUM_QUEUED_ATTESTATIONS: usize = 1_024; + /// Messages that the scheduler can receive. pub enum ReprocessQueueMessage { /// A block that has been received early and we should queue for later processing. @@ -173,7 +182,7 @@ impl Stream for ReprocessQueue { ))); } Poll::Ready(Some(Err(e))) => { - return Poll::Ready(Some(InboundEvent::DelayQueueError(e, "unaggregates_queue"))); + return Poll::Ready(Some(InboundEvent::DelayQueueError(e, "attestations_queue"))); } // `Poll::Ready(None)` means that there are no more entries in the delay queue and we // will continue to get this result until something else is added into the queue. @@ -191,15 +200,17 @@ impl Stream for ReprocessQueue { } /// Starts the job that manages scheduling works that need re-processing. The returned `Sender` -/// gives the communicating channel to receive those works. Once a work is ready, send them back -/// out via `ready_work_tx`. +/// gives the communicating channel to receive those works. Once a work is ready, it is sent back +/// via `ready_work_tx`. pub fn spawn_reprocess_scheduler( ready_work_tx: Sender>, executor: &TaskExecutor, slot_clock: T::SlotClock, log: Logger, ) -> Sender> { - let (work_reprocessing_tx, work_reprocessing_rx) = mpsc::channel(MAX_DELAYED_BLOCK_QUEUE_LEN); + let (work_reprocessing_tx, work_reprocessing_rx) = mpsc::channel(MAX_SCHEDULED_WORK_QUEUE_LEN); + // basic sanity check + assert!(ADDITIONAL_QUEUED_BLOCK_DELAY < MAXIMUM_GOSSIP_CLOCK_DISPARITY); let mut queue = ReprocessQueue { work_reprocessing_rx, @@ -258,9 +269,11 @@ impl ReprocessQueue { self.queued_block_roots.insert(block_root); // Queue the block until the start of the appropriate slot, plus - // `ADDITIONAL_DELAY`. - self.block_delay_queue - .insert(early_block, duration_till_slot + ADDITIONAL_DELAY); + // `ADDITIONAL_QUEUED_BLOCK_DELAY`. + self.block_delay_queue.insert( + early_block, + duration_till_slot + ADDITIONAL_QUEUED_BLOCK_DELAY, + ); } else { // If there is no duration till the next slot, check to see if the slot // has already arrived. If it has already arrived, send it out for @@ -288,11 +301,11 @@ impl ReprocessQueue { } } Some(InboundEvent::Msg(UnknownBlockAggregate(queued_aggregate))) => { - if self.attestations_delay_queue.len() >= MAXIMUM_QUEUED_BLOCKS { + if self.attestations_delay_queue.len() >= MAXIMUM_QUEUED_ATTESTATIONS { error!( self.log, "Attestation delay queue is full"; - "queue_size" => MAXIMUM_QUEUED_BLOCKS, + "queue_size" => MAXIMUM_QUEUED_ATTESTATIONS, "msg" => "check system clock" ); // Drop the attestation. @@ -304,7 +317,7 @@ impl ReprocessQueue { // Register the delay. let delay_key = self .attestations_delay_queue - .insert(att_id, ADDITIONAL_DELAY); + .insert(att_id, QUEUED_ATTESTATION_DELAY); // Register this attestation for the corresponding root. self.awaiting_attestations_per_root @@ -319,11 +332,11 @@ impl ReprocessQueue { self.next_attestation += 1; } Some(InboundEvent::Msg(UnknownBlockUnaggregate(queued_unaggregate))) => { - if self.attestations_delay_queue.len() >= MAXIMUM_QUEUED_BLOCKS { + if self.attestations_delay_queue.len() >= MAXIMUM_QUEUED_ATTESTATIONS { error!( self.log, "Attestation delay queue is full"; - "queue_size" => MAXIMUM_QUEUED_BLOCKS, + "queue_size" => MAXIMUM_QUEUED_ATTESTATIONS, "msg" => "check system clock" ); // Drop the attestation. @@ -335,7 +348,7 @@ impl ReprocessQueue { // Register the delay. let delay_key = self .attestations_delay_queue - .insert(att_id, ADDITIONAL_DELAY); + .insert(att_id, QUEUED_ATTESTATION_DELAY); // Register this attestation for the corresponding root. self.awaiting_attestations_per_root From 1b88a1b411bf9a16635e0318149c43290ab3268c Mon Sep 17 00:00:00 2001 From: Diva M Date: Fri, 23 Apr 2021 11:41:08 -0500 Subject: [PATCH 15/43] add metrics --- .../src/attestation_verification.rs | 7 +-- .../network/src/beacon_processor/mod.rs | 2 +- .../network/src/beacon_processor/tests.rs | 1 - .../work_reprocessing_queue.rs | 9 ++++ .../beacon_processor/worker/gossip_methods.rs | 48 +++++++++++-------- beacon_node/network/src/metrics.rs | 8 ++++ 6 files changed, 50 insertions(+), 25 deletions(-) diff --git a/beacon_node/beacon_chain/src/attestation_verification.rs b/beacon_node/beacon_chain/src/attestation_verification.rs index a5287119420..7caad41d1d1 100644 --- a/beacon_node/beacon_chain/src/attestation_verification.rs +++ b/beacon_node/beacon_chain/src/attestation_verification.rs @@ -137,7 +137,7 @@ pub enum Error { /// /// The attestation points to a block we have not yet imported. It's unclear if the attestation /// is valid or not. - UnknownHeadBlock, + UnknownHeadBlock { beacon_block_root: Hash256 }, /// The `attestation.data.slot` is not from the same epoch as `data.target.epoch`. /// /// ## Peer scoring @@ -748,7 +748,6 @@ impl VerifiedUnaggregatedAttestation { use AttestationSlashInfo::*; if let Err(e) = Self::verify_early_checks(&attestation, chain) { - // TODO: check how to avoid copying this return Err((SignatureNotChecked(attestation.clone(), e), attestation)); } @@ -854,7 +853,9 @@ fn verify_head_block_is_known( Ok(block) } else { - Err(Error::UnknownHeadBlock) + Err(Error::UnknownHeadBlock { + beacon_block_root: attestation.data.beacon_block_root, + }) } } diff --git a/beacon_node/network/src/beacon_processor/mod.rs b/beacon_node/network/src/beacon_processor/mod.rs index 431558c9a75..0b3389f701f 100644 --- a/beacon_node/network/src/beacon_processor/mod.rs +++ b/beacon_node/network/src/beacon_processor/mod.rs @@ -1233,7 +1233,7 @@ impl BeaconProcessor { *attestation, subnet_id, should_import, - None, // do not allow this attestation to be re processed beyond this point + None, // do not allow this attestation to be re-processed beyond this point seen_timestamp, ), Work::UnkonwnBlockAggregate { diff --git a/beacon_node/network/src/beacon_processor/tests.rs b/beacon_node/network/src/beacon_processor/tests.rs index d6f07cb7310..01a21838f4a 100644 --- a/beacon_node/network/src/beacon_processor/tests.rs +++ b/beacon_node/network/src/beacon_processor/tests.rs @@ -65,7 +65,6 @@ impl Drop for TestRig { impl TestRig { pub fn new(chain_length: u64) -> Self { - let mut harness = BeaconChainHarness::new( MainnetEthSpec, generate_deterministic_keypairs(VALIDATOR_COUNT), diff --git a/beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs b/beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs index c749fb80d8c..05dbaa27bc6 100644 --- a/beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs +++ b/beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs @@ -390,6 +390,15 @@ impl ReprocessQueue { "Failed to send scheduled attestation"; ); } + } else { + // there is a mismatch between the attestation ids registered for this + // root and the queued attestations. This should never happen + error!( + self.log, + "Unknown queued attestation for block root"; + "block_root" => ?root, + "att_id" => ?id, + ); } } } diff --git a/beacon_node/network/src/beacon_processor/worker/gossip_methods.rs b/beacon_node/network/src/beacon_processor/worker/gossip_methods.rs index 140f8835cc6..fbba02c7243 100644 --- a/beacon_node/network/src/beacon_processor/worker/gossip_methods.rs +++ b/beacon_node/network/src/beacon_processor/worker/gossip_methods.rs @@ -852,7 +852,7 @@ impl Worker { self.propagate_validation_result(message_id, peer_id, MessageAcceptance::Reject); self.gossip_penalize_peer(peer_id, PeerAction::LowToleranceError); } - AttnError::UnknownHeadBlock => { + AttnError::UnknownHeadBlock { beacon_block_root } => { debug!( self.log, "Attestation for unknown block"; @@ -861,12 +861,9 @@ impl Worker { ); if let Some(sender) = reprocess_tx { // we don't know the block, get the sync manager to handle the block lookup, and - // send the attestation to be scheduled for re-processing. self.sync_tx + // send the attestation to be scheduled for re-processing. self.sync_tx - .send(SyncMessage::UnknownBlockHash( - peer_id, - beacon_block_root.clone(), - )) + .send(SyncMessage::UnknownBlockHash(peer_id, *beacon_block_root)) .unwrap_or_else(|_| { warn!( self.log, @@ -878,26 +875,37 @@ impl Worker { FailedAtt::Aggregate { attestation, seen_timestamp, - } => ReprocessQueueMessage::UnknownBlockAggregate(QueuedAggregate { - peer_id, - attestation, - message_id: message_id.clone(), - seen_timestamp, - }), + } => { + metrics::inc_counter( + &metrics::BEACON_PROCESSOR_AGGREGATED_ATTESTATION_REQUEUED_TOTAL, + ); + ReprocessQueueMessage::UnknownBlockAggregate(QueuedAggregate { + peer_id, + attestation, + message_id: message_id.clone(), + seen_timestamp, + }) + } FailedAtt::Unaggregate { attestation, subnet_id, should_import, seen_timestamp, - } => ReprocessQueueMessage::UnknownBlockUnaggregate(QueuedUnaggregate { - peer_id, - should_import, - message_id: message_id.clone(), - attestation, - subnet_id, - seen_timestamp, - }), + } => { + metrics::inc_counter( + &metrics::BEACON_PROCESSOR_UNAGGREGATED_ATTESTATION_REQUEUED_TOTAL, + ); + ReprocessQueueMessage::UnknownBlockUnaggregate(QueuedUnaggregate { + peer_id, + should_import, + message_id: message_id.clone(), + attestation, + subnet_id, + seen_timestamp, + }) + } }; + if sender.try_send(msg).is_err() { error!( self.log, diff --git a/beacon_node/network/src/metrics.rs b/beacon_node/network/src/metrics.rs index f04cf8d7ed5..f8387c37086 100644 --- a/beacon_node/network/src/metrics.rs +++ b/beacon_node/network/src/metrics.rs @@ -301,6 +301,10 @@ lazy_static! { "beacon_processor_unaggregated_attestation_imported_total", "Total number of unaggregated attestations imported to fork choice, etc." ); + pub static ref BEACON_PROCESSOR_UNAGGREGATED_ATTESTATION_REQUEUED_TOTAL: Result = try_create_int_counter( + "beacon_processor_unaggregated_attestation_requeued_total", + "Total number of unaggregated attestations that referenced an unknwon block and were re-queued." + ); // Aggregated attestations. pub static ref BEACON_PROCESSOR_AGGREGATED_ATTESTATION_QUEUE_TOTAL: Result = try_create_int_gauge( "beacon_processor_aggregated_attestation_queue_total", @@ -314,6 +318,10 @@ lazy_static! { "beacon_processor_aggregated_attestation_imported_total", "Total number of aggregated attestations imported to fork choice, etc." ); + pub static ref BEACON_PROCESSOR_AGGREGATED_ATTESTATION_REQUEUED_TOTAL: Result = try_create_int_counter( + "beacon_processor_aggregated_attestation_requeued_total", + "Total number of aggregated attestations that referenced an unknwon block and were re-queued." + ); } lazy_static! { From 4213847996beb29fe908a38c38ce50fb0efdfbce Mon Sep 17 00:00:00 2001 From: Diva M Date: Tue, 27 Apr 2021 10:15:34 -0500 Subject: [PATCH 16/43] fix delay --- .../work_reprocessing_queue.rs | 23 +++++++++++-------- 1 file changed, 14 insertions(+), 9 deletions(-) diff --git a/beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs b/beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs index 05dbaa27bc6..4099dfd2cc3 100644 --- a/beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs +++ b/beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs @@ -34,9 +34,6 @@ const TASK_NAME: &str = "beacon_processor_reprocess_queue"; /// This is to account for any slight drift in the system clock. const ADDITIONAL_QUEUED_BLOCK_DELAY: Duration = Duration::from_millis(5); -/// For how long attestations will be kept before sending them back for reprocessing. -const QUEUED_ATTESTATION_DELAY: Duration = Duration::from_secs(20); - /// Set an arbitrary upper-bound on the number of queued blocks to avoid DoS attacks. The fact that /// we signature-verify blocks before putting them in the queue *should* protect against this, but /// it's nice to have extra protection. @@ -312,12 +309,16 @@ impl ReprocessQueue { return; } + // Check for how long can we keep this attestation before it is too late. + let delay = self + .slot_clock + .duration_to_next_slot() + .unwrap_or(self.slot_clock.slot_duration()); + let att_id = QueuedAttestationId::Aggregate(self.next_attestation); // Register the delay. - let delay_key = self - .attestations_delay_queue - .insert(att_id, QUEUED_ATTESTATION_DELAY); + let delay_key = self.attestations_delay_queue.insert(att_id, delay); // Register this attestation for the corresponding root. self.awaiting_attestations_per_root @@ -343,12 +344,16 @@ impl ReprocessQueue { return; } + // Check for how long can we keep this attestation before it is too late. + let delay = self + .slot_clock + .duration_to_next_slot() + .unwrap_or(self.slot_clock.slot_duration()); + let att_id = QueuedAttestationId::Unaggregate(self.next_attestation); // Register the delay. - let delay_key = self - .attestations_delay_queue - .insert(att_id, QUEUED_ATTESTATION_DELAY); + let delay_key = self.attestations_delay_queue.insert(att_id, delay); // Register this attestation for the corresponding root. self.awaiting_attestations_per_root From 3b5b253278bcff3b0bf9f40ef13d17ca3611b684 Mon Sep 17 00:00:00 2001 From: Diva M Date: Tue, 27 Apr 2021 11:40:40 -0500 Subject: [PATCH 17/43] add initial test --- .../src/attestation_verification.rs | 11 +++- beacon_node/beacon_chain/src/beacon_chain.rs | 8 ++- .../network/src/beacon_processor/tests.rs | 61 ++++++++++++++++--- consensus/types/src/eth_spec.rs | 16 ++++- 4 files changed, 79 insertions(+), 17 deletions(-) diff --git a/beacon_node/beacon_chain/src/attestation_verification.rs b/beacon_node/beacon_chain/src/attestation_verification.rs index 7caad41d1d1..f9de3eeb1bd 100644 --- a/beacon_node/beacon_chain/src/attestation_verification.rs +++ b/beacon_node/beacon_chain/src/attestation_verification.rs @@ -383,7 +383,9 @@ impl VerifiedAggregatedAttestation { } verified_aggregate }) - .map_err(|(slash_info, original_aggregate)| (process_slash_info(slash_info, chain), original_aggregate)) + .map_err(|(slash_info, original_aggregate)| { + (process_slash_info(slash_info, chain), original_aggregate) + }) } /// Run the checks that happen before an indexed attestation is constructed. @@ -557,7 +559,12 @@ impl VerifiedAggregatedAttestation { .map_err(|e| BeaconChainError::from(e).into()) }) { Ok(indexed_attestation) => indexed_attestation, - Err(e) => return Err((SignatureNotChecked(signed_aggregate.message.aggregate.clone(), e), signed_aggregate)), + Err(e) => { + return Err(( + SignatureNotChecked(signed_aggregate.message.aggregate.clone(), e), + signed_aggregate, + )) + } }; // Ensure that all signatures are valid. diff --git a/beacon_node/beacon_chain/src/beacon_chain.rs b/beacon_node/beacon_chain/src/beacon_chain.rs index 162488135ce..dbc18bf6ada 100644 --- a/beacon_node/beacon_chain/src/beacon_chain.rs +++ b/beacon_node/beacon_chain/src/beacon_chain.rs @@ -1017,7 +1017,8 @@ impl BeaconChain { &self, unaggregated_attestation: Attestation, subnet_id: Option, - ) -> Result, (AttestationError, Attestation)> { + ) -> Result, (AttestationError, Attestation)> + { metrics::inc_counter(&metrics::UNAGGREGATED_ATTESTATION_PROCESSING_REQUESTS); let _timer = metrics::start_timer(&metrics::UNAGGREGATED_ATTESTATION_GOSSIP_VERIFICATION_TIMES); @@ -1041,7 +1042,10 @@ impl BeaconChain { pub fn verify_aggregated_attestation_for_gossip( &self, signed_aggregate: SignedAggregateAndProof, - ) -> Result, (AttestationError, SignedAggregateAndProof)> { + ) -> Result< + VerifiedAggregatedAttestation, + (AttestationError, SignedAggregateAndProof), + > { metrics::inc_counter(&metrics::AGGREGATED_ATTESTATION_PROCESSING_REQUESTS); let _timer = metrics::start_timer(&metrics::AGGREGATED_ATTESTATION_GOSSIP_VERIFICATION_TIMES); diff --git a/beacon_node/network/src/beacon_processor/tests.rs b/beacon_node/network/src/beacon_processor/tests.rs index 01a21838f4a..8cca0c411ee 100644 --- a/beacon_node/network/src/beacon_processor/tests.rs +++ b/beacon_node/network/src/beacon_processor/tests.rs @@ -42,12 +42,12 @@ struct TestRig { chain: Arc>, next_block: SignedBeaconBlock, attestations: Vec<(Attestation, SubnetId)>, + next_block_attestations: Vec<(Attestation, SubnetId)>, attester_slashing: AttesterSlashing, proposer_slashing: ProposerSlashing, voluntary_exit: SignedVoluntaryExit, beacon_processor_tx: mpsc::Sender>, - /// TODO: document this guy - work_journal_rx: mpsc::Receiver, + work_journal_rx: mpsc::Receiver<&'static str>, _network_rx: mpsc::UnboundedReceiver>, _sync_rx: mpsc::UnboundedReceiver>, environment: Option>, @@ -90,7 +90,7 @@ impl TestRig { "precondition: current slot is one after head" ); - let (next_block, _next_state) = + let (next_block, next_state) = harness.make_block(head.beacon_state.clone(), harness.chain.slot().unwrap()); let head_state_root = head.beacon_state_root(); @@ -111,6 +111,23 @@ impl TestRig { "precondition: attestations for testing" ); + let next_block_attestations = harness + .get_unaggregated_attestations( + &AttestationStrategy::AllValidators, + &next_state, + next_block.state_root(), + next_state.canonical_root(), + next_block.slot(), + ) + .into_iter() + .flatten() + .collect::>(); + + assert!( + !next_block_attestations.is_empty(), + "precondition: attestation for next block are not empty" + ); + let attester_slashing = harness.make_attester_slashing(vec![0, 1]); let proposer_slashing = harness.make_proposer_slashing(2); let voluntary_exit = harness.make_voluntary_exit(3, harness.chain.epoch().unwrap()); @@ -174,6 +191,7 @@ impl TestRig { chain, next_block, attestations, + next_block_attestations, attester_slashing, proposer_slashing, voluntary_exit, @@ -305,13 +323,7 @@ impl TestRig { events }); - assert_eq!( - events, - expected - .into_iter() - .map(|s| s.to_string()) - .collect::>() - ); + assert_eq!(events, expected); } } @@ -445,6 +457,35 @@ fn import_gossip_attestation() { ); } +/// Ensure that attestations that reference an unkown block get properly re-queued and +/// re-processed. +#[test] +fn import_unknown_block_gossip_attestation() { + let mut rig = TestRig::new(SMALL_CHAIN); + + let initial_attns = rig.chain.naive_aggregation_pool.read().num_attestations(); + + let (attestation, subnet_id) = rig.next_block_attestations.first().unwrap().clone(); + rig.beacon_processor_tx + .try_send(WorkEvent::unaggregated_attestation( + junk_message_id(), + junk_peer_id(), + attestation, + subnet_id, + true, + Duration::from_secs(0), + )) + .unwrap(); + + rig.assert_event_journal(&[GOSSIP_ATTESTATION, WORKER_FREED, NOTHING_TO_DO]); + + assert_eq!( + rig.chain.naive_aggregation_pool.read().num_attestations(), + initial_attns, + "op pool should not have been included" + ); +} + /// Ensure a bunch of valid operations can be imported. #[test] fn import_misc_gossip_ops() { diff --git a/consensus/types/src/eth_spec.rs b/consensus/types/src/eth_spec.rs index 6156b40204b..3671460c6c5 100644 --- a/consensus/types/src/eth_spec.rs +++ b/consensus/types/src/eth_spec.rs @@ -6,8 +6,11 @@ use ssz_types::typenum::{ Unsigned, U0, U1024, U1099511627776, U128, U16, U16777216, U2, U2048, U32, U4, U4096, U64, U65536, U8, U8192, }; -use std::{fmt::{self, Debug}, marker::Unpin}; use std::str::FromStr; +use std::{ + fmt::{self, Debug}, + marker::Unpin, +}; const MAINNET: &str = "mainnet"; const MINIMAL: &str = "minimal"; @@ -51,7 +54,14 @@ pub trait EthSpec: 'static + Default + Sync + Send + Clone + Debug + PartialEq + * Constants */ type GenesisEpoch: Unsigned + Clone + Sync + Send + Debug + PartialEq + Unpin; - type JustificationBitsLength: Unsigned + Clone + Sync + Send + Debug + PartialEq + Default + Unpin; + type JustificationBitsLength: Unsigned + + Clone + + Sync + + Send + + Debug + + PartialEq + + Default + + Unpin; type SubnetBitfieldLength: Unsigned + Clone + Sync + Send + Debug + PartialEq + Default + Unpin; /* * Misc @@ -73,7 +83,7 @@ pub trait EthSpec: 'static + Default + Sync + Send + Clone + Debug + PartialEq + /* * Max operations per block */ - type MaxProposerSlashings: Unsigned + Clone + Sync + Send + Debug + PartialEq + Unpin; + type MaxProposerSlashings: Unsigned + Clone + Sync + Send + Debug + PartialEq + Unpin; type MaxAttesterSlashings: Unsigned + Clone + Sync + Send + Debug + PartialEq + Unpin; type MaxAttestations: Unsigned + Clone + Sync + Send + Debug + PartialEq + Unpin; type MaxDeposits: Unsigned + Clone + Sync + Send + Debug + PartialEq + Unpin; From a62bbdf5051eef77890a5dac5571bb289f884b53 Mon Sep 17 00:00:00 2001 From: Diva M Date: Tue, 27 Apr 2021 17:52:32 -0500 Subject: [PATCH 18/43] better first test --- .../beacon_chain/src/validator_monitor.rs | 1 + .../network/src/beacon_processor/tests.rs | 121 +++++++++++++++--- .../work_reprocessing_queue.rs | 20 +++ .../beacon_processor/worker/gossip_methods.rs | 7 + 4 files changed, 130 insertions(+), 19 deletions(-) diff --git a/beacon_node/beacon_chain/src/validator_monitor.rs b/beacon_node/beacon_chain/src/validator_monitor.rs index f3ef3aa8813..e26077a71b4 100644 --- a/beacon_node/beacon_chain/src/validator_monitor.rs +++ b/beacon_node/beacon_chain/src/validator_monitor.rs @@ -540,6 +540,7 @@ impl ValidatorMonitor { indexed_attestation: &IndexedAttestation, slot_clock: &S, ) { + println!("attestation registered"); self.register_unaggregated_attestation( "gossip", seen_timestamp, diff --git a/beacon_node/network/src/beacon_processor/tests.rs b/beacon_node/network/src/beacon_processor/tests.rs index 8cca0c411ee..b337def7d81 100644 --- a/beacon_node/network/src/beacon_processor/tests.rs +++ b/beacon_node/network/src/beacon_processor/tests.rs @@ -116,7 +116,7 @@ impl TestRig { &AttestationStrategy::AllValidators, &next_state, next_block.state_root(), - next_state.canonical_root(), + next_block.canonical_root(), next_block.slot(), ) .into_iter() @@ -203,6 +203,10 @@ impl TestRig { } } + pub fn head_root(&self) -> Hash256 { + self.chain.head().unwrap().beacon_block_root + } + pub fn enqueue_gossip_block(&self) { self.beacon_processor_tx .try_send(WorkEvent::gossip_beacon_block( @@ -258,6 +262,20 @@ impl TestRig { .unwrap(); } + pub fn enqueue_next_block_unaggregated_attestation(&self) { + let (attestation, subnet_id) = self.next_block_attestations.first().unwrap().clone(); + self.beacon_processor_tx + .try_send(WorkEvent::unaggregated_attestation( + junk_message_id(), + junk_peer_id(), + attestation, + subnet_id, + true, + Duration::from_secs(0), + )) + .unwrap(); + } + fn runtime(&mut self) -> Arc { self.environment .as_mut() @@ -283,6 +301,61 @@ impl TestRig { }) } + /// Checks that the `BeaconProcessor` event journal contains the `expected` events in the given + /// order with a matching number of `WORKER_FREED` events in between. `NOTHING_TO_DO` events + /// are ignored. + /// + /// Given the described logic, `expected` must not contain `WORKER_FREED` or `NOTHING_TO_DO` + /// events. + pub fn assert_event_journal_contains_ordered(&mut self, expected: &[&str]) { + assert!(expected + .iter() + .all(|ev| ev != &WORKER_FREED && ev != &NOTHING_TO_DO)); + + let (events, worker_freed_remaining) = self.runtime().block_on(async { + let mut events = Vec::with_capacity(expected.len()); + let mut worker_freed_remaining = expected.len(); + + let drain_future = async { + loop { + match self.work_journal_rx.recv().await { + Some(event) if event == WORKER_FREED => { + worker_freed_remaining -= 1; + if worker_freed_remaining == 0 { + // Break when all expected events are finished. + break; + } + } + Some(event) if event == NOTHING_TO_DO => { + // Ignore these. + } + Some(event) => { + events.push(event); + } + None => break, + } + } + }; + + // Drain the expected number of events from the channel, or time out and give up. + tokio::select! { + _ = tokio::time::sleep(STANDARD_TIMEOUT) => panic!( + "timeout ({:?}) expired waiting for events. expected {:?} but got {:?} waiting for {} `WORKER_FREED` events", + STANDARD_TIMEOUT, + expected, + events, + worker_freed_remaining, + ), + _ = drain_future => {}, + } + + (events, worker_freed_remaining) + }); + + assert_eq!(events, expected); + assert_eq!(worker_freed_remaining, 0); + } + /// Assert that the `BeaconProcessor` event journal is as `expected`. /// /// ## Note @@ -365,18 +438,18 @@ fn import_gossip_block_acceptably_early() { // processing. // // If this causes issues we might be able to make the block delay queue add a longer delay for - // processing, instead of just MAXIMUM_GOSSIP_CLOCK_DISPARITY. Speak to @paulhauner if this test + // processing, instead of just ADDITIONAL_QUEUED_BLOCK_DELAY. Speak to @paulhauner if this test // starts failing. rig.chain.slot_clock.set_slot(rig.next_block.slot().into()); assert!( - rig.chain.head().unwrap().beacon_block_root != rig.next_block.canonical_root(), + rig.head_root() != rig.next_block.canonical_root(), "block not yet imported" ); rig.assert_event_journal(&[DELAYED_IMPORT_BLOCK, WORKER_FREED, NOTHING_TO_DO]); assert_eq!( - rig.chain.head().unwrap().beacon_block_root, + rig.head_root(), rig.next_block.canonical_root(), "block should be imported and become head" ); @@ -412,7 +485,7 @@ fn import_gossip_block_unacceptably_early() { rig.assert_no_events_for(Duration::from_secs(5)); assert!( - rig.chain.head().unwrap().beacon_block_root != rig.next_block.canonical_root(), + rig.head_root() != rig.next_block.canonical_root(), "block should not be imported" ); } @@ -433,7 +506,7 @@ fn import_gossip_block_at_current_slot() { rig.assert_event_journal(&[GOSSIP_BLOCK, WORKER_FREED, NOTHING_TO_DO]); assert_eq!( - rig.chain.head().unwrap().beacon_block_root, + rig.head_root(), rig.next_block.canonical_root(), "block should be imported and become head" ); @@ -458,31 +531,41 @@ fn import_gossip_attestation() { } /// Ensure that attestations that reference an unkown block get properly re-queued and -/// re-processed. +/// re-processed upon seeing the block. #[test] fn import_unknown_block_gossip_attestation() { let mut rig = TestRig::new(SMALL_CHAIN); + // Send the attestation but not the block, and check that it was not imported. + let initial_attns = rig.chain.naive_aggregation_pool.read().num_attestations(); - let (attestation, subnet_id) = rig.next_block_attestations.first().unwrap().clone(); - rig.beacon_processor_tx - .try_send(WorkEvent::unaggregated_attestation( - junk_message_id(), - junk_peer_id(), - attestation, - subnet_id, - true, - Duration::from_secs(0), - )) - .unwrap(); + rig.enqueue_next_block_unaggregated_attestation(); rig.assert_event_journal(&[GOSSIP_ATTESTATION, WORKER_FREED, NOTHING_TO_DO]); assert_eq!( rig.chain.naive_aggregation_pool.read().num_attestations(), initial_attns, - "op pool should not have been included" + "Attestation should not have been included." + ); + + // Send the block and ensure that the attestation is received back and imported. + + rig.enqueue_gossip_block(); + + rig.assert_event_journal_contains_ordered(&[GOSSIP_BLOCK, UNKNOWN_BLOCK_ATTESTATION]); + + assert_eq!( + rig.head_root(), + rig.next_block.canonical_root(), + "Block should be imported and become head." + ); + + assert_eq!( + rig.chain.naive_aggregation_pool.read().num_attestations(), + initial_attns + 1, + "Attestation should have been included." ); } diff --git a/beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs b/beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs index 4099dfd2cc3..1fd78f4d974 100644 --- a/beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs +++ b/beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs @@ -333,6 +333,10 @@ impl ReprocessQueue { self.next_attestation += 1; } Some(InboundEvent::Msg(UnknownBlockUnaggregate(queued_unaggregate))) => { + println!( + "received unaggreated attestation for root {:?}", + queued_unaggregate.root() + ); if self.attestations_delay_queue.len() >= MAXIMUM_QUEUED_ATTESTATIONS { error!( self.log, @@ -361,6 +365,13 @@ impl ReprocessQueue { .or_default() .push(att_id); + println!( + "added attestation for root {:?} with id {:?}. Number of queued attestations: {}", + queued_unaggregate.root(), + att_id, + self.awaiting_attestations_per_root.len(), + ); + // Store the attestation and its info. self.queued_unaggregates .insert(self.next_attestation, (queued_unaggregate, delay_key)); @@ -370,6 +381,11 @@ impl ReprocessQueue { Some(InboundEvent::Msg(BlockImported(root))) => { // Unqueue the attestations we have for this root, if any. if let Some(queued_ids) = self.awaiting_attestations_per_root.remove(&root) { + println!( + "There are attestations queued for this root {:?}: [{}]", + root, + queued_ids.len() + ); for id in queued_ids { if let Some((work, delay_key)) = match id { QueuedAttestationId::Aggregate(id) => self @@ -406,6 +422,8 @@ impl ReprocessQueue { ); } } + } else { + println!("no attestations queued for this root {:?}", root); } } // A block that was queued for later processing is now ready to be processed. @@ -442,6 +460,7 @@ impl ReprocessQueue { ) } Some(InboundEvent::ReadyAttestation(queued_id)) => { + println!("attestation with id became ready {:?}", queued_id); if let Some((root, work)) = match queued_id { QueuedAttestationId::Aggregate(id) => { self.queued_aggregates @@ -457,6 +476,7 @@ impl ReprocessQueue { (*unaggregate.root(), ReadyWork::Unaggregate(unaggregate)) }), } { + println!("ready for root {:?}", root); if self.ready_work_tx.try_send(work).is_err() { error!( self.log, diff --git a/beacon_node/network/src/beacon_processor/worker/gossip_methods.rs b/beacon_node/network/src/beacon_processor/worker/gossip_methods.rs index fbba02c7243..2a07b59ba01 100644 --- a/beacon_node/network/src/beacon_processor/worker/gossip_methods.rs +++ b/beacon_node/network/src/beacon_processor/worker/gossip_methods.rs @@ -130,6 +130,8 @@ impl Worker { } }; + println!("Attestation was verified for root {:?}", beacon_block_root); + // Register the attestation with any monitored validators. self.chain .validator_monitor @@ -148,9 +150,11 @@ impl Worker { return; } + println!("importing thing"); metrics::inc_counter(&metrics::BEACON_PROCESSOR_UNAGGREGATED_ATTESTATION_VERIFIED_TOTAL); if let Err(e) = self.chain.apply_attestation_to_fork_choice(&attestation) { + println!("apply to fork choice failed with error {:?}", e); match e { BeaconChainError::ForkChoiceError(ForkChoiceError::InvalidAttestation(e)) => { debug!( @@ -172,6 +176,7 @@ impl Worker { } if let Err(e) = self.chain.add_to_naive_aggregation_pool(attestation) { + println!("attestation was invalid for aggregation pool {:?}", e); debug!( self.log, "Attestation invalid for agg pool"; @@ -180,6 +185,7 @@ impl Worker { "beacon_block_root" => ?beacon_block_root ) } + println!("attestation was added to aggregation pool"); metrics::inc_counter(&metrics::BEACON_PROCESSOR_UNAGGREGATED_ATTESTATION_IMPORTED_TOTAL); } @@ -695,6 +701,7 @@ impl Worker { reprocess_tx: Option>>, error: AttnError, ) { + println!("error {:?}", error); let beacon_block_root = failed_att.root(); let attestation_type = failed_att.kind(); metrics::register_attestation_error(&error); From 117af5d701134079284e91f4280b94fa56b99504 Mon Sep 17 00:00:00 2001 From: Diva M Date: Tue, 27 Apr 2021 18:30:26 -0500 Subject: [PATCH 19/43] second test --- .../network/src/beacon_processor/tests.rs | 31 +++++++++++++++++++ .../work_reprocessing_queue.rs | 14 ++++----- 2 files changed, 38 insertions(+), 7 deletions(-) diff --git a/beacon_node/network/src/beacon_processor/tests.rs b/beacon_node/network/src/beacon_processor/tests.rs index b337def7d81..e56cf7ee491 100644 --- a/beacon_node/network/src/beacon_processor/tests.rs +++ b/beacon_node/network/src/beacon_processor/tests.rs @@ -569,6 +569,37 @@ fn import_unknown_block_gossip_attestation() { ); } +/// Ensure that attestations that reference an unkown block get properly re-queued and re-processed +/// when the block is not seen. +#[test] +fn requeue_unknown_block_gossip_attestation_without_import() { + let mut rig = TestRig::new(SMALL_CHAIN); + + // Send the attestation but not the block, and check that it was not imported. + + let initial_attns = rig.chain.naive_aggregation_pool.read().num_attestations(); + + rig.enqueue_next_block_unaggregated_attestation(); + + rig.assert_event_journal(&[GOSSIP_ATTESTATION, WORKER_FREED, NOTHING_TO_DO]); + + assert_eq!( + rig.chain.naive_aggregation_pool.read().num_attestations(), + initial_attns, + "Attestation should not have been included." + ); + + // Ensure that the attestation is received back but not imported. + + rig.assert_event_journal(&[UNKNOWN_BLOCK_ATTESTATION, WORKER_FREED, NOTHING_TO_DO]); + + assert_eq!( + rig.chain.naive_aggregation_pool.read().num_attestations(), + initial_attns, + "Attestation should have been included." + ); +} + /// Ensure a bunch of valid operations can be imported. #[test] fn import_misc_gossip_ops() { diff --git a/beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs b/beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs index 1fd78f4d974..6e1e2a576d6 100644 --- a/beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs +++ b/beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs @@ -356,6 +356,13 @@ impl ReprocessQueue { let att_id = QueuedAttestationId::Unaggregate(self.next_attestation); + println!( + "added attestation for root {:?} with id {:?}. Number of queued attestations: {} for {:?}", + queued_unaggregate.root(), + att_id, + self.awaiting_attestations_per_root.len(), + delay + ); // Register the delay. let delay_key = self.attestations_delay_queue.insert(att_id, delay); @@ -365,13 +372,6 @@ impl ReprocessQueue { .or_default() .push(att_id); - println!( - "added attestation for root {:?} with id {:?}. Number of queued attestations: {}", - queued_unaggregate.root(), - att_id, - self.awaiting_attestations_per_root.len(), - ); - // Store the attestation and its info. self.queued_unaggregates .insert(self.next_attestation, (queued_unaggregate, delay_key)); From 23d3a6e42b2058b23131340c34bf6b6c37dfde1b Mon Sep 17 00:00:00 2001 From: Diva M Date: Tue, 27 Apr 2021 19:05:34 -0500 Subject: [PATCH 20/43] clippy lints --- beacon_node/beacon_chain/src/attestation_verification.rs | 6 ++++++ beacon_node/network/src/beacon_processor/mod.rs | 4 ++-- .../src/beacon_processor/work_reprocessing_queue.rs | 9 ++++----- .../src/beacon_processor/worker/gossip_methods.rs | 8 ++++---- 4 files changed, 16 insertions(+), 11 deletions(-) diff --git a/beacon_node/beacon_chain/src/attestation_verification.rs b/beacon_node/beacon_chain/src/attestation_verification.rs index f9de3eeb1bd..c98c5dd18fc 100644 --- a/beacon_node/beacon_chain/src/attestation_verification.rs +++ b/beacon_node/beacon_chain/src/attestation_verification.rs @@ -511,6 +511,9 @@ impl VerifiedAggregatedAttestation { } /// Verify the attestation, producing extra information about whether it might be slashable. + // NOTE: clippy considers the return too complex. This tuple is not used elsewhere so it is not + // worth creating an alias. + #[allow(clippy::type_complexity)] pub fn verify_slashable( signed_aggregate: SignedAggregateAndProof, chain: &BeaconChain, @@ -747,6 +750,9 @@ impl VerifiedUnaggregatedAttestation { } /// Verify the attestation, producing extra information about whether it might be slashable. + // NOTE: clippy considers the return too complex. This tuple is not used elsewhere so it is not + // worth creating an alias. + #[allow(clippy::type_complexity)] pub fn verify_slashable( attestation: Attestation, subnet_id: Option, diff --git a/beacon_node/network/src/beacon_processor/mod.rs b/beacon_node/network/src/beacon_processor/mod.rs index 0b3389f701f..9deee10d549 100644 --- a/beacon_node/network/src/beacon_processor/mod.rs +++ b/beacon_node/network/src/beacon_processor/mod.rs @@ -472,7 +472,7 @@ impl std::convert::From> for WorkEvent { work: Work::UnknownBlockAttestation { message_id, peer_id, - attestation: Box::new(attestation), + attestation, subnet_id, should_import, seen_timestamp, @@ -488,7 +488,7 @@ impl std::convert::From> for WorkEvent { work: Work::UnkonwnBlockAggregate { message_id, peer_id, - aggregate: Box::new(attestation), + aggregate: attestation, seen_timestamp, }, }, diff --git a/beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs b/beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs index 6e1e2a576d6..f7407e93306 100644 --- a/beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs +++ b/beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs @@ -65,7 +65,7 @@ pub enum ReadyWork { pub struct QueuedUnaggregate { pub peer_id: PeerId, pub message_id: MessageId, - pub attestation: Attestation, + pub attestation: Box>, pub subnet_id: SubnetId, pub should_import: bool, pub seen_timestamp: Duration, @@ -76,7 +76,7 @@ pub struct QueuedUnaggregate { pub struct QueuedAggregate { pub peer_id: PeerId, pub message_id: MessageId, - pub attestation: SignedAggregateAndProof, + pub attestation: Box>, pub seen_timestamp: Duration, } @@ -313,7 +313,7 @@ impl ReprocessQueue { let delay = self .slot_clock .duration_to_next_slot() - .unwrap_or(self.slot_clock.slot_duration()); + .unwrap_or_else(|| self.slot_clock.slot_duration()); let att_id = QueuedAttestationId::Aggregate(self.next_attestation); @@ -352,7 +352,7 @@ impl ReprocessQueue { let delay = self .slot_clock .duration_to_next_slot() - .unwrap_or(self.slot_clock.slot_duration()); + .unwrap_or_else(|| self.slot_clock.slot_duration()); let att_id = QueuedAttestationId::Unaggregate(self.next_attestation); @@ -497,7 +497,6 @@ impl ReprocessQueue { "Block delay queue stopped"; "msg" => "shutting down" ); - return; } } } diff --git a/beacon_node/network/src/beacon_processor/worker/gossip_methods.rs b/beacon_node/network/src/beacon_processor/worker/gossip_methods.rs index 2a07b59ba01..a3684a715ad 100644 --- a/beacon_node/network/src/beacon_processor/worker/gossip_methods.rs +++ b/beacon_node/network/src/beacon_processor/worker/gossip_methods.rs @@ -27,13 +27,13 @@ use super::{ /// Data for an aggregated or unaggregated attestation that failed verification. enum FailedAtt { Unaggregate { - attestation: Attestation, + attestation: Box>, subnet_id: SubnetId, should_import: bool, seen_timestamp: Duration, }, Aggregate { - attestation: SignedAggregateAndProof, + attestation: Box>, seen_timestamp: Duration, }, } @@ -118,7 +118,7 @@ impl Worker { peer_id, message_id, FailedAtt::Unaggregate { - attestation, + attestation: Box::new(attestation), subnet_id, should_import, seen_timestamp, @@ -218,7 +218,7 @@ impl Worker { peer_id, message_id, FailedAtt::Aggregate { - attestation, + attestation: Box::new(attestation), seen_timestamp, }, reprocess_tx, From cbd74a90529aa6231af2813a6c798bab86983ef5 Mon Sep 17 00:00:00 2001 From: Diva M Date: Tue, 27 Apr 2021 19:08:12 -0500 Subject: [PATCH 21/43] cleanup --- .../beacon_chain/src/validator_monitor.rs | 1 - .../work_reprocessing_queue.rs | 20 ------------------- .../beacon_processor/worker/gossip_methods.rs | 7 ------- 3 files changed, 28 deletions(-) diff --git a/beacon_node/beacon_chain/src/validator_monitor.rs b/beacon_node/beacon_chain/src/validator_monitor.rs index e26077a71b4..f3ef3aa8813 100644 --- a/beacon_node/beacon_chain/src/validator_monitor.rs +++ b/beacon_node/beacon_chain/src/validator_monitor.rs @@ -540,7 +540,6 @@ impl ValidatorMonitor { indexed_attestation: &IndexedAttestation, slot_clock: &S, ) { - println!("attestation registered"); self.register_unaggregated_attestation( "gossip", seen_timestamp, diff --git a/beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs b/beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs index f7407e93306..f6a65572759 100644 --- a/beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs +++ b/beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs @@ -333,10 +333,6 @@ impl ReprocessQueue { self.next_attestation += 1; } Some(InboundEvent::Msg(UnknownBlockUnaggregate(queued_unaggregate))) => { - println!( - "received unaggreated attestation for root {:?}", - queued_unaggregate.root() - ); if self.attestations_delay_queue.len() >= MAXIMUM_QUEUED_ATTESTATIONS { error!( self.log, @@ -356,13 +352,6 @@ impl ReprocessQueue { let att_id = QueuedAttestationId::Unaggregate(self.next_attestation); - println!( - "added attestation for root {:?} with id {:?}. Number of queued attestations: {} for {:?}", - queued_unaggregate.root(), - att_id, - self.awaiting_attestations_per_root.len(), - delay - ); // Register the delay. let delay_key = self.attestations_delay_queue.insert(att_id, delay); @@ -381,11 +370,6 @@ impl ReprocessQueue { Some(InboundEvent::Msg(BlockImported(root))) => { // Unqueue the attestations we have for this root, if any. if let Some(queued_ids) = self.awaiting_attestations_per_root.remove(&root) { - println!( - "There are attestations queued for this root {:?}: [{}]", - root, - queued_ids.len() - ); for id in queued_ids { if let Some((work, delay_key)) = match id { QueuedAttestationId::Aggregate(id) => self @@ -422,8 +406,6 @@ impl ReprocessQueue { ); } } - } else { - println!("no attestations queued for this root {:?}", root); } } // A block that was queued for later processing is now ready to be processed. @@ -460,7 +442,6 @@ impl ReprocessQueue { ) } Some(InboundEvent::ReadyAttestation(queued_id)) => { - println!("attestation with id became ready {:?}", queued_id); if let Some((root, work)) = match queued_id { QueuedAttestationId::Aggregate(id) => { self.queued_aggregates @@ -476,7 +457,6 @@ impl ReprocessQueue { (*unaggregate.root(), ReadyWork::Unaggregate(unaggregate)) }), } { - println!("ready for root {:?}", root); if self.ready_work_tx.try_send(work).is_err() { error!( self.log, diff --git a/beacon_node/network/src/beacon_processor/worker/gossip_methods.rs b/beacon_node/network/src/beacon_processor/worker/gossip_methods.rs index a3684a715ad..5e73518f2d5 100644 --- a/beacon_node/network/src/beacon_processor/worker/gossip_methods.rs +++ b/beacon_node/network/src/beacon_processor/worker/gossip_methods.rs @@ -130,8 +130,6 @@ impl Worker { } }; - println!("Attestation was verified for root {:?}", beacon_block_root); - // Register the attestation with any monitored validators. self.chain .validator_monitor @@ -150,11 +148,9 @@ impl Worker { return; } - println!("importing thing"); metrics::inc_counter(&metrics::BEACON_PROCESSOR_UNAGGREGATED_ATTESTATION_VERIFIED_TOTAL); if let Err(e) = self.chain.apply_attestation_to_fork_choice(&attestation) { - println!("apply to fork choice failed with error {:?}", e); match e { BeaconChainError::ForkChoiceError(ForkChoiceError::InvalidAttestation(e)) => { debug!( @@ -176,7 +172,6 @@ impl Worker { } if let Err(e) = self.chain.add_to_naive_aggregation_pool(attestation) { - println!("attestation was invalid for aggregation pool {:?}", e); debug!( self.log, "Attestation invalid for agg pool"; @@ -185,7 +180,6 @@ impl Worker { "beacon_block_root" => ?beacon_block_root ) } - println!("attestation was added to aggregation pool"); metrics::inc_counter(&metrics::BEACON_PROCESSOR_UNAGGREGATED_ATTESTATION_IMPORTED_TOTAL); } @@ -701,7 +695,6 @@ impl Worker { reprocess_tx: Option>>, error: AttnError, ) { - println!("error {:?}", error); let beacon_block_root = failed_att.root(); let attestation_type = failed_att.kind(); metrics::register_attestation_error(&error); From 0edffed97f868aa08789567c9769a0d006dcf55d Mon Sep 17 00:00:00 2001 From: Diva M Date: Tue, 27 Apr 2021 19:13:56 -0500 Subject: [PATCH 22/43] ignore attestation message for unknown block only after retry --- .../network/src/beacon_processor/worker/gossip_methods.rs | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/beacon_node/network/src/beacon_processor/worker/gossip_methods.rs b/beacon_node/network/src/beacon_processor/worker/gossip_methods.rs index 5e73518f2d5..e8f0310ce8d 100644 --- a/beacon_node/network/src/beacon_processor/worker/gossip_methods.rs +++ b/beacon_node/network/src/beacon_processor/worker/gossip_methods.rs @@ -916,9 +916,13 @@ impl Worker { // We shouldn't make any further attempts to process this attestation. // Downscore the peer. self.gossip_penalize_peer(peer_id, PeerAction::LowToleranceError); + self.propagate_validation_result( + message_id, + peer_id, + MessageAcceptance::Ignore, + ); } - self.propagate_validation_result(message_id, peer_id, MessageAcceptance::Ignore); return; } AttnError::UnknownTargetRoot(_) => { @@ -978,7 +982,6 @@ impl Worker { self.propagate_validation_result(message_id, peer_id, MessageAcceptance::Reject); self.gossip_penalize_peer(peer_id, PeerAction::LowToleranceError); } - AttnError::InvalidSubnetId { received, expected } => { /* * The attestation was received on an incorrect subnet id. From aea9c3b06306ad0f73c59043769ec81c54738f77 Mon Sep 17 00:00:00 2001 From: Diva M Date: Tue, 27 Apr 2021 19:16:39 -0500 Subject: [PATCH 23/43] Comment style --- beacon_node/beacon_chain/src/attestation_verification.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/beacon_node/beacon_chain/src/attestation_verification.rs b/beacon_node/beacon_chain/src/attestation_verification.rs index c98c5dd18fc..e4efc612f9f 100644 --- a/beacon_node/beacon_chain/src/attestation_verification.rs +++ b/beacon_node/beacon_chain/src/attestation_verification.rs @@ -511,7 +511,7 @@ impl VerifiedAggregatedAttestation { } /// Verify the attestation, producing extra information about whether it might be slashable. - // NOTE: clippy considers the return too complex. This tuple is not used elsewhere so it is not + // NOTE: Clippy considers the return too complex. This tuple is not used elsewhere so it is not // worth creating an alias. #[allow(clippy::type_complexity)] pub fn verify_slashable( @@ -750,7 +750,7 @@ impl VerifiedUnaggregatedAttestation { } /// Verify the attestation, producing extra information about whether it might be slashable. - // NOTE: clippy considers the return too complex. This tuple is not used elsewhere so it is not + // NOTE: Clippy considers the return too complex. This tuple is not used elsewhere so it is not // worth creating an alias. #[allow(clippy::type_complexity)] pub fn verify_slashable( From 07b8b16ec74fc37d2da36fcdf118fcd6d0cb54a8 Mon Sep 17 00:00:00 2001 From: Diva M Date: Tue, 27 Apr 2021 20:28:32 -0500 Subject: [PATCH 24/43] fmt and spelling --- beacon_node/network/src/beacon_processor/mod.rs | 4 ++-- .../network/src/beacon_processor/tests.rs | 16 ++++++++-------- .../beacon_processor/work_reprocessing_queue.rs | 16 +++++++++------- .../beacon_processor/worker/gossip_methods.rs | 2 +- consensus/types/src/eth_spec.rs | 6 ++---- 5 files changed, 22 insertions(+), 22 deletions(-) diff --git a/beacon_node/network/src/beacon_processor/mod.rs b/beacon_node/network/src/beacon_processor/mod.rs index 9deee10d549..0ba1e50959b 100644 --- a/beacon_node/network/src/beacon_processor/mod.rs +++ b/beacon_node/network/src/beacon_processor/mod.rs @@ -637,7 +637,7 @@ struct InboundEvents { idle_rx: mpsc::Receiver<()>, /// Used by upstream processes to send new work to the `BeaconProcessor`. event_rx: mpsc::Receiver>, - /// Used internally for queuing work ready to be reprocessed. + /// Used internally for queuing work ready to be re-processed. reprocess_work_rx: mpsc::Receiver>, } @@ -1233,7 +1233,7 @@ impl BeaconProcessor { *attestation, subnet_id, should_import, - None, // do not allow this attestation to be re-processed beyond this point + None, // Do not allow this attestation to be re-processed beyond this point. seen_timestamp, ), Work::UnkonwnBlockAggregate { diff --git a/beacon_node/network/src/beacon_processor/tests.rs b/beacon_node/network/src/beacon_processor/tests.rs index e56cf7ee491..004d25c0bc3 100644 --- a/beacon_node/network/src/beacon_processor/tests.rs +++ b/beacon_node/network/src/beacon_processor/tests.rs @@ -1,4 +1,4 @@ -// #![cfg(not(debug_assertions))] // Tests are too slow in debug. +#![cfg(not(debug_assertions))] // Tests are too slow in debug. #![cfg(test)] use crate::beacon_processor::*; @@ -340,7 +340,7 @@ impl TestRig { // Drain the expected number of events from the channel, or time out and give up. tokio::select! { _ = tokio::time::sleep(STANDARD_TIMEOUT) => panic!( - "timeout ({:?}) expired waiting for events. expected {:?} but got {:?} waiting for {} `WORKER_FREED` events", + "Timeout ({:?}) expired waiting for events. Expected {:?} but got {:?} waiting for {} `WORKER_FREED` events.", STANDARD_TIMEOUT, expected, events, @@ -385,7 +385,7 @@ impl TestRig { // Drain the expected number of events from the channel, or time out and give up. tokio::select! { _ = tokio::time::sleep(STANDARD_TIMEOUT) => panic!( - "timeout ({:?}) expired waiting for events. expected {:?} but got {:?}", + "Timeout ({:?}) expired waiting for events. Expected {:?} but got {:?}", STANDARD_TIMEOUT, expected, events @@ -480,7 +480,7 @@ fn import_gossip_block_unacceptably_early() { rig.assert_event_journal(&[GOSSIP_BLOCK, WORKER_FREED, NOTHING_TO_DO]); - // Waiting for 5 seconds is a bit arbtirary, however it *should* be long enough to ensure the + // Waiting for 5 seconds is a bit arbitrary, however it *should* be long enough to ensure the // block isn't imported. rig.assert_no_events_for(Duration::from_secs(5)); @@ -530,8 +530,8 @@ fn import_gossip_attestation() { ); } -/// Ensure that attestations that reference an unkown block get properly re-queued and -/// re-processed upon seeing the block. +/// Ensure that attestations that reference an unknown block get properly re-queued and +/// re-processed upon importing the block. #[test] fn import_unknown_block_gossip_attestation() { let mut rig = TestRig::new(SMALL_CHAIN); @@ -569,7 +569,7 @@ fn import_unknown_block_gossip_attestation() { ); } -/// Ensure that attestations that reference an unkown block get properly re-queued and re-processed +/// Ensure that attestations that reference an unknown block get properly re-queued and re-processed /// when the block is not seen. #[test] fn requeue_unknown_block_gossip_attestation_without_import() { @@ -596,7 +596,7 @@ fn requeue_unknown_block_gossip_attestation_without_import() { assert_eq!( rig.chain.naive_aggregation_pool.read().num_attestations(), initial_attns, - "Attestation should have been included." + "Attestation should not have been included." ); } diff --git a/beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs b/beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs index f6a65572759..1b3f4fd121b 100644 --- a/beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs +++ b/beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs @@ -8,8 +8,8 @@ //! There is the edge-case where the slot arrives before this queue manages to process it. In that //! case, the block will be sent off for immediate processing (skipping the `DelayQueue`). //! -//! Aggregated and unaggreated attestations that failed verification due to referencing an unknown -//! block will be delayed for `` +//! Aggregated and unaggregated attestations that failed verification due to referencing an unknown +//! block will be re-queued until their block is imported, or until they expire. use super::MAX_SCHEDULED_WORK_QUEUE_LEN; use beacon_chain::{BeaconChainTypes, GossipVerifiedBlock, MAXIMUM_GOSSIP_CLOCK_DISPARITY}; use eth2_libp2p::{MessageId, PeerId}; @@ -46,10 +46,12 @@ const MAXIMUM_QUEUED_ATTESTATIONS: usize = 1_024; pub enum ReprocessQueueMessage { /// A block that has been received early and we should queue for later processing. EarlyBlock(QueuedBlock), - /// A block that was succesfully processed. We use this to handle attestations for unknown + /// A block that was successfully processed. We use this to handle attestations for unknown /// blocks. BlockImported(Hash256), + /// An unaggregated attestation that references an unknown block. UnknownBlockUnaggregate(QueuedUnaggregate), + /// An aggregated attestation that references an unknown block. UnknownBlockAggregate(QueuedAggregate), } @@ -91,7 +93,7 @@ pub struct QueuedBlock { enum InboundEvent { /// A block that was queued for later processing and is ready for import. ReadyBlock(QueuedBlock), - /// An aggregated or unaggreated attestation is ready for re-processing. + /// An aggregated or unaggregated attestation is ready for re-processing. ReadyAttestation(QueuedAttestationId), /// A `DelayQueue` returned an error. DelayQueueError(TimeError, &'static str), @@ -115,15 +117,15 @@ struct ReprocessQueue { /* Queued items */ /// Queued blocks. queued_block_roots: HashSet, - /// Queued aggreated attestations. + /// Queued aggregated attestations. queued_aggregates: FnvHashMap, DelayKey)>, /// Queued attestations. queued_unaggregates: FnvHashMap, DelayKey)>, - /// Attestations (aggreated and unaggreated) per root. + /// Attestations (aggregated and unaggregated) per root. awaiting_attestations_per_root: HashMap>, /* Aux */ - /// Next attestation id, used for both aggreated and unaggreated attestations + /// Next attestation id, used for both aggregated and unaggregated attestations next_attestation: usize, slot_clock: T::SlotClock, diff --git a/beacon_node/network/src/beacon_processor/worker/gossip_methods.rs b/beacon_node/network/src/beacon_processor/worker/gossip_methods.rs index e8f0310ce8d..955c4d1da6d 100644 --- a/beacon_node/network/src/beacon_processor/worker/gossip_methods.rs +++ b/beacon_node/network/src/beacon_processor/worker/gossip_methods.rs @@ -860,7 +860,7 @@ impl Worker { "block" => %beacon_block_root ); if let Some(sender) = reprocess_tx { - // we don't know the block, get the sync manager to handle the block lookup, and + // We don't know the block, get the sync manager to handle the block lookup, and // send the attestation to be scheduled for re-processing. self.sync_tx .send(SyncMessage::UnknownBlockHash(peer_id, *beacon_block_root)) diff --git a/consensus/types/src/eth_spec.rs b/consensus/types/src/eth_spec.rs index 3671460c6c5..9083b55db22 100644 --- a/consensus/types/src/eth_spec.rs +++ b/consensus/types/src/eth_spec.rs @@ -6,11 +6,9 @@ use ssz_types::typenum::{ Unsigned, U0, U1024, U1099511627776, U128, U16, U16777216, U2, U2048, U32, U4, U4096, U64, U65536, U8, U8192, }; +use std::fmt::{self, Debug}; +use std::marker::Unpin; use std::str::FromStr; -use std::{ - fmt::{self, Debug}, - marker::Unpin, -}; const MAINNET: &str = "mainnet"; const MINIMAL: &str = "minimal"; From 5e9c6b07a7d69acbd53d3434b5a012a5d0ae131d Mon Sep 17 00:00:00 2001 From: Diva M Date: Tue, 27 Apr 2021 20:32:58 -0500 Subject: [PATCH 25/43] update delay --- .../work_reprocessing_queue.rs | 23 ++++++++----------- 1 file changed, 9 insertions(+), 14 deletions(-) diff --git a/beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs b/beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs index 1b3f4fd121b..b3347860ef3 100644 --- a/beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs +++ b/beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs @@ -34,6 +34,9 @@ const TASK_NAME: &str = "beacon_processor_reprocess_queue"; /// This is to account for any slight drift in the system clock. const ADDITIONAL_QUEUED_BLOCK_DELAY: Duration = Duration::from_millis(5); +/// For how long to queue aggregated and unaggregated attestations for re-processing. +const QUEUED_ATTESTATION_DELAY: Duration = Duration::from_secs(30); + /// Set an arbitrary upper-bound on the number of queued blocks to avoid DoS attacks. The fact that /// we signature-verify blocks before putting them in the queue *should* protect against this, but /// it's nice to have extra protection. @@ -311,16 +314,12 @@ impl ReprocessQueue { return; } - // Check for how long can we keep this attestation before it is too late. - let delay = self - .slot_clock - .duration_to_next_slot() - .unwrap_or_else(|| self.slot_clock.slot_duration()); - let att_id = QueuedAttestationId::Aggregate(self.next_attestation); // Register the delay. - let delay_key = self.attestations_delay_queue.insert(att_id, delay); + let delay_key = self + .attestations_delay_queue + .insert(att_id, QUEUED_ATTESTATION_DELAY); // Register this attestation for the corresponding root. self.awaiting_attestations_per_root @@ -346,16 +345,12 @@ impl ReprocessQueue { return; } - // Check for how long can we keep this attestation before it is too late. - let delay = self - .slot_clock - .duration_to_next_slot() - .unwrap_or_else(|| self.slot_clock.slot_duration()); - let att_id = QueuedAttestationId::Unaggregate(self.next_attestation); // Register the delay. - let delay_key = self.attestations_delay_queue.insert(att_id, delay); + let delay_key = self + .attestations_delay_queue + .insert(att_id, QUEUED_ATTESTATION_DELAY); // Register this attestation for the corresponding root. self.awaiting_attestations_per_root From c3e1e28cc3cfc7403169fbc761357589f2d10754 Mon Sep 17 00:00:00 2001 From: Diva M Date: Tue, 27 Apr 2021 21:52:12 -0500 Subject: [PATCH 26/43] update delay --- .../network/src/beacon_processor/tests.rs | 18 +++++++++++++----- .../work_reprocessing_queue.rs | 19 ++++++++++++++++++- 2 files changed, 31 insertions(+), 6 deletions(-) diff --git a/beacon_node/network/src/beacon_processor/tests.rs b/beacon_node/network/src/beacon_processor/tests.rs index 004d25c0bc3..333b3f40c8d 100644 --- a/beacon_node/network/src/beacon_processor/tests.rs +++ b/beacon_node/network/src/beacon_processor/tests.rs @@ -1,6 +1,7 @@ -#![cfg(not(debug_assertions))] // Tests are too slow in debug. +//#![cfg(not(debug_assertions))] // Tests are too slow in debug. #![cfg(test)] +use crate::beacon_processor::work_reprocessing_queue::QUEUED_ATTESTATION_DELAY; use crate::beacon_processor::*; use crate::{service::NetworkMessage, sync::SyncMessage}; use beacon_chain::{ @@ -356,13 +357,17 @@ impl TestRig { assert_eq!(worker_freed_remaining, 0); } + pub fn assert_event_journal(&mut self, expected: &[&str]) { + self.assert_event_journal_with_timeout(expected, STANDARD_TIMEOUT); + } + /// Assert that the `BeaconProcessor` event journal is as `expected`. /// /// ## Note /// /// We won't attempt to listen for any more than `expected.len()` events. As such, it makes sense /// to use the `NOTHING_TO_DO` event to ensure that execution has completed. - pub fn assert_event_journal(&mut self, expected: &[&str]) { + pub fn assert_event_journal_with_timeout(&mut self, expected: &[&str], timeout: Duration) { let events = self.runtime().block_on(async { let mut events = Vec::with_capacity(expected.len()); @@ -384,9 +389,9 @@ impl TestRig { // Drain the expected number of events from the channel, or time out and give up. tokio::select! { - _ = tokio::time::sleep(STANDARD_TIMEOUT) => panic!( + _ = tokio::time::sleep(timeout) => panic!( "Timeout ({:?}) expired waiting for events. Expected {:?} but got {:?}", - STANDARD_TIMEOUT, + timeout, expected, events ), @@ -591,7 +596,10 @@ fn requeue_unknown_block_gossip_attestation_without_import() { // Ensure that the attestation is received back but not imported. - rig.assert_event_journal(&[UNKNOWN_BLOCK_ATTESTATION, WORKER_FREED, NOTHING_TO_DO]); + rig.assert_event_journal_with_timeout( + &[UNKNOWN_BLOCK_ATTESTATION, WORKER_FREED, NOTHING_TO_DO], + Duration::from_secs(1) + QUEUED_ATTESTATION_DELAY, + ); assert_eq!( rig.chain.naive_aggregation_pool.read().num_attestations(), diff --git a/beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs b/beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs index b3347860ef3..4548fd78ade 100644 --- a/beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs +++ b/beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs @@ -35,7 +35,7 @@ const TASK_NAME: &str = "beacon_processor_reprocess_queue"; const ADDITIONAL_QUEUED_BLOCK_DELAY: Duration = Duration::from_millis(5); /// For how long to queue aggregated and unaggregated attestations for re-processing. -const QUEUED_ATTESTATION_DELAY: Duration = Duration::from_secs(30); +pub const QUEUED_ATTESTATION_DELAY: Duration = Duration::from_secs(30); /// Set an arbitrary upper-bound on the number of queued blocks to avoid DoS attacks. The fact that /// we signature-verify blocks before putting them in the queue *should* protect against this, but @@ -133,6 +133,9 @@ struct ReprocessQueue { slot_clock: T::SlotClock, + /// The waker for the current thread. + waker: Option, + log: Logger, } @@ -161,6 +164,15 @@ impl Stream for ReprocessQueue { // NOTE: implementing `Stream` is not necessary but allows to maintain the future selection // order fine-grained and separate from the logic of handling each message, which is nice. + // update the waker if needed + if let Some(waker) = &self.waker { + if waker.will_wake(cx.waker()) { + self.waker = Some(cx.waker().clone()); + } + } else { + self.waker = Some(cx.waker().clone()); + } + // Poll for expired blocks *before* we try to process new blocks. // // The sequential nature of blockchains means it is generally better to try and import all @@ -225,6 +237,7 @@ pub fn spawn_reprocess_scheduler( awaiting_attestations_per_root: HashMap::new(), next_attestation: 0, slot_clock, + waker: None, log, }; @@ -233,6 +246,10 @@ pub fn spawn_reprocess_scheduler( loop { let msg = queue.next().await; queue.handle_message(msg); + // pre-emptively wake the thread to check for new events + if let Some(ref waker) = queue.waker { + waker.wake_by_ref(); + } } }, TASK_NAME, From 67bc7913ae2cb25590fe5ba3d67436e1f30aca56 Mon Sep 17 00:00:00 2001 From: Diva M Date: Tue, 27 Apr 2021 22:01:47 -0500 Subject: [PATCH 27/43] update tests --- beacon_node/beacon_chain/tests/store_tests.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/beacon_node/beacon_chain/tests/store_tests.rs b/beacon_node/beacon_chain/tests/store_tests.rs index a25663178cd..5831d29f976 100644 --- a/beacon_node/beacon_chain/tests/store_tests.rs +++ b/beacon_node/beacon_chain/tests/store_tests.rs @@ -327,7 +327,7 @@ fn epoch_boundary_state_attestation_processing() { { checked_pre_fin = true; assert!(matches!( - res.err().unwrap(), + res.err().unwrap().0, AttnError::PastSlot { attestation_slot, earliest_permissible_slot, From a18a04bdb27c1b3f08a307d0c314bcb0ff3d2f70 Mon Sep 17 00:00:00 2001 From: Diva M Date: Tue, 27 Apr 2021 22:29:50 -0500 Subject: [PATCH 28/43] update tests --- beacon_node/beacon_chain/tests/attestation_verification.rs | 4 ++-- beacon_node/beacon_chain/tests/tests.rs | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/beacon_node/beacon_chain/tests/attestation_verification.rs b/beacon_node/beacon_chain/tests/attestation_verification.rs index 0b7e7ef8d1f..061795cd19e 100644 --- a/beacon_node/beacon_chain/tests/attestation_verification.rs +++ b/beacon_node/beacon_chain/tests/attestation_verification.rs @@ -222,7 +222,7 @@ fn aggregated_gossip_verification() { .expect(&format!( "{} should error during verify_aggregated_attestation_for_gossip", $desc - )), + )).0, $( $error ) |+ $( if $guard )? ), "case: {}", @@ -605,7 +605,7 @@ fn unaggregated_gossip_verification() { .expect(&format!( "{} should error during verify_unaggregated_attestation_for_gossip", $desc - )), + )).0, $( $error ) |+ $( if $guard )? ), "case: {}", diff --git a/beacon_node/beacon_chain/tests/tests.rs b/beacon_node/beacon_chain/tests/tests.rs index 616f2b544ff..9879bdbbb24 100644 --- a/beacon_node/beacon_chain/tests/tests.rs +++ b/beacon_node/beacon_chain/tests/tests.rs @@ -462,7 +462,7 @@ fn attestations_with_increasing_slots() { if expected_attestation_slot < expected_earliest_permissible_slot { assert!(matches!( - res.err().unwrap(), + res.err().unwrap().0, AttnError::PastSlot { attestation_slot, earliest_permissible_slot, From 82408fdc9e24ec022d6a052fdbd9a008518d6c52 Mon Sep 17 00:00:00 2001 From: Diva M Date: Wed, 28 Apr 2021 09:28:07 -0500 Subject: [PATCH 29/43] clippy lints --- .../network/src/beacon_processor/tests.rs | 15 +++++---------- .../beacon_processor/work_reprocessing_queue.rs | 17 ----------------- .../beacon_processor/worker/gossip_methods.rs | 5 +++-- 3 files changed, 8 insertions(+), 29 deletions(-) diff --git a/beacon_node/network/src/beacon_processor/tests.rs b/beacon_node/network/src/beacon_processor/tests.rs index 333b3f40c8d..7da5a78a283 100644 --- a/beacon_node/network/src/beacon_processor/tests.rs +++ b/beacon_node/network/src/beacon_processor/tests.rs @@ -372,17 +372,12 @@ impl TestRig { let mut events = Vec::with_capacity(expected.len()); let drain_future = async { - loop { - match self.work_journal_rx.recv().await { - Some(event) => { - events.push(event); + while let Some(event) = self.work_journal_rx.recv().await { + events.push(event); - // Break as soon as we collect the desired number of events. - if events.len() >= expected.len() { - break; - } - } - None => break, + // Break as soon as we collect the desired number of events. + if events.len() >= expected.len() { + break; } } }; diff --git a/beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs b/beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs index 4548fd78ade..5e8aece5fcb 100644 --- a/beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs +++ b/beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs @@ -133,9 +133,6 @@ struct ReprocessQueue { slot_clock: T::SlotClock, - /// The waker for the current thread. - waker: Option, - log: Logger, } @@ -164,15 +161,6 @@ impl Stream for ReprocessQueue { // NOTE: implementing `Stream` is not necessary but allows to maintain the future selection // order fine-grained and separate from the logic of handling each message, which is nice. - // update the waker if needed - if let Some(waker) = &self.waker { - if waker.will_wake(cx.waker()) { - self.waker = Some(cx.waker().clone()); - } - } else { - self.waker = Some(cx.waker().clone()); - } - // Poll for expired blocks *before* we try to process new blocks. // // The sequential nature of blockchains means it is generally better to try and import all @@ -237,7 +225,6 @@ pub fn spawn_reprocess_scheduler( awaiting_attestations_per_root: HashMap::new(), next_attestation: 0, slot_clock, - waker: None, log, }; @@ -246,10 +233,6 @@ pub fn spawn_reprocess_scheduler( loop { let msg = queue.next().await; queue.handle_message(msg); - // pre-emptively wake the thread to check for new events - if let Some(ref waker) = queue.waker { - waker.wake_by_ref(); - } } }, TASK_NAME, diff --git a/beacon_node/network/src/beacon_processor/worker/gossip_methods.rs b/beacon_node/network/src/beacon_processor/worker/gossip_methods.rs index 955c4d1da6d..e75f03978c5 100644 --- a/beacon_node/network/src/beacon_processor/worker/gossip_methods.rs +++ b/beacon_node/network/src/beacon_processor/worker/gossip_methods.rs @@ -96,6 +96,7 @@ impl Worker { /// - Attempt to add it to the naive aggregation pool. /// /// Raises a log if there are errors. + #[allow(clippy::too_many_arguments)] pub fn process_gossip_attestation( self, message_id: MessageId, @@ -882,7 +883,7 @@ impl Worker { ReprocessQueueMessage::UnknownBlockAggregate(QueuedAggregate { peer_id, attestation, - message_id: message_id.clone(), + message_id, seen_timestamp, }) } @@ -898,7 +899,7 @@ impl Worker { ReprocessQueueMessage::UnknownBlockUnaggregate(QueuedUnaggregate { peer_id, should_import, - message_id: message_id.clone(), + message_id, attestation, subnet_id, seen_timestamp, From 1861f0abd2a30a238760c3eaf10bc86320a81095 Mon Sep 17 00:00:00 2001 From: divma <26765164+divagant-martian@users.noreply.github.com> Date: Thu, 29 Apr 2021 09:03:24 -0500 Subject: [PATCH 30/43] spelling Co-authored-by: Michael Sproul --- beacon_node/network/src/metrics.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/beacon_node/network/src/metrics.rs b/beacon_node/network/src/metrics.rs index f8387c37086..a4be93bbbd1 100644 --- a/beacon_node/network/src/metrics.rs +++ b/beacon_node/network/src/metrics.rs @@ -303,7 +303,7 @@ lazy_static! { ); pub static ref BEACON_PROCESSOR_UNAGGREGATED_ATTESTATION_REQUEUED_TOTAL: Result = try_create_int_counter( "beacon_processor_unaggregated_attestation_requeued_total", - "Total number of unaggregated attestations that referenced an unknwon block and were re-queued." + "Total number of unaggregated attestations that referenced an unknown block and were re-queued." ); // Aggregated attestations. pub static ref BEACON_PROCESSOR_AGGREGATED_ATTESTATION_QUEUE_TOTAL: Result = try_create_int_gauge( From 49a0e8ec374942d14085489104997439ad777364 Mon Sep 17 00:00:00 2001 From: divma <26765164+divagant-martian@users.noreply.github.com> Date: Thu, 29 Apr 2021 09:04:00 -0500 Subject: [PATCH 31/43] spelling Co-authored-by: Michael Sproul --- beacon_node/network/src/metrics.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/beacon_node/network/src/metrics.rs b/beacon_node/network/src/metrics.rs index a4be93bbbd1..7b325f8a83c 100644 --- a/beacon_node/network/src/metrics.rs +++ b/beacon_node/network/src/metrics.rs @@ -320,7 +320,7 @@ lazy_static! { ); pub static ref BEACON_PROCESSOR_AGGREGATED_ATTESTATION_REQUEUED_TOTAL: Result = try_create_int_counter( "beacon_processor_aggregated_attestation_requeued_total", - "Total number of aggregated attestations that referenced an unknwon block and were re-queued." + "Total number of aggregated attestations that referenced an unknown block and were re-queued." ); } From 681d1e7f6c77d0f16101a13907115d97344e949d Mon Sep 17 00:00:00 2001 From: divma <26765164+divagant-martian@users.noreply.github.com> Date: Thu, 29 Apr 2021 09:12:13 -0500 Subject: [PATCH 32/43] docs update Co-authored-by: Michael Sproul --- beacon_node/network/src/beacon_processor/mod.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/beacon_node/network/src/beacon_processor/mod.rs b/beacon_node/network/src/beacon_processor/mod.rs index 0ba1e50959b..6231193a2da 100644 --- a/beacon_node/network/src/beacon_processor/mod.rs +++ b/beacon_node/network/src/beacon_processor/mod.rs @@ -624,7 +624,7 @@ enum InboundEvent { WorkerIdle, /// There is new work to be done. WorkEvent(WorkEvent), - /// A work that was queued for re-processing has become ready. + /// A work event that was queued for re-processing has become ready. ReprocessingWork(WorkEvent), } From c09c364e0a39e3a8304134fc51ac4929891fcfbf Mon Sep 17 00:00:00 2001 From: Diva M Date: Thu, 29 Apr 2021 09:17:09 -0500 Subject: [PATCH 33/43] review suggestions --- beacon_node/network/src/beacon_processor/mod.rs | 10 +++++----- beacon_node/network/src/beacon_processor/tests.rs | 2 +- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/beacon_node/network/src/beacon_processor/mod.rs b/beacon_node/network/src/beacon_processor/mod.rs index 6231193a2da..cd05663836b 100644 --- a/beacon_node/network/src/beacon_processor/mod.rs +++ b/beacon_node/network/src/beacon_processor/mod.rs @@ -485,7 +485,7 @@ impl std::convert::From> for WorkEvent { seen_timestamp, }) => Self { drop_during_sync: true, - work: Work::UnkonwnBlockAggregate { + work: Work::UnknownBlockAggregate { message_id, peer_id, aggregate: attestation, @@ -521,7 +521,7 @@ pub enum Work { aggregate: Box>, seen_timestamp: Duration, }, - UnkonwnBlockAggregate { + UnknownBlockAggregate { message_id: MessageId, peer_id: PeerId, aggregate: Box>, @@ -594,7 +594,7 @@ impl Work { Work::BlocksByRangeRequest { .. } => BLOCKS_BY_RANGE_REQUEST, Work::BlocksByRootsRequest { .. } => BLOCKS_BY_ROOTS_REQUEST, Work::UnknownBlockAttestation { .. } => UNKNOWN_BLOCK_ATTESTATION, - Work::UnkonwnBlockAggregate { .. } => UNKNOWN_BLOCK_AGGREGATE, + Work::UnknownBlockAggregate { .. } => UNKNOWN_BLOCK_AGGREGATE, } } } @@ -966,7 +966,7 @@ impl BeaconProcessor { Work::UnknownBlockAttestation { .. } => { unknown_block_attestation_queue.push(work) } - Work::UnkonwnBlockAggregate { .. } => { + Work::UnknownBlockAggregate { .. } => { unknown_block_aggregate_queue.push(work) } } @@ -1236,7 +1236,7 @@ impl BeaconProcessor { None, // Do not allow this attestation to be re-processed beyond this point. seen_timestamp, ), - Work::UnkonwnBlockAggregate { + Work::UnknownBlockAggregate { message_id, peer_id, aggregate, diff --git a/beacon_node/network/src/beacon_processor/tests.rs b/beacon_node/network/src/beacon_processor/tests.rs index 7da5a78a283..25f008de79a 100644 --- a/beacon_node/network/src/beacon_processor/tests.rs +++ b/beacon_node/network/src/beacon_processor/tests.rs @@ -1,4 +1,4 @@ -//#![cfg(not(debug_assertions))] // Tests are too slow in debug. +#![cfg(not(debug_assertions))] // Tests are too slow in debug. #![cfg(test)] use crate::beacon_processor::work_reprocessing_queue::QUEUED_ATTESTATION_DELAY; From 7823704b9c59610403691f4fc246369c0974fd35 Mon Sep 17 00:00:00 2001 From: Diva M Date: Thu, 29 Apr 2021 18:17:39 -0500 Subject: [PATCH 34/43] remove unpin contrains --- .../work_reprocessing_queue.rs | 66 ++++++++----------- common/slot_clock/src/lib.rs | 4 +- consensus/types/src/eth_spec.rs | 44 +++++-------- 3 files changed, 49 insertions(+), 65 deletions(-) diff --git a/beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs b/beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs index 5e8aece5fcb..9f288c1027d 100644 --- a/beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs +++ b/beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs @@ -130,10 +130,6 @@ struct ReprocessQueue { /* Aux */ /// Next attestation id, used for both aggregated and unaggregated attestations next_attestation: usize, - - slot_clock: T::SlotClock, - - log: Logger, } #[derive(Debug, Clone, Copy, PartialEq, Eq)] @@ -211,7 +207,7 @@ pub fn spawn_reprocess_scheduler( log: Logger, ) -> Sender> { let (work_reprocessing_tx, work_reprocessing_rx) = mpsc::channel(MAX_SCHEDULED_WORK_QUEUE_LEN); - // basic sanity check + // Basic sanity check. assert!(ADDITIONAL_QUEUED_BLOCK_DELAY < MAXIMUM_GOSSIP_CLOCK_DISPARITY); let mut queue = ReprocessQueue { @@ -224,16 +220,19 @@ pub fn spawn_reprocess_scheduler( queued_unaggregates: FnvHashMap::default(), awaiting_attestations_per_root: HashMap::new(), next_attestation: 0, - slot_clock, - log, }; executor.spawn( async move { - loop { - let msg = queue.next().await; - queue.handle_message(msg); + while let Some(msg) = queue.next().await { + queue.handle_message(msg, &slot_clock, &log); } + + debug!( + log, + "Re-process queue stopped"; + "msg" => "shutting down" + ); }, TASK_NAME, ); @@ -242,12 +241,12 @@ pub fn spawn_reprocess_scheduler( } impl ReprocessQueue { - fn handle_message(&mut self, msg: Option>) { + fn handle_message(&mut self, msg: InboundEvent, slot_clock: &T::SlotClock, log: &Logger) { use ReprocessQueueMessage::*; match msg { // Some block has been indicated as "early" and should be processed when the // appropriate slot arrives. - Some(InboundEvent::Msg(EarlyBlock(early_block))) => { + InboundEvent::Msg(EarlyBlock(early_block)) => { let block_slot = early_block.block.block.slot(); let block_root = early_block.block.block_root; @@ -256,11 +255,11 @@ impl ReprocessQueue { return; } - if let Some(duration_till_slot) = self.slot_clock.duration_to_slot(block_slot) { + if let Some(duration_till_slot) = slot_clock.duration_to_slot(block_slot) { // Check to ensure this won't over-fill the queue. if self.queued_block_roots.len() >= MAXIMUM_QUEUED_BLOCKS { error!( - self.log, + log, "Early blocks queue is full"; "queue_size" => MAXIMUM_QUEUED_BLOCKS, "msg" => "check system clock" @@ -287,7 +286,7 @@ impl ReprocessQueue { // This logic is slightly awkward since `SlotClock::duration_to_slot` // doesn't distinguish between a slot that has already arrived and an // error reading the slot clock. - if let Some(now) = self.slot_clock.now() { + if let Some(now) = slot_clock.now() { if block_slot <= now && self .ready_work_tx @@ -295,17 +294,17 @@ impl ReprocessQueue { .is_err() { error!( - self.log, + log, "Failed to send block"; ); } } } } - Some(InboundEvent::Msg(UnknownBlockAggregate(queued_aggregate))) => { + InboundEvent::Msg(UnknownBlockAggregate(queued_aggregate)) => { if self.attestations_delay_queue.len() >= MAXIMUM_QUEUED_ATTESTATIONS { error!( - self.log, + log, "Attestation delay queue is full"; "queue_size" => MAXIMUM_QUEUED_ATTESTATIONS, "msg" => "check system clock" @@ -333,10 +332,10 @@ impl ReprocessQueue { self.next_attestation += 1; } - Some(InboundEvent::Msg(UnknownBlockUnaggregate(queued_unaggregate))) => { + InboundEvent::Msg(UnknownBlockUnaggregate(queued_unaggregate)) => { if self.attestations_delay_queue.len() >= MAXIMUM_QUEUED_ATTESTATIONS { error!( - self.log, + log, "Attestation delay queue is full"; "queue_size" => MAXIMUM_QUEUED_ATTESTATIONS, "msg" => "check system clock" @@ -364,7 +363,7 @@ impl ReprocessQueue { self.next_attestation += 1; } - Some(InboundEvent::Msg(BlockImported(root))) => { + InboundEvent::Msg(BlockImported(root)) => { // Unqueue the attestations we have for this root, if any. if let Some(queued_ids) = self.awaiting_attestations_per_root.remove(&root) { for id in queued_ids { @@ -388,7 +387,7 @@ impl ReprocessQueue { // send the work if self.ready_work_tx.try_send(work).is_err() { error!( - self.log, + log, "Failed to send scheduled attestation"; ); } @@ -396,7 +395,7 @@ impl ReprocessQueue { // there is a mismatch between the attestation ids registered for this // root and the queued attestations. This should never happen error!( - self.log, + log, "Unknown queued attestation for block root"; "block_root" => ?root, "att_id" => ?id, @@ -406,14 +405,14 @@ impl ReprocessQueue { } } // A block that was queued for later processing is now ready to be processed. - Some(InboundEvent::ReadyBlock(ready_block)) => { + InboundEvent::ReadyBlock(ready_block) => { let block_root = ready_block.block.block_root; if !self.queued_block_roots.remove(&block_root) { // Log an error to alert that we've made a bad assumption about how this // program works, but still process the block anyway. error!( - self.log, + log, "Unknown block in delay queue"; "block_root" => ?block_root ); @@ -425,20 +424,20 @@ impl ReprocessQueue { .is_err() { error!( - self.log, + log, "Failed to pop queued block"; ); } } - Some(InboundEvent::DelayQueueError(e, queue_name)) => { + InboundEvent::DelayQueueError(e, queue_name) => { crit!( - self.log, + log, "Failed to poll queue"; "queue" => queue_name, "e" => ?e ) } - Some(InboundEvent::ReadyAttestation(queued_id)) => { + InboundEvent::ReadyAttestation(queued_id) => { if let Some((root, work)) = match queued_id { QueuedAttestationId::Aggregate(id) => { self.queued_aggregates @@ -456,7 +455,7 @@ impl ReprocessQueue { } { if self.ready_work_tx.try_send(work).is_err() { error!( - self.log, + log, "Failed to send scheduled attestation"; ); } @@ -468,13 +467,6 @@ impl ReprocessQueue { } } } - None => { - debug!( - self.log, - "Block delay queue stopped"; - "msg" => "shutting down" - ); - } } } } diff --git a/common/slot_clock/src/lib.rs b/common/slot_clock/src/lib.rs index 04a86ebd113..94538141c5f 100644 --- a/common/slot_clock/src/lib.rs +++ b/common/slot_clock/src/lib.rs @@ -5,7 +5,7 @@ mod manual_slot_clock; mod metrics; mod system_time_slot_clock; -use std::{marker::Unpin, time::Duration}; +use std::time::Duration; pub use crate::manual_slot_clock::ManualSlotClock; pub use crate::manual_slot_clock::ManualSlotClock as TestingSlotClock; @@ -16,7 +16,7 @@ pub use types::Slot; /// A clock that reports the current slot. /// /// The clock is not required to be monotonically increasing and may go backwards. -pub trait SlotClock: Send + Sync + Sized + Clone + Unpin { +pub trait SlotClock: Send + Sync + Sized + Clone { /// Creates a new slot clock where the first slot is `genesis_slot`, genesis occurred /// `genesis_duration` after the `UNIX_EPOCH` and each slot is `slot_duration` apart. fn new(genesis_slot: Slot, genesis_duration: Duration, slot_duration: Duration) -> Self; diff --git a/consensus/types/src/eth_spec.rs b/consensus/types/src/eth_spec.rs index 9083b55db22..9eeee557f89 100644 --- a/consensus/types/src/eth_spec.rs +++ b/consensus/types/src/eth_spec.rs @@ -7,7 +7,6 @@ use ssz_types::typenum::{ U65536, U8, U8192, }; use std::fmt::{self, Debug}; -use std::marker::Unpin; use std::str::FromStr; const MAINNET: &str = "mainnet"; @@ -51,41 +50,34 @@ pub trait EthSpec: 'static + Default + Sync + Send + Clone + Debug + PartialEq + /* * Constants */ - type GenesisEpoch: Unsigned + Clone + Sync + Send + Debug + PartialEq + Unpin; - type JustificationBitsLength: Unsigned - + Clone - + Sync - + Send - + Debug - + PartialEq - + Default - + Unpin; - type SubnetBitfieldLength: Unsigned + Clone + Sync + Send + Debug + PartialEq + Default + Unpin; + type GenesisEpoch: Unsigned + Clone + Sync + Send + Debug + PartialEq; + type JustificationBitsLength: Unsigned + Clone + Sync + Send + Debug + PartialEq + Default; + type SubnetBitfieldLength: Unsigned + Clone + Sync + Send + Debug + PartialEq + Default; /* * Misc */ - type MaxValidatorsPerCommittee: Unsigned + Clone + Sync + Send + Debug + PartialEq + Eq + Unpin; + type MaxValidatorsPerCommittee: Unsigned + Clone + Sync + Send + Debug + PartialEq + Eq; /* * Time parameters */ - type SlotsPerEpoch: Unsigned + Clone + Sync + Send + Debug + PartialEq + Unpin; - type EpochsPerEth1VotingPeriod: Unsigned + Clone + Sync + Send + Debug + PartialEq + Unpin; - type SlotsPerHistoricalRoot: Unsigned + Clone + Sync + Send + Debug + PartialEq + Unpin; + type SlotsPerEpoch: Unsigned + Clone + Sync + Send + Debug + PartialEq; + type EpochsPerEth1VotingPeriod: Unsigned + Clone + Sync + Send + Debug + PartialEq; + type SlotsPerHistoricalRoot: Unsigned + Clone + Sync + Send + Debug + PartialEq; /* * State list lengths */ - type EpochsPerHistoricalVector: Unsigned + Clone + Sync + Send + Debug + PartialEq + Unpin; - type EpochsPerSlashingsVector: Unsigned + Clone + Sync + Send + Debug + PartialEq + Unpin; - type HistoricalRootsLimit: Unsigned + Clone + Sync + Send + Debug + PartialEq + Unpin; - type ValidatorRegistryLimit: Unsigned + Clone + Sync + Send + Debug + PartialEq + Unpin; + type EpochsPerHistoricalVector: Unsigned + Clone + Sync + Send + Debug + PartialEq; + type EpochsPerSlashingsVector: Unsigned + Clone + Sync + Send + Debug + PartialEq; + type HistoricalRootsLimit: Unsigned + Clone + Sync + Send + Debug + PartialEq; + type ValidatorRegistryLimit: Unsigned + Clone + Sync + Send + Debug + PartialEq; /* * Max operations per block */ - type MaxProposerSlashings: Unsigned + Clone + Sync + Send + Debug + PartialEq + Unpin; - type MaxAttesterSlashings: Unsigned + Clone + Sync + Send + Debug + PartialEq + Unpin; - type MaxAttestations: Unsigned + Clone + Sync + Send + Debug + PartialEq + Unpin; - type MaxDeposits: Unsigned + Clone + Sync + Send + Debug + PartialEq + Unpin; - type MaxVoluntaryExits: Unsigned + Clone + Sync + Send + Debug + PartialEq + Unpin; + type MaxProposerSlashings: Unsigned + Clone + Sync + Send + Debug + PartialEq; + type MaxAttesterSlashings: Unsigned + Clone + Sync + Send + Debug + PartialEq; + type MaxAttestations: Unsigned + Clone + Sync + Send + Debug + PartialEq; + type MaxDeposits: Unsigned + Clone + Sync + Send + Debug + PartialEq; + type MaxVoluntaryExits: Unsigned + Clone + Sync + Send + Debug + PartialEq; /* * Derived values (set these CAREFULLY) */ @@ -94,11 +86,11 @@ pub trait EthSpec: 'static + Default + Sync + Send + Clone + Debug + PartialEq + /// Must be set to `MaxAttestations * SlotsPerEpoch` // NOTE: we could safely instantiate these by using type-level arithmetic, but doing // so adds ~25s to the time required to type-check this crate - type MaxPendingAttestations: Unsigned + Clone + Sync + Send + Debug + PartialEq + Unpin; + type MaxPendingAttestations: Unsigned + Clone + Sync + Send + Debug + PartialEq; /// The length of `eth1_data_votes`. /// /// Must be set to `EpochsPerEth1VotingPeriod * SlotsPerEpoch` - type SlotsPerEth1VotingPeriod: Unsigned + Clone + Sync + Send + Debug + PartialEq + Unpin; + type SlotsPerEth1VotingPeriod: Unsigned + Clone + Sync + Send + Debug + PartialEq; fn default_spec() -> ChainSpec; From ca66900555538c86723ebc5f12f2552bfdb2bb59 Mon Sep 17 00:00:00 2001 From: Diva M Date: Thu, 29 Apr 2021 18:47:31 -0500 Subject: [PATCH 35/43] comment style --- .../src/beacon_processor/work_reprocessing_queue.rs | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs b/beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs index 9f288c1027d..c62d98ae45e 100644 --- a/beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs +++ b/beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs @@ -381,10 +381,10 @@ impl ReprocessQueue { (ReadyWork::Unaggregate(unaggregate), delay_key) }), } { - // remove the delay + // Remove the delay. self.attestations_delay_queue.remove(&delay_key); - // send the work + // Send the work. if self.ready_work_tx.try_send(work).is_err() { error!( log, @@ -392,8 +392,8 @@ impl ReprocessQueue { ); } } else { - // there is a mismatch between the attestation ids registered for this - // root and the queued attestations. This should never happen + // There is a mismatch between the attestation ids registered for this + // root and the queued attestations. This should never happen. error!( log, "Unknown queued attestation for block root"; From e762f175b753fd61469a532ff139844265a76998 Mon Sep 17 00:00:00 2001 From: Diva M Date: Fri, 18 Jun 2021 10:28:22 -0500 Subject: [PATCH 36/43] our savior clippy --- .../network/src/beacon_processor/worker/gossip_methods.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/beacon_node/network/src/beacon_processor/worker/gossip_methods.rs b/beacon_node/network/src/beacon_processor/worker/gossip_methods.rs index 6daf63747af..5b81384fd64 100644 --- a/beacon_node/network/src/beacon_processor/worker/gossip_methods.rs +++ b/beacon_node/network/src/beacon_processor/worker/gossip_methods.rs @@ -882,8 +882,8 @@ impl Worker { ); ReprocessQueueMessage::UnknownBlockAggregate(QueuedAggregate { peer_id, - attestation, message_id, + attestation, seen_timestamp, }) } @@ -898,10 +898,10 @@ impl Worker { ); ReprocessQueueMessage::UnknownBlockUnaggregate(QueuedUnaggregate { peer_id, - should_import, message_id, attestation, subnet_id, + should_import, seen_timestamp, }) } From d9b69be17f88fc7a7b7636298d1be3cdd527ff8e Mon Sep 17 00:00:00 2001 From: Paul Hauner Date: Mon, 21 Jun 2021 10:58:50 +1000 Subject: [PATCH 37/43] Add some requeue tests for aggregates --- .../network/src/beacon_processor/tests.rs | 109 ++++++++++++++++++ 1 file changed, 109 insertions(+) diff --git a/beacon_node/network/src/beacon_processor/tests.rs b/beacon_node/network/src/beacon_processor/tests.rs index 25f008de79a..918c22a6ab9 100644 --- a/beacon_node/network/src/beacon_processor/tests.rs +++ b/beacon_node/network/src/beacon_processor/tests.rs @@ -44,6 +44,7 @@ struct TestRig { next_block: SignedBeaconBlock, attestations: Vec<(Attestation, SubnetId)>, next_block_attestations: Vec<(Attestation, SubnetId)>, + next_block_aggregate_attestations: Vec>, attester_slashing: AttesterSlashing, proposer_slashing: ProposerSlashing, voluntary_exit: SignedVoluntaryExit, @@ -124,6 +125,18 @@ impl TestRig { .flatten() .collect::>(); + let next_block_aggregate_attestations = harness + .make_attestations( + &harness.get_all_validators(), + &next_state, + next_block.state_root(), + next_block.canonical_root().into(), + next_block.slot(), + ) + .into_iter() + .filter_map(|(_, aggregate_opt)| aggregate_opt) + .collect::>(); + assert!( !next_block_attestations.is_empty(), "precondition: attestation for next block are not empty" @@ -193,6 +206,7 @@ impl TestRig { next_block, attestations, next_block_attestations, + next_block_aggregate_attestations, attester_slashing, proposer_slashing, voluntary_exit, @@ -277,6 +291,22 @@ impl TestRig { .unwrap(); } + pub fn enqueue_next_block_aggregated_attestation(&self) { + let aggregate = self + .next_block_aggregate_attestations + .first() + .unwrap() + .clone(); + self.beacon_processor_tx + .try_send(WorkEvent::aggregated_attestation( + junk_message_id(), + junk_peer_id(), + aggregate, + Duration::from_secs(0), + )) + .unwrap(); + } + fn runtime(&mut self) -> Arc { self.environment .as_mut() @@ -569,6 +599,51 @@ fn import_unknown_block_gossip_attestation() { ); } +/// Ensure that attestations that reference an unknown block get properly re-queued and +/// re-processed upon importing the block. +#[test] +fn import_unknown_block_gossip_aggregated_attestation() { + let mut rig = TestRig::new(SMALL_CHAIN); + + // Empty the op pool. + rig.chain + .op_pool + .prune_attestations(u64::max_value().into()); + assert_eq!(rig.chain.op_pool.num_attestations(), 0); + + // Send the attestation but not the block, and check that it was not imported. + + let initial_attns = rig.chain.op_pool.num_attestations(); + + rig.enqueue_next_block_aggregated_attestation(); + + rig.assert_event_journal(&[GOSSIP_AGGREGATE, WORKER_FREED, NOTHING_TO_DO]); + + assert_eq!( + rig.chain.op_pool.num_attestations(), + initial_attns, + "Attestation should not have been included." + ); + + // Send the block and ensure that the attestation is received back and imported. + + rig.enqueue_gossip_block(); + + rig.assert_event_journal_contains_ordered(&[GOSSIP_BLOCK, UNKNOWN_BLOCK_AGGREGATE]); + + assert_eq!( + rig.head_root(), + rig.next_block.canonical_root(), + "Block should be imported and become head." + ); + + assert_eq!( + rig.chain.op_pool.num_attestations(), + initial_attns + 1, + "Attestation should have been included." + ); +} + /// Ensure that attestations that reference an unknown block get properly re-queued and re-processed /// when the block is not seen. #[test] @@ -603,6 +678,40 @@ fn requeue_unknown_block_gossip_attestation_without_import() { ); } +/// Ensure that aggregate that reference an unknown block get properly re-queued and re-processed +/// when the block is not seen. +#[test] +fn requeue_unknown_block_gossip_aggregated_attestation_without_import() { + let mut rig = TestRig::new(SMALL_CHAIN); + + // Send the attestation but not the block, and check that it was not imported. + + let initial_attns = rig.chain.op_pool.num_attestations(); + + rig.enqueue_next_block_aggregated_attestation(); + + rig.assert_event_journal(&[GOSSIP_AGGREGATE, WORKER_FREED, NOTHING_TO_DO]); + + assert_eq!( + rig.chain.naive_aggregation_pool.read().num_attestations(), + initial_attns, + "Attestation should not have been included." + ); + + // Ensure that the attestation is received back but not imported. + + rig.assert_event_journal_with_timeout( + &[UNKNOWN_BLOCK_AGGREGATE, WORKER_FREED, NOTHING_TO_DO], + Duration::from_secs(1) + QUEUED_ATTESTATION_DELAY, + ); + + assert_eq!( + rig.chain.op_pool.num_attestations(), + initial_attns, + "Attestation should not have been included." + ); +} + /// Ensure a bunch of valid operations can be imported. #[test] fn import_misc_gossip_ops() { From 6744e2a7df3e0d6227cf4cb77478719f7616b8cb Mon Sep 17 00:00:00 2001 From: Paul Hauner Date: Mon, 21 Jun 2021 11:19:47 +1000 Subject: [PATCH 38/43] Try reprocess attns after RPC block import --- .../network/src/beacon_processor/mod.rs | 2 +- .../network/src/beacon_processor/tests.rs | 70 ++++++++++++++++--- .../beacon_processor/worker/gossip_methods.rs | 1 + .../beacon_processor/worker/sync_methods.rs | 16 ++++- 4 files changed, 79 insertions(+), 10 deletions(-) diff --git a/beacon_node/network/src/beacon_processor/mod.rs b/beacon_node/network/src/beacon_processor/mod.rs index cd05663836b..fb8fbc9dcd0 100644 --- a/beacon_node/network/src/beacon_processor/mod.rs +++ b/beacon_node/network/src/beacon_processor/mod.rs @@ -1192,7 +1192,7 @@ impl BeaconProcessor { * Verification for beacon blocks received during syncing via RPC. */ Work::RpcBlock { block, result_tx } => { - worker.process_rpc_block(*block, result_tx) + worker.process_rpc_block(*block, result_tx, work_reprocessing_tx) } /* * Verification for a chain segment (multiple blocks). diff --git a/beacon_node/network/src/beacon_processor/tests.rs b/beacon_node/network/src/beacon_processor/tests.rs index 918c22a6ab9..4d49385fa51 100644 --- a/beacon_node/network/src/beacon_processor/tests.rs +++ b/beacon_node/network/src/beacon_processor/tests.rs @@ -233,6 +233,11 @@ impl TestRig { .unwrap(); } + pub fn enqueue_rpc_block(&self) { + let (event, _rx) = WorkEvent::rpc_beacon_block(Box::new(self.next_block.clone())); + self.beacon_processor_tx.try_send(event).unwrap(); + } + pub fn enqueue_unaggregated_attestation(&self) { let (attestation, subnet_id) = self.attestations.first().unwrap().clone(); self.beacon_processor_tx @@ -560,10 +565,14 @@ fn import_gossip_attestation() { ); } +enum BlockImportMethod { + Gossip, + Rpc, +} + /// Ensure that attestations that reference an unknown block get properly re-queued and /// re-processed upon importing the block. -#[test] -fn import_unknown_block_gossip_attestation() { +fn attestation_to_unknown_block_processed(import_method: BlockImportMethod) { let mut rig = TestRig::new(SMALL_CHAIN); // Send the attestation but not the block, and check that it was not imported. @@ -582,9 +591,22 @@ fn import_unknown_block_gossip_attestation() { // Send the block and ensure that the attestation is received back and imported. - rig.enqueue_gossip_block(); + let block_event = match import_method { + BlockImportMethod::Gossip => { + rig.enqueue_gossip_block(); + GOSSIP_BLOCK + } + BlockImportMethod::Rpc => { + rig.enqueue_rpc_block(); + RPC_BLOCK + } + }; + + rig.assert_event_journal_contains_ordered(&[block_event, UNKNOWN_BLOCK_ATTESTATION]); - rig.assert_event_journal_contains_ordered(&[GOSSIP_BLOCK, UNKNOWN_BLOCK_ATTESTATION]); + // Run fork choice, since it isn't run when processing an RPC block. At runtime it is the + // responsibility of the sync manager to do this. + rig.chain.fork_choice().unwrap(); assert_eq!( rig.head_root(), @@ -599,10 +621,19 @@ fn import_unknown_block_gossip_attestation() { ); } +#[test] +fn attestation_to_unknown_block_processed_after_gossip_block() { + attestation_to_unknown_block_processed(BlockImportMethod::Gossip) +} + +#[test] +fn attestation_to_unknown_block_processed_after_rpc_block() { + attestation_to_unknown_block_processed(BlockImportMethod::Rpc) +} + /// Ensure that attestations that reference an unknown block get properly re-queued and /// re-processed upon importing the block. -#[test] -fn import_unknown_block_gossip_aggregated_attestation() { +fn aggregate_attestation_to_unknown_block(import_method: BlockImportMethod) { let mut rig = TestRig::new(SMALL_CHAIN); // Empty the op pool. @@ -627,9 +658,22 @@ fn import_unknown_block_gossip_aggregated_attestation() { // Send the block and ensure that the attestation is received back and imported. - rig.enqueue_gossip_block(); + let block_event = match import_method { + BlockImportMethod::Gossip => { + rig.enqueue_gossip_block(); + GOSSIP_BLOCK + } + BlockImportMethod::Rpc => { + rig.enqueue_rpc_block(); + RPC_BLOCK + } + }; + + rig.assert_event_journal_contains_ordered(&[block_event, UNKNOWN_BLOCK_AGGREGATE]); - rig.assert_event_journal_contains_ordered(&[GOSSIP_BLOCK, UNKNOWN_BLOCK_AGGREGATE]); + // Run fork choice, since it isn't run when processing an RPC block. At runtime it is the + // responsibility of the sync manager to do this. + rig.chain.fork_choice().unwrap(); assert_eq!( rig.head_root(), @@ -644,6 +688,16 @@ fn import_unknown_block_gossip_aggregated_attestation() { ); } +#[test] +fn aggregate_attestation_to_unknown_block_processed_after_gossip_block() { + aggregate_attestation_to_unknown_block(BlockImportMethod::Gossip) +} + +#[test] +fn aggregate_attestation_to_unknown_block_processed_after_rpc_block() { + aggregate_attestation_to_unknown_block(BlockImportMethod::Rpc) +} + /// Ensure that attestations that reference an unknown block get properly re-queued and re-processed /// when the block is not seen. #[test] diff --git a/beacon_node/network/src/beacon_processor/worker/gossip_methods.rs b/beacon_node/network/src/beacon_processor/worker/gossip_methods.rs index 5b81384fd64..1c2956dfb49 100644 --- a/beacon_node/network/src/beacon_processor/worker/gossip_methods.rs +++ b/beacon_node/network/src/beacon_processor/worker/gossip_methods.rs @@ -468,6 +468,7 @@ impl Worker { error!( self.log, "Failed to inform block import"; + "source" => "gossip", "block_root" => %block_root, ) }; diff --git a/beacon_node/network/src/beacon_processor/worker/sync_methods.rs b/beacon_node/network/src/beacon_processor/worker/sync_methods.rs index 65118e3de5f..10ac409ff43 100644 --- a/beacon_node/network/src/beacon_processor/worker/sync_methods.rs +++ b/beacon_node/network/src/beacon_processor/worker/sync_methods.rs @@ -1,4 +1,4 @@ -use super::Worker; +use super::{super::work_reprocessing_queue::ReprocessQueueMessage, Worker}; use crate::beacon_processor::worker::FUTURE_SLOT_TOLERANCE; use crate::beacon_processor::BlockResultSender; use crate::metrics; @@ -7,6 +7,7 @@ use crate::sync::{BatchProcessResult, ChainId}; use beacon_chain::{BeaconChainTypes, BlockError, ChainSegmentResult}; use eth2_libp2p::PeerId; use slog::{crit, debug, error, info, trace, warn}; +use tokio::sync::mpsc; use types::{Epoch, Hash256, SignedBeaconBlock}; /// Id associated to a block processing request, either a batch or a single block. @@ -27,6 +28,7 @@ impl Worker { self, block: SignedBeaconBlock, result_tx: BlockResultSender, + reprocess_tx: mpsc::Sender>, ) { let slot = block.slot(); let block_result = self.chain.process_block(block); @@ -40,6 +42,18 @@ impl Worker { "slot" => slot, "hash" => %root ); + + if reprocess_tx + .try_send(ReprocessQueueMessage::BlockImported(*root)) + .is_err() + { + error!( + self.log, + "Failed to inform block import"; + "source" => "rpc", + "block_root" => %root, + ) + }; } if result_tx.send(block_result).is_err() { From 8fae9050e4d34859d793e74adee399033282afea Mon Sep 17 00:00:00 2001 From: Paul Hauner Date: Mon, 21 Jun 2021 11:21:40 +1000 Subject: [PATCH 39/43] Rename root -> beacon_block_root --- .../work_reprocessing_queue.rs | 18 ++++++++++++------ 1 file changed, 12 insertions(+), 6 deletions(-) diff --git a/beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs b/beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs index c62d98ae45e..8bfac10de7e 100644 --- a/beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs +++ b/beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs @@ -139,13 +139,13 @@ enum QueuedAttestationId { } impl QueuedAggregate { - pub fn root(&self) -> &Hash256 { + pub fn beacon_block_root(&self) -> &Hash256 { &self.attestation.message.aggregate.data.beacon_block_root } } impl QueuedUnaggregate { - pub fn root(&self) -> &Hash256 { + pub fn beacon_block_root(&self) -> &Hash256 { &self.attestation.data.beacon_block_root } } @@ -322,7 +322,7 @@ impl ReprocessQueue { // Register this attestation for the corresponding root. self.awaiting_attestations_per_root - .entry(*queued_aggregate.root()) + .entry(*queued_aggregate.beacon_block_root()) .or_default() .push(att_id); @@ -353,7 +353,7 @@ impl ReprocessQueue { // Register this attestation for the corresponding root. self.awaiting_attestations_per_root - .entry(*queued_unaggregate.root()) + .entry(*queued_unaggregate.beacon_block_root()) .or_default() .push(att_id); @@ -443,14 +443,20 @@ impl ReprocessQueue { self.queued_aggregates .remove(&id) .map(|(aggregate, _delay_key)| { - (*aggregate.root(), ReadyWork::Aggregate(aggregate)) + ( + *aggregate.beacon_block_root(), + ReadyWork::Aggregate(aggregate), + ) }) } QueuedAttestationId::Unaggregate(id) => self .queued_unaggregates .remove(&id) .map(|(unaggregate, _delay_key)| { - (*unaggregate.root(), ReadyWork::Unaggregate(unaggregate)) + ( + *unaggregate.beacon_block_root(), + ReadyWork::Unaggregate(unaggregate), + ) }), } { if self.ready_work_tx.try_send(work).is_err() { From 863d286c3644f040a1c2b731a685b0d2133e085d Mon Sep 17 00:00:00 2001 From: Paul Hauner Date: Mon, 21 Jun 2021 11:43:54 +1000 Subject: [PATCH 40/43] Add metrics for reprocessing queue size --- .../src/beacon_processor/work_reprocessing_queue.rs | 12 ++++++++++++ beacon_node/network/src/metrics.rs | 10 ++++++++++ 2 files changed, 22 insertions(+) diff --git a/beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs b/beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs index 8bfac10de7e..d373e7830bc 100644 --- a/beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs +++ b/beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs @@ -11,6 +11,7 @@ //! Aggregated and unaggregated attestations that failed verification due to referencing an unknown //! block will be re-queued until their block is imported, or until they expire. use super::MAX_SCHEDULED_WORK_QUEUE_LEN; +use crate::metrics; use beacon_chain::{BeaconChainTypes, GossipVerifiedBlock, MAXIMUM_GOSSIP_CLOCK_DISPARITY}; use eth2_libp2p::{MessageId, PeerId}; use fnv::FnvHashMap; @@ -474,5 +475,16 @@ impl ReprocessQueue { } } } + + metrics::set_gauge_vec( + &metrics::BEACON_PROCESSOR_REPROCESSING_QUEUE_TOTAL, + &["blocks"], + self.block_delay_queue.len() as i64, + ); + metrics::set_gauge_vec( + &metrics::BEACON_PROCESSOR_REPROCESSING_QUEUE_TOTAL, + &["attestations"], + self.attestations_delay_queue.len() as i64, + ); } } diff --git a/beacon_node/network/src/metrics.rs b/beacon_node/network/src/metrics.rs index 7b325f8a83c..b744a179353 100644 --- a/beacon_node/network/src/metrics.rs +++ b/beacon_node/network/src/metrics.rs @@ -378,6 +378,16 @@ lazy_static! { "beacon_block_gossip_slot_start_delay_time", "Duration between when the block is received and the start of the slot it belongs to.", ); + + /* + * Attestation reprocessing queue metrics. + */ + pub static ref BEACON_PROCESSOR_REPROCESSING_QUEUE_TOTAL: Result = + try_create_int_gauge_vec( + "beacon_processor_reprocessing_queue_total", + "Count of items in a reprocessing queue.", + &["type"] + ); } pub fn register_attestation_error(error: &AttnError) { From dca09d012155f6224291b0a56f009aa6a61ef0a2 Mon Sep 17 00:00:00 2001 From: Paul Hauner Date: Mon, 21 Jun 2021 11:52:21 +1000 Subject: [PATCH 41/43] Add metrics to reprocessing queue --- .../beacon_processor/work_reprocessing_queue.rs | 14 ++++++++++++-- beacon_node/network/src/metrics.rs | 8 ++++++++ 2 files changed, 20 insertions(+), 2 deletions(-) diff --git a/beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs b/beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs index d373e7830bc..9ad9b82285f 100644 --- a/beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs +++ b/beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs @@ -30,6 +30,8 @@ use tokio_util::time::delay_queue::{DelayQueue, Key as DelayKey}; use types::{Attestation, EthSpec, Hash256, SignedAggregateAndProof, SubnetId}; const TASK_NAME: &str = "beacon_processor_reprocess_queue"; +const BLOCKS: &str = "blocks"; +const ATTESTATIONS: &str = "attestations"; /// Queue blocks for re-processing with an `ADDITIONAL_QUEUED_BLOCK_DELAY` after the slot starts. /// This is to account for any slight drift in the system clock. @@ -368,6 +370,10 @@ impl ReprocessQueue { // Unqueue the attestations we have for this root, if any. if let Some(queued_ids) = self.awaiting_attestations_per_root.remove(&root) { for id in queued_ids { + metrics::inc_counter( + &metrics::BEACON_PROCESSOR_REPROCESSING_QUEUE_MATCHED_ATTESTATIONS, + ); + if let Some((work, delay_key)) = match id { QueuedAttestationId::Aggregate(id) => self .queued_aggregates @@ -439,6 +445,10 @@ impl ReprocessQueue { ) } InboundEvent::ReadyAttestation(queued_id) => { + metrics::inc_counter( + &metrics::BEACON_PROCESSOR_REPROCESSING_QUEUE_EXPIRED_ATTESTATIONS, + ); + if let Some((root, work)) = match queued_id { QueuedAttestationId::Aggregate(id) => { self.queued_aggregates @@ -478,12 +488,12 @@ impl ReprocessQueue { metrics::set_gauge_vec( &metrics::BEACON_PROCESSOR_REPROCESSING_QUEUE_TOTAL, - &["blocks"], + &[BLOCKS], self.block_delay_queue.len() as i64, ); metrics::set_gauge_vec( &metrics::BEACON_PROCESSOR_REPROCESSING_QUEUE_TOTAL, - &["attestations"], + &[ATTESTATIONS], self.attestations_delay_queue.len() as i64, ); } diff --git a/beacon_node/network/src/metrics.rs b/beacon_node/network/src/metrics.rs index b744a179353..a201a97d83b 100644 --- a/beacon_node/network/src/metrics.rs +++ b/beacon_node/network/src/metrics.rs @@ -388,6 +388,14 @@ lazy_static! { "Count of items in a reprocessing queue.", &["type"] ); + pub static ref BEACON_PROCESSOR_REPROCESSING_QUEUE_EXPIRED_ATTESTATIONS: Result = try_create_int_counter( + "beacon_processor_reprocessing_queue_expired_attestations", + "Number of queued attestations which have expired before a matching block has been found" + ); + pub static ref BEACON_PROCESSOR_REPROCESSING_QUEUE_MATCHED_ATTESTATIONS: Result = try_create_int_counter( + "beacon_processor_reprocessing_queue_matched_attestations", + "Number of queued attestations where as matching block has been imported" + ); } pub fn register_attestation_error(error: &AttnError) { From c33d3cf904f19acb1dd93cf54f9195a59d0d25fb Mon Sep 17 00:00:00 2001 From: divma <26765164+divagant-martian@users.noreply.github.com> Date: Mon, 5 Jul 2021 10:56:44 -0500 Subject: [PATCH 42/43] improve logs Co-authored-by: Paul Hauner --- .../network/src/beacon_processor/work_reprocessing_queue.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs b/beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs index 9ad9b82285f..c92842920b1 100644 --- a/beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs +++ b/beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs @@ -308,7 +308,7 @@ impl ReprocessQueue { if self.attestations_delay_queue.len() >= MAXIMUM_QUEUED_ATTESTATIONS { error!( log, - "Attestation delay queue is full"; + "Aggregate attestation delay queue is full"; "queue_size" => MAXIMUM_QUEUED_ATTESTATIONS, "msg" => "check system clock" ); From 4ba921e6a365974872b91175e87c403343f542c2 Mon Sep 17 00:00:00 2001 From: divma <26765164+divagant-martian@users.noreply.github.com> Date: Tue, 6 Jul 2021 10:13:10 -0500 Subject: [PATCH 43/43] adjust attestations delay before re-processing Co-authored-by: Paul Hauner --- .../network/src/beacon_processor/work_reprocessing_queue.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs b/beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs index c92842920b1..b9501680528 100644 --- a/beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs +++ b/beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs @@ -38,7 +38,7 @@ const ATTESTATIONS: &str = "attestations"; const ADDITIONAL_QUEUED_BLOCK_DELAY: Duration = Duration::from_millis(5); /// For how long to queue aggregated and unaggregated attestations for re-processing. -pub const QUEUED_ATTESTATION_DELAY: Duration = Duration::from_secs(30); +pub const QUEUED_ATTESTATION_DELAY: Duration = Duration::from_secs(12); /// Set an arbitrary upper-bound on the number of queued blocks to avoid DoS attacks. The fact that /// we signature-verify blocks before putting them in the queue *should* protect against this, but