Self rate limiting dev flag (#3928)

## Issue Addressed
Adds self rate limiting options, mainly with the idea to comply with peer's rate limits in small testnets

## Proposed Changes
Add a hidden flag `self-limiter` this can take no value, or customs values to configure quotas per protocol

## Additional Info
### How to use
`--self-limiter` will turn on the self rate limiter applying the same params we apply to inbound requests (requests from other peers)
`--self-limiter "beacon_blocks_by_range:64/1"` will turn on the self rate limiter for ALL protocols, but change the quota for bbrange to 64 requested blocks per 1 second.
`--self-limiter "beacon_blocks_by_range:64/1;ping:1/10"` same as previous one, changing the quota for ping as well.

### Caveats
- The rate limiter is either on or off for all protocols. I added the custom values to be able to change the quotas per protocol so that some protocols can be given extremely loose or tight quotas. I think this should satisfy every need even if we can't technically turn off rate limits per protocol.
- This reuses the rate limiter struct for the inbound requests so there is this ugly part of the code in which we need to deal with the inbound only protocols (light client stuff) if this becomes too ugly as we add lc protocols, we might want to split the rate limiters. I've checked this and looks doable with const generics to avoid so much code duplication

### Knowing if this is on
```
Feb 06 21:12:05.493 DEBG Using self rate limiting params         config: OutboundRateLimiterConfig { ping: 2/10s, metadata: 1/15s, status: 5/15s, goodbye: 1/10s, blocks_by_range: 1024/10s, blocks_by_root: 128/10s }, service: libp2p_rpc, service: libp2p
```
This commit is contained in:
Divma 2023-02-08 02:18:53 +00:00
parent 7934485aef
commit ceb986549d
10 changed files with 528 additions and 39 deletions

View File

@ -1,3 +1,4 @@
use crate::rpc::config::OutboundRateLimiterConfig;
use crate::types::GossipKind; use crate::types::GossipKind;
use crate::{Enr, PeerIdSerialized}; use crate::{Enr, PeerIdSerialized};
use directory::{ use directory::{
@ -133,6 +134,9 @@ pub struct Config {
/// Whether light client protocols should be enabled. /// Whether light client protocols should be enabled.
pub enable_light_client_server: bool, pub enable_light_client_server: bool,
/// Configuration for the outbound rate limiter (requests made by this node).
pub outbound_rate_limiter_config: Option<OutboundRateLimiterConfig>,
} }
impl Default for Config { impl Default for Config {
@ -211,6 +215,7 @@ impl Default for Config {
topics: Vec::new(), topics: Vec::new(),
metrics_enabled: false, metrics_enabled: false,
enable_light_client_server: false, enable_light_client_server: false,
outbound_rate_limiter_config: None,
} }
} }
} }

View File

