Refactor block processing results, some sync logic

This commit is contained in:
Paul Hauner 2019-06-12 23:54:16 -04:00
parent b9e832216b
commit 4c0724fba6
No known key found for this signature in database
GPG Key ID: 303E4494BB28068C
5 changed files with 215 additions and 359 deletions

View File

@ -15,7 +15,7 @@ use state_processing::per_block_processing::errors::{
}; };
use state_processing::{ use state_processing::{
per_block_processing, per_block_processing_without_verifying_block_signature, per_block_processing, per_block_processing_without_verifying_block_signature,
per_slot_processing, BlockProcessingError, SlotProcessingError, per_slot_processing, BlockProcessingError,
}; };
use std::sync::Arc; use std::sync::Arc;
use store::{Error as DBError, Store}; use store::{Error as DBError, Store};
@ -23,15 +23,11 @@ use tree_hash::TreeHash;
use types::*; use types::*;
#[derive(Debug, PartialEq)] #[derive(Debug, PartialEq)]
pub enum ValidBlock { pub enum BlockProcessingOutcome {
/// The block was successfully processed. /// Block was valid and imported into the block graph.
Processed, Processed,
} /// The blocks parent_root is unknown.
ParentUnknown { parent: Hash256 },
#[derive(Debug, PartialEq)]
pub enum InvalidBlock {
/// Don't re-process the genesis block.
GenesisBlock,
/// The block slot is greater than the present slot. /// The block slot is greater than the present slot.
FutureSlot { FutureSlot {
present_slot: Slot, present_slot: Slot,
@ -39,53 +35,16 @@ pub enum InvalidBlock {
}, },
/// The block state_root does not match the generated state. /// The block state_root does not match the generated state.
StateRootMismatch, StateRootMismatch,
/// The blocks parent_root is unknown. /// The block was a genesis block, these blocks cannot be re-imported.
ParentUnknown { parent: Hash256 }, GenesisBlock,
/// There was an error whilst advancing the parent state to the present slot. This condition /// The slot is finalized, no need to import.
/// should not occur, it likely represents an internal error. FinalizedSlot,
SlotProcessingError(SlotProcessingError), /// Block is already known, no need to re-import.
BlockIsAlreadyKnown,
/// The block could not be applied to the state, it is invalid. /// The block could not be applied to the state, it is invalid.
PerBlockProcessingError(BlockProcessingError), PerBlockProcessingError(BlockProcessingError),
} }
#[derive(Debug, PartialEq)]
pub enum BlockProcessingOutcome {
/// The block was successfully validated.
ValidBlock(ValidBlock),
/// The block was not successfully validated.
InvalidBlock(InvalidBlock),
}
impl BlockProcessingOutcome {
/// Returns `true` if the block was objectively invalid and we should disregard the peer who
/// sent it.
pub fn is_invalid(&self) -> bool {
match self {
BlockProcessingOutcome::ValidBlock(_) => false,
BlockProcessingOutcome::InvalidBlock(r) => match r {
InvalidBlock::GenesisBlock { .. } => true,
InvalidBlock::FutureSlot { .. } => true,
InvalidBlock::StateRootMismatch => true,
InvalidBlock::ParentUnknown { .. } => false,
InvalidBlock::SlotProcessingError(_) => false,
InvalidBlock::PerBlockProcessingError(e) => match e {
BlockProcessingError::Invalid(_) => true,
BlockProcessingError::BeaconStateError(_) => false,
},
},
}
}
/// Returns `true` if the block was successfully processed and can be removed from any import
/// queues or temporary storage.
pub fn sucessfully_processed(&self) -> bool {
match self {
BlockProcessingOutcome::ValidBlock(_) => true,
_ => false,
}
}
}
pub trait BeaconChainTypes { pub trait BeaconChainTypes {
type Store: store::Store; type Store: store::Store;
type SlotClock: slot_clock::SlotClock; type SlotClock: slot_clock::SlotClock;
@ -257,88 +216,6 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
BlockRootsIterator::new(self.store.clone(), self.state.read().clone(), slot) BlockRootsIterator::new(self.store.clone(), self.state.read().clone(), slot)
} }
/*
/// Returns `count `beacon block roots, starting from `start_slot` with an
/// interval of `skip` slots between each root.
///
/// ## Errors:
///
/// - `SlotOutOfBounds`: Unable to return the full specified range.
/// - `SlotOutOfBounds`: Unable to load a state from the DB.
/// - `SlotOutOfBounds`: Start slot is higher than the first slot.
/// - Other: BeaconState` is inconsistent.
pub fn get_block_roots(
&self,
earliest_slot: Slot,
count: usize,
skip: usize,
) -> Result<Vec<Hash256>, Error> {
let step_by = Slot::from(skip + 1);
let mut roots: Vec<Hash256> = vec![];
// The state for reading block roots. Will be updated with an older state if slots go too
// far back in history.
let mut state = self.state.read().clone();
// The final slot in this series, will be reduced by `skip` each loop iteration.
let mut slot = earliest_slot + Slot::from(count * (skip + 1)) - 1;
// If the highest slot requested is that of the current state insert the root of the
// head block, unless the head block's slot is not matching.
if slot == state.slot && self.head().beacon_block.slot == slot {
roots.push(self.head().beacon_block_root);
slot -= step_by;
} else if slot >= state.slot {
return Err(BeaconStateError::SlotOutOfBounds.into());
}
loop {
// If the slot is within the range of the current state's block roots, append the root
// to the output vec.
//
// If we get `SlotOutOfBounds` error, load the oldest available historic
// state from the DB.
match state.get_block_root(slot) {
Ok(root) => {
if slot < earliest_slot {
break;
} else {
roots.push(*root);
slot -= step_by;
}
}
Err(BeaconStateError::SlotOutOfBounds) => {
// Read the earliest historic state in the current slot.
let earliest_historic_slot =
state.slot - Slot::from(T::EthSpec::slots_per_historical_root());
// Load the earlier state from disk.
let new_state_root = state.get_state_root(earliest_historic_slot)?;
// Break if the DB is unable to load the state.
state = match self.store.get(&new_state_root) {
Ok(Some(state)) => state,
_ => break,
}
}
Err(e) => return Err(e.into()),
};
}
// Return the results if they pass a sanity check.
if (slot <= earliest_slot) && (roots.len() == count) {
// Reverse the ordering of the roots. We extracted them in reverse order to make it
// simpler to lookup historic states.
//
// This is a potential optimisation target.
Ok(roots.iter().rev().cloned().collect())
} else {
Err(BeaconStateError::SlotOutOfBounds.into())
}
}
*/
/// Returns the block at the given root, if any. /// Returns the block at the given root, if any.
/// ///
/// ## Errors /// ## Errors
@ -649,31 +526,39 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
/// ///
/// Will accept blocks from prior slots, however it will reject any block from a future slot. /// Will accept blocks from prior slots, however it will reject any block from a future slot.
pub fn process_block(&self, block: BeaconBlock) -> Result<BlockProcessingOutcome, Error> { pub fn process_block(&self, block: BeaconBlock) -> Result<BlockProcessingOutcome, Error> {
debug!("Processing block with slot {}...", block.slot);
self.metrics.block_processing_requests.inc(); self.metrics.block_processing_requests.inc();
let timer = self.metrics.block_processing_times.start_timer(); let timer = self.metrics.block_processing_times.start_timer();
let finalized_slot = self
.state
.read()
.finalized_epoch
.start_slot(T::EthSpec::slots_per_epoch());
if block.slot <= finalized_slot {
return Ok(BlockProcessingOutcome::FinalizedSlot);
}
if block.slot == 0 { if block.slot == 0 {
return Ok(BlockProcessingOutcome::InvalidBlock( return Ok(BlockProcessingOutcome::GenesisBlock);
InvalidBlock::GenesisBlock,
));
} }
let block_root = block.block_header().canonical_root(); let block_root = block.block_header().canonical_root();
if block_root == self.genesis_block_root { if block_root == self.genesis_block_root {
return Ok(BlockProcessingOutcome::ValidBlock(ValidBlock::Processed)); return Ok(BlockProcessingOutcome::GenesisBlock);
} }
let present_slot = self.present_slot(); let present_slot = self.present_slot();
if block.slot > present_slot { if block.slot > present_slot {
return Ok(BlockProcessingOutcome::InvalidBlock( return Ok(BlockProcessingOutcome::FutureSlot {
InvalidBlock::FutureSlot { present_slot,
present_slot, block_slot: block.slot,
block_slot: block.slot, });
}, }
));
if self.store.exists::<BeaconBlock>(&block_root)? {
return Ok(BlockProcessingOutcome::BlockIsAlreadyKnown);
} }
// Load the blocks parent block from the database, returning invalid if that block is not // Load the blocks parent block from the database, returning invalid if that block is not
@ -682,11 +567,9 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
let parent_block: BeaconBlock = match self.store.get(&parent_block_root)? { let parent_block: BeaconBlock = match self.store.get(&parent_block_root)? {
Some(previous_block_root) => previous_block_root, Some(previous_block_root) => previous_block_root,
None => { None => {
return Ok(BlockProcessingOutcome::InvalidBlock( return Ok(BlockProcessingOutcome::ParentUnknown {
InvalidBlock::ParentUnknown { parent: parent_block_root,
parent: parent_block_root, });
},
));
} }
}; };
@ -704,29 +587,25 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
// Transition the parent state to the block slot. // Transition the parent state to the block slot.
let mut state: BeaconState<T::EthSpec> = parent_state; let mut state: BeaconState<T::EthSpec> = parent_state;
for _ in state.slot.as_u64()..block.slot.as_u64() { for _ in state.slot.as_u64()..block.slot.as_u64() {
if let Err(e) = per_slot_processing(&mut state, &self.spec) { per_slot_processing(&mut state, &self.spec)?;
return Ok(BlockProcessingOutcome::InvalidBlock(
InvalidBlock::SlotProcessingError(e),
));
}
} }
state.build_committee_cache(RelativeEpoch::Current, &self.spec)?; state.build_committee_cache(RelativeEpoch::Current, &self.spec)?;
// Apply the received block to its parent state (which has been transitioned into this // Apply the received block to its parent state (which has been transitioned into this
// slot). // slot).
if let Err(e) = per_block_processing(&mut state, &block, &self.spec) { match per_block_processing(&mut state, &block, &self.spec) {
return Ok(BlockProcessingOutcome::InvalidBlock( Err(BlockProcessingError::BeaconStateError(e)) => {
InvalidBlock::PerBlockProcessingError(e), return Err(Error::BeaconStateError(e))
)); }
Err(e) => return Ok(BlockProcessingOutcome::PerBlockProcessingError(e)),
_ => {}
} }
let state_root = state.canonical_root(); let state_root = state.canonical_root();
if block.state_root != state_root { if block.state_root != state_root {
return Ok(BlockProcessingOutcome::InvalidBlock( return Ok(BlockProcessingOutcome::StateRootMismatch);
InvalidBlock::StateRootMismatch,
));
} }
// Store the block and state. // Store the block and state.
@ -750,7 +629,7 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
.observe(block.body.attestations.len() as f64); .observe(block.body.attestations.len() as f64);
timer.observe_duration(); timer.observe_duration();
Ok(BlockProcessingOutcome::ValidBlock(ValidBlock::Processed)) Ok(BlockProcessingOutcome::Processed)
} }
/// Produce a new block at the present slot. /// Produce a new block at the present slot.

View File

@ -5,9 +5,7 @@ pub mod iter;
mod metrics; mod metrics;
mod persisted_beacon_chain; mod persisted_beacon_chain;
pub use self::beacon_chain::{ pub use self::beacon_chain::{BeaconChain, BeaconChainTypes, BlockProcessingOutcome};
BeaconChain, BeaconChainTypes, BlockProcessingOutcome, InvalidBlock, ValidBlock,
};
pub use self::checkpoint::CheckPoint; pub use self::checkpoint::CheckPoint;
pub use self::errors::{BeaconChainError, BlockProductionError}; pub use self::errors::{BeaconChainError, BlockProductionError};
pub use fork_choice; pub use fork_choice;

View File

@ -212,7 +212,7 @@ impl<T: BeaconChainTypes> ImportQueue<T> {
// Case 2: there was no partial with a matching block root. // Case 2: there was no partial with a matching block root.
// //
// A new partial is added. This case permits adding a header without already known the // A new partial is added. This case permits adding a header without already known the
// root -- this is not possible in the wire protocol however we support it anyway. // root.
self.partials.push(PartialBeaconBlock { self.partials.push(PartialBeaconBlock {
slot: header.slot, slot: header.slot,
block_root, block_root,

View File

@ -1,6 +1,6 @@
use super::import_queue::ImportQueue; use super::import_queue::ImportQueue;
use crate::message_handler::NetworkContext; use crate::message_handler::NetworkContext;
use beacon_chain::{BeaconChain, BeaconChainTypes, BlockProcessingOutcome, InvalidBlock}; use beacon_chain::{BeaconChain, BeaconChainTypes, BlockProcessingOutcome};
use eth2_libp2p::rpc::methods::*; use eth2_libp2p::rpc::methods::*;
use eth2_libp2p::rpc::{RPCRequest, RPCResponse, RequestId}; use eth2_libp2p::rpc::{RPCRequest, RPCResponse, RequestId};
use eth2_libp2p::PeerId; use eth2_libp2p::PeerId;
@ -9,7 +9,6 @@ use std::collections::HashMap;
use std::sync::Arc; use std::sync::Arc;
use std::time::Duration; use std::time::Duration;
use store::Store; use store::Store;
use tree_hash::TreeHash;
use types::{ use types::{
Attestation, BeaconBlock, BeaconBlockBody, BeaconBlockHeader, Epoch, EthSpec, Hash256, Slot, Attestation, BeaconBlock, BeaconBlockBody, BeaconBlockHeader, Epoch, EthSpec, Hash256, Slot,
}; };
@ -24,6 +23,9 @@ const QUEUE_STALE_SECS: u64 = 600;
/// Otherwise we queue it. /// Otherwise we queue it.
const FUTURE_SLOT_TOLERANCE: u64 = 1; const FUTURE_SLOT_TOLERANCE: u64 = 1;
const SHOULD_FORWARD_GOSSIP_BLOCK: bool = true;
const SHOULD_NOT_FORWARD_GOSSIP_BLOCK: bool = false;
/// Keeps track of syncing information for known connected peers. /// Keeps track of syncing information for known connected peers.
#[derive(Clone, Copy, Debug)] #[derive(Clone, Copy, Debug)]
pub struct PeerSyncInfo { pub struct PeerSyncInfo {
@ -393,6 +395,7 @@ impl<T: BeaconChainTypes> SimpleSync<T> {
.collect(); .collect();
roots.reverse(); roots.reverse();
roots.dedup();
let headers: Vec<BeaconBlockHeader> = roots let headers: Vec<BeaconBlockHeader> = roots
.into_iter() .into_iter()
@ -509,6 +512,8 @@ impl<T: BeaconChainTypes> SimpleSync<T> {
/// Process a gossip message declaring a new block. /// Process a gossip message declaring a new block.
/// ///
/// Attempts to apply to block to the beacon chain. May queue the block for later processing.
///
/// Returns a `bool` which, if `true`, indicates we should forward the block to our peers. /// Returns a `bool` which, if `true`, indicates we should forward the block to our peers.
pub fn on_block_gossip( pub fn on_block_gossip(
&mut self, &mut self,
@ -516,133 +521,35 @@ impl<T: BeaconChainTypes> SimpleSync<T> {
block: BeaconBlock, block: BeaconBlock,
network: &mut NetworkContext, network: &mut NetworkContext,
) -> bool { ) -> bool {
// Ignore any block from a finalized slot. if let Some(outcome) =
if self.slot_is_finalized(block.slot) { self.process_block(peer_id.clone(), block.clone(), network, &"gossip")
debug!( {
self.log, "IgnoredFinalizedBlock"; match outcome {
"source" => "gossip", BlockProcessingOutcome::Processed => SHOULD_FORWARD_GOSSIP_BLOCK,
"msg" => "chain is finalized at block slot", BlockProcessingOutcome::ParentUnknown { .. } => {
"block_slot" => block.slot, self.import_queue
); .enqueue_full_blocks(vec![block], peer_id.clone());
return false;
}
let block_root = Hash256::from_slice(&block.tree_hash_root()); SHOULD_FORWARD_GOSSIP_BLOCK
// Ignore any block that the chain already knows about.
if self.chain_has_seen_block(&block_root) {
// TODO: Age confirm that we shouldn't forward a block if we already know of it.
return false;
}
match self.chain.process_block(block.clone()) {
Ok(BlockProcessingOutcome::InvalidBlock(InvalidBlock::ParentUnknown { .. })) => {
// The block was valid and we processed it successfully.
debug!(
self.log, "ParentBlockUnknown";
"source" => "gossip",
"parent_root" => format!("{}", block.previous_block_root),
"peer" => format!("{:?}", peer_id),
);
// Queue the block for later processing.
self.import_queue
.enqueue_full_blocks(vec![block], peer_id.clone());
// Send a hello to learn of the clients best slot so we can then sync the require
// parent(s).
network.send_rpc_request(
peer_id.clone(),
RPCRequest::Hello(hello_message(&self.chain)),
);
// Forward the block onto our peers.
//
// Note: this may need to be changed if we decide to only forward blocks if we have
// all required info.
true
}
Ok(BlockProcessingOutcome::InvalidBlock(InvalidBlock::FutureSlot {
present_slot,
block_slot,
})) => {
if block_slot - present_slot > FUTURE_SLOT_TOLERANCE {
// The block is too far in the future, drop it.
warn!(
self.log, "FutureBlock";
"source" => "gossip",
"msg" => "block for future slot rejected, check your time",
"present_slot" => present_slot,
"block_slot" => block_slot,
"FUTURE_SLOT_TOLERANCE" => FUTURE_SLOT_TOLERANCE,
"peer" => format!("{:?}", peer_id),
);
// Do not forward the block around to peers.
false
} else {
// The block is in the future, but not too far.
warn!(
self.log, "QueuedFutureBlock";
"source" => "gossip",
"msg" => "queuing future block, check your time",
"present_slot" => present_slot,
"block_slot" => block_slot,
"FUTURE_SLOT_TOLERANCE" => FUTURE_SLOT_TOLERANCE,
"peer" => format!("{:?}", peer_id),
);
// Queue the block for later processing.
self.import_queue.enqueue_full_blocks(vec![block], peer_id);
// Forward the block around to peers.
true
} }
} BlockProcessingOutcome::FutureSlot {
Ok(outcome) => { present_slot,
if outcome.is_invalid() { block_slot,
// The peer has sent a block which is fundamentally invalid. } if present_slot + FUTURE_SLOT_TOLERANCE >= block_slot => {
warn!( self.import_queue
self.log, "InvalidBlock"; .enqueue_full_blocks(vec![block], peer_id.clone());
"source" => "gossip",
"msg" => "peer sent objectively invalid block", SHOULD_FORWARD_GOSSIP_BLOCK
"outcome" => format!("{:?}", outcome),
"peer" => format!("{:?}", peer_id),
);
// Disconnect the peer
network.disconnect(peer_id, GoodbyeReason::Fault);
// Do not forward the block to peers.
false
} else if outcome.sucessfully_processed() {
// The block was valid and we processed it successfully.
info!(
self.log, "ImportedBlock";
"source" => "gossip",
"peer" => format!("{:?}", peer_id),
);
// Forward the block to peers
true
} else {
// The block wasn't necessarily invalid but we didn't process it successfully.
// This condition shouldn't be reached.
error!(
self.log, "BlockProcessingFailure";
"source" => "gossip",
"msg" => "unexpected condition in processing block.",
"outcome" => format!("{:?}", outcome),
);
// Do not forward the block on.
false
} }
} // Note: known blocks are forwarded on the gossip network.
Err(e) => {
// We encountered an error whilst processing the block.
// //
// Blocks should not be able to trigger errors, instead they should be flagged as // We rely upon the lower layers (libp2p) to stop loops occuring from re-gossiped
// invalid. // blocks.
error!( BlockProcessingOutcome::BlockIsAlreadyKnown => SHOULD_FORWARD_GOSSIP_BLOCK,
self.log, "BlockProcessingError"; _ => SHOULD_NOT_FORWARD_GOSSIP_BLOCK,
"msg" => "internal error in processing block.",
"source" => "gossip",
"error" => format!("{:?}", e),
);
// Do not forward the block to peers.
false
} }
} else {
SHOULD_NOT_FORWARD_GOSSIP_BLOCK
} }
} }
@ -669,57 +576,32 @@ impl<T: BeaconChainTypes> SimpleSync<T> {
/// the queue. /// the queue.
pub fn process_import_queue(&mut self, network: &mut NetworkContext) { pub fn process_import_queue(&mut self, network: &mut NetworkContext) {
let mut successful = 0; let mut successful = 0;
let mut invalid = 0;
let mut errored = 0;
// Loop through all of the complete blocks in the queue. // Loop through all of the complete blocks in the queue.
for (block_root, block, sender) in self.import_queue.complete_blocks() { for (block_root, block, sender) in self.import_queue.complete_blocks() {
let slot = block.slot; let processing_result = self.process_block(sender, block.clone(), network, &"gossip");
let parent_root = block.previous_block_root;
match self.chain.process_block(block) { let should_dequeue = match processing_result {
Ok(outcome) => { Some(BlockProcessingOutcome::ParentUnknown { .. }) => false,
if outcome.is_invalid() { Some(BlockProcessingOutcome::FutureSlot {
invalid += 1; present_slot,
warn!( block_slot,
self.log, }) if present_slot + FUTURE_SLOT_TOLERANCE >= block_slot => false,
"InvalidBlock"; _ => true,
"sender_peer_id" => format!("{:?}", sender.clone()), };
"block_root" => format!("{}", block_root),
"reason" => format!("{:?}", outcome), if processing_result == Some(BlockProcessingOutcome::Processed) {
); successful += 1;
network.disconnect(sender, GoodbyeReason::Fault); }
} else if outcome.sucessfully_processed() {
successful += 1; if should_dequeue {
self.import_queue.remove(block_root); self.import_queue.remove(block_root);
} else {
debug!(
self.log,
"ProcessImportQueue";
"msg" => "Block not imported",
"outcome" => format!("{:?}", outcome),
"block_slot" => format!("{:?}", slot),
"parent_root" => format!("{}", parent_root),
"peer" => format!("{:?}", sender),
);
}
}
Err(e) => {
errored += 1;
error!(self.log, "BlockProcessingError"; "error" => format!("{:?}", e));
}
} }
} }
if successful > 0 { if successful > 0 {
info!(self.log, "Imported {} blocks", successful) info!(self.log, "Imported {} blocks", successful)
} }
if invalid > 0 {
warn!(self.log, "Rejected {} invalid blocks", invalid)
}
if errored > 0 {
warn!(self.log, "Failed to process {} blocks", errored)
}
} }
/// Request some `BeaconBlockRoots` from the remote peer. /// Request some `BeaconBlockRoots` from the remote peer.
@ -791,17 +673,128 @@ impl<T: BeaconChainTypes> SimpleSync<T> {
}) })
} }
/// Returns `true` if the given slot is finalized in our chain.
fn slot_is_finalized(&self, slot: Slot) -> bool {
slot <= hello_message(&self.chain)
.latest_finalized_epoch
.start_slot(T::EthSpec::slots_per_epoch())
}
/// Generates our current state in the form of a HELLO RPC message. /// Generates our current state in the form of a HELLO RPC message.
pub fn generate_hello(&self) -> HelloMessage { pub fn generate_hello(&self) -> HelloMessage {
hello_message(&self.chain) hello_message(&self.chain)
} }
/// Processes the `block` that was received from `peer_id`.
///
/// If the block was submitted to the beacon chain without internal error, `Some(outcome)` is
/// returned, otherwise `None` is returned. Note: `Some(_)` does not necessarily indicate that
/// the block was successfully processed or valid.
///
/// This function performs the following duties:
///
/// - Attempting to import the block into the beacon chain.
/// - Logging
/// - Requesting unavailable blocks (e.g., if parent is unknown).
/// - Disconnecting faulty nodes.
///
/// This function does not remove processed blocks from the import queue.
fn process_block(
&mut self,
peer_id: PeerId,
block: BeaconBlock,
network: &mut NetworkContext,
source: &str,
) -> Option<BlockProcessingOutcome> {
let processing_result = self.chain.process_block(block.clone());
if let Ok(outcome) = processing_result {
match outcome {
BlockProcessingOutcome::Processed => {
info!(
self.log, "Imported block from network";
"source" => source,
"slot" => block.slot,
"peer" => format!("{:?}", peer_id),
);
}
BlockProcessingOutcome::ParentUnknown { parent } => {
// The block was valid and we processed it successfully.
debug!(
self.log, "ParentBlockUnknown";
"source" => source,
"parent_root" => format!("{}", parent),
"peer" => format!("{:?}", peer_id),
);
// Send a hello to learn of the clients best slot so we can then sync the require
// parent(s).
network.send_rpc_request(
peer_id.clone(),
RPCRequest::Hello(hello_message(&self.chain)),
);
// Explicitly request the parent block from the peer.
//
// It is likely that this is duplicate work, given we already send a hello
// request. However, I believe there are some edge-cases where the hello
// message doesn't suffice, so we perform this request as well.
self.request_block_headers(
peer_id,
BeaconBlockHeadersRequest {
start_root: parent,
start_slot: block.slot - 1,
max_headers: 1,
skip_slots: 0,
},
network,
)
}
BlockProcessingOutcome::FutureSlot {
present_slot,
block_slot,
} => {
if present_slot + FUTURE_SLOT_TOLERANCE >= block_slot {
// The block is too far in the future, drop it.
warn!(
self.log, "FutureBlock";
"source" => source,
"msg" => "block for future slot rejected, check your time",
"present_slot" => present_slot,
"block_slot" => block_slot,
"FUTURE_SLOT_TOLERANCE" => FUTURE_SLOT_TOLERANCE,
"peer" => format!("{:?}", peer_id),
);
network.disconnect(peer_id, GoodbyeReason::Fault);
} else {
// The block is in the future, but not too far.
debug!(
self.log, "QueuedFutureBlock";
"source" => source,
"msg" => "queuing future block, check your time",
"present_slot" => present_slot,
"block_slot" => block_slot,
"FUTURE_SLOT_TOLERANCE" => FUTURE_SLOT_TOLERANCE,
"peer" => format!("{:?}", peer_id),
);
}
}
_ => {
debug!(
self.log, "InvalidBlock";
"source" => source,
"msg" => "peer sent invalid block",
"outcome" => format!("{:?}", outcome),
"peer" => format!("{:?}", peer_id),
);
}
}
Some(outcome)
} else {
error!(
self.log, "BlockProcessingFailure";
"source" => source,
"msg" => "unexpected condition in processing block.",
"outcome" => format!("{:?}", processing_result)
);
None
}
}
} }
/// Build a `HelloMessage` representing the state of the given `beacon_chain`. /// Build a `HelloMessage` representing the state of the given `beacon_chain`.

View File

@ -1,4 +1,4 @@
use beacon_chain::{BeaconChain, BeaconChainTypes}; use beacon_chain::{BeaconChain, BeaconChainTypes, BlockProcessingOutcome};
use crossbeam_channel; use crossbeam_channel;
use eth2_libp2p::PubsubMessage; use eth2_libp2p::PubsubMessage;
use futures::Future; use futures::Future;
@ -95,14 +95,12 @@ impl<T: BeaconChainTypes> BeaconBlockService for BeaconBlockServiceInstance<T> {
Ok(block) => { Ok(block) => {
match self.chain.process_block(block.clone()) { match self.chain.process_block(block.clone()) {
Ok(outcome) => { Ok(outcome) => {
if outcome.sucessfully_processed() { if outcome == BlockProcessingOutcome::Processed {
// Block was successfully processed. // Block was successfully processed.
info!( info!(
self.log, self.log,
"PublishBeaconBlock"; "Valid block from RPC";
"type" => "valid_block",
"block_slot" => block.slot, "block_slot" => block.slot,
"outcome" => format!("{:?}", outcome)
); );
// TODO: Obtain topics from the network service properly. // TODO: Obtain topics from the network service properly.
@ -126,12 +124,11 @@ impl<T: BeaconChainTypes> BeaconBlockService for BeaconBlockServiceInstance<T> {
}); });
resp.set_success(true); resp.set_success(true);
} else if outcome.is_invalid() { } else {
// Block was invalid. // Block was not successfully processed.
warn!( warn!(
self.log, self.log,
"PublishBeaconBlock"; "Invalid block from RPC";
"type" => "invalid_block",
"outcome" => format!("{:?}", outcome) "outcome" => format!("{:?}", outcome)
); );
@ -139,17 +136,6 @@ impl<T: BeaconChainTypes> BeaconBlockService for BeaconBlockServiceInstance<T> {
resp.set_msg( resp.set_msg(
format!("InvalidBlock: {:?}", outcome).as_bytes().to_vec(), format!("InvalidBlock: {:?}", outcome).as_bytes().to_vec(),
); );
} else {
// Some failure during processing.
warn!(
self.log,
"PublishBeaconBlock";
"type" => "unable_to_import",
"outcome" => format!("{:?}", outcome)
);
resp.set_success(false);
resp.set_msg(format!("other: {:?}", outcome).as_bytes().to_vec());
} }
} }
Err(e) => { Err(e) => {