Increase network limits (#2796)

Fix max packet sizes

Fix max_payload_size function

Add merge block test

Fix max size calculation; fix up test

Clear comments

Add a payload_size_function

Use safe arith for payload calculation

Return an error if block too big in block production

Separate test to check if block is over limit
This commit is contained in:
pawan 2021-11-09 10:42:02 -06:00 committed by Paul Hauner
parent afe59afacd
commit 44a7b37ce3
No known key found for this signature in database
GPG Key ID: 5E2CFF9B75FA63DF
14 changed files with 268 additions and 46 deletions

View File

@ -61,6 +61,7 @@ use safe_arith::SafeArith;
use slasher::Slasher;
use slog::{crit, debug, error, info, trace, warn, Logger};
use slot_clock::SlotClock;
use ssz::Encode;
use state_processing::{
common::get_indexed_attestation,
per_block_processing,
@ -3006,6 +3007,19 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
Signature::empty(),
);
let block_size = block.ssz_bytes_len();
debug!(
self.log,
"Produced block on state";
"block_size" => block_size,
);
metrics::observe(&metrics::BLOCK_SIZE, block_size as f64);
if block_size > self.config.max_network_size {
return Err(BlockProductionError::BlockTooLarge(block_size));
}
let process_timer = metrics::start_timer(&metrics::BLOCK_PRODUCTION_PROCESS_TIMES);
per_block_processing(
&mut state,

View File

@ -16,6 +16,8 @@ pub struct ChainConfig {
pub reconstruct_historic_states: bool,
/// Whether timeouts on `TimeoutRwLock`s are enabled or not.
pub enable_lock_timeouts: bool,
/// The max size of a message that can be sent over the network.
pub max_network_size: usize,
}
impl Default for ChainConfig {
@ -25,6 +27,7 @@ impl Default for ChainConfig {
weak_subjectivity_checkpoint: None,
reconstruct_historic_states: false,
enable_lock_timeouts: true,
max_network_size: 10 * 1_048_576, // 10M
}
}
}

View File

@ -185,6 +185,7 @@ pub enum BlockProductionError {
GetPayloadFailed(execution_layer::Error),
FailedToReadFinalizedBlock(store::Error),
MissingFinalizedBlock(Hash256),
BlockTooLarge(usize),
}
easy_from_to!(BlockProcessingError, BlockProductionError);

View File

@ -107,6 +107,11 @@ lazy_static! {
"Number of attestations in a block"
);
pub static ref BLOCK_SIZE: Result<Histogram> = try_create_histogram(
"beacon_block_total_size",
"Size of a signed beacon block"
);
/*
* Unaggregated Attestation Verification
*/

View File

@ -17,7 +17,7 @@ use std::time::Duration;
use types::{ForkContext, ForkName};
/// The maximum transmit size of gossip messages in bytes.
pub const GOSSIP_MAX_SIZE: usize = 1_048_576;
pub const GOSSIP_MAX_SIZE: usize = 10 * 1_048_576; // 10M
/// This is a constant to be used in discovery. The lower bound of the gossipsub mesh.
pub const MESH_N_LOW: usize = 6;

View File

@ -16,6 +16,8 @@ pub mod rpc;
mod service;
pub mod types;
pub use config::GOSSIP_MAX_SIZE;
use serde::{de, Deserialize, Deserializer, Serialize, Serializer};
use std::str::FromStr;

View File

@ -145,7 +145,7 @@ impl<TSpec: EthSpec> Decoder for SSZSnappyInboundCodec<TSpec> {
// Should not attempt to decode rpc chunks with `length > max_packet_size` or not within bounds of
// packet size for ssz container corresponding to `self.protocol`.
let ssz_limits = self.protocol.rpc_request_limits();
if length > self.max_packet_size || ssz_limits.is_out_of_bounds(length) {
if ssz_limits.is_out_of_bounds(length, self.max_packet_size) {
return Err(RPCError::InvalidData);
}
// Calculate worst case compression length for given uncompressed length
@ -280,7 +280,7 @@ impl<TSpec: EthSpec> Decoder for SSZSnappyOutboundCodec<TSpec> {
// Should not attempt to decode rpc chunks with `length > max_packet_size` or not within bounds of
// packet size for ssz container corresponding to `self.protocol`.
let ssz_limits = self.protocol.rpc_response_limits::<TSpec>();
if length > self.max_packet_size || ssz_limits.is_out_of_bounds(length) {
if ssz_limits.is_out_of_bounds(length, self.max_packet_size) {
return Err(RPCError::InvalidData);
}
// Calculate worst case compression length for given uncompressed length

View File

@ -30,7 +30,7 @@ pub use methods::{
RPCResponseErrorCode, RequestId, ResponseTermination, StatusMessage, MAX_REQUEST_BLOCKS,
};
pub(crate) use outbound::OutboundRequest;
pub use protocol::{Protocol, RPCError};
pub use protocol::{Protocol, RPCError, MAX_RPC_SIZE};
pub(crate) mod codec;
mod handler;

View File

@ -60,12 +60,10 @@ lazy_static! {
)
.as_ssz_bytes()
.len();
pub static ref SIGNED_BEACON_BLOCK_MERGE_MAX: usize = SignedBeaconBlock::<MainnetEthSpec>::from_block(
BeaconBlock::Merge(BeaconBlockMerge::full(&MainnetEthSpec::default_spec())),
Signature::empty(),
)
.as_ssz_bytes()
.len();
/// The `BeaconBlockMerge` block has an `ExecutionPayload` field which has a max size ~16 GiB for future proofing.
/// We calculate the value from its fields instead of constructing the block and checking the length.
pub static ref SIGNED_BEACON_BLOCK_MERGE_MAX: usize = types::ExecutionPayload::<MainnetEthSpec>::max_execution_payload_size();
pub static ref BLOCKS_BY_ROOT_REQUEST_MIN: usize =
VariableList::<Hash256, MaxRequestBlocks>::from(Vec::<Hash256>::new())
@ -95,7 +93,7 @@ lazy_static! {
}
/// The maximum bytes that can be sent across the RPC.
pub const MAX_RPC_SIZE: usize = 1_048_576; // 1M
pub const MAX_RPC_SIZE: usize = 10 * 1_048_576; // 10M
/// The protocol prefix the RPC protocol id.
const PROTOCOL_PREFIX: &str = "/eth2/beacon_chain/req";
/// Time allowed for the first byte of a request to arrive before we time out (Time To First Byte).
@ -208,9 +206,10 @@ impl RpcLimits {
Self { min, max }
}
/// Returns true if the given length is out of bounds, false otherwise.
pub fn is_out_of_bounds(&self, length: usize) -> bool {
length > self.max || length < self.min
/// Returns true if the given length is is greater than `MAX_RPC_SIZE` or out of
/// bounds for the given ssz type, returns false otherwise.
pub fn is_out_of_bounds(&self, length: usize, max_rpc_size: usize) -> bool {
length > std::cmp::min(self.max, max_rpc_size) || length < self.min
}
}

View File

@ -22,6 +22,7 @@ fn fork_context() -> ForkContext {
// Set fork_epoch to `Some` to ensure that the `ForkContext` object
// includes altair in the list of forks
chain_spec.altair_fork_epoch = Some(types::Epoch::new(42));
chain_spec.merge_fork_epoch = Some(types::Epoch::new(84));
ForkContext::new::<E>(types::Slot::new(0), Hash256::zero(), &chain_spec)
}

View File

@ -1,21 +1,52 @@
#![cfg(test)]
use lighthouse_network::rpc::methods::*;
use lighthouse_network::{BehaviourEvent, Libp2pEvent, ReportSource, Request, Response};
use lighthouse_network::{
rpc::MAX_RPC_SIZE, BehaviourEvent, Libp2pEvent, ReportSource, Request, Response,
};
use slog::{debug, warn, Level};
use ssz::Encode;
use ssz_types::VariableList;
use std::sync::Arc;
use std::time::Duration;
use tokio::runtime::Runtime;
use tokio::time::sleep;
use types::{
BeaconBlock, BeaconBlockAltair, BeaconBlockBase, Epoch, EthSpec, Hash256, MinimalEthSpec,
Signature, SignedBeaconBlock, Slot,
BeaconBlock, BeaconBlockAltair, BeaconBlockBase, BeaconBlockMerge, Epoch, EthSpec, Hash256,
MinimalEthSpec, Signature, SignedBeaconBlock, Slot,
};
mod common;
type E = MinimalEthSpec;
/// Merge block with length < MAX_RPC_SIZE.
fn merge_block_small() -> BeaconBlock<E> {
let mut block = BeaconBlockMerge::empty(&E::default_spec());
let tx = VariableList::from(vec![0; 1024]);
let txs = VariableList::from(std::iter::repeat(tx).take(100).collect::<Vec<_>>());
block.body.execution_payload.transactions = txs;
let block = BeaconBlock::Merge(block);
assert!(block.ssz_bytes_len() <= MAX_RPC_SIZE);
block
}
/// Merge block with length > MAX_RPC_SIZE.
/// The max limit for a merge block is in the order of ~16GiB which wouldn't fit in memory.
/// Hence, we generate a merge block just greater than `MAX_RPC_SIZE` to test rejection on the rpc layer.
fn merge_block_large() -> BeaconBlock<E> {
let mut block = BeaconBlockMerge::empty(&E::default_spec());
let tx = VariableList::from(vec![0; 1024]);
let txs = VariableList::from(std::iter::repeat(tx).take(100000).collect::<Vec<_>>());
block.body.execution_payload.transactions = txs;
let block = BeaconBlock::Merge(block);
assert!(block.ssz_bytes_len() > MAX_RPC_SIZE);
block
}
// Tests the STATUS RPC message
#[test]
#[allow(clippy::single_match)]
@ -118,10 +149,10 @@ fn test_status_rpc() {
#[allow(clippy::single_match)]
fn test_blocks_by_range_chunked_rpc() {
// set up the logging. The level and enabled logging or not
let log_level = Level::Trace;
let log_level = Level::Debug;
let enable_logging = false;
let messages_to_send = 10;
let messages_to_send = 6;
let log = common::build_log(log_level, enable_logging);
@ -149,8 +180,13 @@ fn test_blocks_by_range_chunked_rpc() {
let signed_full_block = SignedBeaconBlock::from_block(full_block, Signature::empty());
let rpc_response_altair = Response::BlocksByRange(Some(Box::new(signed_full_block)));
let full_block = merge_block_small();
let signed_full_block = SignedBeaconBlock::from_block(full_block, Signature::empty());
let rpc_response_merge_small = Response::BlocksByRange(Some(Box::new(signed_full_block)));
// keep count of the number of messages received
let mut messages_received = 0;
let request_id = RequestId::Sync(messages_to_send as usize);
// build the sender future
let sender_future = async {
loop {
@ -160,28 +196,30 @@ fn test_blocks_by_range_chunked_rpc() {
debug!(log, "Sending RPC");
sender.swarm.behaviour_mut().send_request(
peer_id,
RequestId::Sync(10),
request_id,
rpc_request.clone(),
);
}
Libp2pEvent::Behaviour(BehaviourEvent::ResponseReceived {
peer_id: _,
id: RequestId::Sync(10),
id: _,
response,
}) => {
warn!(log, "Sender received a response");
match response {
Response::BlocksByRange(Some(_)) => {
if messages_received < 5 {
if messages_received < 2 {
assert_eq!(response, rpc_response_base.clone());
} else {
} else if messages_received < 4 {
assert_eq!(response, rpc_response_altair.clone());
} else {
assert_eq!(response, rpc_response_merge_small.clone());
}
messages_received += 1;
warn!(log, "Chunk received");
}
Response::BlocksByRange(None) => {
// should be exactly 10 messages before terminating
// should be exactly `messages_to_send` messages before terminating
assert_eq!(messages_received, messages_to_send);
// end the test
return;
@ -207,12 +245,14 @@ fn test_blocks_by_range_chunked_rpc() {
// send the response
warn!(log, "Receiver got request");
for i in 0..messages_to_send {
// Send first half of responses as base blocks and
// second half as altair blocks.
let rpc_response = if i < 5 {
// Send first third of responses as base blocks,
// second as altair and third as merge.
let rpc_response = if i < 2 {
rpc_response_base.clone()
} else {
} else if i < 4 {
rpc_response_altair.clone()
} else {
rpc_response_merge_small.clone()
};
receiver.swarm.behaviour_mut().send_successful_response(
peer_id,
@ -236,8 +276,105 @@ fn test_blocks_by_range_chunked_rpc() {
tokio::select! {
_ = sender_future => {}
_ = receiver_future => {}
_ = sleep(Duration::from_secs(10)) => {
panic!("Future timed out");
_ = sleep(Duration::from_secs(30)) => {
panic!("Future timed out");
}
}
})
}
// Tests rejection of blocks over `MAX_RPC_SIZE`.
#[test]
#[allow(clippy::single_match)]
fn test_blocks_by_range_over_limit() {
// set up the logging. The level and enabled logging or not
let log_level = Level::Debug;
let enable_logging = false;
let messages_to_send = 5;
let log = common::build_log(log_level, enable_logging);
let rt = Arc::new(Runtime::new().unwrap());
rt.block_on(async {
// get sender/receiver
let (mut sender, mut receiver) = common::build_node_pair(Arc::downgrade(&rt), &log).await;
// BlocksByRange Request
let rpc_request = Request::BlocksByRange(BlocksByRangeRequest {
start_slot: 0,
count: messages_to_send,
step: 0,
});
// BlocksByRange Response
let full_block = merge_block_large();
let signed_full_block = SignedBeaconBlock::from_block(full_block, Signature::empty());
let rpc_response_merge_large = Response::BlocksByRange(Some(Box::new(signed_full_block)));
let request_id = RequestId::Sync(messages_to_send as usize);
// build the sender future
let sender_future = async {
loop {
match sender.next_event().await {
Libp2pEvent::Behaviour(BehaviourEvent::PeerConnectedOutgoing(peer_id)) => {
// Send a STATUS message
debug!(log, "Sending RPC");
sender.swarm.behaviour_mut().send_request(
peer_id,
request_id,
rpc_request.clone(),
);
}
// The request will fail because the sender will refuse to send anything > MAX_RPC_SIZE
Libp2pEvent::Behaviour(BehaviourEvent::RPCFailed { id, .. }) => {
assert_eq!(id, request_id);
return;
}
_ => {} // Ignore other behaviour events
}
}
};
// build the receiver future
let receiver_future = async {
loop {
match receiver.next_event().await {
Libp2pEvent::Behaviour(BehaviourEvent::RequestReceived {
peer_id,
id,
request,
}) => {
if request == rpc_request {
// send the response
warn!(log, "Receiver got request");
for _ in 0..messages_to_send {
let rpc_response = rpc_response_merge_large.clone();
receiver.swarm.behaviour_mut().send_successful_response(
peer_id,
id,
rpc_response.clone(),
);
}
// send the stream termination
receiver.swarm.behaviour_mut().send_successful_response(
peer_id,
id,
Response::BlocksByRange(None),
);
}
}
_ => {} // Ignore other events
}
}
};
tokio::select! {
_ = sender_future => {}
_ = receiver_future => {}
_ = sleep(Duration::from_secs(30)) => {
panic!("Future timed out");
}
}
})
@ -276,6 +413,7 @@ fn test_blocks_by_range_chunked_rpc_terminates_correctly() {
// keep count of the number of messages received
let mut messages_received: u64 = 0;
let request_id = RequestId::Sync(messages_to_send as usize);
// build the sender future
let sender_future = async {
loop {
@ -285,13 +423,13 @@ fn test_blocks_by_range_chunked_rpc_terminates_correctly() {
debug!(log, "Sending RPC");
sender.swarm.behaviour_mut().send_request(
peer_id,
RequestId::Sync(10),
request_id,
rpc_request.clone(),
);
}
Libp2pEvent::Behaviour(BehaviourEvent::ResponseReceived {
peer_id: _,
id: RequestId::Sync(10),
id: _,
response,
}) =>
// Should receive the RPC response
@ -497,7 +635,7 @@ fn test_blocks_by_root_chunked_rpc() {
let log_level = Level::Debug;
let enable_logging = false;
let messages_to_send = 10;
let messages_to_send = 6;
let log = common::build_log(log_level, enable_logging);
let spec = E::default_spec();
@ -516,10 +654,6 @@ fn test_blocks_by_root_chunked_rpc() {
Hash256::from_low_u64_be(0),
Hash256::from_low_u64_be(0),
Hash256::from_low_u64_be(0),
Hash256::from_low_u64_be(0),
Hash256::from_low_u64_be(0),
Hash256::from_low_u64_be(0),
Hash256::from_low_u64_be(0),
]),
});
@ -532,6 +666,10 @@ fn test_blocks_by_root_chunked_rpc() {
let signed_full_block = SignedBeaconBlock::from_block(full_block, Signature::empty());
let rpc_response_altair = Response::BlocksByRoot(Some(Box::new(signed_full_block)));
let full_block = merge_block_small();
let signed_full_block = SignedBeaconBlock::from_block(full_block, Signature::empty());
let rpc_response_merge_small = Response::BlocksByRoot(Some(Box::new(signed_full_block)));
// keep count of the number of messages received
let mut messages_received = 0;
// build the sender future
@ -543,20 +681,22 @@ fn test_blocks_by_root_chunked_rpc() {
debug!(log, "Sending RPC");
sender.swarm.behaviour_mut().send_request(
peer_id,
RequestId::Sync(10),
RequestId::Sync(6),
rpc_request.clone(),
);
}
Libp2pEvent::Behaviour(BehaviourEvent::ResponseReceived {
peer_id: _,
id: RequestId::Sync(10),
id: RequestId::Sync(6),
response,
}) => match response {
Response::BlocksByRoot(Some(_)) => {
if messages_received < 5 {
if messages_received < 2 {
assert_eq!(response, rpc_response_base.clone());
} else {
} else if messages_received < 4 {
assert_eq!(response, rpc_response_altair.clone());
} else {
assert_eq!(response, rpc_response_merge_small.clone());
}
messages_received += 1;
debug!(log, "Chunk received");
@ -588,12 +728,13 @@ fn test_blocks_by_root_chunked_rpc() {
debug!(log, "Receiver got request");
for i in 0..messages_to_send {
// Send first half of responses as base blocks and
// second half as altair blocks.
let rpc_response = if i < 5 {
// Send equal base, altair and merge blocks
let rpc_response = if i < 2 {
rpc_response_base.clone()
} else {
} else if i < 4 {
rpc_response_altair.clone()
} else {
rpc_response_merge_small.clone()
};
receiver.swarm.behaviour_mut().send_successful_response(
peer_id,
@ -619,7 +760,7 @@ fn test_blocks_by_root_chunked_rpc() {
_ = sender_future => {}
_ = receiver_future => {}
_ = sleep(Duration::from_secs(30)) => {
panic!("Future timed out");
panic!("Future timed out");
}
}
})

View File

@ -452,6 +452,8 @@ pub fn get_config<E: EthSpec>(
};
}
client_config.chain.max_network_size = lighthouse_network::GOSSIP_MAX_SIZE;
if cli_args.is_present("slasher") {
let slasher_dir = if let Some(slasher_dir) = cli_args.value_of("slasher-dir") {
PathBuf::from(slasher_dir)

View File

@ -210,6 +210,11 @@ pub trait EthSpec: 'static + Default + Sync + Send + Clone + Debug + PartialEq +
Self::MaxTransactionsPerPayload::to_usize()
}
/// Returns the `MAX_EXTRA_DATA_BYTES` constant for this specification.
fn max_extra_data_bytes() -> usize {
Self::MaxExtraDataBytes::to_usize()
}
/// Returns the `BYTES_PER_LOGS_BLOOM` constant for this specification.
fn bytes_per_logs_bloom() -> usize {
Self::BytesPerLogsBloom::to_usize()

View File

@ -1,5 +1,7 @@
use crate::{test_utils::TestRandom, *};
use safe_arith::{ArithError, SafeArith};
use serde_derive::{Deserialize, Serialize};
use ssz::Encode;
use ssz_derive::{Decode, Encode};
use test_random_derive::TestRandom;
use tree_hash_derive::TreeHash;
@ -57,4 +59,51 @@ impl<T: EthSpec> ExecutionPayload<T> {
transactions: VariableList::empty(),
}
}
/// Returns the ssz size of `self`.
pub fn payload_size(&self) -> Result<usize, ArithError> {
let mut tx_size = ssz::BYTES_PER_LENGTH_OFFSET.safe_mul(self.transactions.len())?;
for tx in self.transactions.iter() {
tx_size.safe_add_assign(tx.len())?;
}
Self::empty()
.as_ssz_bytes()
.len()
.safe_add(<u8 as Encode>::ssz_fixed_len().safe_mul(self.extra_data.len())?)?
.safe_add(tx_size)
}
#[allow(clippy::integer_arithmetic)]
/// Returns the maximum size of an execution payload.
pub fn max_execution_payload_size() -> usize {
// Fixed part
Self::empty().as_ssz_bytes().len()
// Max size of variable length `extra_data` field
+ (T::max_extra_data_bytes() * <u8 as Encode>::ssz_fixed_len())
// Max size of variable length `transactions` field
+ (T::max_transactions_per_payload() * (ssz::BYTES_PER_LENGTH_OFFSET + T::max_bytes_per_transaction()))
}
}
#[cfg(test)]
mod tests {
use super::*;
#[test]
fn test_payload_size() {
let mut payload = ExecutionPayload::<crate::MainnetEthSpec>::empty();
assert_eq!(
payload.as_ssz_bytes().len(),
payload.payload_size().unwrap()
);
payload.extra_data = VariableList::from(vec![42; 16]);
payload.transactions = VariableList::from(vec![VariableList::from(vec![42; 42])]);
assert_eq!(
payload.as_ssz_bytes().len(),
payload.payload_size().unwrap()
);
}
}