@ -0,0 +1,173 @@
use std::{
fmt::{Debug, Display},
str::FromStr,
time::Duration,
};
use super::{methods, rate_limiter::Quota, Protocol};
use serde_derive::{Deserialize, Serialize};
/// Auxiliary struct to aid on configuration parsing.
///
/// A protocol's quota is specified as `protocol_name:tokens/time_in_seconds`.
#[derive(Debug, PartialEq, Eq)]
struct ProtocolQuota {
protocol: Protocol,
quota: Quota,
}
impl Display for ProtocolQuota {
fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
write!(
f,
"{}:{}/{}",
self.protocol.as_ref(),
self.quota.max_tokens,
self.quota.replenish_all_every.as_secs()
)
}
}
impl FromStr for ProtocolQuota {
type Err = &'static str;
fn from_str(s: &str) -> Result<Self, Self::Err> {
let (protocol_str, quota_str) = s
.split_once(':')
.ok_or("Missing ':' from quota definition.")?;
let protocol = protocol_str
.parse()
.map_err(|_parse_err| "Wrong protocol representation in quota")?;
let (tokens_str, time_str) = quota_str
.split_once('/')
.ok_or("Quota should be defined as \"n/t\" (t in seconds). Missing '/' from quota.")?;
let tokens = tokens_str
.parse()
.map_err(|_| "Failed to parse tokens from quota.")?;
let seconds = time_str
.parse::<u64>()
.map_err(|_| "Failed to parse time in seconds from quota.")?;
Ok(ProtocolQuota {
protocol,
quota: Quota {
replenish_all_every: Duration::from_secs(seconds),
max_tokens: tokens,
},
})
}
}
/// Configurations for the rate limiter applied to outbound requests (made by the node itself).
#[derive(Clone, Serialize, Deserialize, PartialEq, Eq)]
pub struct OutboundRateLimiterConfig {
pub(super) ping_quota: Quota,
pub(super) meta_data_quota: Quota,
pub(super) status_quota: Quota,
pub(super) goodbye_quota: Quota,
pub(super) blocks_by_range_quota: Quota,
pub(super) blocks_by_root_quota: Quota,
}
impl OutboundRateLimiterConfig {
pub const DEFAULT_PING_QUOTA: Quota = Quota::n_every(2, 10);
pub const DEFAULT_META_DATA_QUOTA: Quota = Quota::n_every(2, 5);
pub const DEFAULT_STATUS_QUOTA: Quota = Quota::n_every(5, 15);
pub const DEFAULT_GOODBYE_QUOTA: Quota = Quota::one_every(10);
pub const DEFAULT_BLOCKS_BY_RANGE_QUOTA: Quota =
Quota::n_every(methods::MAX_REQUEST_BLOCKS, 10);
pub const DEFAULT_BLOCKS_BY_ROOT_QUOTA: Quota = Quota::n_every(128, 10);
}
impl Default for OutboundRateLimiterConfig {
fn default() -> Self {
OutboundRateLimiterConfig {
ping_quota: Self::DEFAULT_PING_QUOTA,
meta_data_quota: Self::DEFAULT_META_DATA_QUOTA,
status_quota: Self::DEFAULT_STATUS_QUOTA,
goodbye_quota: Self::DEFAULT_GOODBYE_QUOTA,
blocks_by_range_quota: Self::DEFAULT_BLOCKS_BY_RANGE_QUOTA,
blocks_by_root_quota: Self::DEFAULT_BLOCKS_BY_ROOT_QUOTA,
}
}
}
impl Debug for OutboundRateLimiterConfig {
fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
macro_rules! fmt_q {
($quota:expr) => {
&format_args!(
"{}/{}s",
$quota.max_tokens,
$quota.replenish_all_every.as_secs()
)
};
}
f.debug_struct("OutboundRateLimiterConfig")
.field("ping", fmt_q!(&self.ping_quota))
.field("metadata", fmt_q!(&self.meta_data_quota))
.field("status", fmt_q!(&self.status_quota))
.field("goodbye", fmt_q!(&self.goodbye_quota))
.field("blocks_by_range", fmt_q!(&self.blocks_by_range_quota))
.field("blocks_by_root", fmt_q!(&self.blocks_by_root_quota))
.finish()
}
}
/// Parse configurations for the outbound rate limiter. Protocols that are not specified use
/// the default values. Protocol specified more than once use only the first given Quota.
///
/// The expected format is a ';' separated list of [`ProtocolQuota`].
impl FromStr for OutboundRateLimiterConfig {
type Err = &'static str;
fn from_str(s: &str) -> Result<Self, Self::Err> {
let mut ping_quota = None;
let mut meta_data_quota = None;
let mut status_quota = None;
let mut goodbye_quota = None;
let mut blocks_by_range_quota = None;
let mut blocks_by_root_quota = None;
for proto_def in s.split(';') {
let ProtocolQuota { protocol, quota } = proto_def.parse()?;
let quota = Some(quota);
match protocol {
Protocol::Status => status_quota = status_quota.or(quota),
Protocol::Goodbye => goodbye_quota = goodbye_quota.or(quota),
Protocol::BlocksByRange => blocks_by_range_quota = blocks_by_range_quota.or(quota),
Protocol::BlocksByRoot => blocks_by_root_quota = blocks_by_root_quota.or(quota),
Protocol::Ping => ping_quota = ping_quota.or(quota),
Protocol::MetaData => meta_data_quota = meta_data_quota.or(quota),
Protocol::LightClientBootstrap => return Err("Lighthouse does not send LightClientBootstrap requests. Quota should not be set."),
}
}
Ok(OutboundRateLimiterConfig {
ping_quota: ping_quota.unwrap_or(Self::DEFAULT_PING_QUOTA),
meta_data_quota: meta_data_quota.unwrap_or(Self::DEFAULT_META_DATA_QUOTA),
status_quota: status_quota.unwrap_or(Self::DEFAULT_STATUS_QUOTA),
goodbye_quota: goodbye_quota.unwrap_or(Self::DEFAULT_GOODBYE_QUOTA),
blocks_by_range_quota: blocks_by_range_quota
.unwrap_or(Self::DEFAULT_BLOCKS_BY_RANGE_QUOTA),
blocks_by_root_quota: blocks_by_root_quota
.unwrap_or(Self::DEFAULT_BLOCKS_BY_ROOT_QUOTA),
})
}
}
#[cfg(test)]
mod tests {
use super::*;
#[test]
fn test_quota_inverse() {
let quota = ProtocolQuota {
protocol: Protocol::Goodbye,
quota: Quota {
replenish_all_every: Duration::from_secs(10),
max_tokens: 8,
},
};
assert_eq!(quota.to_string().parse(), Ok(quota))
}
}

