Reduce false positive logging for late builder blocks (#4073)

## Issue Addressed

NA

## Proposed Changes

When producing a block from a builder, there are two points where we could consider the block "broadcast":

1. When the blinded block is published to the builder.
2. When the un-blinded block is published to the P2P network (this is always *after* the previous step).

Our logging for late block broadcasts was using (2) for builder-blocks, which was creating a lot of false-positive logs. This is because the builder publishes the block on the P2P network themselves before returning it to us and we perform (2). For clarity, the logs were false-positives because we claim that the block was published late by us when it was actually published earlier by the builder.

This PR changes our logging behavior so we do our logging at (1) instead. It also updates our metrics for block broadcast to distinguish between local and builder blocks. I believe the metrics change will be natively compatible with existing Grafana dashboards.

## Additional Info

One could argue that the builder *should* return the block to us faster, however that's not the case. I think it's more important that we don't desensitize users with false-positives.
This commit is contained in:
Paul Hauner 2023-03-17 00:44:03 +00:00
parent 4c2d4af6cd
commit c3e5053612
6 changed files with 130 additions and 58 deletions

View File

@ -15,6 +15,7 @@ use std::io;
use std::marker::PhantomData; use std::marker::PhantomData;
use std::str::Utf8Error; use std::str::Utf8Error;
use std::time::{Duration, SystemTime, UNIX_EPOCH}; use std::time::{Duration, SystemTime, UNIX_EPOCH};
use store::AbstractExecPayload;
use types::{ use types::{
AttesterSlashing, BeaconBlockRef, BeaconState, ChainSpec, Epoch, EthSpec, Hash256, AttesterSlashing, BeaconBlockRef, BeaconState, ChainSpec, Epoch, EthSpec, Hash256,
IndexedAttestation, ProposerSlashing, PublicKeyBytes, SignedAggregateAndProof, IndexedAttestation, ProposerSlashing, PublicKeyBytes, SignedAggregateAndProof,
@ -1736,9 +1737,9 @@ fn u64_to_i64(n: impl Into<u64>) -> i64 {
} }
/// Returns the delay between the start of `block.slot` and `seen_timestamp`. /// Returns the delay between the start of `block.slot` and `seen_timestamp`.
pub fn get_block_delay_ms<T: EthSpec, S: SlotClock>( pub fn get_block_delay_ms<T: EthSpec, S: SlotClock, P: AbstractExecPayload<T>>(
seen_timestamp: Duration, seen_timestamp: Duration,
block: BeaconBlockRef<'_, T>, block: BeaconBlockRef<'_, T, P>,
slot_clock: &S, slot_clock: &S,
) -> Duration { ) -> Duration {
get_slot_delay_ms::<S>(seen_timestamp, block.slot(), slot_clock) get_slot_delay_ms::<S>(seen_timestamp, block.slot(), slot_clock)

View File

@ -77,7 +77,7 @@ const DEFAULT_SUGGESTED_FEE_RECIPIENT: [u8; 20] =
const CONFIG_POLL_INTERVAL: Duration = Duration::from_secs(60); const CONFIG_POLL_INTERVAL: Duration = Duration::from_secs(60);
/// A payload alongside some information about where it came from. /// A payload alongside some information about where it came from.
enum ProvenancedPayload<P> { pub enum ProvenancedPayload<P> {
/// A good ol' fashioned farm-to-table payload from your local EE. /// A good ol' fashioned farm-to-table payload from your local EE.
Local(P), Local(P),
/// A payload from a builder (e.g. mev-boost). /// A payload from a builder (e.g. mev-boost).

View File

@ -37,6 +37,7 @@ use lighthouse_version::version_with_platform;
use network::{NetworkMessage, NetworkSenders, ValidatorSubscriptionMessage}; use network::{NetworkMessage, NetworkSenders, ValidatorSubscriptionMessage};
use operation_pool::ReceivedPreCapella; use operation_pool::ReceivedPreCapella;
use parking_lot::RwLock; use parking_lot::RwLock;
use publish_blocks::ProvenancedBlock;
use serde::{Deserialize, Serialize}; use serde::{Deserialize, Serialize};
use slog::{crit, debug, error, info, warn, Logger}; use slog::{crit, debug, error, info, warn, Logger};
use slot_clock::SlotClock; use slot_clock::SlotClock;
@ -1123,7 +1124,13 @@ pub fn serve<T: BeaconChainTypes>(
chain: Arc<BeaconChain<T>>, chain: Arc<BeaconChain<T>>,
network_tx: UnboundedSender<NetworkMessage<T::EthSpec>>, network_tx: UnboundedSender<NetworkMessage<T::EthSpec>>,
log: Logger| async move { log: Logger| async move {
publish_blocks::publish_block(None, block, chain, &network_tx, log) publish_blocks::publish_block(
None,
ProvenancedBlock::Local(block),
chain,
&network_tx,
log,
)
.await .await
.map(|()| warp::reply().into_response()) .map(|()| warp::reply().into_response())
}, },

View File

@ -29,9 +29,10 @@ lazy_static::lazy_static! {
"http_api_beacon_proposer_cache_misses_total", "http_api_beacon_proposer_cache_misses_total",
"Count of times the proposer cache has been missed", "Count of times the proposer cache has been missed",
); );
pub static ref HTTP_API_BLOCK_BROADCAST_DELAY_TIMES: Result<Histogram> = try_create_histogram( pub static ref HTTP_API_BLOCK_BROADCAST_DELAY_TIMES: Result<HistogramVec> = try_create_histogram_vec(
"http_api_block_broadcast_delay_times", "http_api_block_broadcast_delay_times",
"Time between start of the slot and when the block was broadcast" "Time between start of the slot and when the block was broadcast",
&["provenance"]
); );
pub static ref HTTP_API_BLOCK_PUBLISHED_LATE_TOTAL: Result<IntCounter> = try_create_int_counter( pub static ref HTTP_API_BLOCK_PUBLISHED_LATE_TOTAL: Result<IntCounter> = try_create_int_counter(
"http_api_block_published_late_total", "http_api_block_published_late_total",

View File

@ -3,28 +3,43 @@ use beacon_chain::validator_monitor::{get_block_delay_ms, timestamp_now};
use beacon_chain::{ use beacon_chain::{
BeaconChain, BeaconChainTypes, BlockError, CountUnrealized, NotifyExecutionLayer, BeaconChain, BeaconChainTypes, BlockError, CountUnrealized, NotifyExecutionLayer,
}; };
use execution_layer::ProvenancedPayload;
use lighthouse_network::PubsubMessage; use lighthouse_network::PubsubMessage;
use network::NetworkMessage; use network::NetworkMessage;
use slog::{debug, error, info, warn, Logger}; use slog::{debug, error, info, warn, Logger};
use slot_clock::SlotClock; use slot_clock::SlotClock;
use std::sync::Arc; use std::sync::Arc;
use std::time::Duration;
use tokio::sync::mpsc::UnboundedSender; use tokio::sync::mpsc::UnboundedSender;
use tree_hash::TreeHash; use tree_hash::TreeHash;
use types::{ use types::{
AbstractExecPayload, BlindedPayload, EthSpec, ExecPayload, ExecutionBlockHash, FullPayload, AbstractExecPayload, BeaconBlockRef, BlindedPayload, EthSpec, ExecPayload, ExecutionBlockHash,
Hash256, SignedBeaconBlock, FullPayload, Hash256, SignedBeaconBlock,
}; };
use warp::Rejection; use warp::Rejection;
pub enum ProvenancedBlock<T: EthSpec> {
/// The payload was built using a local EE.
Local(Arc<SignedBeaconBlock<T, FullPayload<T>>>),
/// The payload was build using a remote builder (e.g., via a mev-boost
/// compatible relay).
Builder(Arc<SignedBeaconBlock<T, FullPayload<T>>>),
}
/// Handles a request from the HTTP API for full blocks. /// Handles a request from the HTTP API for full blocks.
pub async fn publish_block<T: BeaconChainTypes>( pub async fn publish_block<T: BeaconChainTypes>(
block_root: Option<Hash256>, block_root: Option<Hash256>,
block: Arc<SignedBeaconBlock<T::EthSpec>>, provenanced_block: ProvenancedBlock<T::EthSpec>,
chain: Arc<BeaconChain<T>>, chain: Arc<BeaconChain<T>>,
network_tx: &UnboundedSender<NetworkMessage<T::EthSpec>>, network_tx: &UnboundedSender<NetworkMessage<T::EthSpec>>,
log: Logger, log: Logger,
) -> Result<(), Rejection> { ) -> Result<(), Rejection> {
let seen_timestamp = timestamp_now(); let seen_timestamp = timestamp_now();
let (block, is_locally_built_block) = match provenanced_block {
ProvenancedBlock::Local(block) => (block, true),
ProvenancedBlock::Builder(block) => (block, false),
};
let delay = get_block_delay_ms(seen_timestamp, block.message(), &chain.slot_clock);
debug!( debug!(
log, log,
@ -38,10 +53,6 @@ pub async fn publish_block<T: BeaconChainTypes>(
let message = PubsubMessage::BeaconBlock(block.clone()); let message = PubsubMessage::BeaconBlock(block.clone());
crate::publish_pubsub_message(network_tx, message)?; crate::publish_pubsub_message(network_tx, message)?;
// Determine the delay after the start of the slot, register it with metrics.
let delay = get_block_delay_ms(seen_timestamp, block.message(), &chain.slot_clock);
metrics::observe_duration(&metrics::HTTP_API_BLOCK_BROADCAST_DELAY_TIMES, delay);
let block_root = block_root.unwrap_or_else(|| block.canonical_root()); let block_root = block_root.unwrap_or_else(|| block.canonical_root());
match chain match chain
@ -75,31 +86,11 @@ pub async fn publish_block<T: BeaconChainTypes>(
// head. // head.
chain.recompute_head_at_current_slot().await; chain.recompute_head_at_current_slot().await;
// Perform some logging to inform users if their blocks are being produced // Only perform late-block logging here if the block is local. For
// late. // blocks built with builders we consider the broadcast time to be
// // when the blinded block is published to the builder.
// Check to see the thresholds are non-zero to avoid logging errors with small if is_locally_built_block {
// slot times (e.g., during testing) late_block_logging(&chain, seen_timestamp, block.message(), root, "local", &log)
let too_late_threshold = chain.slot_clock.unagg_attestation_production_delay();
let delayed_threshold = too_late_threshold / 2;
if delay >= too_late_threshold {
error!(
log,
"Block was broadcast too late";
"msg" => "system may be overloaded, block likely to be orphaned",
"delay_ms" => delay.as_millis(),
"slot" => block.slot(),
"root" => ?root,
)
} else if delay >= delayed_threshold {
error!(
log,
"Block broadcast was delayed";
"msg" => "system may be overloaded, block may be orphaned",
"delay_ms" => delay.as_millis(),
"slot" => block.slot(),
"root" => ?root,
)
} }
Ok(()) Ok(())
@ -147,14 +138,7 @@ pub async fn publish_blinded_block<T: BeaconChainTypes>(
) -> Result<(), Rejection> { ) -> Result<(), Rejection> {
let block_root = block.canonical_root(); let block_root = block.canonical_root();
let full_block = reconstruct_block(chain.clone(), block_root, block, log.clone()).await?; let full_block = reconstruct_block(chain.clone(), block_root, block, log.clone()).await?;
publish_block::<T>( publish_block::<T>(Some(block_root), full_block, chain, network_tx, log).await
Some(block_root),
Arc::new(full_block),
chain,
network_tx,
log,
)
.await
} }
/// Deconstruct the given blinded block, and construct a full block. This attempts to use the /// Deconstruct the given blinded block, and construct a full block. This attempts to use the
@ -165,15 +149,15 @@ async fn reconstruct_block<T: BeaconChainTypes>(
block_root: Hash256, block_root: Hash256,
block: SignedBeaconBlock<T::EthSpec, BlindedPayload<T::EthSpec>>, block: SignedBeaconBlock<T::EthSpec, BlindedPayload<T::EthSpec>>,
log: Logger, log: Logger,
) -> Result<SignedBeaconBlock<T::EthSpec, FullPayload<T::EthSpec>>, Rejection> { ) -> Result<ProvenancedBlock<T::EthSpec>, Rejection> {
let full_payload = if let Ok(payload_header) = block.message().body().execution_payload() { let full_payload_opt = if let Ok(payload_header) = block.message().body().execution_payload() {
let el = chain.execution_layer.as_ref().ok_or_else(|| { let el = chain.execution_layer.as_ref().ok_or_else(|| {
warp_utils::reject::custom_server_error("Missing execution layer".to_string()) warp_utils::reject::custom_server_error("Missing execution layer".to_string())
})?; })?;
// If the execution block hash is zero, use an empty payload. // If the execution block hash is zero, use an empty payload.
let full_payload = if payload_header.block_hash() == ExecutionBlockHash::zero() { let full_payload = if payload_header.block_hash() == ExecutionBlockHash::zero() {
FullPayload::default_at_fork( let payload = FullPayload::default_at_fork(
chain chain
.spec .spec
.fork_name_at_epoch(block.slot().epoch(T::EthSpec::slots_per_epoch())), .fork_name_at_epoch(block.slot().epoch(T::EthSpec::slots_per_epoch())),
@ -183,15 +167,30 @@ async fn reconstruct_block<T: BeaconChainTypes>(
"Default payload construction error: {e:?}" "Default payload construction error: {e:?}"
)) ))
})? })?
.into() .into();
ProvenancedPayload::Local(payload)
// If we already have an execution payload with this transactions root cached, use it. // If we already have an execution payload with this transactions root cached, use it.
} else if let Some(cached_payload) = } else if let Some(cached_payload) =
el.get_payload_by_root(&payload_header.tree_hash_root()) el.get_payload_by_root(&payload_header.tree_hash_root())
{ {
info!(log, "Reconstructing a full block using a local payload"; "block_hash" => ?cached_payload.block_hash()); info!(log, "Reconstructing a full block using a local payload"; "block_hash" => ?cached_payload.block_hash());
cached_payload ProvenancedPayload::Local(cached_payload)
// Otherwise, this means we are attempting a blind block proposal. // Otherwise, this means we are attempting a blind block proposal.
} else { } else {
// Perform the logging for late blocks when we publish to the
// builder, rather than when we publish to the network. This helps
// prevent false positive logs when the builder publishes to the P2P
// network significantly earlier than when they return the block to
// us.
late_block_logging(
&chain,
timestamp_now(),
block.message(),
block_root,
"builder",
&log,
);
let full_payload = el let full_payload = el
.propose_blinded_beacon_block(block_root, &block) .propose_blinded_beacon_block(block_root, &block)
.await .await
@ -202,7 +201,7 @@ async fn reconstruct_block<T: BeaconChainTypes>(
)) ))
})?; })?;
info!(log, "Successfully published a block to the builder network"; "block_hash" => ?full_payload.block_hash()); info!(log, "Successfully published a block to the builder network"; "block_hash" => ?full_payload.block_hash());
full_payload ProvenancedPayload::Builder(full_payload)
}; };
Some(full_payload) Some(full_payload)
@ -210,7 +209,71 @@ async fn reconstruct_block<T: BeaconChainTypes>(
None None
}; };
block.try_into_full_block(full_payload).ok_or_else(|| { match full_payload_opt {
// A block without a payload is pre-merge and we consider it locally
// built.
None => block
.try_into_full_block(None)
.map(Arc::new)
.map(ProvenancedBlock::Local),
Some(ProvenancedPayload::Local(full_payload)) => block
.try_into_full_block(Some(full_payload))
.map(Arc::new)
.map(ProvenancedBlock::Local),
Some(ProvenancedPayload::Builder(full_payload)) => block
.try_into_full_block(Some(full_payload))
.map(Arc::new)
.map(ProvenancedBlock::Builder),
}
.ok_or_else(|| {
warp_utils::reject::custom_server_error("Unable to add payload to block".to_string()) warp_utils::reject::custom_server_error("Unable to add payload to block".to_string())
}) })
} }
/// If the `seen_timestamp` is some time after the start of the slot for
/// `block`, create some logs to indicate that the block was published late.
fn late_block_logging<T: BeaconChainTypes, P: AbstractExecPayload<T::EthSpec>>(
chain: &BeaconChain<T>,
seen_timestamp: Duration,
block: BeaconBlockRef<T::EthSpec, P>,
root: Hash256,
provenance: &str,
log: &Logger,
) {
let delay = get_block_delay_ms(seen_timestamp, block, &chain.slot_clock);
metrics::observe_timer_vec(
&metrics::HTTP_API_BLOCK_BROADCAST_DELAY_TIMES,
&[provenance],
delay,
);
// Perform some logging to inform users if their blocks are being produced
// late.
//
// Check to see the thresholds are non-zero to avoid logging errors with small
// slot times (e.g., during testing)
let too_late_threshold = chain.slot_clock.unagg_attestation_production_delay();
let delayed_threshold = too_late_threshold / 2;
if delay >= too_late_threshold {
error!(
log,
"Block was broadcast too late";
"msg" => "system may be overloaded, block likely to be orphaned",
"provenance" => provenance,
"delay_ms" => delay.as_millis(),
"slot" => block.slot(),
"root" => ?root,
)
} else if delay >= delayed_threshold {
error!(
log,
"Block broadcast was delayed";
"msg" => "system may be overloaded, block may be orphaned",
"provenance" => provenance,
"delay_ms" => delay.as_millis(),
"slot" => block.slot(),
"root" => ?root,
)
}
}

View File

@ -12,7 +12,7 @@ use test_random_derive::TestRandom;
use tree_hash::TreeHash; use tree_hash::TreeHash;
use tree_hash_derive::TreeHash; use tree_hash_derive::TreeHash;
#[derive(Debug)] #[derive(Debug, PartialEq)]
pub enum BlockType { pub enum BlockType {
Blinded, Blinded,
Full, Full,