Ignored sync jobs 2 (#3317)
## Issue Addressed Duplicate of #3269. Making this since @divagant-martian opened the previous PR and she can't approve her own PR 😄 Co-authored-by: Diva M <divma@protonmail.com>
This commit is contained in:
parent
98a9626ef5
commit
28b0ff27ff
@ -66,7 +66,7 @@ use types::{
|
|||||||
SyncCommitteeMessage, SyncSubnetId,
|
SyncCommitteeMessage, SyncSubnetId,
|
||||||
};
|
};
|
||||||
use work_reprocessing_queue::{
|
use work_reprocessing_queue::{
|
||||||
spawn_reprocess_scheduler, QueuedAggregate, QueuedUnaggregate, ReadyWork,
|
spawn_reprocess_scheduler, QueuedAggregate, QueuedRpcBlock, QueuedUnaggregate, ReadyWork,
|
||||||
};
|
};
|
||||||
|
|
||||||
use worker::{Toolbox, Worker};
|
use worker::{Toolbox, Worker};
|
||||||
@ -75,7 +75,7 @@ mod tests;
|
|||||||
mod work_reprocessing_queue;
|
mod work_reprocessing_queue;
|
||||||
mod worker;
|
mod worker;
|
||||||
|
|
||||||
use crate::beacon_processor::work_reprocessing_queue::QueuedBlock;
|
use crate::beacon_processor::work_reprocessing_queue::QueuedGossipBlock;
|
||||||
pub use worker::{
|
pub use worker::{
|
||||||
ChainSegmentProcessId, FailureMode, GossipAggregatePackage, GossipAttestationPackage,
|
ChainSegmentProcessId, FailureMode, GossipAggregatePackage, GossipAttestationPackage,
|
||||||
};
|
};
|
||||||
@ -501,6 +501,7 @@ impl<T: BeaconChainTypes> WorkEvent<T> {
|
|||||||
block,
|
block,
|
||||||
seen_timestamp,
|
seen_timestamp,
|
||||||
process_type,
|
process_type,
|
||||||
|
should_process: true,
|
||||||
},
|
},
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@ -565,7 +566,7 @@ impl<T: BeaconChainTypes> WorkEvent<T> {
|
|||||||
impl<T: BeaconChainTypes> std::convert::From<ReadyWork<T>> for WorkEvent<T> {
|
impl<T: BeaconChainTypes> std::convert::From<ReadyWork<T>> for WorkEvent<T> {
|
||||||
fn from(ready_work: ReadyWork<T>) -> Self {
|
fn from(ready_work: ReadyWork<T>) -> Self {
|
||||||
match ready_work {
|
match ready_work {
|
||||||
ReadyWork::Block(QueuedBlock {
|
ReadyWork::Block(QueuedGossipBlock {
|
||||||
peer_id,
|
peer_id,
|
||||||
block,
|
block,
|
||||||
seen_timestamp,
|
seen_timestamp,
|
||||||
@ -577,6 +578,20 @@ impl<T: BeaconChainTypes> std::convert::From<ReadyWork<T>> for WorkEvent<T> {
|
|||||||
seen_timestamp,
|
seen_timestamp,
|
||||||
},
|
},
|
||||||
},
|
},
|
||||||
|
ReadyWork::RpcBlock(QueuedRpcBlock {
|
||||||
|
block,
|
||||||
|
seen_timestamp,
|
||||||
|
process_type,
|
||||||
|
should_process,
|
||||||
|
}) => Self {
|
||||||
|
drop_during_sync: false,
|
||||||
|
work: Work::RpcBlock {
|
||||||
|
block,
|
||||||
|
seen_timestamp,
|
||||||
|
process_type,
|
||||||
|
should_process,
|
||||||
|
},
|
||||||
|
},
|
||||||
ReadyWork::Unaggregate(QueuedUnaggregate {
|
ReadyWork::Unaggregate(QueuedUnaggregate {
|
||||||
peer_id,
|
peer_id,
|
||||||
message_id,
|
message_id,
|
||||||
@ -695,6 +710,7 @@ pub enum Work<T: BeaconChainTypes> {
|
|||||||
block: Arc<SignedBeaconBlock<T::EthSpec>>,
|
block: Arc<SignedBeaconBlock<T::EthSpec>>,
|
||||||
seen_timestamp: Duration,
|
seen_timestamp: Duration,
|
||||||
process_type: BlockProcessType,
|
process_type: BlockProcessType,
|
||||||
|
should_process: bool,
|
||||||
},
|
},
|
||||||
ChainSegment {
|
ChainSegment {
|
||||||
process_id: ChainSegmentProcessId,
|
process_id: ChainSegmentProcessId,
|
||||||
@ -1521,12 +1537,14 @@ impl<T: BeaconChainTypes> BeaconProcessor<T> {
|
|||||||
block,
|
block,
|
||||||
seen_timestamp,
|
seen_timestamp,
|
||||||
process_type,
|
process_type,
|
||||||
|
should_process,
|
||||||
} => task_spawner.spawn_async(worker.process_rpc_block(
|
} => task_spawner.spawn_async(worker.process_rpc_block(
|
||||||
block,
|
block,
|
||||||
seen_timestamp,
|
seen_timestamp,
|
||||||
process_type,
|
process_type,
|
||||||
work_reprocessing_tx,
|
work_reprocessing_tx,
|
||||||
duplicate_cache,
|
duplicate_cache,
|
||||||
|
should_process,
|
||||||
)),
|
)),
|
||||||
/*
|
/*
|
||||||
* Verification for a chain segment (multiple blocks).
|
* Verification for a chain segment (multiple blocks).
|
||||||
|
@ -1,7 +1,9 @@
|
|||||||
#![cfg(not(debug_assertions))] // Tests are too slow in debug.
|
#![cfg(not(debug_assertions))] // Tests are too slow in debug.
|
||||||
#![cfg(test)]
|
#![cfg(test)]
|
||||||
|
|
||||||
use crate::beacon_processor::work_reprocessing_queue::QUEUED_ATTESTATION_DELAY;
|
use crate::beacon_processor::work_reprocessing_queue::{
|
||||||
|
QUEUED_ATTESTATION_DELAY, QUEUED_RPC_BLOCK_DELAY,
|
||||||
|
};
|
||||||
use crate::beacon_processor::*;
|
use crate::beacon_processor::*;
|
||||||
use crate::{service::NetworkMessage, sync::SyncMessage};
|
use crate::{service::NetworkMessage, sync::SyncMessage};
|
||||||
use beacon_chain::test_utils::{
|
use beacon_chain::test_utils::{
|
||||||
@ -54,6 +56,7 @@ struct TestRig {
|
|||||||
work_journal_rx: mpsc::Receiver<&'static str>,
|
work_journal_rx: mpsc::Receiver<&'static str>,
|
||||||
_network_rx: mpsc::UnboundedReceiver<NetworkMessage<E>>,
|
_network_rx: mpsc::UnboundedReceiver<NetworkMessage<E>>,
|
||||||
_sync_rx: mpsc::UnboundedReceiver<SyncMessage<E>>,
|
_sync_rx: mpsc::UnboundedReceiver<SyncMessage<E>>,
|
||||||
|
duplicate_cache: DuplicateCache,
|
||||||
_harness: BeaconChainHarness<T>,
|
_harness: BeaconChainHarness<T>,
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -185,6 +188,7 @@ impl TestRig {
|
|||||||
|
|
||||||
let (work_journal_tx, work_journal_rx) = mpsc::channel(16_364);
|
let (work_journal_tx, work_journal_rx) = mpsc::channel(16_364);
|
||||||
|
|
||||||
|
let duplicate_cache = DuplicateCache::default();
|
||||||
BeaconProcessor {
|
BeaconProcessor {
|
||||||
beacon_chain: Arc::downgrade(&chain),
|
beacon_chain: Arc::downgrade(&chain),
|
||||||
network_tx,
|
network_tx,
|
||||||
@ -193,7 +197,7 @@ impl TestRig {
|
|||||||
executor,
|
executor,
|
||||||
max_workers: cmp::max(1, num_cpus::get()),
|
max_workers: cmp::max(1, num_cpus::get()),
|
||||||
current_workers: 0,
|
current_workers: 0,
|
||||||
importing_blocks: Default::default(),
|
importing_blocks: duplicate_cache.clone(),
|
||||||
log: log.clone(),
|
log: log.clone(),
|
||||||
}
|
}
|
||||||
.spawn_manager(beacon_processor_rx, Some(work_journal_tx));
|
.spawn_manager(beacon_processor_rx, Some(work_journal_tx));
|
||||||
@ -211,6 +215,7 @@ impl TestRig {
|
|||||||
work_journal_rx,
|
work_journal_rx,
|
||||||
_network_rx,
|
_network_rx,
|
||||||
_sync_rx,
|
_sync_rx,
|
||||||
|
duplicate_cache,
|
||||||
_harness: harness,
|
_harness: harness,
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@ -246,6 +251,15 @@ impl TestRig {
|
|||||||
self.beacon_processor_tx.try_send(event).unwrap();
|
self.beacon_processor_tx.try_send(event).unwrap();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
pub fn enqueue_single_lookup_rpc_block(&self) {
|
||||||
|
let event = WorkEvent::rpc_beacon_block(
|
||||||
|
self.next_block.clone(),
|
||||||
|
std::time::Duration::default(),
|
||||||
|
BlockProcessType::SingleBlock { id: 1 },
|
||||||
|
);
|
||||||
|
self.beacon_processor_tx.try_send(event).unwrap();
|
||||||
|
}
|
||||||
|
|
||||||
pub fn enqueue_unaggregated_attestation(&self) {
|
pub fn enqueue_unaggregated_attestation(&self) {
|
||||||
let (attestation, subnet_id) = self.attestations.first().unwrap().clone();
|
let (attestation, subnet_id) = self.attestations.first().unwrap().clone();
|
||||||
self.beacon_processor_tx
|
self.beacon_processor_tx
|
||||||
@ -828,3 +842,33 @@ async fn import_misc_gossip_ops() {
|
|||||||
"op pool should have one more exit"
|
"op pool should have one more exit"
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/// Ensure that rpc block going to the reprocessing queue flow
|
||||||
|
/// works when the duplicate cache handle is held by another task.
|
||||||
|
#[tokio::test]
|
||||||
|
async fn test_rpc_block_reprocessing() {
|
||||||
|
let mut rig = TestRig::new(SMALL_CHAIN).await;
|
||||||
|
let next_block_root = rig.next_block.canonical_root();
|
||||||
|
// Insert the next block into the duplicate cache manually
|
||||||
|
let handle = rig.duplicate_cache.check_and_insert(next_block_root);
|
||||||
|
rig.enqueue_single_lookup_rpc_block();
|
||||||
|
|
||||||
|
rig.assert_event_journal(&[RPC_BLOCK, WORKER_FREED, NOTHING_TO_DO])
|
||||||
|
.await;
|
||||||
|
// next_block shouldn't be processed since it couldn't get the
|
||||||
|
// duplicate cache handle
|
||||||
|
assert_ne!(next_block_root, rig.head_root());
|
||||||
|
|
||||||
|
drop(handle);
|
||||||
|
|
||||||
|
// The block should arrive at the beacon processor again after
|
||||||
|
// the specified delay.
|
||||||
|
tokio::time::sleep(QUEUED_RPC_BLOCK_DELAY).await;
|
||||||
|
|
||||||
|
rig.assert_event_journal(&[RPC_BLOCK]).await;
|
||||||
|
// Add an extra delay for block processing
|
||||||
|
tokio::time::sleep(Duration::from_millis(10)).await;
|
||||||
|
// head should update to next block now since the duplicate
|
||||||
|
// cache handle was dropped.
|
||||||
|
assert_eq!(next_block_root, rig.head_root());
|
||||||
|
}
|
||||||
|
@ -12,6 +12,7 @@
|
|||||||
//! block will be re-queued until their block is imported, or until they expire.
|
//! block will be re-queued until their block is imported, or until they expire.
|
||||||
use super::MAX_SCHEDULED_WORK_QUEUE_LEN;
|
use super::MAX_SCHEDULED_WORK_QUEUE_LEN;
|
||||||
use crate::metrics;
|
use crate::metrics;
|
||||||
|
use crate::sync::manager::BlockProcessType;
|
||||||
use beacon_chain::{BeaconChainTypes, GossipVerifiedBlock, MAXIMUM_GOSSIP_CLOCK_DISPARITY};
|
use beacon_chain::{BeaconChainTypes, GossipVerifiedBlock, MAXIMUM_GOSSIP_CLOCK_DISPARITY};
|
||||||
use fnv::FnvHashMap;
|
use fnv::FnvHashMap;
|
||||||
use futures::task::Poll;
|
use futures::task::Poll;
|
||||||
@ -22,16 +23,18 @@ use slog::{crit, debug, error, warn, Logger};
|
|||||||
use slot_clock::SlotClock;
|
use slot_clock::SlotClock;
|
||||||
use std::collections::{HashMap, HashSet};
|
use std::collections::{HashMap, HashSet};
|
||||||
use std::pin::Pin;
|
use std::pin::Pin;
|
||||||
|
use std::sync::Arc;
|
||||||
use std::task::Context;
|
use std::task::Context;
|
||||||
use std::time::Duration;
|
use std::time::Duration;
|
||||||
use task_executor::TaskExecutor;
|
use task_executor::TaskExecutor;
|
||||||
use tokio::sync::mpsc::{self, Receiver, Sender};
|
use tokio::sync::mpsc::{self, Receiver, Sender};
|
||||||
use tokio::time::error::Error as TimeError;
|
use tokio::time::error::Error as TimeError;
|
||||||
use tokio_util::time::delay_queue::{DelayQueue, Key as DelayKey};
|
use tokio_util::time::delay_queue::{DelayQueue, Key as DelayKey};
|
||||||
use types::{Attestation, EthSpec, Hash256, SignedAggregateAndProof, SubnetId};
|
use types::{Attestation, EthSpec, Hash256, SignedAggregateAndProof, SignedBeaconBlock, SubnetId};
|
||||||
|
|
||||||
const TASK_NAME: &str = "beacon_processor_reprocess_queue";
|
const TASK_NAME: &str = "beacon_processor_reprocess_queue";
|
||||||
const BLOCKS: &str = "blocks";
|
const GOSSIP_BLOCKS: &str = "gossip_blocks";
|
||||||
|
const RPC_BLOCKS: &str = "rpc_blocks";
|
||||||
const ATTESTATIONS: &str = "attestations";
|
const ATTESTATIONS: &str = "attestations";
|
||||||
|
|
||||||
/// Queue blocks for re-processing with an `ADDITIONAL_QUEUED_BLOCK_DELAY` after the slot starts.
|
/// Queue blocks for re-processing with an `ADDITIONAL_QUEUED_BLOCK_DELAY` after the slot starts.
|
||||||
@ -41,6 +44,9 @@ const ADDITIONAL_QUEUED_BLOCK_DELAY: Duration = Duration::from_millis(5);
|
|||||||
/// For how long to queue aggregated and unaggregated attestations for re-processing.
|
/// For how long to queue aggregated and unaggregated attestations for re-processing.
|
||||||
pub const QUEUED_ATTESTATION_DELAY: Duration = Duration::from_secs(12);
|
pub const QUEUED_ATTESTATION_DELAY: Duration = Duration::from_secs(12);
|
||||||
|
|
||||||
|
/// For how long to queue rpc blocks before sending them back for reprocessing.
|
||||||
|
pub const QUEUED_RPC_BLOCK_DELAY: Duration = Duration::from_secs(3);
|
||||||
|
|
||||||
/// Set an arbitrary upper-bound on the number of queued blocks to avoid DoS attacks. The fact that
|
/// 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
|
/// we signature-verify blocks before putting them in the queue *should* protect against this, but
|
||||||
/// it's nice to have extra protection.
|
/// it's nice to have extra protection.
|
||||||
@ -52,7 +58,10 @@ const MAXIMUM_QUEUED_ATTESTATIONS: usize = 16_384;
|
|||||||
/// Messages that the scheduler can receive.
|
/// Messages that the scheduler can receive.
|
||||||
pub enum ReprocessQueueMessage<T: BeaconChainTypes> {
|
pub enum ReprocessQueueMessage<T: BeaconChainTypes> {
|
||||||
/// A block that has been received early and we should queue for later processing.
|
/// A block that has been received early and we should queue for later processing.
|
||||||
EarlyBlock(QueuedBlock<T>),
|
EarlyBlock(QueuedGossipBlock<T>),
|
||||||
|
/// A gossip block for hash `X` is being imported, we should queue the rpc block for the same
|
||||||
|
/// hash until the gossip block is imported.
|
||||||
|
RpcBlock(QueuedRpcBlock<T::EthSpec>),
|
||||||
/// A block that was successfully processed. We use this to handle attestations for unknown
|
/// A block that was successfully processed. We use this to handle attestations for unknown
|
||||||
/// blocks.
|
/// blocks.
|
||||||
BlockImported(Hash256),
|
BlockImported(Hash256),
|
||||||
@ -64,7 +73,8 @@ pub enum ReprocessQueueMessage<T: BeaconChainTypes> {
|
|||||||
|
|
||||||
/// Events sent by the scheduler once they are ready for re-processing.
|
/// Events sent by the scheduler once they are ready for re-processing.
|
||||||
pub enum ReadyWork<T: BeaconChainTypes> {
|
pub enum ReadyWork<T: BeaconChainTypes> {
|
||||||
Block(QueuedBlock<T>),
|
Block(QueuedGossipBlock<T>),
|
||||||
|
RpcBlock(QueuedRpcBlock<T::EthSpec>),
|
||||||
Unaggregate(QueuedUnaggregate<T::EthSpec>),
|
Unaggregate(QueuedUnaggregate<T::EthSpec>),
|
||||||
Aggregate(QueuedAggregate<T::EthSpec>),
|
Aggregate(QueuedAggregate<T::EthSpec>),
|
||||||
}
|
}
|
||||||
@ -90,16 +100,30 @@ pub struct QueuedAggregate<T: EthSpec> {
|
|||||||
}
|
}
|
||||||
|
|
||||||
/// A block that arrived early and has been queued for later import.
|
/// A block that arrived early and has been queued for later import.
|
||||||
pub struct QueuedBlock<T: BeaconChainTypes> {
|
pub struct QueuedGossipBlock<T: BeaconChainTypes> {
|
||||||
pub peer_id: PeerId,
|
pub peer_id: PeerId,
|
||||||
pub block: Box<GossipVerifiedBlock<T>>,
|
pub block: Box<GossipVerifiedBlock<T>>,
|
||||||
pub seen_timestamp: Duration,
|
pub seen_timestamp: Duration,
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/// A block that arrived for processing when the same block was being imported over gossip.
|
||||||
|
/// It is queued for later import.
|
||||||
|
pub struct QueuedRpcBlock<T: EthSpec> {
|
||||||
|
pub block: Arc<SignedBeaconBlock<T>>,
|
||||||
|
pub process_type: BlockProcessType,
|
||||||
|
pub seen_timestamp: Duration,
|
||||||
|
/// Indicates if the beacon chain should process this block or not.
|
||||||
|
/// We use this to ignore block processing when rpc block queues are full.
|
||||||
|
pub should_process: bool,
|
||||||
|
}
|
||||||
|
|
||||||
/// Unifies the different messages processed by the block delay queue.
|
/// Unifies the different messages processed by the block delay queue.
|
||||||
enum InboundEvent<T: BeaconChainTypes> {
|
enum InboundEvent<T: BeaconChainTypes> {
|
||||||
/// A block that was queued for later processing and is ready for import.
|
/// A gossip block that was queued for later processing and is ready for import.
|
||||||
ReadyBlock(QueuedBlock<T>),
|
ReadyGossipBlock(QueuedGossipBlock<T>),
|
||||||
|
/// A rpc block that was queued because the same gossip block was being imported
|
||||||
|
/// will now be retried for import.
|
||||||
|
ReadyRpcBlock(QueuedRpcBlock<T::EthSpec>),
|
||||||
/// An aggregated or unaggregated attestation is ready for re-processing.
|
/// An aggregated or unaggregated attestation is ready for re-processing.
|
||||||
ReadyAttestation(QueuedAttestationId),
|
ReadyAttestation(QueuedAttestationId),
|
||||||
/// A `DelayQueue` returned an error.
|
/// A `DelayQueue` returned an error.
|
||||||
@ -117,13 +141,15 @@ struct ReprocessQueue<T: BeaconChainTypes> {
|
|||||||
|
|
||||||
/* Queues */
|
/* Queues */
|
||||||
/// Queue to manage scheduled early blocks.
|
/// Queue to manage scheduled early blocks.
|
||||||
block_delay_queue: DelayQueue<QueuedBlock<T>>,
|
gossip_block_delay_queue: DelayQueue<QueuedGossipBlock<T>>,
|
||||||
|
/// Queue to manage scheduled early blocks.
|
||||||
|
rpc_block_delay_queue: DelayQueue<QueuedRpcBlock<T::EthSpec>>,
|
||||||
/// Queue to manage scheduled attestations.
|
/// Queue to manage scheduled attestations.
|
||||||
attestations_delay_queue: DelayQueue<QueuedAttestationId>,
|
attestations_delay_queue: DelayQueue<QueuedAttestationId>,
|
||||||
|
|
||||||
/* Queued items */
|
/* Queued items */
|
||||||
/// Queued blocks.
|
/// Queued blocks.
|
||||||
queued_block_roots: HashSet<Hash256>,
|
queued_gossip_block_roots: HashSet<Hash256>,
|
||||||
/// Queued aggregated attestations.
|
/// Queued aggregated attestations.
|
||||||
queued_aggregates: FnvHashMap<usize, (QueuedAggregate<T::EthSpec>, DelayKey)>,
|
queued_aggregates: FnvHashMap<usize, (QueuedAggregate<T::EthSpec>, DelayKey)>,
|
||||||
/// Queued attestations.
|
/// Queued attestations.
|
||||||
@ -135,6 +161,7 @@ struct ReprocessQueue<T: BeaconChainTypes> {
|
|||||||
/// Next attestation id, used for both aggregated and unaggregated attestations
|
/// Next attestation id, used for both aggregated and unaggregated attestations
|
||||||
next_attestation: usize,
|
next_attestation: usize,
|
||||||
early_block_debounce: TimeLatch,
|
early_block_debounce: TimeLatch,
|
||||||
|
rpc_block_debounce: TimeLatch,
|
||||||
attestation_delay_debounce: TimeLatch,
|
attestation_delay_debounce: TimeLatch,
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -167,12 +194,26 @@ impl<T: BeaconChainTypes> Stream for ReprocessQueue<T> {
|
|||||||
//
|
//
|
||||||
// The sequential nature of blockchains means it is generally better to try and import all
|
// The sequential nature of blockchains means it is generally better to try and import all
|
||||||
// existing blocks before new ones.
|
// existing blocks before new ones.
|
||||||
match self.block_delay_queue.poll_expired(cx) {
|
match self.gossip_block_delay_queue.poll_expired(cx) {
|
||||||
Poll::Ready(Some(Ok(queued_block))) => {
|
Poll::Ready(Some(Ok(queued_block))) => {
|
||||||
return Poll::Ready(Some(InboundEvent::ReadyBlock(queued_block.into_inner())));
|
return Poll::Ready(Some(InboundEvent::ReadyGossipBlock(
|
||||||
|
queued_block.into_inner(),
|
||||||
|
)));
|
||||||
}
|
}
|
||||||
Poll::Ready(Some(Err(e))) => {
|
Poll::Ready(Some(Err(e))) => {
|
||||||
return Poll::Ready(Some(InboundEvent::DelayQueueError(e, "block_queue")));
|
return Poll::Ready(Some(InboundEvent::DelayQueueError(e, "gossip_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 => (),
|
||||||
|
}
|
||||||
|
|
||||||
|
match self.rpc_block_delay_queue.poll_expired(cx) {
|
||||||
|
Poll::Ready(Some(Ok(queued_block))) => {
|
||||||
|
return Poll::Ready(Some(InboundEvent::ReadyRpcBlock(queued_block.into_inner())));
|
||||||
|
}
|
||||||
|
Poll::Ready(Some(Err(e))) => {
|
||||||
|
return Poll::Ready(Some(InboundEvent::DelayQueueError(e, "rpc_block_queue")));
|
||||||
}
|
}
|
||||||
// `Poll::Ready(None)` means that there are no more entries in the delay queue and we
|
// `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.
|
// will continue to get this result until something else is added into the queue.
|
||||||
@ -219,14 +260,16 @@ pub fn spawn_reprocess_scheduler<T: BeaconChainTypes>(
|
|||||||
let mut queue = ReprocessQueue {
|
let mut queue = ReprocessQueue {
|
||||||
work_reprocessing_rx,
|
work_reprocessing_rx,
|
||||||
ready_work_tx,
|
ready_work_tx,
|
||||||
block_delay_queue: DelayQueue::new(),
|
gossip_block_delay_queue: DelayQueue::new(),
|
||||||
|
rpc_block_delay_queue: DelayQueue::new(),
|
||||||
attestations_delay_queue: DelayQueue::new(),
|
attestations_delay_queue: DelayQueue::new(),
|
||||||
queued_block_roots: HashSet::new(),
|
queued_gossip_block_roots: HashSet::new(),
|
||||||
queued_aggregates: FnvHashMap::default(),
|
queued_aggregates: FnvHashMap::default(),
|
||||||
queued_unaggregates: FnvHashMap::default(),
|
queued_unaggregates: FnvHashMap::default(),
|
||||||
awaiting_attestations_per_root: HashMap::new(),
|
awaiting_attestations_per_root: HashMap::new(),
|
||||||
next_attestation: 0,
|
next_attestation: 0,
|
||||||
early_block_debounce: TimeLatch::default(),
|
early_block_debounce: TimeLatch::default(),
|
||||||
|
rpc_block_debounce: TimeLatch::default(),
|
||||||
attestation_delay_debounce: TimeLatch::default(),
|
attestation_delay_debounce: TimeLatch::default(),
|
||||||
};
|
};
|
||||||
|
|
||||||
@ -259,13 +302,13 @@ impl<T: BeaconChainTypes> ReprocessQueue<T> {
|
|||||||
let block_root = early_block.block.block_root;
|
let block_root = early_block.block.block_root;
|
||||||
|
|
||||||
// Don't add the same block to the queue twice. This prevents DoS attacks.
|
// Don't add the same block to the queue twice. This prevents DoS attacks.
|
||||||
if self.queued_block_roots.contains(&block_root) {
|
if self.queued_gossip_block_roots.contains(&block_root) {
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
|
||||||
if let Some(duration_till_slot) = 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.
|
// Check to ensure this won't over-fill the queue.
|
||||||
if self.queued_block_roots.len() >= MAXIMUM_QUEUED_BLOCKS {
|
if self.queued_gossip_block_roots.len() >= MAXIMUM_QUEUED_BLOCKS {
|
||||||
if self.early_block_debounce.elapsed() {
|
if self.early_block_debounce.elapsed() {
|
||||||
warn!(
|
warn!(
|
||||||
log,
|
log,
|
||||||
@ -278,10 +321,10 @@ impl<T: BeaconChainTypes> ReprocessQueue<T> {
|
|||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
|
||||||
self.queued_block_roots.insert(block_root);
|
self.queued_gossip_block_roots.insert(block_root);
|
||||||
// Queue the block until the start of the appropriate slot, plus
|
// Queue the block until the start of the appropriate slot, plus
|
||||||
// `ADDITIONAL_QUEUED_BLOCK_DELAY`.
|
// `ADDITIONAL_QUEUED_BLOCK_DELAY`.
|
||||||
self.block_delay_queue.insert(
|
self.gossip_block_delay_queue.insert(
|
||||||
early_block,
|
early_block,
|
||||||
duration_till_slot + ADDITIONAL_QUEUED_BLOCK_DELAY,
|
duration_till_slot + ADDITIONAL_QUEUED_BLOCK_DELAY,
|
||||||
);
|
);
|
||||||
@ -311,6 +354,58 @@ impl<T: BeaconChainTypes> ReprocessQueue<T> {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
// A rpc block arrived for processing at the same time when a gossip block
|
||||||
|
// for the same block hash is being imported. We wait for `QUEUED_RPC_BLOCK_DELAY`
|
||||||
|
// and then send the rpc block back for processing assuming the gossip import
|
||||||
|
// has completed by then.
|
||||||
|
InboundEvent::Msg(RpcBlock(mut rpc_block)) => {
|
||||||
|
// Check to ensure this won't over-fill the queue.
|
||||||
|
if self.rpc_block_delay_queue.len() >= MAXIMUM_QUEUED_BLOCKS {
|
||||||
|
if self.rpc_block_debounce.elapsed() {
|
||||||
|
warn!(
|
||||||
|
log,
|
||||||
|
"RPC blocks queue is full";
|
||||||
|
"queue_size" => MAXIMUM_QUEUED_BLOCKS,
|
||||||
|
"msg" => "check system clock"
|
||||||
|
);
|
||||||
|
}
|
||||||
|
// Return the block to the beacon processor signalling to
|
||||||
|
// ignore processing for this block
|
||||||
|
rpc_block.should_process = false;
|
||||||
|
if self
|
||||||
|
.ready_work_tx
|
||||||
|
.try_send(ReadyWork::RpcBlock(rpc_block))
|
||||||
|
.is_err()
|
||||||
|
{
|
||||||
|
error!(
|
||||||
|
log,
|
||||||
|
"Failed to send rpc block to beacon processor";
|
||||||
|
);
|
||||||
|
}
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
|
||||||
|
// Queue the block for 1/4th of a slot
|
||||||
|
self.rpc_block_delay_queue
|
||||||
|
.insert(rpc_block, QUEUED_RPC_BLOCK_DELAY);
|
||||||
|
}
|
||||||
|
InboundEvent::ReadyRpcBlock(queued_rpc_block) => {
|
||||||
|
debug!(
|
||||||
|
log,
|
||||||
|
"Sending rpc block for reprocessing";
|
||||||
|
"block_root" => %queued_rpc_block.block.canonical_root()
|
||||||
|
);
|
||||||
|
if self
|
||||||
|
.ready_work_tx
|
||||||
|
.try_send(ReadyWork::RpcBlock(queued_rpc_block))
|
||||||
|
.is_err()
|
||||||
|
{
|
||||||
|
error!(
|
||||||
|
log,
|
||||||
|
"Failed to send rpc block to beacon processor";
|
||||||
|
);
|
||||||
|
}
|
||||||
|
}
|
||||||
InboundEvent::Msg(UnknownBlockAggregate(queued_aggregate)) => {
|
InboundEvent::Msg(UnknownBlockAggregate(queued_aggregate)) => {
|
||||||
if self.attestations_delay_queue.len() >= MAXIMUM_QUEUED_ATTESTATIONS {
|
if self.attestations_delay_queue.len() >= MAXIMUM_QUEUED_ATTESTATIONS {
|
||||||
if self.attestation_delay_debounce.elapsed() {
|
if self.attestation_delay_debounce.elapsed() {
|
||||||
@ -423,10 +518,10 @@ impl<T: BeaconChainTypes> ReprocessQueue<T> {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
// A block that was queued for later processing is now ready to be processed.
|
// A block that was queued for later processing is now ready to be processed.
|
||||||
InboundEvent::ReadyBlock(ready_block) => {
|
InboundEvent::ReadyGossipBlock(ready_block) => {
|
||||||
let block_root = ready_block.block.block_root;
|
let block_root = ready_block.block.block_root;
|
||||||
|
|
||||||
if !self.queued_block_roots.remove(&block_root) {
|
if !self.queued_gossip_block_roots.remove(&block_root) {
|
||||||
// Log an error to alert that we've made a bad assumption about how this
|
// Log an error to alert that we've made a bad assumption about how this
|
||||||
// program works, but still process the block anyway.
|
// program works, but still process the block anyway.
|
||||||
error!(
|
error!(
|
||||||
@ -499,8 +594,13 @@ impl<T: BeaconChainTypes> ReprocessQueue<T> {
|
|||||||
|
|
||||||
metrics::set_gauge_vec(
|
metrics::set_gauge_vec(
|
||||||
&metrics::BEACON_PROCESSOR_REPROCESSING_QUEUE_TOTAL,
|
&metrics::BEACON_PROCESSOR_REPROCESSING_QUEUE_TOTAL,
|
||||||
&[BLOCKS],
|
&[GOSSIP_BLOCKS],
|
||||||
self.block_delay_queue.len() as i64,
|
self.gossip_block_delay_queue.len() as i64,
|
||||||
|
);
|
||||||
|
metrics::set_gauge_vec(
|
||||||
|
&metrics::BEACON_PROCESSOR_REPROCESSING_QUEUE_TOTAL,
|
||||||
|
&[RPC_BLOCKS],
|
||||||
|
self.rpc_block_delay_queue.len() as i64,
|
||||||
);
|
);
|
||||||
metrics::set_gauge_vec(
|
metrics::set_gauge_vec(
|
||||||
&metrics::BEACON_PROCESSOR_REPROCESSING_QUEUE_TOTAL,
|
&metrics::BEACON_PROCESSOR_REPROCESSING_QUEUE_TOTAL,
|
||||||
|
@ -25,7 +25,7 @@ use types::{
|
|||||||
|
|
||||||
use super::{
|
use super::{
|
||||||
super::work_reprocessing_queue::{
|
super::work_reprocessing_queue::{
|
||||||
QueuedAggregate, QueuedBlock, QueuedUnaggregate, ReprocessQueueMessage,
|
QueuedAggregate, QueuedGossipBlock, QueuedUnaggregate, ReprocessQueueMessage,
|
||||||
},
|
},
|
||||||
Worker,
|
Worker,
|
||||||
};
|
};
|
||||||
@ -857,7 +857,7 @@ impl<T: BeaconChainTypes> Worker<T> {
|
|||||||
metrics::inc_counter(&metrics::BEACON_PROCESSOR_GOSSIP_BLOCK_REQUEUED_TOTAL);
|
metrics::inc_counter(&metrics::BEACON_PROCESSOR_GOSSIP_BLOCK_REQUEUED_TOTAL);
|
||||||
|
|
||||||
if reprocess_tx
|
if reprocess_tx
|
||||||
.try_send(ReprocessQueueMessage::EarlyBlock(QueuedBlock {
|
.try_send(ReprocessQueueMessage::EarlyBlock(QueuedGossipBlock {
|
||||||
peer_id,
|
peer_id,
|
||||||
block: Box::new(verified_block),
|
block: Box::new(verified_block),
|
||||||
seen_timestamp: seen_duration,
|
seen_timestamp: seen_duration,
|
||||||
|
@ -1,6 +1,7 @@
|
|||||||
use std::time::Duration;
|
use std::time::Duration;
|
||||||
|
|
||||||
use super::{super::work_reprocessing_queue::ReprocessQueueMessage, Worker};
|
use super::{super::work_reprocessing_queue::ReprocessQueueMessage, Worker};
|
||||||
|
use crate::beacon_processor::work_reprocessing_queue::QueuedRpcBlock;
|
||||||
use crate::beacon_processor::worker::FUTURE_SLOT_TOLERANCE;
|
use crate::beacon_processor::worker::FUTURE_SLOT_TOLERANCE;
|
||||||
use crate::beacon_processor::DuplicateCache;
|
use crate::beacon_processor::DuplicateCache;
|
||||||
use crate::metrics;
|
use crate::metrics;
|
||||||
@ -53,16 +54,37 @@ impl<T: BeaconChainTypes> Worker<T> {
|
|||||||
process_type: BlockProcessType,
|
process_type: BlockProcessType,
|
||||||
reprocess_tx: mpsc::Sender<ReprocessQueueMessage<T>>,
|
reprocess_tx: mpsc::Sender<ReprocessQueueMessage<T>>,
|
||||||
duplicate_cache: DuplicateCache,
|
duplicate_cache: DuplicateCache,
|
||||||
|
should_process: bool,
|
||||||
) {
|
) {
|
||||||
|
if !should_process {
|
||||||
|
// Sync handles these results
|
||||||
|
self.send_sync_message(SyncMessage::BlockProcessed {
|
||||||
|
process_type,
|
||||||
|
result: crate::sync::manager::BlockProcessResult::Ignored,
|
||||||
|
});
|
||||||
|
return;
|
||||||
|
}
|
||||||
// Check if the block is already being imported through another source
|
// Check if the block is already being imported through another source
|
||||||
let handle = match duplicate_cache.check_and_insert(block.canonical_root()) {
|
let handle = match duplicate_cache.check_and_insert(block.canonical_root()) {
|
||||||
Some(handle) => handle,
|
Some(handle) => handle,
|
||||||
None => {
|
None => {
|
||||||
// Sync handles these results
|
debug!(
|
||||||
self.send_sync_message(SyncMessage::BlockProcessed {
|
self.log,
|
||||||
|
"Gossip block is being processed";
|
||||||
|
"action" => "sending rpc block to reprocessing queue",
|
||||||
|
"block_root" => %block.canonical_root(),
|
||||||
|
);
|
||||||
|
// Send message to work reprocess queue to retry the block
|
||||||
|
let reprocess_msg = ReprocessQueueMessage::RpcBlock(QueuedRpcBlock {
|
||||||
|
block: block.clone(),
|
||||||
process_type,
|
process_type,
|
||||||
result: Err(BlockError::BlockIsAlreadyKnown),
|
seen_timestamp,
|
||||||
|
should_process: true,
|
||||||
});
|
});
|
||||||
|
|
||||||
|
if reprocess_tx.try_send(reprocess_msg).is_err() {
|
||||||
|
error!(self.log, "Failed to inform block import"; "source" => "rpc", "block_root" => %block.canonical_root())
|
||||||
|
};
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
@ -95,7 +117,7 @@ impl<T: BeaconChainTypes> Worker<T> {
|
|||||||
// Sync handles these results
|
// Sync handles these results
|
||||||
self.send_sync_message(SyncMessage::BlockProcessed {
|
self.send_sync_message(SyncMessage::BlockProcessed {
|
||||||
process_type,
|
process_type,
|
||||||
result: result.map(|_| ()),
|
result: result.into(),
|
||||||
});
|
});
|
||||||
|
|
||||||
// Drop the handle to remove the entry from the cache
|
// Drop the handle to remove the entry from the cache
|
||||||
|
@ -19,6 +19,7 @@ use self::{
|
|||||||
single_block_lookup::SingleBlockRequest,
|
single_block_lookup::SingleBlockRequest,
|
||||||
};
|
};
|
||||||
|
|
||||||
|
use super::manager::BlockProcessResult;
|
||||||
use super::BatchProcessResult;
|
use super::BatchProcessResult;
|
||||||
use super::{
|
use super::{
|
||||||
manager::{BlockProcessType, Id},
|
manager::{BlockProcessType, Id},
|
||||||
@ -247,7 +248,7 @@ impl<T: BeaconChainTypes> BlockLookups<T> {
|
|||||||
| VerifyError::ExtraBlocksReturned => {
|
| VerifyError::ExtraBlocksReturned => {
|
||||||
let e = e.into();
|
let e = e.into();
|
||||||
warn!(self.log, "Peer sent invalid response to parent request.";
|
warn!(self.log, "Peer sent invalid response to parent request.";
|
||||||
"peer_id" => %peer_id, "reason" => e);
|
"peer_id" => %peer_id, "reason" => %e);
|
||||||
|
|
||||||
// We do not tolerate these kinds of errors. We will accept a few but these are signs
|
// We do not tolerate these kinds of errors. We will accept a few but these are signs
|
||||||
// of a faulty peer.
|
// of a faulty peer.
|
||||||
@ -381,7 +382,7 @@ impl<T: BeaconChainTypes> BlockLookups<T> {
|
|||||||
pub fn single_block_processed(
|
pub fn single_block_processed(
|
||||||
&mut self,
|
&mut self,
|
||||||
id: Id,
|
id: Id,
|
||||||
result: Result<(), BlockError<T::EthSpec>>,
|
result: BlockProcessResult<T::EthSpec>,
|
||||||
cx: &mut SyncNetworkContext<T::EthSpec>,
|
cx: &mut SyncNetworkContext<T::EthSpec>,
|
||||||
) {
|
) {
|
||||||
let mut req = match self.single_block_lookups.remove(&id) {
|
let mut req = match self.single_block_lookups.remove(&id) {
|
||||||
@ -403,52 +404,62 @@ impl<T: BeaconChainTypes> BlockLookups<T> {
|
|||||||
Err(_) => return,
|
Err(_) => return,
|
||||||
};
|
};
|
||||||
|
|
||||||
if let Err(e) = &result {
|
match result {
|
||||||
trace!(self.log, "Single block processing failed"; "block" => %root, "error" => %e);
|
BlockProcessResult::Ok => {
|
||||||
} else {
|
trace!(self.log, "Single block processing succeeded"; "block" => %root);
|
||||||
trace!(self.log, "Single block processing succeeded"; "block" => %root);
|
}
|
||||||
}
|
BlockProcessResult::Ignored => {
|
||||||
|
// Beacon processor signalled to ignore the block processing result.
|
||||||
if let Err(e) = result {
|
// This implies that the cpu is overloaded. Drop the request.
|
||||||
match e {
|
warn!(
|
||||||
BlockError::BlockIsAlreadyKnown => {
|
self.log,
|
||||||
// No error here
|
"Single block processing was ignored, cpu might be overloaded";
|
||||||
}
|
"action" => "dropping single block request"
|
||||||
BlockError::BeaconChainError(e) => {
|
);
|
||||||
// Internal error
|
}
|
||||||
error!(self.log, "Beacon chain error processing single block"; "block_root" => %root, "error" => ?e);
|
BlockProcessResult::Err(e) => {
|
||||||
}
|
trace!(self.log, "Single block processing failed"; "block" => %root, "error" => %e);
|
||||||
BlockError::ParentUnknown(block) => {
|
match e {
|
||||||
self.search_parent(block, peer_id, cx);
|
BlockError::BlockIsAlreadyKnown => {
|
||||||
}
|
// No error here
|
||||||
|
}
|
||||||
e @ BlockError::ExecutionPayloadError(ExecutionPayloadError::RequestFailed(_))
|
BlockError::BeaconChainError(e) => {
|
||||||
| e @ BlockError::ExecutionPayloadError(
|
// Internal error
|
||||||
ExecutionPayloadError::NoExecutionConnection,
|
error!(self.log, "Beacon chain error processing single block"; "block_root" => %root, "error" => ?e);
|
||||||
) => {
|
}
|
||||||
// These errors indicate that the execution layer is offline
|
BlockError::ParentUnknown(block) => {
|
||||||
// and failed to validate the execution payload. Do not downscore peer.
|
self.search_parent(block, peer_id, cx);
|
||||||
debug!(
|
}
|
||||||
self.log,
|
e @ BlockError::ExecutionPayloadError(ExecutionPayloadError::RequestFailed(
|
||||||
"Single block lookup failed. Execution layer is offline";
|
_,
|
||||||
"root" => %root,
|
))
|
||||||
"error" => ?e
|
| e @ BlockError::ExecutionPayloadError(
|
||||||
);
|
ExecutionPayloadError::NoExecutionConnection,
|
||||||
}
|
) => {
|
||||||
other => {
|
// These errors indicate that the execution layer is offline
|
||||||
warn!(self.log, "Peer sent invalid block in single block lookup"; "root" => %root, "error" => ?other, "peer_id" => %peer_id);
|
// and failed to validate the execution payload. Do not downscore peer.
|
||||||
cx.report_peer(
|
debug!(
|
||||||
peer_id,
|
self.log,
|
||||||
PeerAction::MidToleranceError,
|
"Single block lookup failed. Execution layer is offline";
|
||||||
"single_block_failure",
|
"root" => %root,
|
||||||
);
|
"error" => ?e
|
||||||
|
);
|
||||||
// Try it again if possible.
|
}
|
||||||
req.register_failure();
|
other => {
|
||||||
if let Ok((peer_id, request)) = req.request_block() {
|
warn!(self.log, "Peer sent invalid block in single block lookup"; "root" => %root, "error" => ?other, "peer_id" => %peer_id);
|
||||||
if let Ok(request_id) = cx.single_block_lookup_request(peer_id, request) {
|
cx.report_peer(
|
||||||
// insert with the new id
|
peer_id,
|
||||||
self.single_block_lookups.insert(request_id, req);
|
PeerAction::MidToleranceError,
|
||||||
|
"single_block_failure",
|
||||||
|
);
|
||||||
|
// Try it again if possible.
|
||||||
|
req.register_failure();
|
||||||
|
if let Ok((peer_id, request)) = req.request_block() {
|
||||||
|
if let Ok(request_id) = cx.single_block_lookup_request(peer_id, request)
|
||||||
|
{
|
||||||
|
// insert with the new id
|
||||||
|
self.single_block_lookups.insert(request_id, req);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@ -464,7 +475,7 @@ impl<T: BeaconChainTypes> BlockLookups<T> {
|
|||||||
pub fn parent_block_processed(
|
pub fn parent_block_processed(
|
||||||
&mut self,
|
&mut self,
|
||||||
chain_hash: Hash256,
|
chain_hash: Hash256,
|
||||||
result: Result<(), BlockError<T::EthSpec>>,
|
result: BlockProcessResult<T::EthSpec>,
|
||||||
cx: &mut SyncNetworkContext<T::EthSpec>,
|
cx: &mut SyncNetworkContext<T::EthSpec>,
|
||||||
) {
|
) {
|
||||||
let (mut parent_lookup, peer_id) = if let Some((pos, peer)) = self
|
let (mut parent_lookup, peer_id) = if let Some((pos, peer)) = self
|
||||||
@ -487,20 +498,32 @@ impl<T: BeaconChainTypes> BlockLookups<T> {
|
|||||||
return crit!(self.log, "Process response for a parent lookup request that was not found"; "chain_hash" => %chain_hash);
|
return crit!(self.log, "Process response for a parent lookup request that was not found"; "chain_hash" => %chain_hash);
|
||||||
};
|
};
|
||||||
|
|
||||||
if let Err(e) = &result {
|
match &result {
|
||||||
trace!(self.log, "Parent block processing failed"; &parent_lookup, "error" => %e);
|
BlockProcessResult::Ok => {
|
||||||
} else {
|
trace!(self.log, "Parent block processing succeeded"; &parent_lookup)
|
||||||
trace!(self.log, "Parent block processing succeeded"; &parent_lookup);
|
}
|
||||||
|
BlockProcessResult::Err(e) => {
|
||||||
|
trace!(self.log, "Parent block processing failed"; &parent_lookup, "error" => %e)
|
||||||
|
}
|
||||||
|
BlockProcessResult::Ignored => {
|
||||||
|
trace!(
|
||||||
|
self.log,
|
||||||
|
"Parent block processing job was ignored";
|
||||||
|
"action" => "re-requesting block",
|
||||||
|
&parent_lookup
|
||||||
|
);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
match result {
|
match result {
|
||||||
Err(BlockError::ParentUnknown(block)) => {
|
BlockProcessResult::Err(BlockError::ParentUnknown(block)) => {
|
||||||
// need to keep looking for parents
|
// need to keep looking for parents
|
||||||
// add the block back to the queue and continue the search
|
// add the block back to the queue and continue the search
|
||||||
parent_lookup.add_block(block);
|
parent_lookup.add_block(block);
|
||||||
self.request_parent(parent_lookup, cx);
|
self.request_parent(parent_lookup, cx);
|
||||||
}
|
}
|
||||||
Ok(_) | Err(BlockError::BlockIsAlreadyKnown { .. }) => {
|
BlockProcessResult::Ok
|
||||||
|
| BlockProcessResult::Err(BlockError::BlockIsAlreadyKnown { .. }) => {
|
||||||
let chain_hash = parent_lookup.chain_hash();
|
let chain_hash = parent_lookup.chain_hash();
|
||||||
let blocks = parent_lookup.chain_blocks();
|
let blocks = parent_lookup.chain_blocks();
|
||||||
let process_id = ChainSegmentProcessId::ParentLookup(chain_hash);
|
let process_id = ChainSegmentProcessId::ParentLookup(chain_hash);
|
||||||
@ -521,8 +544,10 @@ impl<T: BeaconChainTypes> BlockLookups<T> {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
Err(e @ BlockError::ExecutionPayloadError(ExecutionPayloadError::RequestFailed(_)))
|
BlockProcessResult::Err(
|
||||||
| Err(
|
e @ BlockError::ExecutionPayloadError(ExecutionPayloadError::RequestFailed(_)),
|
||||||
|
)
|
||||||
|
| BlockProcessResult::Err(
|
||||||
e @ BlockError::ExecutionPayloadError(ExecutionPayloadError::NoExecutionConnection),
|
e @ BlockError::ExecutionPayloadError(ExecutionPayloadError::NoExecutionConnection),
|
||||||
) => {
|
) => {
|
||||||
// These errors indicate that the execution layer is offline
|
// These errors indicate that the execution layer is offline
|
||||||
@ -534,7 +559,7 @@ impl<T: BeaconChainTypes> BlockLookups<T> {
|
|||||||
"error" => ?e
|
"error" => ?e
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
Err(outcome) => {
|
BlockProcessResult::Err(outcome) => {
|
||||||
// all else we consider the chain a failure and downvote the peer that sent
|
// all else we consider the chain a failure and downvote the peer that sent
|
||||||
// us the last block
|
// us the last block
|
||||||
warn!(
|
warn!(
|
||||||
@ -551,6 +576,15 @@ impl<T: BeaconChainTypes> BlockLookups<T> {
|
|||||||
// ambiguity.
|
// ambiguity.
|
||||||
cx.report_peer(peer_id, PeerAction::MidToleranceError, "parent_request_err");
|
cx.report_peer(peer_id, PeerAction::MidToleranceError, "parent_request_err");
|
||||||
}
|
}
|
||||||
|
BlockProcessResult::Ignored => {
|
||||||
|
// Beacon processor signalled to ignore the block processing result.
|
||||||
|
// This implies that the cpu is overloaded. Drop the request.
|
||||||
|
warn!(
|
||||||
|
self.log,
|
||||||
|
"Parent block processing was ignored, cpu might be overloaded";
|
||||||
|
"action" => "dropping parent request"
|
||||||
|
);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
metrics::set_gauge(
|
metrics::set_gauge(
|
||||||
|
@ -168,7 +168,7 @@ fn test_single_block_lookup_happy_path() {
|
|||||||
// Send the stream termination. Peer should have not been penalized, and the request removed
|
// Send the stream termination. Peer should have not been penalized, and the request removed
|
||||||
// after processing.
|
// after processing.
|
||||||
bl.single_block_lookup_response(id, peer_id, None, D, &mut cx);
|
bl.single_block_lookup_response(id, peer_id, None, D, &mut cx);
|
||||||
bl.single_block_processed(id, Ok(()), &mut cx);
|
bl.single_block_processed(id, Ok(()).into(), &mut cx);
|
||||||
rig.expect_empty_network();
|
rig.expect_empty_network();
|
||||||
assert_eq!(bl.single_block_lookups.len(), 0);
|
assert_eq!(bl.single_block_lookups.len(), 0);
|
||||||
}
|
}
|
||||||
@ -252,7 +252,11 @@ fn test_single_block_lookup_becomes_parent_request() {
|
|||||||
|
|
||||||
// Send the stream termination. Peer should have not been penalized, and the request moved to a
|
// Send the stream termination. Peer should have not been penalized, and the request moved to a
|
||||||
// parent request after processing.
|
// parent request after processing.
|
||||||
bl.single_block_processed(id, Err(BlockError::ParentUnknown(Arc::new(block))), &mut cx);
|
bl.single_block_processed(
|
||||||
|
id,
|
||||||
|
BlockError::ParentUnknown(Arc::new(block)).into(),
|
||||||
|
&mut cx,
|
||||||
|
);
|
||||||
assert_eq!(bl.single_block_lookups.len(), 0);
|
assert_eq!(bl.single_block_lookups.len(), 0);
|
||||||
rig.expect_parent_request();
|
rig.expect_parent_request();
|
||||||
rig.expect_empty_network();
|
rig.expect_empty_network();
|
||||||
@ -278,7 +282,7 @@ fn test_parent_lookup_happy_path() {
|
|||||||
rig.expect_empty_network();
|
rig.expect_empty_network();
|
||||||
|
|
||||||
// Processing succeeds, now the rest of the chain should be sent for processing.
|
// Processing succeeds, now the rest of the chain should be sent for processing.
|
||||||
bl.parent_block_processed(chain_hash, Err(BlockError::BlockIsAlreadyKnown), &mut cx);
|
bl.parent_block_processed(chain_hash, BlockError::BlockIsAlreadyKnown.into(), &mut cx);
|
||||||
rig.expect_parent_chain_process();
|
rig.expect_parent_chain_process();
|
||||||
bl.parent_chain_processed(chain_hash, BatchProcessResult::Success(true), &mut cx);
|
bl.parent_chain_processed(chain_hash, BatchProcessResult::Success(true), &mut cx);
|
||||||
assert_eq!(bl.parent_queue.len(), 0);
|
assert_eq!(bl.parent_queue.len(), 0);
|
||||||
@ -312,7 +316,7 @@ fn test_parent_lookup_wrong_response() {
|
|||||||
rig.expect_block_process();
|
rig.expect_block_process();
|
||||||
|
|
||||||
// Processing succeeds, now the rest of the chain should be sent for processing.
|
// Processing succeeds, now the rest of the chain should be sent for processing.
|
||||||
bl.parent_block_processed(chain_hash, Ok(()), &mut cx);
|
bl.parent_block_processed(chain_hash, Ok(()).into(), &mut cx);
|
||||||
rig.expect_parent_chain_process();
|
rig.expect_parent_chain_process();
|
||||||
bl.parent_chain_processed(chain_hash, BatchProcessResult::Success(true), &mut cx);
|
bl.parent_chain_processed(chain_hash, BatchProcessResult::Success(true), &mut cx);
|
||||||
assert_eq!(bl.parent_queue.len(), 0);
|
assert_eq!(bl.parent_queue.len(), 0);
|
||||||
@ -341,7 +345,7 @@ fn test_parent_lookup_empty_response() {
|
|||||||
rig.expect_block_process();
|
rig.expect_block_process();
|
||||||
|
|
||||||
// Processing succeeds, now the rest of the chain should be sent for processing.
|
// Processing succeeds, now the rest of the chain should be sent for processing.
|
||||||
bl.parent_block_processed(chain_hash, Ok(()), &mut cx);
|
bl.parent_block_processed(chain_hash, Ok(()).into(), &mut cx);
|
||||||
rig.expect_parent_chain_process();
|
rig.expect_parent_chain_process();
|
||||||
bl.parent_chain_processed(chain_hash, BatchProcessResult::Success(true), &mut cx);
|
bl.parent_chain_processed(chain_hash, BatchProcessResult::Success(true), &mut cx);
|
||||||
assert_eq!(bl.parent_queue.len(), 0);
|
assert_eq!(bl.parent_queue.len(), 0);
|
||||||
@ -369,7 +373,7 @@ fn test_parent_lookup_rpc_failure() {
|
|||||||
rig.expect_block_process();
|
rig.expect_block_process();
|
||||||
|
|
||||||
// Processing succeeds, now the rest of the chain should be sent for processing.
|
// Processing succeeds, now the rest of the chain should be sent for processing.
|
||||||
bl.parent_block_processed(chain_hash, Ok(()), &mut cx);
|
bl.parent_block_processed(chain_hash, Ok(()).into(), &mut cx);
|
||||||
rig.expect_parent_chain_process();
|
rig.expect_parent_chain_process();
|
||||||
bl.parent_chain_processed(chain_hash, BatchProcessResult::Success(true), &mut cx);
|
bl.parent_chain_processed(chain_hash, BatchProcessResult::Success(true), &mut cx);
|
||||||
assert_eq!(bl.parent_queue.len(), 0);
|
assert_eq!(bl.parent_queue.len(), 0);
|
||||||
@ -440,7 +444,7 @@ fn test_parent_lookup_too_deep() {
|
|||||||
// the processing result
|
// the processing result
|
||||||
bl.parent_block_processed(
|
bl.parent_block_processed(
|
||||||
chain_hash,
|
chain_hash,
|
||||||
Err(BlockError::ParentUnknown(Arc::new(block))),
|
BlockError::ParentUnknown(Arc::new(block)).into(),
|
||||||
&mut cx,
|
&mut cx,
|
||||||
)
|
)
|
||||||
}
|
}
|
||||||
@ -458,3 +462,56 @@ fn test_parent_lookup_disconnection() {
|
|||||||
bl.peer_disconnected(&peer_id, &mut cx);
|
bl.peer_disconnected(&peer_id, &mut cx);
|
||||||
assert!(bl.parent_queue.is_empty());
|
assert!(bl.parent_queue.is_empty());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
#[test]
|
||||||
|
fn test_single_block_lookup_ignored_response() {
|
||||||
|
let (mut bl, mut cx, mut rig) = TestRig::test_setup(None);
|
||||||
|
|
||||||
|
let block = rig.rand_block();
|
||||||
|
let peer_id = PeerId::random();
|
||||||
|
|
||||||
|
// Trigger the request
|
||||||
|
bl.search_block(block.canonical_root(), peer_id, &mut cx);
|
||||||
|
let id = rig.expect_block_request();
|
||||||
|
|
||||||
|
// The peer provides the correct block, should not be penalized. Now the block should be sent
|
||||||
|
// for processing.
|
||||||
|
bl.single_block_lookup_response(id, peer_id, Some(Arc::new(block)), D, &mut cx);
|
||||||
|
rig.expect_empty_network();
|
||||||
|
rig.expect_block_process();
|
||||||
|
|
||||||
|
// The request should still be active.
|
||||||
|
assert_eq!(bl.single_block_lookups.len(), 1);
|
||||||
|
|
||||||
|
// Send the stream termination. Peer should have not been penalized, and the request removed
|
||||||
|
// after processing.
|
||||||
|
bl.single_block_lookup_response(id, peer_id, None, D, &mut cx);
|
||||||
|
// Send an Ignored response, the request should be dropped
|
||||||
|
bl.single_block_processed(id, BlockProcessResult::Ignored, &mut cx);
|
||||||
|
rig.expect_empty_network();
|
||||||
|
assert_eq!(bl.single_block_lookups.len(), 0);
|
||||||
|
}
|
||||||
|
|
||||||
|
#[test]
|
||||||
|
fn test_parent_lookup_ignored_response() {
|
||||||
|
let (mut bl, mut cx, mut rig) = TestRig::test_setup(None);
|
||||||
|
|
||||||
|
let parent = rig.rand_block();
|
||||||
|
let block = rig.block_with_parent(parent.canonical_root());
|
||||||
|
let chain_hash = block.canonical_root();
|
||||||
|
let peer_id = PeerId::random();
|
||||||
|
|
||||||
|
// Trigger the request
|
||||||
|
bl.search_parent(Arc::new(block), peer_id, &mut cx);
|
||||||
|
let id = rig.expect_parent_request();
|
||||||
|
|
||||||
|
// Peer sends the right block, it should be sent for processing. Peer should not be penalized.
|
||||||
|
bl.parent_lookup_response(id, peer_id, Some(Arc::new(parent)), D, &mut cx);
|
||||||
|
rig.expect_block_process();
|
||||||
|
rig.expect_empty_network();
|
||||||
|
|
||||||
|
// Return an Ignored result. The request should be dropped
|
||||||
|
bl.parent_block_processed(chain_hash, BlockProcessResult::Ignored, &mut cx);
|
||||||
|
rig.expect_empty_network();
|
||||||
|
assert_eq!(bl.parent_queue.len(), 0);
|
||||||
|
}
|
||||||
|
@ -117,7 +117,7 @@ pub enum SyncMessage<T: EthSpec> {
|
|||||||
/// Block processed
|
/// Block processed
|
||||||
BlockProcessed {
|
BlockProcessed {
|
||||||
process_type: BlockProcessType,
|
process_type: BlockProcessType,
|
||||||
result: Result<(), BlockError<T>>,
|
result: BlockProcessResult<T>,
|
||||||
},
|
},
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -128,6 +128,13 @@ pub enum BlockProcessType {
|
|||||||
ParentLookup { chain_hash: Hash256 },
|
ParentLookup { chain_hash: Hash256 },
|
||||||
}
|
}
|
||||||
|
|
||||||
|
#[derive(Debug)]
|
||||||
|
pub enum BlockProcessResult<T: EthSpec> {
|
||||||
|
Ok,
|
||||||
|
Err(BlockError<T>),
|
||||||
|
Ignored,
|
||||||
|
}
|
||||||
|
|
||||||
/// The result of processing multiple blocks (a chain segment).
|
/// The result of processing multiple blocks (a chain segment).
|
||||||
#[derive(Debug)]
|
#[derive(Debug)]
|
||||||
pub enum BatchProcessResult {
|
pub enum BatchProcessResult {
|
||||||
@ -620,3 +627,18 @@ impl<T: BeaconChainTypes> SyncManager<T> {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
impl<IgnoredOkVal, T: EthSpec> From<Result<IgnoredOkVal, BlockError<T>>> for BlockProcessResult<T> {
|
||||||
|
fn from(result: Result<IgnoredOkVal, BlockError<T>>) -> Self {
|
||||||
|
match result {
|
||||||
|
Ok(_) => BlockProcessResult::Ok,
|
||||||
|
Err(e) => e.into(),
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
impl<T: EthSpec> From<BlockError<T>> for BlockProcessResult<T> {
|
||||||
|
fn from(e: BlockError<T>) -> Self {
|
||||||
|
BlockProcessResult::Err(e)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
Loading…
Reference in New Issue
Block a user