View File

@ -12,7 +12,7 @@ use libp2p::swarm::{
PollParameters, SubstreamProtocol, PollParameters, SubstreamProtocol,
}; };
use libp2p::PeerId; use libp2p::PeerId;
use rate_limiter::{RPCRateLimiter as RateLimiter, RPCRateLimiterBuilder, RateLimitedErr}; use rate_limiter::{RPCRateLimiter as RateLimiter, RateLimitedErr};
use slog::{crit, debug, o}; use slog::{crit, debug, o};
use std::marker::PhantomData; use std::marker::PhantomData;
use std::sync::Arc; use std::sync::Arc;
@ -32,12 +32,17 @@ pub use methods::{
pub(crate) use outbound::OutboundRequest; pub(crate) use outbound::OutboundRequest;
pub use protocol::{max_rpc_size, Protocol, RPCError}; pub use protocol::{max_rpc_size, Protocol, RPCError};
use self::config::OutboundRateLimiterConfig;
use self::self_limiter::SelfRateLimiter;
pub(crate) mod codec; pub(crate) mod codec;
pub mod config;
mod handler; mod handler;
pub mod methods; pub mod methods;
mod outbound; mod outbound;
mod protocol; mod protocol;
mod rate_limiter; mod rate_limiter;
mod self_limiter;
/// Composite trait for a request id. /// Composite trait for a request id.
pub trait ReqId: Send + 'static + std::fmt::Debug + Copy + Clone {} pub trait ReqId: Send + 'static + std::fmt::Debug + Copy + Clone {}
@ -100,13 +105,18 @@ pub struct RPCMessage<Id, TSpec: EthSpec> {
pub event: HandlerEvent<Id, TSpec>, pub event: HandlerEvent<Id, TSpec>,
} }
type BehaviourAction<Id, TSpec> =
NetworkBehaviourAction<RPCMessage<Id, TSpec>, RPCHandler<Id, TSpec>>;
/// Implements the libp2p `NetworkBehaviour` trait and therefore manages network-level /// Implements the libp2p `NetworkBehaviour` trait and therefore manages network-level
/// logic. /// logic.
pub struct RPC<Id: ReqId, TSpec: EthSpec> { pub struct RPC<Id: ReqId, TSpec: EthSpec> {
/// Rate limiter /// Rate limiter
limiter: RateLimiter, limiter: RateLimiter,
/// Rate limiter for our own requests.
self_limiter: Option<SelfRateLimiter<Id, TSpec>>,
/// Queue of events to be processed. /// Queue of events to be processed.
events: Vec<NetworkBehaviourAction<RPCMessage<Id, TSpec>, RPCHandler<Id, TSpec>>>, events: Vec<BehaviourAction<Id, TSpec>>,
fork_context: Arc<ForkContext>, fork_context: Arc<ForkContext>,
enable_light_client_server: bool, enable_light_client_server: bool,
/// Slog logger for RPC behaviour. /// Slog logger for RPC behaviour.
@ -117,10 +127,12 @@ impl<Id: ReqId, TSpec: EthSpec> RPC<Id, TSpec> {
pub fn new( pub fn new(
fork_context: Arc<ForkContext>, fork_context: Arc<ForkContext>,
enable_light_client_server: bool, enable_light_client_server: bool,
outbound_rate_limiter_config: Option<OutboundRateLimiterConfig>,
log: slog::Logger, log: slog::Logger,
) -> Self { ) -> Self {
let log = log.new(o!("service" => "libp2p_rpc")); let log = log.new(o!("service" => "libp2p_rpc"));
let limiter = RPCRateLimiterBuilder::new()
let limiter = RateLimiter::builder()
.n_every(Protocol::MetaData, 2, Duration::from_secs(5)) .n_every(Protocol::MetaData, 2, Duration::from_secs(5))
.n_every(Protocol::Ping, 2, Duration::from_secs(10)) .n_every(Protocol::Ping, 2, Duration::from_secs(10))
.n_every(Protocol::Status, 5, Duration::from_secs(15)) .n_every(Protocol::Status, 5, Duration::from_secs(15))
@ -134,8 +146,14 @@ impl<Id: ReqId, TSpec: EthSpec> RPC<Id, TSpec> {
.n_every(Protocol::BlocksByRoot, 128, Duration::from_secs(10)) .n_every(Protocol::BlocksByRoot, 128, Duration::from_secs(10))
.build() .build()
.expect("Configuration parameters are valid"); .expect("Configuration parameters are valid");
let self_limiter = outbound_rate_limiter_config.map(|config| {
SelfRateLimiter::new(config, log.clone()).expect("Configuration parameters are valid")
});
RPC { RPC {
limiter, limiter,
self_limiter,
events: Vec::new(), events: Vec::new(),
fork_context, fork_context,
enable_light_client_server, enable_light_client_server,
@ -162,12 +180,24 @@ impl<Id: ReqId, TSpec: EthSpec> RPC<Id, TSpec> {
/// Submits an RPC request. /// Submits an RPC request.
/// ///
/// The peer must be connected for this to succeed. /// The peer must be connected for this to succeed.
pub fn send_request(&mut self, peer_id: PeerId, request_id: Id, event: OutboundRequest<TSpec>) { pub fn send_request(&mut self, peer_id: PeerId, request_id: Id, req: OutboundRequest<TSpec>) {
self.events.push(NetworkBehaviourAction::NotifyHandler { let event = if let Some(self_limiter) = self.self_limiter.as_mut() {
match self_limiter.allows(peer_id, request_id, req) {
Ok(event) => event,
Err(_e) => {
// Request is logged and queued internally in the self rate limiter.
return;
}
}
} else {
NetworkBehaviourAction::NotifyHandler {
peer_id, peer_id,
handler: NotifyHandler::Any, handler: NotifyHandler::Any,
event: RPCSend::Request(request_id, event), event: RPCSend::Request(request_id, req),
}); }
};
self.events.push(event);
} }
/// Lighthouse wishes to disconnect from this peer by sending a Goodbye message. This /// Lighthouse wishes to disconnect from this peer by sending a Goodbye message. This
@ -272,11 +302,19 @@ where
cx: &mut Context, cx: &mut Context,
_: &mut impl PollParameters, _: &mut impl PollParameters,
) -> Poll<NetworkBehaviourAction<Self::OutEvent, Self::ConnectionHandler>> { ) -> Poll<NetworkBehaviourAction<Self::OutEvent, Self::ConnectionHandler>> {
// let the rate limiter prune // let the rate limiter prune.
let _ = self.limiter.poll_unpin(cx); let _ = self.limiter.poll_unpin(cx);
if let Some(self_limiter) = self.self_limiter.as_mut() {
if let Poll::Ready(event) = self_limiter.poll_ready(cx) {
self.events.push(event)
}
}
if !self.events.is_empty() { if !self.events.is_empty() {
return Poll::Ready(self.events.remove(0)); return Poll::Ready(self.events.remove(0));
} }
Poll::Pending Poll::Pending
} }
} }

View File

@ -14,7 +14,7 @@ use std::io;
use std::marker::PhantomData; use std::marker::PhantomData;
use std::sync::Arc; use std::sync::Arc;
use std::time::Duration; use std::time::Duration;
use strum::IntoStaticStr; use strum::{AsRefStr, Display, EnumString, IntoStaticStr};
use tokio_io_timeout::TimeoutStream; use tokio_io_timeout::TimeoutStream;
use tokio_util::{ use tokio_util::{
codec::Framed, codec::Framed,
@ -139,21 +139,26 @@ pub fn rpc_block_limits_by_fork(current_fork: ForkName) -> RpcLimits {
} }
/// Protocol names to be used. /// Protocol names to be used.
#[derive(Debug, Clone, Copy, PartialEq, Eq)] #[derive(Debug, Clone, Copy, PartialEq, Eq, Hash, EnumString, AsRefStr, Display)]
#[strum(serialize_all = "snake_case")]
pub enum Protocol { pub enum Protocol {
/// The Status protocol name. /// The Status protocol name.
Status, Status,
/// The Goodbye protocol name. /// The Goodbye protocol name.
Goodbye, Goodbye,
/// The `BlocksByRange` protocol name. /// The `BlocksByRange` protocol name.
#[strum(serialize = "beacon_blocks_by_range")]
BlocksByRange, BlocksByRange,
/// The `BlocksByRoot` protocol name. /// The `BlocksByRoot` protocol name.
#[strum(serialize = "beacon_blocks_by_root")]
BlocksByRoot, BlocksByRoot,
/// The `Ping` protocol name. /// The `Ping` protocol name.
Ping, Ping,
/// The `MetaData` protocol name. /// The `MetaData` protocol name.
#[strum(serialize = "metadata")]
MetaData, MetaData,
/// The `LightClientBootstrap` protocol name. /// The `LightClientBootstrap` protocol name.
#[strum(serialize = "light_client_bootstrap")]
LightClientBootstrap, LightClientBootstrap,
} }
@ -172,21 +177,6 @@ pub enum Encoding {
SSZSnappy, SSZSnappy,
} }
impl std::fmt::Display for Protocol {
fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
let repr = match self {
Protocol::Status => "status",
Protocol::Goodbye => "goodbye",
Protocol::BlocksByRange => "beacon_blocks_by_range",
Protocol::BlocksByRoot => "beacon_blocks_by_root",
Protocol::Ping => "ping",
Protocol::MetaData => "metadata",
Protocol::LightClientBootstrap => "light_client_bootstrap",
};
f.write_str(repr)
}
}
impl std::fmt::Display for Encoding { impl std::fmt::Display for Encoding {
fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
let repr = match self { let repr = match self {

View File

@ -1,6 +1,7 @@
use crate::rpc::{InboundRequest, Protocol}; use crate::rpc::Protocol;
use fnv::FnvHashMap; use fnv::FnvHashMap;
use libp2p::PeerId; use libp2p::PeerId;
use serde_derive::{Deserialize, Serialize};
use std::convert::TryInto; use std::convert::TryInto;
use std::future::Future; use std::future::Future;
use std::hash::Hash; use std::hash::Hash;
@ -47,12 +48,31 @@ type Nanosecs = u64;
/// n*`replenish_all_every`/`max_tokens` units of time since their last request. /// n*`replenish_all_every`/`max_tokens` units of time since their last request.
/// ///
/// To produce hard limits, set `max_tokens` to 1. /// To produce hard limits, set `max_tokens` to 1.
#[derive(Clone, Debug, PartialEq, Eq, Serialize, Deserialize)]
pub struct Quota { pub struct Quota {
/// How often are `max_tokens` fully replenished. /// How often are `max_tokens` fully replenished.
replenish_all_every: Duration, pub(super) replenish_all_every: Duration,
/// Token limit. This translates on how large can an instantaneous batch of /// Token limit. This translates on how large can an instantaneous batch of
/// tokens be. /// tokens be.
max_tokens: u64, pub(super) max_tokens: u64,
}
impl Quota {
/// A hard limit of one token every `seconds`.
pub const fn one_every(seconds: u64) -> Self {
Quota {
replenish_all_every: Duration::from_secs(seconds),
max_tokens: 1,
}
}
/// Allow `n` tokens to be use used every `seconds`.
pub const fn n_every(n: u64, seconds: u64) -> Self {
Quota {
replenish_all_every: Duration::from_secs(seconds),
max_tokens: n,
}
}
} }
/// Manages rate limiting of requests per peer, with differentiated rates per protocol. /// Manages rate limiting of requests per peer, with differentiated rates per protocol.
@ -78,6 +98,7 @@ pub struct RPCRateLimiter {
} }
/// Error type for non conformant requests /// Error type for non conformant requests
#[derive(Debug)]
pub enum RateLimitedErr { pub enum RateLimitedErr {
/// Required tokens for this request exceed the maximum /// Required tokens for this request exceed the maximum
TooLarge, TooLarge,
@ -86,7 +107,7 @@ pub enum RateLimitedErr {
} }
/// User-friendly builder of a `RPCRateLimiter` /// User-friendly builder of a `RPCRateLimiter`
#[derive(Default)] #[derive(Default, Clone)]
pub struct RPCRateLimiterBuilder { pub struct RPCRateLimiterBuilder {
/// Quota for the Goodbye protocol. /// Quota for the Goodbye protocol.
goodbye_quota: Option<Quota>, goodbye_quota: Option<Quota>,
@ -105,13 +126,8 @@ pub struct RPCRateLimiterBuilder {
} }
impl RPCRateLimiterBuilder { impl RPCRateLimiterBuilder {
/// Get an empty `RPCRateLimiterBuilder`.
pub fn new() -> Self {
Default::default()
}
/// Set a quota for a protocol. /// Set a quota for a protocol.
fn set_quota(mut self, protocol: Protocol, quota: Quota) -> Self { pub fn set_quota(mut self, protocol: Protocol, quota: Quota) -> Self {
let q = Some(quota); let q = Some(quota);
match protocol { match protocol {
Protocol::Ping => self.ping_quota = q, Protocol::Ping => self.ping_quota = q,
@ -191,11 +207,40 @@ impl RPCRateLimiterBuilder {
} }
} }
pub trait RateLimiterItem {
fn protocol(&self) -> Protocol;
fn expected_responses(&self) -> u64;
}
impl<T: EthSpec> RateLimiterItem for super::InboundRequest<T> {
fn protocol(&self) -> Protocol {
self.protocol()
}
fn expected_responses(&self) -> u64 {
self.expected_responses()
}
}
impl<T: EthSpec> RateLimiterItem for super::OutboundRequest<T> {
fn protocol(&self) -> Protocol {
self.protocol()
}
fn expected_responses(&self) -> u64 {
self.expected_responses()
}
}
impl RPCRateLimiter { impl RPCRateLimiter {
pub fn allows<T: EthSpec>( /// Get a builder instance.
pub fn builder() -> RPCRateLimiterBuilder {
RPCRateLimiterBuilder::default()
}
pub fn allows<Item: RateLimiterItem>(
&mut self, &mut self,
peer_id: &PeerId, peer_id: &PeerId,
request: &InboundRequest<T>, request: &Item,
) -> Result<(), RateLimitedErr> { ) -> Result<(), RateLimitedErr> {
let time_since_start = self.init_time.elapsed(); let time_since_start = self.init_time.elapsed();
let tokens = request.expected_responses().max(1); let tokens = request.expected_responses().max(1);

View File

@ -0,0 +1,202 @@
use std::{
collections::{hash_map::Entry, HashMap, VecDeque},
task::{Context, Poll},
time::Duration,
};
use futures::FutureExt;
use libp2p::{swarm::NotifyHandler, PeerId};
use slog::{crit, debug, Logger};
use smallvec::SmallVec;
use tokio_util::time::DelayQueue;
use types::EthSpec;
use super::{
config::OutboundRateLimiterConfig,
rate_limiter::{RPCRateLimiter as RateLimiter, RateLimitedErr},
BehaviourAction, OutboundRequest, Protocol, RPCSend, ReqId,
};
/// A request that was rate limited or waiting on rate limited requests for the same peer and
/// protocol.
struct QueuedRequest<Id: ReqId, TSpec: EthSpec> {
req: OutboundRequest<TSpec>,
request_id: Id,
}
pub(crate) struct SelfRateLimiter<Id: ReqId, TSpec: EthSpec> {
/// Requests queued for sending per peer. This requests are stored when the self rate
/// limiter rejects them. Rate limiting is based on a Peer and Protocol basis, therefore
/// are stored in the same way.
delayed_requests: HashMap<(PeerId, Protocol), VecDeque<QueuedRequest<Id, TSpec>>>,
/// The delay required to allow a peer's outbound request per protocol.
next_peer_request: DelayQueue<(PeerId, Protocol)>,
/// Rate limiter for our own requests.
limiter: RateLimiter,
/// Requests that are ready to be sent.
ready_requests: SmallVec<[BehaviourAction<Id, TSpec>; 3]>,
/// Slog logger.
log: Logger,
}
/// Error returned when the rate limiter does not accept a request.
// NOTE: this is currently not used, but might be useful for debugging.
pub enum Error {
/// There are queued requests for this same peer and protocol.
PendingRequests,
/// Request was tried but rate limited.
RateLimited,
}
impl<Id: ReqId, TSpec: EthSpec> SelfRateLimiter<Id, TSpec> {
/// Creates a new [`SelfRateLimiter`] based on configration values.
pub fn new(config: OutboundRateLimiterConfig, log: Logger) -> Result<Self, &'static str> {
debug!(log, "Using self rate limiting params"; "config" => ?config);
// Destructure to make sure every configuration value is used.
let OutboundRateLimiterConfig {
ping_quota,
meta_data_quota,
status_quota,
goodbye_quota,
blocks_by_range_quota,
blocks_by_root_quota,
} = config;
let limiter = RateLimiter::builder()
.set_quota(Protocol::Ping, ping_quota)
.set_quota(Protocol::MetaData, meta_data_quota)
.set_quota(Protocol::Status, status_quota)
.set_quota(Protocol::Goodbye, goodbye_quota)
.set_quota(Protocol::BlocksByRange, blocks_by_range_quota)
.set_quota(Protocol::BlocksByRoot, blocks_by_root_quota)
// Manually set the LightClientBootstrap quota, since we use the same rate limiter for
// inbound and outbound requests, and the LightClientBootstrap is an only inbound
// protocol.
.one_every(Protocol::LightClientBootstrap, Duration::from_secs(10))
.build()?;
Ok(SelfRateLimiter {
delayed_requests: Default::default(),
next_peer_request: Default::default(),
limiter,
ready_requests: Default::default(),
log,
})
}
/// Checks if the rate limiter allows the request. If it's allowed, returns the
/// [`NetworkBehaviourAction`] that should be emitted. When not allowed, the request is delayed
/// until it can be sent.
pub fn allows(
&mut self,
peer_id: PeerId,
request_id: Id,
req: OutboundRequest<TSpec>,
) -> Result<BehaviourAction<Id, TSpec>, Error> {
let protocol = req.protocol();
// First check that there are not already other requests waiting to be sent.
if let Some(queued_requests) = self.delayed_requests.get_mut(&(peer_id, protocol)) {
queued_requests.push_back(QueuedRequest { req, request_id });
return Err(Error::PendingRequests);
}
match Self::try_send_request(&mut self.limiter, peer_id, request_id, req, &self.log) {
Err((rate_limited_req, wait_time)) => {
let key = (peer_id, protocol);
self.next_peer_request.insert(key, wait_time);
self.delayed_requests
.entry(key)
.or_default()
.push_back(rate_limited_req);
Err(Error::RateLimited)
}
Ok(event) => Ok(event),
}
}
/// Auxiliary function to deal with self rate limiting outcomes. If the rate limiter allows the
/// request, the [`NetworkBehaviourAction`] that should be emitted is returned. If the request
/// should be delayed, it's returned with the duration to wait.
fn try_send_request(
limiter: &mut RateLimiter,
peer_id: PeerId,
request_id: Id,
req: OutboundRequest<TSpec>,
log: &Logger,
) -> Result<BehaviourAction<Id, TSpec>, (QueuedRequest<Id, TSpec>, Duration)> {
match limiter.allows(&peer_id, &req) {
Ok(()) => Ok(BehaviourAction::NotifyHandler {
peer_id,
handler: NotifyHandler::Any,
event: RPCSend::Request(request_id, req),
}),
Err(e) => {
let protocol = req.protocol();
match e {
RateLimitedErr::TooLarge => {
// this should never happen with default parameters. Let's just send the request.
// Log a crit since this is a config issue.
crit!(
log,
"Self rate limiting error for a batch that will never fit. Sending request anyway. Check configuration parameters.";
"protocol" => %req.protocol()
);
Ok(BehaviourAction::NotifyHandler {
peer_id,
handler: NotifyHandler::Any,
event: RPCSend::Request(request_id, req),
})
}
RateLimitedErr::TooSoon(wait_time) => {
debug!(log, "Self rate limiting"; "protocol" => %protocol, "wait_time_ms" => wait_time.as_millis(), "peer_id" => %peer_id);
Err((QueuedRequest { req, request_id }, wait_time))
}
}
}
}
}
/// When a peer and protocol are allowed to send a next request, this function checks the
/// queued requests and attempts marking as ready as many as the limiter allows.
fn next_peer_request_ready(&mut self, peer_id: PeerId, protocol: Protocol) {
if let Entry::Occupied(mut entry) = self.delayed_requests.entry((peer_id, protocol)) {
let queued_requests = entry.get_mut();
while let Some(QueuedRequest { req, request_id }) = queued_requests.pop_front() {
match Self::try_send_request(&mut self.limiter, peer_id, request_id, req, &self.log)
{
Err((rate_limited_req, wait_time)) => {
let key = (peer_id, protocol);
self.next_peer_request.insert(key, wait_time);
queued_requests.push_back(rate_limited_req);
// If one fails just wait for the next window that allows sending requests.
return;
}
Ok(event) => self.ready_requests.push(event),
}
}
if queued_requests.is_empty() {
entry.remove();
}
}
}
pub fn poll_ready(&mut self, cx: &mut Context<'_>) -> Poll<BehaviourAction<Id, TSpec>> {
// First check the requests that were self rate limited, since those might add events to
// the queue. Also do this this before rate limiter prunning to avoid removing and
// immediately adding rate limiting keys.
if let Poll::Ready(Some(Ok(expired))) = self.next_peer_request.poll_expired(cx) {
let (peer_id, protocol) = expired.into_inner();
self.next_peer_request_ready(peer_id, protocol);
}
// Prune the rate limiter.
let _ = self.limiter.poll_unpin(cx);
// Finally return any queued events.
if !self.ready_requests.is_empty() {
return Poll::Ready(self.ready_requests.remove(0));
}
Poll::Pending
}
}

View File

@ -262,6 +262,7 @@ impl<AppReqId: ReqId, TSpec: EthSpec> Network<AppReqId, TSpec> {
let eth2_rpc = RPC::new( let eth2_rpc = RPC::new(
ctx.fork_context.clone(), ctx.fork_context.clone(),
config.enable_light_client_server, config.enable_light_client_server,
config.outbound_rate_limiter_config.clone(),
log.clone(), log.clone(),
); );

View File

@ -194,6 +194,21 @@ pub fn cli_app<'a, 'b>() -> App<'a, 'b> {
.help("Lighthouse by default does not discover private IP addresses. Set this flag to enable connection attempts to local addresses.") .help("Lighthouse by default does not discover private IP addresses. Set this flag to enable connection attempts to local addresses.")
.takes_value(false), .takes_value(false),
) )
.arg(
Arg::with_name("self-limiter")
.long("self-limiter")
.help(
"Enables the outbound rate limiter (requests made by this node).\
\
Rate limit quotas per protocol can be set in the form of \
<protocol_name>:<tokens>/<time_in_seconds>. To set quotas for multiple protocols, \
separate them by ';'. If the self rate limiter is enabled and a protocol is not \
present in the configuration, the quotas used for the inbound rate limiter will be \
used."
)
.min_values(0)
.hidden(true)
)
/* REST API related arguments */ /* REST API related arguments */
.arg( .arg(
Arg::with_name("http") Arg::with_name("http")

View File

@ -967,6 +967,13 @@ pub fn set_network_config(
// Light client server config. // Light client server config.
config.enable_light_client_server = cli_args.is_present("light-client-server"); config.enable_light_client_server = cli_args.is_present("light-client-server");
// This flag can be used both with or without a value. Try to parse it first with a value, if
// no value is defined but the flag is present, use the default params.
config.outbound_rate_limiter_config = clap_utils::parse_optional(cli_args, "self-limiter")?;
if cli_args.is_present("self-limiter") && config.outbound_rate_limiter_config.is_none() {
config.outbound_rate_limiter_config = Some(Default::default());
}
Ok(()) Ok(())
} }

View File

@ -1079,6 +1079,19 @@ fn http_port_flag() {
.with_config(|config| assert_eq!(config.http_api.listen_port, port1)); .with_config(|config| assert_eq!(config.http_api.listen_port, port1));
} }
#[test] #[test]
fn empty_self_limiter_flag() {
// Test that empty rate limiter is accepted using the default rate limiting configurations.
CommandLineTest::new()
.flag("self-limiter", None)
.run_with_zero_port()
.with_config(|config| {
assert_eq!(
config.network.outbound_rate_limiter_config,
Some(lighthouse_network::rpc::config::OutboundRateLimiterConfig::default())
)
});
}
#[test]
fn http_allow_origin_flag() { fn http_allow_origin_flag() {
CommandLineTest::new() CommandLineTest::new()
.flag("http-allow-origin", Some("127.0.0.99")) .flag("http-allow-origin", Some("127.0.0.99"))