diff --git a/beacon_node/beacon_chain/src/beacon_chain.rs b/beacon_node/beacon_chain/src/beacon_chain.rs index 945c37617..dc2cc16df 100644 --- a/beacon_node/beacon_chain/src/beacon_chain.rs +++ b/beacon_node/beacon_chain/src/beacon_chain.rs @@ -15,7 +15,7 @@ use state_processing::per_block_processing::errors::{ }; use state_processing::{ per_block_processing, per_block_processing_without_verifying_block_signature, - per_slot_processing, BlockProcessingError, SlotProcessingError, + per_slot_processing, BlockProcessingError, }; use std::sync::Arc; use store::{Error as DBError, Store}; @@ -23,15 +23,11 @@ use tree_hash::TreeHash; use types::*; #[derive(Debug, PartialEq)] -pub enum ValidBlock { - /// The block was successfully processed. +pub enum BlockProcessingOutcome { + /// Block was valid and imported into the block graph. Processed, -} - -#[derive(Debug, PartialEq)] -pub enum InvalidBlock { - /// Don't re-process the genesis block. - GenesisBlock, + /// The blocks parent_root is unknown. + ParentUnknown { parent: Hash256 }, /// The block slot is greater than the present slot. FutureSlot { present_slot: Slot, @@ -39,53 +35,16 @@ pub enum InvalidBlock { }, /// The block state_root does not match the generated state. StateRootMismatch, - /// The blocks parent_root is unknown. - ParentUnknown { parent: Hash256 }, - /// There was an error whilst advancing the parent state to the present slot. This condition - /// should not occur, it likely represents an internal error. - SlotProcessingError(SlotProcessingError), + /// The block was a genesis block, these blocks cannot be re-imported. + GenesisBlock, + /// The slot is finalized, no need to import. + FinalizedSlot, + /// Block is already known, no need to re-import. + BlockIsAlreadyKnown, /// The block could not be applied to the state, it is invalid. 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 { type Store: store::Store; type SlotClock: slot_clock::SlotClock; @@ -257,88 +216,6 @@ impl BeaconChain { 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, Error> { - let step_by = Slot::from(skip + 1); - - let mut roots: Vec = 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. /// /// ## Errors @@ -649,31 +526,39 @@ impl BeaconChain { /// /// Will accept blocks from prior slots, however it will reject any block from a future slot. pub fn process_block(&self, block: BeaconBlock) -> Result { - debug!("Processing block with slot {}...", block.slot); self.metrics.block_processing_requests.inc(); 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 { - return Ok(BlockProcessingOutcome::InvalidBlock( - InvalidBlock::GenesisBlock, - )); + return Ok(BlockProcessingOutcome::GenesisBlock); } let block_root = block.block_header().canonical_root(); if block_root == self.genesis_block_root { - return Ok(BlockProcessingOutcome::ValidBlock(ValidBlock::Processed)); + return Ok(BlockProcessingOutcome::GenesisBlock); } let present_slot = self.present_slot(); if block.slot > present_slot { - return Ok(BlockProcessingOutcome::InvalidBlock( - InvalidBlock::FutureSlot { - present_slot, - block_slot: block.slot, - }, - )); + return Ok(BlockProcessingOutcome::FutureSlot { + present_slot, + block_slot: block.slot, + }); + } + + if self.store.exists::(&block_root)? { + return Ok(BlockProcessingOutcome::BlockIsAlreadyKnown); } // Load the blocks parent block from the database, returning invalid if that block is not @@ -682,11 +567,9 @@ impl BeaconChain { let parent_block: BeaconBlock = match self.store.get(&parent_block_root)? { Some(previous_block_root) => previous_block_root, None => { - return Ok(BlockProcessingOutcome::InvalidBlock( - InvalidBlock::ParentUnknown { - parent: parent_block_root, - }, - )); + return Ok(BlockProcessingOutcome::ParentUnknown { + parent: parent_block_root, + }); } }; @@ -704,29 +587,25 @@ impl BeaconChain { // Transition the parent state to the block slot. let mut state: BeaconState = parent_state; for _ in state.slot.as_u64()..block.slot.as_u64() { - if let Err(e) = per_slot_processing(&mut state, &self.spec) { - return Ok(BlockProcessingOutcome::InvalidBlock( - InvalidBlock::SlotProcessingError(e), - )); - } + per_slot_processing(&mut state, &self.spec)?; } state.build_committee_cache(RelativeEpoch::Current, &self.spec)?; // Apply the received block to its parent state (which has been transitioned into this // slot). - if let Err(e) = per_block_processing(&mut state, &block, &self.spec) { - return Ok(BlockProcessingOutcome::InvalidBlock( - InvalidBlock::PerBlockProcessingError(e), - )); + match per_block_processing(&mut state, &block, &self.spec) { + Err(BlockProcessingError::BeaconStateError(e)) => { + return Err(Error::BeaconStateError(e)) + } + Err(e) => return Ok(BlockProcessingOutcome::PerBlockProcessingError(e)), + _ => {} } let state_root = state.canonical_root(); if block.state_root != state_root { - return Ok(BlockProcessingOutcome::InvalidBlock( - InvalidBlock::StateRootMismatch, - )); + return Ok(BlockProcessingOutcome::StateRootMismatch); } // Store the block and state. @@ -750,7 +629,7 @@ impl BeaconChain { .observe(block.body.attestations.len() as f64); timer.observe_duration(); - Ok(BlockProcessingOutcome::ValidBlock(ValidBlock::Processed)) + Ok(BlockProcessingOutcome::Processed) } /// Produce a new block at the present slot. diff --git a/beacon_node/beacon_chain/src/lib.rs b/beacon_node/beacon_chain/src/lib.rs index bde541fce..21edb7859 100644 --- a/beacon_node/beacon_chain/src/lib.rs +++ b/beacon_node/beacon_chain/src/lib.rs @@ -5,9 +5,7 @@ pub mod iter; mod metrics; mod persisted_beacon_chain; -pub use self::beacon_chain::{ - BeaconChain, BeaconChainTypes, BlockProcessingOutcome, InvalidBlock, ValidBlock, -}; +pub use self::beacon_chain::{BeaconChain, BeaconChainTypes, BlockProcessingOutcome}; pub use self::checkpoint::CheckPoint; pub use self::errors::{BeaconChainError, BlockProductionError}; pub use fork_choice; diff --git a/beacon_node/network/src/sync/import_queue.rs b/beacon_node/network/src/sync/import_queue.rs index 16a277f0b..90c354cfd 100644 --- a/beacon_node/network/src/sync/import_queue.rs +++ b/beacon_node/network/src/sync/import_queue.rs @@ -212,7 +212,7 @@ impl ImportQueue { // 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 - // root -- this is not possible in the wire protocol however we support it anyway. + // root. self.partials.push(PartialBeaconBlock { slot: header.slot, block_root, diff --git a/beacon_node/network/src/sync/simple_sync.rs b/beacon_node/network/src/sync/simple_sync.rs index 7f6421e32..403a8c54b 100644 --- a/beacon_node/network/src/sync/simple_sync.rs +++ b/beacon_node/network/src/sync/simple_sync.rs @@ -1,6 +1,6 @@ use super::import_queue::ImportQueue; 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::{RPCRequest, RPCResponse, RequestId}; use eth2_libp2p::PeerId; @@ -9,7 +9,6 @@ use std::collections::HashMap; use std::sync::Arc; use std::time::Duration; use store::Store; -use tree_hash::TreeHash; use types::{ Attestation, BeaconBlock, BeaconBlockBody, BeaconBlockHeader, Epoch, EthSpec, Hash256, Slot, }; @@ -24,6 +23,9 @@ const QUEUE_STALE_SECS: u64 = 600; /// Otherwise we queue it. 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. #[derive(Clone, Copy, Debug)] pub struct PeerSyncInfo { @@ -393,6 +395,7 @@ impl SimpleSync { .collect(); roots.reverse(); + roots.dedup(); let headers: Vec = roots .into_iter() @@ -509,6 +512,8 @@ impl SimpleSync { /// 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. pub fn on_block_gossip( &mut self, @@ -516,133 +521,35 @@ impl SimpleSync { block: BeaconBlock, network: &mut NetworkContext, ) -> bool { - // Ignore any block from a finalized slot. - if self.slot_is_finalized(block.slot) { - debug!( - self.log, "IgnoredFinalizedBlock"; - "source" => "gossip", - "msg" => "chain is finalized at block slot", - "block_slot" => block.slot, - ); - return false; - } + if let Some(outcome) = + self.process_block(peer_id.clone(), block.clone(), network, &"gossip") + { + match outcome { + BlockProcessingOutcome::Processed => SHOULD_FORWARD_GOSSIP_BLOCK, + BlockProcessingOutcome::ParentUnknown { .. } => { + self.import_queue + .enqueue_full_blocks(vec![block], peer_id.clone()); - let block_root = Hash256::from_slice(&block.tree_hash_root()); - - // 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 + SHOULD_FORWARD_GOSSIP_BLOCK } - } - Ok(outcome) => { - if outcome.is_invalid() { - // The peer has sent a block which is fundamentally invalid. - warn!( - self.log, "InvalidBlock"; - "source" => "gossip", - "msg" => "peer sent objectively invalid 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 + BlockProcessingOutcome::FutureSlot { + present_slot, + block_slot, + } if present_slot + FUTURE_SLOT_TOLERANCE >= block_slot => { + self.import_queue + .enqueue_full_blocks(vec![block], peer_id.clone()); + + SHOULD_FORWARD_GOSSIP_BLOCK } - } - Err(e) => { - // We encountered an error whilst processing the block. + // Note: known blocks are forwarded on the gossip network. // - // Blocks should not be able to trigger errors, instead they should be flagged as - // invalid. - error!( - self.log, "BlockProcessingError"; - "msg" => "internal error in processing block.", - "source" => "gossip", - "error" => format!("{:?}", e), - ); - // Do not forward the block to peers. - false + // We rely upon the lower layers (libp2p) to stop loops occuring from re-gossiped + // blocks. + BlockProcessingOutcome::BlockIsAlreadyKnown => SHOULD_FORWARD_GOSSIP_BLOCK, + _ => SHOULD_NOT_FORWARD_GOSSIP_BLOCK, } + } else { + SHOULD_NOT_FORWARD_GOSSIP_BLOCK } } @@ -669,57 +576,32 @@ impl SimpleSync { /// the queue. pub fn process_import_queue(&mut self, network: &mut NetworkContext) { let mut successful = 0; - let mut invalid = 0; - let mut errored = 0; // Loop through all of the complete blocks in the queue. for (block_root, block, sender) in self.import_queue.complete_blocks() { - let slot = block.slot; - let parent_root = block.previous_block_root; + let processing_result = self.process_block(sender, block.clone(), network, &"gossip"); - match self.chain.process_block(block) { - Ok(outcome) => { - if outcome.is_invalid() { - invalid += 1; - warn!( - self.log, - "InvalidBlock"; - "sender_peer_id" => format!("{:?}", sender.clone()), - "block_root" => format!("{}", block_root), - "reason" => format!("{:?}", outcome), - ); - network.disconnect(sender, GoodbyeReason::Fault); - } else if outcome.sucessfully_processed() { - successful += 1; - 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)); - } + let should_dequeue = match processing_result { + Some(BlockProcessingOutcome::ParentUnknown { .. }) => false, + Some(BlockProcessingOutcome::FutureSlot { + present_slot, + block_slot, + }) if present_slot + FUTURE_SLOT_TOLERANCE >= block_slot => false, + _ => true, + }; + + if processing_result == Some(BlockProcessingOutcome::Processed) { + successful += 1; + } + + if should_dequeue { + self.import_queue.remove(block_root); } } if successful > 0 { 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. @@ -791,17 +673,128 @@ impl SimpleSync { }) } - /// 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. pub fn generate_hello(&self) -> HelloMessage { 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 { + 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`. diff --git a/beacon_node/rpc/src/beacon_block.rs b/beacon_node/rpc/src/beacon_block.rs index c28c4f111..d36cb1f31 100644 --- a/beacon_node/rpc/src/beacon_block.rs +++ b/beacon_node/rpc/src/beacon_block.rs @@ -1,4 +1,4 @@ -use beacon_chain::{BeaconChain, BeaconChainTypes}; +use beacon_chain::{BeaconChain, BeaconChainTypes, BlockProcessingOutcome}; use crossbeam_channel; use eth2_libp2p::PubsubMessage; use futures::Future; @@ -95,14 +95,12 @@ impl BeaconBlockService for BeaconBlockServiceInstance { Ok(block) => { match self.chain.process_block(block.clone()) { Ok(outcome) => { - if outcome.sucessfully_processed() { + if outcome == BlockProcessingOutcome::Processed { // Block was successfully processed. info!( self.log, - "PublishBeaconBlock"; - "type" => "valid_block", + "Valid block from RPC"; "block_slot" => block.slot, - "outcome" => format!("{:?}", outcome) ); // TODO: Obtain topics from the network service properly. @@ -126,12 +124,11 @@ impl BeaconBlockService for BeaconBlockServiceInstance { }); resp.set_success(true); - } else if outcome.is_invalid() { - // Block was invalid. + } else { + // Block was not successfully processed. warn!( self.log, - "PublishBeaconBlock"; - "type" => "invalid_block", + "Invalid block from RPC"; "outcome" => format!("{:?}", outcome) ); @@ -139,17 +136,6 @@ impl BeaconBlockService for BeaconBlockServiceInstance { resp.set_msg( 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) => {