Handle early blocks (#2155)

## Issue Addressed

NA

## Problem this PR addresses

There's an issue where Lighthouse is banning a lot of peers due to the following sequence of events:

1. Gossip block 0xabc arrives ~200ms early
    - It is propagated across the network, with respect to [`MAXIMUM_GOSSIP_CLOCK_DISPARITY`](https://github.com/ethereum/eth2.0-specs/blob/v1.0.0/specs/phase0/p2p-interface.md#why-is-there-maximum_gossip_clock_disparity-when-validating-slot-ranges-of-messages-in-gossip-subnets).
    - However, it is not imported to our database since the block is early.
2. Attestations for 0xabc arrive, but the block was not imported.
    - The peer that sent the attestation is down-voted.
        - Each unknown-block attestation causes a score loss of 1, the peer is banned at -100.
        - When the peer is on an attestation subnet there can be hundreds of attestations, so the peer is banned quickly (before the missed block can be obtained via rpc).

## Potential solutions

I can think of three solutions to this:

1. Wait for attestation-queuing (#635) to arrive and solve this.
    - Easy
    - Not immediate fix.
    - Whilst this would work, I don't think it's a perfect solution for this particular issue, rather (3) is better.
1. Allow importing blocks with a tolerance of `MAXIMUM_GOSSIP_CLOCK_DISPARITY`.
    - Easy
    - ~~I have implemented this, for now.~~
1. If a block is verified for gossip propagation (i.e., signature verified) and it's within `MAXIMUM_GOSSIP_CLOCK_DISPARITY`, then queue it to be processed at the start of the appropriate slot.
    - More difficult
    - Feels like the best solution, I will try to implement this.
    
    
**This PR takes approach (3).**

## Changes included

- Implement the `block_delay_queue`, based upon a [`DelayQueue`](https://docs.rs/tokio-util/0.6.3/tokio_util/time/delay_queue/struct.DelayQueue.html) which can store blocks until it's time to import them.
- Add a new `DelayedImportBlock` variant to the `beacon_processor::WorkEvent` enum to handle this new event.
- In the `BeaconProcessor`, refactor a `tokio::select!` to a struct with an explicit `Stream` implementation. I experienced some issues with `tokio::select!` in the block delay queue and I also found it hard to debug. I think this explicit implementation is nicer and functionally equivalent (apart from the fact that `tokio::select!` randomly chooses futures to poll, whereas now we're deterministic).
- Add a testing framework to the `beacon_processor` module that tests this new block delay logic. I also tested a handful of other operations in the beacon processor (attns, slashings, exits) since it was super easy to copy-pasta the code from the `http_api` tester.
    - To implement these tests I added the concept of an optional `work_journal_tx` to the `BeaconProcessor` which will spit out a log of events. I used this in the tests to ensure that things were happening as I expect.
    - The tests are a little racey, but it's hard to avoid that when testing timing-based code. If we see CI failures I can revise. I haven't observed *any* failures due to races on my machine or on CI yet.
    - To assist with testing I allowed for directly setting the time on the `ManualSlotClock`.
- I gave the `beacon_processor::Worker` a `Toolbox` for two reasons; (a) it avoids changing tons of function sigs when you want to pass a new object to the worker and (b) it seemed cute.
This commit is contained in:
Paul Hauner 2021-02-24 03:08:52 +00:00
parent 399d073ab4
commit a764c3b247
19 changed files with 1161 additions and 115 deletions

53
Cargo.lock generated
View File

@ -1607,6 +1607,38 @@ version = "1.0.4"
source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "212d0f5754cb6769937f4501cc0e67f4f4483c8d2c3e1e922ee9edbe4ab4c7c0"
[[package]]
name = "discv5"
version = "0.1.0-beta.3"
source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "f6f52d2228d51e8f868a37d5b5b25b82c13552b635d5b47c3a5d53855a6fc4f0"
dependencies = [
"aes-ctr",
"aes-gcm 0.8.0",
"arrayvec",
"digest 0.9.0",
"enr",
"fnv",
"futures 0.3.12",
"hex",
"hkdf",
"k256",
"lazy_static",
"lru_time_cache",
"parking_lot",
"rand 0.7.3",
"rlp 0.5.0",
"sha2 0.9.2",
"smallvec",
"tokio 1.1.0",
"tokio-stream",
"tokio-util 0.6.3",
"tracing",
"tracing-subscriber",
"uint",
"zeroize",
]
[[package]]
name = "discv5"
version = "0.1.0-beta.3"
@ -1775,9 +1807,9 @@ dependencies = [
[[package]]
name = "env_logger"
version = "0.8.2"
version = "0.8.3"
source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "f26ecb66b4bdca6c1409b40fb255eefc2bd4f6d135dab3c3124f80ffa2a9661e"
checksum = "17392a012ea30ef05a610aa97dfb49496e71c9f676b27879922ea5bdf60d9d3f"
dependencies = [
"atty",
"humantime",
@ -1966,7 +1998,7 @@ dependencies = [
"base64 0.13.0",
"directory",
"dirs 3.0.1",
"discv5",
"discv5 0.1.0-beta.3 (git+https://github.com/sigp/discv5?rev=02d2c896c66f8dc2b848c3996fedcd98e1dfec69)",
"error-chain",
"eth2_ssz",
"eth2_ssz_derive",
@ -2797,7 +2829,7 @@ version = "0.1.0"
dependencies = [
"beacon_chain",
"bs58 0.4.0",
"discv5",
"discv5 0.1.0-beta.3 (git+https://github.com/sigp/discv5?rev=02d2c896c66f8dc2b848c3996fedcd98e1dfec69)",
"environment",
"eth1",
"eth2",
@ -3657,7 +3689,7 @@ dependencies = [
"clap",
"clap_utils",
"directory",
"env_logger 0.8.2",
"env_logger 0.8.3",
"environment",
"eth2_network_config",
"futures 0.3.12",
@ -4069,6 +4101,8 @@ name = "network"
version = "0.2.0"
dependencies = [
"beacon_chain",
"discv5 0.1.0-beta.3 (registry+https://github.com/rust-lang/crates.io-index)",
"environment",
"error-chain",
"eth2_libp2p",
"eth2_ssz",
@ -4104,6 +4138,7 @@ dependencies = [
"tempfile",
"tokio 1.1.0",
"tokio-stream",
"tokio-util 0.6.3",
"tree_hash",
"types",
]
@ -5651,7 +5686,7 @@ name = "simulator"
version = "0.2.0"
dependencies = [
"clap",
"env_logger 0.8.2",
"env_logger 0.8.3",
"eth1",
"eth1_test_rig",
"futures 0.3.12",
@ -5942,7 +5977,7 @@ dependencies = [
"arbitrary",
"bls",
"criterion",
"env_logger 0.8.2",
"env_logger 0.8.3",
"eth2_hashing",
"eth2_ssz",
"eth2_ssz_types",
@ -6509,9 +6544,9 @@ dependencies = [
[[package]]
name = "tokio-stream"
version = "0.1.2"
version = "0.1.3"
source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "76066865172052eb8796c686f0b441a93df8b08d40a950b062ffb9a426f00edd"
checksum = "1981ad97df782ab506a1f43bf82c967326960d278acf3bf8279809648c3ff3ea"
dependencies = [
"futures-core",
"pin-project-lite 0.2.4",

View File

@ -385,6 +385,7 @@ pub fn signature_verify_chain_segment<T: BeaconChainTypes>(
/// A wrapper around a `SignedBeaconBlock` that indicates it has been approved for re-gossiping on
/// the p2p network.
#[derive(Debug)]
pub struct GossipVerifiedBlock<T: BeaconChainTypes> {
pub block: SignedBeaconBlock<T::EthSpec>,
pub block_root: Hash256,

View File

@ -30,7 +30,7 @@ mod validator_pubkey_cache;
pub use self::beacon_chain::{
AttestationProcessingOutcome, BeaconChain, BeaconChainTypes, ChainSegmentResult,
ForkChoiceError, StateSkipConfig,
ForkChoiceError, StateSkipConfig, MAXIMUM_GOSSIP_CLOCK_DISPARITY,
};
pub use self::beacon_snapshot::BeaconSnapshot;
pub use self::chain_config::ChainConfig;

View File

@ -228,6 +228,11 @@ impl<E: EthSpec> NaiveAggregationPool<E> {
outcome
}
/// Returns the total number of attestations stored in `self`.
pub fn num_attestations(&self) -> usize {
self.maps.iter().map(|(_, map)| map.len()).sum()
}
/// Returns an aggregated `Attestation` with the given `data`, if any.
pub fn get(&self, data: &AttestationData) -> Option<Attestation<E>> {
self.maps.get(&data.slot).and_then(|map| map.get(data))

View File

@ -8,6 +8,7 @@ use types::{
pub const DEFAULT_SNAPSHOT_CACHE_SIZE: usize = 4;
/// This snapshot is to be used for verifying a child of `self.beacon_block`.
#[derive(Debug)]
pub struct PreProcessingSnapshot<T: EthSpec> {
/// This state is equivalent to the `self.beacon_block.state_root()` state that has been
/// advanced forward one slot using `per_slot_processing`. This state is "primed and ready" for

View File

@ -14,6 +14,8 @@ exit-future = "0.2.0"
slog-term = "2.6.0"
slog-async = "2.5.0"
logging = { path = "../../common/logging" }
environment = { path = "../../lighthouse/environment" }
discv5 = { version = "0.1.0-beta.3" }
[dependencies]
beacon_chain = { path = "../beacon_chain" }
@ -31,7 +33,7 @@ tree_hash = "0.1.1"
futures = "0.3.7"
error-chain = "0.12.4"
tokio = { version = "1.1.0", features = ["full"] }
tokio-stream = "0.1.2"
tokio-stream = "0.1.3"
parking_lot = "0.11.0"
smallvec = "1.6.1"
rand = "0.7.3"
@ -46,3 +48,4 @@ num_cpus = "1.13.0"
lru_cache = { path = "../../common/lru_cache" }
if-addrs = "0.6.4"
strum = { version = "0.20"}
tokio-util = { version = "0.6.3", features = ["time"] }

View File

@ -0,0 +1,210 @@
//! Provides a mechanism which queues blocks for later processing when they arrive too early.
//!
//! 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
//! `DelayQueue` until the slot arrives. Once the block has been determined to be ready, it will be
//! sent back out on a channel to be processed by the `BeaconProcessor` again.
//!
//! 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};
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::pin::Pin;
use std::task::Context;
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;
const TASK_NAME: &str = "beacon_processor_block_delay_queue";
/// 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);
/// 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;
/// A block that arrived early and has been queued for later import.
pub struct QueuedBlock<T: BeaconChainTypes> {
pub peer_id: PeerId,
pub block: GossipVerifiedBlock<T>,
pub seen_timestamp: Duration,
}
/// Unifies the different messages processed by the block delay queue.
enum InboundEvent<T: BeaconChainTypes> {
/// A block that has been received early that we should queue for later processing.
EarlyBlock(QueuedBlock<T>),
/// A block that was queued for later processing and is ready for import.
ReadyBlock(QueuedBlock<T>),
/// The `DelayQueue` returned an error.
DelayQueueError(TimeError),
}
/// 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
/// control (specifically in the ordering of event processing).
struct InboundEvents<T: BeaconChainTypes> {
pub delay_queue: DelayQueue<QueuedBlock<T>>,
early_blocks_rx: Receiver<QueuedBlock<T>>,
}
impl<T: BeaconChainTypes> Stream for InboundEvents<T> {
type Item = InboundEvent<T>;
fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll<Option<Self::Item>> {
// 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) {
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.early_blocks_rx.poll_recv(cx) {
Poll::Ready(Some(queued_block)) => {
return Poll::Ready(Some(InboundEvent::EarlyBlock(queued_block)));
}
Poll::Ready(None) => {
return Poll::Ready(None);
}
Poll::Pending => {}
}
Poll::Pending
}
}
/// 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<T: BeaconChainTypes>(
ready_blocks_tx: Sender<QueuedBlock<T>>,
executor: &TaskExecutor,
slot_clock: T::SlotClock,
log: Logger,
) -> Sender<QueuedBlock<T>> {
let (early_blocks_tx, early_blocks_rx): (_, Receiver<QueuedBlock<_>>) =
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) {
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
.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";
);
}
}
}
}
// 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";
);
}
}
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;
}
}
}
};
executor.spawn(queue_future, TASK_NAME);
early_blocks_tx
}

View File

@ -36,24 +36,32 @@
//! task.
use crate::{metrics, service::NetworkMessage, sync::SyncMessage};
use beacon_chain::{BeaconChain, BeaconChainTypes, BlockError};
use beacon_chain::{BeaconChain, BeaconChainTypes, BlockError, GossipVerifiedBlock};
use block_delay_queue::{spawn_block_delay_queue, QueuedBlock};
use eth2_libp2p::{
rpc::{BlocksByRangeRequest, BlocksByRootRequest, StatusMessage},
MessageId, NetworkGlobals, PeerId, PeerRequestId,
};
use slog::{crit, debug, error, trace, warn, Logger};
use futures::stream::{Stream, StreamExt};
use futures::task::Poll;
use slog::{debug, error, trace, warn, Logger};
use std::collections::VecDeque;
use std::fmt;
use std::pin::Pin;
use std::sync::{Arc, Weak};
use std::task::Context;
use std::time::{Duration, Instant};
use task_executor::TaskExecutor;
use tokio::sync::{mpsc, oneshot};
use types::{
Attestation, AttesterSlashing, EthSpec, Hash256, ProposerSlashing, SignedAggregateAndProof,
Attestation, AttesterSlashing, Hash256, ProposerSlashing, SignedAggregateAndProof,
SignedBeaconBlock, SignedVoluntaryExit, SubnetId,
};
use worker::Worker;
use worker::{Toolbox, Worker};
mod block_delay_queue;
mod tests;
mod worker;
pub use worker::ProcessId;
@ -81,6 +89,10 @@ const MAX_AGGREGATED_ATTESTATION_QUEUE_LEN: usize = 1_024;
/// before we start dropping them.
const MAX_GOSSIP_BLOCK_QUEUE_LEN: usize = 1_024;
/// The maximum number of queued `SignedBeaconBlock` objects received prior to their slot (but
/// within acceptable clock disparity) that will be queued before we start dropping them.
const MAX_DELAYED_BLOCK_QUEUE_LEN: usize = 1_024;
/// The maximum number of queued `SignedVoluntaryExit` objects received on gossip that will be stored
/// before we start dropping them.
const MAX_GOSSIP_EXIT_QUEUE_LEN: usize = 4_096;
@ -121,6 +133,22 @@ const WORKER_TASK_NAME: &str = "beacon_processor_worker";
/// The minimum interval between log messages indicating that a queue is full.
const LOG_DEBOUNCE_INTERVAL: Duration = Duration::from_secs(30);
/// Unique IDs used for metrics and testing.
pub const WORKER_FREED: &str = "worker_freed";
pub const NOTHING_TO_DO: &str = "nothing_to_do";
pub const GOSSIP_ATTESTATION: &str = "gossip_attestation";
pub const GOSSIP_AGGREGATE: &str = "gossip_aggregate";
pub const GOSSIP_BLOCK: &str = "gossip_block";
pub const DELAYED_IMPORT_BLOCK: &str = "delayed_import_block";
pub const GOSSIP_VOLUNTARY_EXIT: &str = "gossip_voluntary_exit";
pub const GOSSIP_PROPOSER_SLASHING: &str = "gossip_proposer_slashing";
pub const GOSSIP_ATTESTER_SLASHING: &str = "gossip_attester_slashing";
pub const RPC_BLOCK: &str = "rpc_block";
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";
/// Used to send/receive results from a rpc block import in a blocking task.
pub type BlockResultSender<E> = oneshot::Sender<Result<Hash256, BlockError<E>>>;
pub type BlockResultReceiver<E> = oneshot::Receiver<Result<Hash256, BlockError<E>>>;
@ -210,18 +238,23 @@ impl<T> LifoQueue<T> {
}
/// An event to be processed by the manager task.
#[derive(Debug)]
pub struct WorkEvent<E: EthSpec> {
pub struct WorkEvent<T: BeaconChainTypes> {
drop_during_sync: bool,
work: Work<E>,
work: Work<T>,
}
impl<E: EthSpec> WorkEvent<E> {
impl<T: BeaconChainTypes> fmt::Debug for WorkEvent<T> {
fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
write!(f, "{:?}", self)
}
}
impl<T: BeaconChainTypes> WorkEvent<T> {
/// Create a new `Work` event for some unaggregated attestation.
pub fn unaggregated_attestation(
message_id: MessageId,
peer_id: PeerId,
attestation: Attestation<E>,
attestation: Attestation<T::EthSpec>,
subnet_id: SubnetId,
should_import: bool,
seen_timestamp: Duration,
@ -243,7 +276,7 @@ impl<E: EthSpec> WorkEvent<E> {
pub fn aggregated_attestation(
message_id: MessageId,
peer_id: PeerId,
aggregate: SignedAggregateAndProof<E>,
aggregate: SignedAggregateAndProof<T::EthSpec>,
seen_timestamp: Duration,
) -> Self {
Self {
@ -261,7 +294,7 @@ impl<E: EthSpec> WorkEvent<E> {
pub fn gossip_beacon_block(
message_id: MessageId,
peer_id: PeerId,
block: Box<SignedBeaconBlock<E>>,
block: Box<SignedBeaconBlock<T::EthSpec>>,
seen_timestamp: Duration,
) -> Self {
Self {
@ -275,6 +308,22 @@ impl<E: EthSpec> WorkEvent<E> {
}
}
/// 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<GossipVerifiedBlock<T>>,
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,
@ -311,7 +360,7 @@ impl<E: EthSpec> WorkEvent<E> {
pub fn gossip_attester_slashing(
message_id: MessageId,
peer_id: PeerId,
attester_slashing: Box<AttesterSlashing<E>>,
attester_slashing: Box<AttesterSlashing<T::EthSpec>>,
) -> Self {
Self {
drop_during_sync: false,
@ -325,7 +374,9 @@ impl<E: EthSpec> WorkEvent<E> {
/// Create a new `Work` event for some block, where the result from computation (if any) is
/// sent to the other side of `result_tx`.
pub fn rpc_beacon_block(block: Box<SignedBeaconBlock<E>>) -> (Self, BlockResultReceiver<E>) {
pub fn rpc_beacon_block(
block: Box<SignedBeaconBlock<T::EthSpec>>,
) -> (Self, BlockResultReceiver<T::EthSpec>) {
let (result_tx, result_rx) = oneshot::channel();
let event = Self {
drop_during_sync: false,
@ -335,7 +386,10 @@ impl<E: EthSpec> WorkEvent<E> {
}
/// Create a new work event to import `blocks` as a beacon chain segment.
pub fn chain_segment(process_id: ProcessId, blocks: Vec<SignedBeaconBlock<E>>) -> Self {
pub fn chain_segment(
process_id: ProcessId,
blocks: Vec<SignedBeaconBlock<T::EthSpec>>,
) -> Self {
Self {
drop_during_sync: false,
work: Work::ChainSegment { process_id, blocks },
@ -390,11 +444,11 @@ impl<E: EthSpec> WorkEvent<E> {
/// A consensus message (or multiple) from the network that requires processing.
#[derive(Debug)]
pub enum Work<E: EthSpec> {
pub enum Work<T: BeaconChainTypes> {
GossipAttestation {
message_id: MessageId,
peer_id: PeerId,
attestation: Box<Attestation<E>>,
attestation: Box<Attestation<T::EthSpec>>,
subnet_id: SubnetId,
should_import: bool,
seen_timestamp: Duration,
@ -402,13 +456,18 @@ pub enum Work<E: EthSpec> {
GossipAggregate {
message_id: MessageId,
peer_id: PeerId,
aggregate: Box<SignedAggregateAndProof<E>>,
aggregate: Box<SignedAggregateAndProof<T::EthSpec>>,
seen_timestamp: Duration,
},
GossipBlock {
message_id: MessageId,
peer_id: PeerId,
block: Box<SignedBeaconBlock<E>>,
block: Box<SignedBeaconBlock<T::EthSpec>>,
seen_timestamp: Duration,
},
DelayedImportBlock {
peer_id: PeerId,
block: Box<GossipVerifiedBlock<T>>,
seen_timestamp: Duration,
},
GossipVoluntaryExit {
@ -424,15 +483,15 @@ pub enum Work<E: EthSpec> {
GossipAttesterSlashing {
message_id: MessageId,
peer_id: PeerId,
attester_slashing: Box<AttesterSlashing<E>>,
attester_slashing: Box<AttesterSlashing<T::EthSpec>>,
},
RpcBlock {
block: Box<SignedBeaconBlock<E>>,
result_tx: BlockResultSender<E>,
block: Box<SignedBeaconBlock<T::EthSpec>>,
result_tx: BlockResultSender<T::EthSpec>,
},
ChainSegment {
process_id: ProcessId,
blocks: Vec<SignedBeaconBlock<E>>,
blocks: Vec<SignedBeaconBlock<T::EthSpec>>,
},
Status {
peer_id: PeerId,
@ -450,21 +509,22 @@ pub enum Work<E: EthSpec> {
},
}
impl<E: EthSpec> Work<E> {
impl<T: BeaconChainTypes> Work<T> {
/// Provides a `&str` that uniquely identifies each enum variant.
fn str_id(&self) -> &'static str {
match self {
Work::GossipAttestation { .. } => "gossip_attestation",
Work::GossipAggregate { .. } => "gossip_aggregate",
Work::GossipBlock { .. } => "gossip_block",
Work::GossipVoluntaryExit { .. } => "gossip_voluntary_exit",
Work::GossipProposerSlashing { .. } => "gossip_proposer_slashing",
Work::GossipAttesterSlashing { .. } => "gossip_attester_slashing",
Work::RpcBlock { .. } => "rpc_block",
Work::ChainSegment { .. } => "chain_segment",
Work::Status { .. } => "status_processing",
Work::BlocksByRangeRequest { .. } => "blocks_by_range_request",
Work::BlocksByRootsRequest { .. } => "blocks_by_roots_request",
Work::GossipAttestation { .. } => GOSSIP_ATTESTATION,
Work::GossipAggregate { .. } => GOSSIP_AGGREGATE,
Work::GossipBlock { .. } => GOSSIP_BLOCK,
Work::DelayedImportBlock { .. } => DELAYED_IMPORT_BLOCK,
Work::GossipVoluntaryExit { .. } => GOSSIP_VOLUNTARY_EXIT,
Work::GossipProposerSlashing { .. } => GOSSIP_PROPOSER_SLASHING,
Work::GossipAttesterSlashing { .. } => GOSSIP_ATTESTER_SLASHING,
Work::RpcBlock { .. } => RPC_BLOCK,
Work::ChainSegment { .. } => CHAIN_SEGMENT,
Work::Status { .. } => STATUS_PROCESSING,
Work::BlocksByRangeRequest { .. } => BLOCKS_BY_RANGE_REQUEST,
Work::BlocksByRootsRequest { .. } => BLOCKS_BY_ROOTS_REQUEST,
}
}
}
@ -488,6 +548,71 @@ impl TimeLatch {
}
}
/// Unifies all the messages processed by the `BeaconProcessor`.
enum InboundEvent<T: BeaconChainTypes> {
/// A worker has completed a task and is free.
WorkerIdle,
/// There is new work to be done.
WorkEvent(WorkEvent<T>),
/// A block that was delayed for import at a later slot has become ready.
QueuedBlock(Box<QueuedBlock<T>>),
}
/// Combines the various incoming event streams for the `BeaconProcessor` into a single stream.
///
/// This 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<T: BeaconChainTypes> {
/// Used by workers when they finish a task.
idle_rx: mpsc::Receiver<()>,
/// Used by upstream processes to send new work to the `BeaconProcessor`.
event_rx: mpsc::Receiver<WorkEvent<T>>,
/// Used internally for queuing blocks for processing once their slot arrives.
post_delay_block_queue_rx: mpsc::Receiver<QueuedBlock<T>>,
}
impl<T: BeaconChainTypes> Stream for InboundEvents<T> {
type Item = InboundEvent<T>;
fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll<Option<Self::Item>> {
// Always check for idle workers before anything else. This allows us to ensure that a big
// stream of new events doesn't suppress the processing of existing events.
match self.idle_rx.poll_recv(cx) {
Poll::Ready(Some(())) => {
return Poll::Ready(Some(InboundEvent::WorkerIdle));
}
Poll::Ready(None) => {
return Poll::Ready(None);
}
Poll::Pending => {}
}
// 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))));
}
Poll::Ready(None) => {
return Poll::Ready(None);
}
Poll::Pending => {}
}
match self.event_rx.poll_recv(cx) {
Poll::Ready(Some(event)) => {
return Poll::Ready(Some(InboundEvent::WorkEvent(event)));
}
Poll::Ready(None) => {
return Poll::Ready(None);
}
Poll::Pending => {}
}
Poll::Pending
}
}
/// A mutli-threaded processor for messages received on the network
/// that need to be processed by the `BeaconChain`
///
@ -512,8 +637,16 @@ impl<T: BeaconChainTypes> BeaconProcessor<T> {
///
/// Only `self.max_workers` will ever be spawned at one time. Each worker is a `tokio` task
/// started with `spawn_blocking`.
pub fn spawn_manager(mut self, mut event_rx: mpsc::Receiver<WorkEvent<T::EthSpec>>) {
let (idle_tx, mut idle_rx) = mpsc::channel::<()>(MAX_IDLE_QUEUE_LEN);
///
/// The optional `work_journal_tx` allows for an outside process to receive a log of all work
/// events processed by `self`. This should only be used during testing.
pub fn spawn_manager(
mut self,
event_rx: mpsc::Receiver<WorkEvent<T>>,
work_journal_tx: Option<mpsc::Sender<String>>,
) {
// Used by workers to communicate that they are finished a task.
let (idle_tx, idle_rx) = mpsc::channel::<()>(MAX_IDLE_QUEUE_LEN);
// Using LIFO queues for attestations since validator profits rely upon getting fresh
// attestations into blocks. Additionally, later attestations contain more information than
@ -538,55 +671,63 @@ impl<T: BeaconChainTypes> BeaconProcessor<T> {
let mut rpc_block_queue = FifoQueue::new(MAX_RPC_BLOCK_QUEUE_LEN);
let mut chain_segment_queue = FifoQueue::new(MAX_CHAIN_SEGMENT_QUEUE_LEN);
let mut gossip_block_queue = FifoQueue::new(MAX_GOSSIP_BLOCK_QUEUE_LEN);
let mut delayed_block_queue = FifoQueue::new(MAX_DELAYED_BLOCK_QUEUE_LEN);
let mut status_queue = FifoQueue::new(MAX_STATUS_QUEUE_LEN);
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 = {
if let Some(chain) = self.beacon_chain.upgrade() {
spawn_block_delay_queue(
post_delay_block_queue_tx,
&self.executor,
chain.slot_clock.clone(),
self.log.clone(),
)
} else {
// No need to proceed any further if the beacon chain has been dropped, the client
// is shutting down.
return;
}
};
let executor = self.executor.clone();
// The manager future will run on the core executor and delegate tasks to worker
// threads on the blocking executor.
let manager_future = async move {
let mut inbound_events = InboundEvents {
idle_rx,
event_rx,
post_delay_block_queue_rx,
};
loop {
// Listen to both the event and idle channels, acting on whichever is ready
// first.
//
// Set `work_event = Some(event)` if there is new work to be done. Otherwise sets
// `event = None` if it was a worker becoming idle.
let work_event = tokio::select! {
// A worker has finished some work.
new_idle_opt = idle_rx.recv() => {
if new_idle_opt.is_some() {
self.current_workers = self.current_workers.saturating_sub(1);
None
} else {
// Exit if all idle senders have been dropped.
//
// This shouldn't happen since this function holds a sender.
crit!(
self.log,
"Gossip processor stopped";
"msg" => "all idle senders dropped"
);
break
}
},
// There is a new piece of work to be handled.
new_work_event_opt = event_rx.recv() => {
if let Some(new_work_event) = new_work_event_opt {
Some(new_work_event)
} else {
// Exit if all event senders have been dropped.
//
// This should happen when the client shuts down.
debug!(
self.log,
"Gossip processor stopped";
"msg" => "all event senders dropped"
);
break
}
let work_event = match inbound_events.next().await {
Some(InboundEvent::WorkerIdle) => {
self.current_workers = self.current_workers.saturating_sub(1);
None
}
Some(InboundEvent::WorkEvent(event)) => Some(event),
Some(InboundEvent::QueuedBlock(queued_block)) => {
Some(WorkEvent::delayed_import_beacon_block(
queued_block.peer_id,
Box::new(queued_block.block),
queued_block.seen_timestamp,
))
}
None => {
debug!(
self.log,
"Gossip processor stopped";
"msg" => "stream ended"
);
break;
}
};
@ -601,6 +742,17 @@ impl<T: BeaconChainTypes> BeaconProcessor<T> {
metrics::inc_counter(&metrics::BEACON_PROCESSOR_IDLE_EVENTS_TOTAL);
}
if let Some(work_journal_tx) = &work_journal_tx {
let id = work_event
.as_ref()
.map(|event| event.work.str_id())
.unwrap_or(WORKER_FREED);
// 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 can_spawn = self.current_workers < self.max_workers;
let drop_during_sync = work_event
.as_ref()
@ -612,46 +764,64 @@ impl<T: BeaconChainTypes> BeaconProcessor<T> {
// We don't check the `work.drop_during_sync` here. We assume that if it made
// it into the queue at any point then we should process it.
None if can_spawn => {
let toolbox = Toolbox {
idle_tx: idle_tx.clone(),
delayed_block_tx: pre_delay_block_queue_tx.clone(),
};
// Check for chain segments first, they're the most efficient way to get
// blocks into the system.
if let Some(item) = chain_segment_queue.pop() {
self.spawn_worker(idle_tx.clone(), item);
self.spawn_worker(item, toolbox);
// Check sync blocks before gossip blocks, since we've already explicitly
// requested these blocks.
} else if let Some(item) = rpc_block_queue.pop() {
self.spawn_worker(idle_tx.clone(), item);
self.spawn_worker(item, toolbox);
// Check delayed blocks before gossip blocks, the gossip blocks might rely
// on the delayed ones.
} else if let Some(item) = delayed_block_queue.pop() {
self.spawn_worker(item, toolbox);
// Check gossip blocks before gossip attestations, since a block might be
// required to verify some attestations.
} else if let Some(item) = gossip_block_queue.pop() {
self.spawn_worker(idle_tx.clone(), item);
self.spawn_worker(item, toolbox);
// Check the aggregates, *then* the unaggregates since we assume that
// aggregates are more valuable to local validators and effectively give us
// more information with less signature verification time.
} else if let Some(item) = aggregate_queue.pop() {
self.spawn_worker(idle_tx.clone(), item);
self.spawn_worker(item, toolbox);
} else if let Some(item) = attestation_queue.pop() {
self.spawn_worker(idle_tx.clone(), item);
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)
} else if let Some(item) = status_queue.pop() {
self.spawn_worker(idle_tx.clone(), item);
self.spawn_worker(item, toolbox);
} else if let Some(item) = bbrange_queue.pop() {
self.spawn_worker(idle_tx.clone(), item);
self.spawn_worker(item, toolbox);
} else if let Some(item) = bbroots_queue.pop() {
self.spawn_worker(idle_tx.clone(), item);
self.spawn_worker(item, toolbox);
// Check slashings after all other consensus messages so we prioritize
// following head.
//
// Check attester slashings before proposer slashings since they have the
// potential to slash multiple validators at once.
} else if let Some(item) = gossip_attester_slashing_queue.pop() {
self.spawn_worker(idle_tx.clone(), item);
self.spawn_worker(item, toolbox);
} else if let Some(item) = gossip_proposer_slashing_queue.pop() {
self.spawn_worker(idle_tx.clone(), item);
self.spawn_worker(item, toolbox);
// Check exits last since our validators don't get rewards from them.
} else if let Some(item) = gossip_voluntary_exit_queue.pop() {
self.spawn_worker(idle_tx.clone(), item);
self.spawn_worker(item, toolbox);
// This statement should always be the final else statement.
} else {
// Let the journal know that a worker is freed and there's nothing else
// for it to do.
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());
}
}
}
// There is no new work event and we are unable to spawn a new worker.
@ -681,16 +851,25 @@ impl<T: BeaconChainTypes> BeaconProcessor<T> {
"work_id" => work_id
);
}
// There is a new work event and the chain is not syncing. Process it.
// There is a new work event and the chain is not syncing. Process it or queue
// it.
Some(WorkEvent { work, .. }) => {
let work_id = work.str_id();
let toolbox = Toolbox {
idle_tx: idle_tx.clone(),
delayed_block_tx: pre_delay_block_queue_tx.clone(),
};
match work {
_ if can_spawn => self.spawn_worker(idle_tx.clone(), work),
_ if can_spawn => self.spawn_worker(work, toolbox),
Work::GossipAttestation { .. } => attestation_queue.push(work),
Work::GossipAggregate { .. } => aggregate_queue.push(work),
Work::GossipBlock { .. } => {
gossip_block_queue.push(work, work_id, &self.log)
}
Work::DelayedImportBlock { .. } => {
delayed_block_queue.push(work, work_id, &self.log)
}
Work::GossipVoluntaryExit { .. } => {
gossip_voluntary_exit_queue.push(work, work_id, &self.log)
}
@ -779,7 +958,10 @@ impl<T: BeaconChainTypes> BeaconProcessor<T> {
/// Spawns a blocking worker thread to process some `Work`.
///
/// Sends an message on `idle_tx` when the work is complete and the task is stopping.
fn spawn_worker(&mut self, idle_tx: mpsc::Sender<()>, work: Work<T::EthSpec>) {
fn spawn_worker(&mut self, work: Work<T>, toolbox: Toolbox<T>) {
let idle_tx = toolbox.idle_tx;
let delayed_block_tx = toolbox.delayed_block_tx;
// Wrap the `idle_tx` in a struct that will fire the idle message whenever it is dropped.
//
// This helps ensure that the worker is always freed in the case of an early exit or panic.
@ -873,7 +1055,21 @@ impl<T: BeaconChainTypes> BeaconProcessor<T> {
peer_id,
block,
seen_timestamp,
} => worker.process_gossip_block(message_id, peer_id, *block, seen_timestamp),
} => worker.process_gossip_block(
message_id,
peer_id,
*block,
delayed_block_tx,
seen_timestamp,
),
/*
* Import for blocks that we received earlier than their intended slot.
*/
Work::DelayedImportBlock {
peer_id,
block,
seen_timestamp,
} => worker.process_gossip_verified_block(peer_id, *block, seen_timestamp),
/*
* Voluntary exits received on gossip.
*/

View File

@ -0,0 +1,499 @@
#![cfg(not(debug_assertions))] // Tests are too slow in debug.
#![cfg(test)]
use crate::beacon_processor::*;
use crate::{service::NetworkMessage, sync::SyncMessage};
use beacon_chain::{
test_utils::{AttestationStrategy, BeaconChainHarness, BlockStrategy, EphemeralHarnessType},
BeaconChain, MAXIMUM_GOSSIP_CLOCK_DISPARITY,
};
use discv5::enr::{CombinedKey, EnrBuilder};
use environment::{null_logger, Environment, EnvironmentBuilder};
use eth2_libp2p::{rpc::methods::MetaData, types::EnrBitfield, MessageId, NetworkGlobals, PeerId};
use slot_clock::SlotClock;
use std::cmp;
use std::iter::Iterator;
use std::sync::Arc;
use std::time::Duration;
use tokio::runtime::Runtime;
use tokio::sync::mpsc;
use types::{
test_utils::generate_deterministic_keypairs, Attestation, AttesterSlashing, MainnetEthSpec,
ProposerSlashing, SignedBeaconBlock, SignedVoluntaryExit, SubnetId,
};
type E = MainnetEthSpec;
type T = EphemeralHarnessType<E>;
const SLOTS_PER_EPOCH: u64 = 32;
const VALIDATOR_COUNT: usize = SLOTS_PER_EPOCH as usize;
const SMALL_CHAIN: u64 = 2;
const LONG_CHAIN: u64 = SLOTS_PER_EPOCH * 2;
const TCP_PORT: u16 = 42;
const UDP_PORT: u16 = 42;
const SEQ_NUMBER: u64 = 0;
/// The default time to wait for `BeaconProcessor` events.
const STANDARD_TIMEOUT: Duration = Duration::from_secs(10);
/// Provides utilities for testing the `BeaconProcessor`.
struct TestRig {
chain: Arc<BeaconChain<T>>,
next_block: SignedBeaconBlock<E>,
attestations: Vec<(Attestation<E>, SubnetId)>,
attester_slashing: AttesterSlashing<E>,
proposer_slashing: ProposerSlashing,
voluntary_exit: SignedVoluntaryExit,
beacon_processor_tx: mpsc::Sender<WorkEvent<T>>,
work_journal_rx: mpsc::Receiver<String>,
_network_rx: mpsc::UnboundedReceiver<NetworkMessage<E>>,
_sync_rx: mpsc::UnboundedReceiver<SyncMessage<E>>,
environment: Option<Environment<E>>,
}
/// This custom drop implementation ensures that we shut down the tokio runtime gracefully. Without
/// it, tests will hang indefinitely.
impl Drop for TestRig {
fn drop(&mut self) {
// Causes the beacon processor to shutdown.
self.beacon_processor_tx = mpsc::channel(MAX_WORK_EVENT_QUEUE_LEN).0;
self.environment.take().unwrap().shutdown_on_idle();
}
}
impl TestRig {
pub fn new(chain_length: u64) -> Self {
let mut harness = BeaconChainHarness::new(
MainnetEthSpec,
generate_deterministic_keypairs(VALIDATOR_COUNT),
);
harness.advance_slot();
for _ in 0..chain_length {
harness.extend_chain(
1,
BlockStrategy::OnCanonicalHead,
AttestationStrategy::AllValidators,
);
harness.advance_slot();
}
let head = harness.chain.head().unwrap();
assert_eq!(
harness.chain.slot().unwrap(),
head.beacon_block.slot() + 1,
"precondition: current slot is one after head"
);
let (next_block, _next_state) =
harness.make_block(head.beacon_state.clone(), harness.chain.slot().unwrap());
let attestations = harness
.get_unaggregated_attestations(
&AttestationStrategy::AllValidators,
&head.beacon_state,
head.beacon_block_root,
harness.chain.slot().unwrap(),
)
.into_iter()
// .map(|vec| vec.into_iter().map(|(attestation, _subnet_id)| attestation))
.flatten()
.collect::<Vec<_>>();
assert!(
!attestations.is_empty(),
"precondition: attestations for testing"
);
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());
// Changing this *after* the chain has been initialized is a bit cheeky, but it shouldn't
// cause issue.
//
// This allows for testing voluntary exits without building out a massive chain.
harness.chain.spec.shard_committee_period = 2;
let chain = Arc::new(harness.chain);
let (network_tx, _network_rx) = mpsc::unbounded_channel();
let log = null_logger().unwrap();
let (beacon_processor_tx, beacon_processor_rx) = mpsc::channel(MAX_WORK_EVENT_QUEUE_LEN);
let (sync_tx, _sync_rx) = mpsc::unbounded_channel();
// Default metadata
let meta_data = MetaData {
seq_number: SEQ_NUMBER,
attnets: EnrBitfield::<MainnetEthSpec>::default(),
};
let enr_key = CombinedKey::generate_secp256k1();
let enr = EnrBuilder::new("v4").build(&enr_key).unwrap();
let network_globals = Arc::new(NetworkGlobals::new(
enr,
TCP_PORT,
UDP_PORT,
meta_data,
vec![],
&log,
));
let mut environment = EnvironmentBuilder::mainnet()
.null_logger()
.unwrap()
.multi_threaded_tokio_runtime()
.unwrap()
.build()
.unwrap();
let executor = environment.core_context().executor;
let (work_journal_tx, work_journal_rx) = mpsc::channel(16_364);
BeaconProcessor {
beacon_chain: Arc::downgrade(&chain),
network_tx,
sync_tx,
network_globals,
executor,
max_workers: cmp::max(1, num_cpus::get()),
current_workers: 0,
log: log.clone(),
}
.spawn_manager(beacon_processor_rx, Some(work_journal_tx));
Self {
chain,
next_block,
attestations,
attester_slashing,
proposer_slashing,
voluntary_exit,
beacon_processor_tx,
work_journal_rx,
_network_rx,
_sync_rx,
environment: Some(environment),
}
}
pub fn enqueue_gossip_block(&self) {
self.beacon_processor_tx
.try_send(WorkEvent::gossip_beacon_block(
junk_message_id(),
junk_peer_id(),
Box::new(self.next_block.clone()),
Duration::from_secs(0),
))
.unwrap();
}
pub fn enqueue_unaggregated_attestation(&self) {
let (attestation, subnet_id) = self.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();
}
pub fn enqueue_gossip_attester_slashing(&self) {
self.beacon_processor_tx
.try_send(WorkEvent::gossip_attester_slashing(
junk_message_id(),
junk_peer_id(),
Box::new(self.attester_slashing.clone()),
))
.unwrap();
}
pub fn enqueue_gossip_proposer_slashing(&self) {
self.beacon_processor_tx
.try_send(WorkEvent::gossip_proposer_slashing(
junk_message_id(),
junk_peer_id(),
Box::new(self.proposer_slashing.clone()),
))
.unwrap();
}
pub fn enqueue_gossip_voluntary_exit(&self) {
self.beacon_processor_tx
.try_send(WorkEvent::gossip_voluntary_exit(
junk_message_id(),
junk_peer_id(),
Box::new(self.voluntary_exit.clone()),
))
.unwrap();
}
fn runtime(&mut self) -> Arc<Runtime> {
self.environment
.as_mut()
.unwrap()
.core_context()
.executor
.runtime()
.upgrade()
.unwrap()
}
/// Assert that the `BeaconProcessor` doesn't produce any events in the given `duration`.
pub fn assert_no_events_for(&mut self, duration: Duration) {
self.runtime().block_on(async {
tokio::select! {
_ = tokio::time::sleep(duration) => (),
event = self.work_journal_rx.recv() => panic!(
"received {:?} within {:?} when expecting no events",
event,
duration
),
}
})
}
/// 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]) {
let events = self.runtime().block_on(async {
let mut events = vec![];
let drain_future = async {
loop {
match self.work_journal_rx.recv().await {
Some(event) => {
events.push(event);
// Break as soon as we collect the desired number of events.
if events.len() >= expected.len() {
break;
}
}
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 {:?}",
STANDARD_TIMEOUT,
expected,
events
),
_ = drain_future => {},
}
events
});
assert_eq!(
events,
expected
.into_iter()
.map(|s| s.to_string())
.collect::<Vec<_>>()
);
}
}
fn junk_peer_id() -> PeerId {
PeerId::random()
}
fn junk_message_id() -> MessageId {
MessageId::new(&[])
}
/// Blocks that arrive early should be queued for later processing.
#[test]
fn import_gossip_block_acceptably_early() {
let mut rig = TestRig::new(SMALL_CHAIN);
let slot_start = rig
.chain
.slot_clock
.start_of(rig.next_block.slot())
.unwrap();
rig.chain
.slot_clock
.set_current_time(slot_start - MAXIMUM_GOSSIP_CLOCK_DISPARITY);
assert_eq!(
rig.chain.slot().unwrap(),
rig.next_block.slot() - 1,
"chain should be at the correct slot"
);
rig.enqueue_gossip_block();
rig.assert_event_journal(&[GOSSIP_BLOCK, WORKER_FREED, NOTHING_TO_DO]);
// Note: this section of the code is a bit race-y. We're assuming that we can set the slot clock
// and check the head in the time between the block arrived early and when its due for
// 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
// 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(),
"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.next_block.canonical_root(),
"block should be imported and become head"
);
}
/// Blocks that are *too* early shouldn't get into the delay queue.
#[test]
fn import_gossip_block_unacceptably_early() {
let mut rig = TestRig::new(SMALL_CHAIN);
let slot_start = rig
.chain
.slot_clock
.start_of(rig.next_block.slot())
.unwrap();
rig.chain
.slot_clock
.set_current_time(slot_start - MAXIMUM_GOSSIP_CLOCK_DISPARITY - Duration::from_millis(1));
assert_eq!(
rig.chain.slot().unwrap(),
rig.next_block.slot() - 1,
"chain should be at the correct slot"
);
rig.enqueue_gossip_block();
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
// block isn't imported.
rig.assert_no_events_for(Duration::from_secs(5));
assert!(
rig.chain.head().unwrap().beacon_block_root != rig.next_block.canonical_root(),
"block should not be imported"
);
}
/// Blocks that arrive on-time should be processed normally.
#[test]
fn import_gossip_block_at_current_slot() {
let mut rig = TestRig::new(SMALL_CHAIN);
assert_eq!(
rig.chain.slot().unwrap(),
rig.next_block.slot(),
"chain should be at the correct slot"
);
rig.enqueue_gossip_block();
rig.assert_event_journal(&[GOSSIP_BLOCK, WORKER_FREED, NOTHING_TO_DO]);
assert_eq!(
rig.chain.head().unwrap().beacon_block_root,
rig.next_block.canonical_root(),
"block should be imported and become head"
);
}
/// Ensure a valid attestation can be imported.
#[test]
fn import_gossip_attestation() {
let mut rig = TestRig::new(SMALL_CHAIN);
let initial_attns = rig.chain.naive_aggregation_pool.read().num_attestations();
rig.enqueue_unaggregated_attestation();
rig.assert_event_journal(&[GOSSIP_ATTESTATION, WORKER_FREED, NOTHING_TO_DO]);
assert_eq!(
rig.chain.naive_aggregation_pool.read().num_attestations(),
initial_attns + 1,
"op pool should have one more attestation"
);
}
/// Ensure a bunch of valid operations can be imported.
#[test]
fn import_misc_gossip_ops() {
// Exits need the long chain so validators aren't too young to exit.
let mut rig = TestRig::new(LONG_CHAIN);
/*
* Attester slashing
*/
let initial_attester_slashings = rig.chain.op_pool.num_attester_slashings();
rig.enqueue_gossip_attester_slashing();
rig.assert_event_journal(&[GOSSIP_ATTESTER_SLASHING, WORKER_FREED, NOTHING_TO_DO]);
assert_eq!(
rig.chain.op_pool.num_attester_slashings(),
initial_attester_slashings + 1,
"op pool should have one more attester slashing"
);
/*
* Proposer slashing
*/
let initial_proposer_slashings = rig.chain.op_pool.num_proposer_slashings();
rig.enqueue_gossip_proposer_slashing();
rig.assert_event_journal(&[GOSSIP_PROPOSER_SLASHING, WORKER_FREED, NOTHING_TO_DO]);
assert_eq!(
rig.chain.op_pool.num_proposer_slashings(),
initial_proposer_slashings + 1,
"op pool should have one more proposer slashing"
);
/*
* Voluntary exit
*/
let initial_voluntary_exits = rig.chain.op_pool.num_voluntary_exits();
rig.enqueue_gossip_voluntary_exit();
rig.assert_event_journal(&[GOSSIP_VOLUNTARY_EXIT, WORKER_FREED, NOTHING_TO_DO]);
assert_eq!(
rig.chain.op_pool.num_voluntary_exits(),
initial_voluntary_exits + 1,
"op pool should have one more exit"
);
}

View File

@ -4,18 +4,20 @@ use beacon_chain::{
attestation_verification::{Error as AttnError, SignatureVerifiedAttestation},
observed_operations::ObservationOutcome,
validator_monitor::get_block_delay_ms,
BeaconChainError, BeaconChainTypes, BlockError, ForkChoiceError,
BeaconChainError, BeaconChainTypes, BlockError, ForkChoiceError, GossipVerifiedBlock,
};
use eth2_libp2p::{MessageAcceptance, MessageId, PeerAction, PeerId, ReportSource};
use slog::{debug, error, info, trace, warn};
use slot_clock::SlotClock;
use ssz::Encode;
use std::time::{Duration, SystemTime, UNIX_EPOCH};
use tokio::sync::mpsc;
use types::{
Attestation, AttesterSlashing, Hash256, ProposerSlashing, SignedAggregateAndProof,
SignedBeaconBlock, SignedVoluntaryExit, SubnetId,
};
use super::Worker;
use super::{super::block_delay_queue::QueuedBlock, Worker};
impl<T: BeaconChainTypes> Worker<T> {
/* Auxiliary functions */
@ -236,6 +238,7 @@ impl<T: BeaconChainTypes> Worker<T> {
message_id: MessageId,
peer_id: PeerId,
block: SignedBeaconBlock<T::EthSpec>,
delayed_import_tx: mpsc::Sender<QueuedBlock<T>>,
seen_duration: Duration,
) {
// Log metrics to track delay from other nodes on the network.
@ -324,6 +327,80 @@ impl<T: BeaconChainTypes> Worker<T> {
&self.chain.slot_clock,
);
let block_slot = verified_block.block.slot();
let block_root = verified_block.block_root;
// Try read the current slot to determine if this block should be imported now or after some
// delay.
match self.chain.slot() {
// We only need to do a simple check about the block slot and the current slot since the
// `verify_block_for_gossip` function already ensures that the block is within the
// tolerance for block imports.
Ok(current_slot) if block_slot > current_slot => {
warn!(
self.log,
"Block arrived early";
"block_slot" => %block_slot,
"block_root" => %block_root,
"msg" => "if this happens consistently, check system clock"
);
// Take note of how early this block arrived.
if let Some(duration) = self
.chain
.slot_clock
.start_of(block_slot)
.and_then(|start| start.checked_sub(seen_duration))
{
metrics::observe_duration(
&metrics::BEACON_PROCESSOR_GOSSIP_BLOCK_EARLY_SECONDS,
duration,
);
}
metrics::inc_counter(&metrics::BEACON_PROCESSOR_GOSSIP_BLOCK_REQUEUED_TOTAL);
if delayed_import_tx
.try_send(QueuedBlock {
peer_id,
block: verified_block,
seen_timestamp: seen_duration,
})
.is_err()
{
error!(
self.log,
"Failed to defer block import";
"block_slot" => %block_slot,
"block_root" => %block_root,
"location" => "block gossip"
)
}
}
Ok(_) => self.process_gossip_verified_block(peer_id, verified_block, seen_duration),
Err(e) => {
error!(
self.log,
"Failed to defer block import";
"error" => ?e,
"block_slot" => %block_slot,
"block_root" => %block_root,
"location" => "block gossip"
)
}
}
}
/// Process the beacon block that has already passed gossip verification.
///
/// Raises a log if there are errors.
pub fn process_gossip_verified_block(
self,
peer_id: PeerId,
verified_block: GossipVerifiedBlock<T>,
// 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) {

View File

@ -1,3 +1,4 @@
use super::QueuedBlock;
use crate::{service::NetworkMessage, sync::SyncMessage};
use beacon_chain::{BeaconChain, BeaconChainTypes};
use slog::{error, Logger};
@ -41,3 +42,9 @@ impl<T: BeaconChainTypes> Worker<T> {
});
}
}
/// Contains the necessary items for a worker to do their job.
pub struct Toolbox<T: BeaconChainTypes> {
pub idle_tx: mpsc::Sender<()>,
pub delayed_block_tx: mpsc::Sender<QueuedBlock<T>>,
}

View File

@ -215,6 +215,14 @@ lazy_static! {
"beacon_processor_gossip_block_imported_total",
"Total number of gossip blocks imported to fork choice, etc."
);
pub static ref BEACON_PROCESSOR_GOSSIP_BLOCK_REQUEUED_TOTAL: Result<IntCounter> = try_create_int_counter(
"beacon_processor_gossip_block_requeued_total",
"Total number of gossip blocks that arrived early and were re-queued for later processing."
);
pub static ref BEACON_PROCESSOR_GOSSIP_BLOCK_EARLY_SECONDS: Result<Histogram> = try_create_histogram(
"beacon_processor_gossip_block_early_seconds",
"Whenever a gossip block is received early this metrics is set to how early that block was."
);
// Gossip Exits.
pub static ref BEACON_PROCESSOR_EXIT_QUEUE_TOTAL: Result<IntGauge> = try_create_int_gauge(
"beacon_processor_exit_queue_total",

View File

@ -26,7 +26,7 @@ pub struct Processor<T: BeaconChainTypes> {
/// A network context to return and handle RPC requests.
network: HandlerNetworkContext<T::EthSpec>,
/// A multi-threaded, non-blocking processor for applying messages to the beacon chain.
beacon_processor_send: mpsc::Sender<BeaconWorkEvent<T::EthSpec>>,
beacon_processor_send: mpsc::Sender<BeaconWorkEvent<T>>,
/// The `RPCHandler` logger.
log: slog::Logger,
}
@ -64,7 +64,7 @@ impl<T: BeaconChainTypes> Processor<T> {
current_workers: 0,
log: log.clone(),
}
.spawn_manager(beacon_processor_receive);
.spawn_manager(beacon_processor_receive, None);
Processor {
chain: beacon_chain,
@ -309,7 +309,7 @@ impl<T: BeaconChainTypes> Processor<T> {
))
}
fn send_beacon_processor_work(&mut self, work: BeaconWorkEvent<T::EthSpec>) {
fn send_beacon_processor_work(&mut self, work: BeaconWorkEvent<T>) {
self.beacon_processor_send
.try_send(work)
.unwrap_or_else(|e| {

View File

@ -179,7 +179,7 @@ pub struct SyncManager<T: BeaconChainTypes> {
single_block_lookups: FnvHashMap<RequestId, SingleBlockRequest>,
/// A multi-threaded, non-blocking processor for applying messages to the beacon chain.
beacon_processor_send: mpsc::Sender<BeaconWorkEvent<T::EthSpec>>,
beacon_processor_send: mpsc::Sender<BeaconWorkEvent<T>>,
/// The logger for the import manager.
log: Logger,
@ -210,7 +210,7 @@ pub fn spawn<T: BeaconChainTypes>(
beacon_chain: Arc<BeaconChain<T>>,
network_globals: Arc<NetworkGlobals<T::EthSpec>>,
network_send: mpsc::UnboundedSender<NetworkMessage<T::EthSpec>>,
beacon_processor_send: mpsc::Sender<BeaconWorkEvent<T::EthSpec>>,
beacon_processor_send: mpsc::Sender<BeaconWorkEvent<T>>,
log: slog::Logger,
) -> mpsc::UnboundedSender<SyncMessage<T::EthSpec>> {
assert!(

View File

@ -96,7 +96,7 @@ pub struct SyncingChain<T: BeaconChainTypes> {
validated_batches: u64,
/// A multi-threaded, non-blocking processor for applying messages to the beacon chain.
beacon_processor_send: Sender<BeaconWorkEvent<T::EthSpec>>,
beacon_processor_send: Sender<BeaconWorkEvent<T>>,
/// The chain's log.
log: slog::Logger,
@ -123,7 +123,7 @@ impl<T: BeaconChainTypes> SyncingChain<T> {
target_head_slot: Slot,
target_head_root: Hash256,
peer_id: PeerId,
beacon_processor_send: Sender<BeaconWorkEvent<T::EthSpec>>,
beacon_processor_send: Sender<BeaconWorkEvent<T>>,
log: &slog::Logger,
) -> Self {
let mut peers = FnvHashMap::default();

View File

@ -194,7 +194,7 @@ impl<T: BeaconChainTypes> ChainCollection<T> {
network: &mut SyncNetworkContext<T::EthSpec>,
local: &SyncInfo,
awaiting_head_peers: &mut HashMap<PeerId, SyncInfo>,
beacon_processor_send: &mpsc::Sender<BeaconWorkEvent<T::EthSpec>>,
beacon_processor_send: &mpsc::Sender<BeaconWorkEvent<T>>,
) {
// Remove any outdated finalized/head chains
self.purge_outdated_chains(local, awaiting_head_peers);
@ -328,7 +328,7 @@ impl<T: BeaconChainTypes> ChainCollection<T> {
local_epoch: Epoch,
local_head_epoch: Epoch,
awaiting_head_peers: &mut HashMap<PeerId, SyncInfo>,
beacon_processor_send: &mpsc::Sender<BeaconWorkEvent<T::EthSpec>>,
beacon_processor_send: &mpsc::Sender<BeaconWorkEvent<T>>,
) {
// Include the awaiting head peers
for (peer_id, peer_sync_info) in awaiting_head_peers.drain() {
@ -463,7 +463,7 @@ impl<T: BeaconChainTypes> ChainCollection<T> {
target_head_slot: Slot,
peer: PeerId,
sync_type: RangeSyncType,
beacon_processor_send: &mpsc::Sender<BeaconWorkEvent<T::EthSpec>>,
beacon_processor_send: &mpsc::Sender<BeaconWorkEvent<T>>,
network: &mut SyncNetworkContext<T::EthSpec>,
) {
let id = SyncingChain::<T>::id(&target_head_root, &target_head_slot);

View File

@ -68,7 +68,7 @@ pub struct RangeSync<T: BeaconChainTypes> {
/// that need to be downloaded.
chains: ChainCollection<T>,
/// A multi-threaded, non-blocking processor for applying messages to the beacon chain.
beacon_processor_send: mpsc::Sender<BeaconWorkEvent<T::EthSpec>>,
beacon_processor_send: mpsc::Sender<BeaconWorkEvent<T>>,
/// The syncing logger.
log: slog::Logger,
}
@ -76,7 +76,7 @@ pub struct RangeSync<T: BeaconChainTypes> {
impl<T: BeaconChainTypes> RangeSync<T> {
pub fn new(
beacon_chain: Arc<BeaconChain<T>>,
beacon_processor_send: mpsc::Sender<BeaconWorkEvent<T::EthSpec>>,
beacon_processor_send: mpsc::Sender<BeaconWorkEvent<T>>,
log: slog::Logger,
) -> Self {
RangeSync {

View File

@ -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 {
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;

View File

@ -37,6 +37,10 @@ impl ManualSlotClock {
self.genesis_duration + self.slot_duration * slots_since_genesis;
}
pub fn set_current_time(&self, duration: Duration) {
*self.current_time.write() = duration;
}
pub fn advance_slot(&self) {
self.set_slot(self.now().unwrap().as_u64() + 1)
}