Connection pool support in connection cache and QUIC connection reliability improvement (#25793)

* Connection pool in connection cache and handle connection errors

1. The connection not has a pool of connections per address, configurable, default 4
2. The connections per address share a lazy initialized endpoint
3. Handle connection issues better, avoid race conditions
4. Various log improvement for help debug connection issues
This commit is contained in:
Lijun Wang 2022-06-10 09:25:24 -07:00 committed by GitHub
parent bea35d5fbe
commit 29b597cea5
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
13 changed files with 463 additions and 147 deletions

View File

@ -5,7 +5,7 @@ use {
log::*, log::*,
rand::{thread_rng, Rng}, rand::{thread_rng, Rng},
rayon::prelude::*, rayon::prelude::*,
solana_client::connection_cache::ConnectionCache, solana_client::connection_cache::{ConnectionCache, DEFAULT_TPU_CONNECTION_POOL_SIZE},
solana_core::banking_stage::BankingStage, solana_core::banking_stage::BankingStage,
solana_gossip::cluster_info::{ClusterInfo, Node}, solana_gossip::cluster_info::{ClusterInfo, Node},
solana_ledger::{ solana_ledger::{
@ -352,7 +352,10 @@ fn main() {
None, None,
replay_vote_sender, replay_vote_sender,
Arc::new(RwLock::new(CostModel::default())), Arc::new(RwLock::new(CostModel::default())),
Arc::new(ConnectionCache::new(tpu_use_quic)), Arc::new(ConnectionCache::new(
tpu_use_quic,
DEFAULT_TPU_CONNECTION_POOL_SIZE,
)),
); );
poh_recorder.lock().unwrap().set_bank(&bank); poh_recorder.lock().unwrap().set_bank(&bank);

View File

@ -2,7 +2,7 @@ use {
clap::{crate_description, crate_name, App, Arg, ArgMatches}, clap::{crate_description, crate_name, App, Arg, ArgMatches},
solana_clap_utils::input_validators::{is_url, is_url_or_moniker}, solana_clap_utils::input_validators::{is_url, is_url_or_moniker},
solana_cli_config::{ConfigInput, CONFIG_FILE}, solana_cli_config::{ConfigInput, CONFIG_FILE},
solana_client::connection_cache::DEFAULT_TPU_USE_QUIC, solana_client::connection_cache::{DEFAULT_TPU_CONNECTION_POOL_SIZE, DEFAULT_TPU_USE_QUIC},
solana_sdk::{ solana_sdk::{
fee_calculator::FeeRateGovernor, fee_calculator::FeeRateGovernor,
pubkey::Pubkey, pubkey::Pubkey,
@ -53,6 +53,7 @@ pub struct Config {
pub target_node: Option<Pubkey>, pub target_node: Option<Pubkey>,
pub external_client_type: ExternalClientType, pub external_client_type: ExternalClientType,
pub use_quic: bool, pub use_quic: bool,
pub tpu_connection_pool_size: usize,
} }
impl Default for Config { impl Default for Config {
@ -79,6 +80,7 @@ impl Default for Config {
target_node: None, target_node: None,
external_client_type: ExternalClientType::default(), external_client_type: ExternalClientType::default(),
use_quic: DEFAULT_TPU_USE_QUIC, use_quic: DEFAULT_TPU_USE_QUIC,
tpu_connection_pool_size: DEFAULT_TPU_CONNECTION_POOL_SIZE,
} }
} }
} }
@ -294,6 +296,13 @@ pub fn build_args<'a, 'b>(version: &'b str) -> App<'a, 'b> {
.help("Submit transactions via QUIC; only affects ThinClient (default) \ .help("Submit transactions via QUIC; only affects ThinClient (default) \
or TpuClient sends"), or TpuClient sends"),
) )
.arg(
Arg::with_name("tpu_connection_pool_size")
.long("tpu-connection-pool-size")
.takes_value(true)
.help("Controls the connection pool size per remote address; only affects ThinClient (default) \
or TpuClient sends"),
)
} }
/// Parses a clap `ArgMatches` structure into a `Config` /// Parses a clap `ArgMatches` structure into a `Config`
@ -343,6 +352,13 @@ pub fn extract_args(matches: &ArgMatches) -> Config {
args.use_quic = true; args.use_quic = true;
} }
if let Some(v) = matches.value_of("tpu_connection_pool_size") {
args.tpu_connection_pool_size = v
.to_string()
.parse()
.expect("can't parse tpu_connection_pool_size");
}
if let Some(addr) = matches.value_of("entrypoint") { if let Some(addr) = matches.value_of("entrypoint") {
args.entrypoint_addr = solana_net_utils::parse_host_port(addr).unwrap_or_else(|e| { args.entrypoint_addr = solana_net_utils::parse_host_port(addr).unwrap_or_else(|e| {
eprintln!("failed to parse entrypoint address: {}", e); eprintln!("failed to parse entrypoint address: {}", e);

View File

@ -49,6 +49,7 @@ fn main() {
target_node, target_node,
external_client_type, external_client_type,
use_quic, use_quic,
tpu_connection_pool_size,
.. ..
} = &cli_config; } = &cli_config;
@ -102,7 +103,9 @@ fn main() {
do_bench_tps(client, cli_config, keypairs); do_bench_tps(client, cli_config, keypairs);
} }
ExternalClientType::ThinClient => { ExternalClientType::ThinClient => {
let connection_cache = Arc::new(ConnectionCache::new(*use_quic)); let connection_cache =
Arc::new(ConnectionCache::new(*use_quic, *tpu_connection_pool_size));
let client = if let Ok(rpc_addr) = value_t!(matches, "rpc_addr", String) { let client = if let Ok(rpc_addr) = value_t!(matches, "rpc_addr", String) {
let rpc = rpc_addr.parse().unwrap_or_else(|e| { let rpc = rpc_addr.parse().unwrap_or_else(|e| {
eprintln!("RPC address should parse as socketaddr {:?}", e); eprintln!("RPC address should parse as socketaddr {:?}", e);
@ -172,7 +175,9 @@ fn main() {
json_rpc_url.to_string(), json_rpc_url.to_string(),
CommitmentConfig::confirmed(), CommitmentConfig::confirmed(),
)); ));
let connection_cache = Arc::new(ConnectionCache::new(*use_quic)); let connection_cache =
Arc::new(ConnectionCache::new(*use_quic, *tpu_connection_pool_size));
let client = Arc::new( let client = Arc::new(
TpuClient::new_with_connection_cache( TpuClient::new_with_connection_cache(
rpc_client, rpc_client,

View File

@ -1,17 +1,18 @@
use { use {
crate::{ crate::{
nonblocking::quic_client::QuicLazyInitializedEndpoint,
quic_client::QuicTpuConnection, quic_client::QuicTpuConnection,
tpu_connection::{ClientStats, Connection}, tpu_connection::{ClientStats, Connection},
udp_client::UdpTpuConnection, udp_client::UdpTpuConnection,
}, },
indexmap::map::IndexMap, indexmap::map::{Entry, IndexMap},
rand::{thread_rng, Rng}, rand::{thread_rng, Rng},
solana_measure::measure::Measure, solana_measure::measure::Measure,
solana_sdk::timing::AtomicInterval, solana_sdk::{quic::QUIC_PORT_OFFSET, timing::AtomicInterval},
std::{ std::{
net::SocketAddr, net::SocketAddr,
sync::{ sync::{
atomic::{AtomicBool, AtomicU64, Ordering}, atomic::{AtomicU64, Ordering},
Arc, RwLock, Arc, RwLock,
}, },
}, },
@ -24,6 +25,9 @@ static MAX_CONNECTIONS: usize = 1024;
/// QUIC connections. /// QUIC connections.
pub const DEFAULT_TPU_USE_QUIC: bool = false; pub const DEFAULT_TPU_USE_QUIC: bool = false;
/// Default TPU connection pool size per remote address
pub const DEFAULT_TPU_CONNECTION_POOL_SIZE: usize = 4;
#[derive(Default)] #[derive(Default)]
pub struct ConnectionCacheStats { pub struct ConnectionCacheStats {
cache_hits: AtomicU64, cache_hits: AtomicU64,
@ -214,22 +218,144 @@ impl ConnectionCacheStats {
} }
pub struct ConnectionCache { pub struct ConnectionCache {
map: RwLock<IndexMap<SocketAddr, Arc<Connection>>>, map: RwLock<IndexMap<SocketAddr, ConnectionPool>>,
stats: Arc<ConnectionCacheStats>, stats: Arc<ConnectionCacheStats>,
last_stats: AtomicInterval, last_stats: AtomicInterval,
use_quic: AtomicBool, use_quic: bool,
connection_pool_size: usize,
}
/// Models the pool of connections
struct ConnectionPool {
/// The connections in the pool
connections: Vec<Arc<Connection>>,
/// Connections in this pool share the same endpoint
endpoint: Option<Arc<QuicLazyInitializedEndpoint>>,
}
impl ConnectionPool {
/// Get a connection from the pool. It must have at least one connection in the pool.
/// This randomly picks a connection in the pool.
fn borrow_connection(&self) -> Arc<Connection> {
let mut rng = thread_rng();
let n = rng.gen_range(0, self.connections.len());
self.connections[n].clone()
}
/// Check if we need to create a new connection. If the count of the connections
/// is smaller than the pool size.
fn need_new_connection(&self, required_pool_size: usize) -> bool {
self.connections.len() < required_pool_size
}
} }
impl ConnectionCache { impl ConnectionCache {
pub fn new(use_quic: bool) -> Self { pub fn new(use_quic: bool, connection_pool_size: usize) -> Self {
// The minimum pool size is 1.
let connection_pool_size = 1.max(connection_pool_size);
Self { Self {
use_quic: AtomicBool::new(use_quic), use_quic,
connection_pool_size,
..Self::default() ..Self::default()
} }
} }
pub fn get_use_quic(&self) -> bool { pub fn get_use_quic(&self) -> bool {
self.use_quic.load(Ordering::Relaxed) self.use_quic
}
fn create_endpoint(&self) -> Option<Arc<QuicLazyInitializedEndpoint>> {
if self.use_quic {
Some(Arc::new(QuicLazyInitializedEndpoint::new()))
} else {
None
}
}
/// Create a lazy connection object under the exclusive lock of the cache map if there is not
/// enough unsed connections in the connection pool for the specified address.
/// Returns CreateConnectionResult.
fn create_connection(
&self,
lock_timing_ms: &mut u64,
addr: &SocketAddr,
) -> CreateConnectionResult {
let mut get_connection_map_lock_measure = Measure::start("get_connection_map_lock_measure");
let mut map = self.map.write().unwrap();
get_connection_map_lock_measure.stop();
*lock_timing_ms = lock_timing_ms.saturating_add(get_connection_map_lock_measure.as_ms());
// Read again, as it is possible that between read lock dropped and the write lock acquired
// another thread could have setup the connection.
let (to_create_connection, endpoint) =
map.get(addr)
.map_or((true, self.create_endpoint()), |pool| {
(
pool.need_new_connection(self.connection_pool_size),
pool.endpoint.clone(),
)
});
let (cache_hit, connection_cache_stats, num_evictions, eviction_timing_ms) =
if to_create_connection {
let connection: Connection = if self.use_quic {
QuicTpuConnection::new(
endpoint.as_ref().unwrap().clone(),
*addr,
self.stats.clone(),
)
.into()
} else {
UdpTpuConnection::new(*addr, self.stats.clone()).into()
};
let connection = Arc::new(connection);
// evict a connection if the cache is reaching upper bounds
let mut num_evictions = 0;
let mut get_connection_cache_eviction_measure =
Measure::start("get_connection_cache_eviction_measure");
while map.len() >= MAX_CONNECTIONS {
let mut rng = thread_rng();
let n = rng.gen_range(0, MAX_CONNECTIONS);
map.swap_remove_index(n);
num_evictions += 1;
}
get_connection_cache_eviction_measure.stop();
match map.entry(*addr) {
Entry::Occupied(mut entry) => {
let pool = entry.get_mut();
pool.connections.push(connection);
}
Entry::Vacant(entry) => {
entry.insert(ConnectionPool {
connections: vec![connection],
endpoint,
});
}
}
(
false,
self.stats.clone(),
num_evictions,
get_connection_cache_eviction_measure.as_ms(),
)
} else {
(true, self.stats.clone(), 0, 0)
};
let pool = map.get(addr).unwrap();
let connection = pool.borrow_connection();
CreateConnectionResult {
connection,
cache_hit,
connection_cache_stats,
num_evictions,
eviction_timing_ms,
}
} }
fn get_or_add_connection(&self, addr: &SocketAddr) -> GetConnectionResult { fn get_or_add_connection(&self, addr: &SocketAddr) -> GetConnectionResult {
@ -237,6 +363,10 @@ impl ConnectionCache {
let map = self.map.read().unwrap(); let map = self.map.read().unwrap();
get_connection_map_lock_measure.stop(); get_connection_map_lock_measure.stop();
let port_offset = if self.use_quic { QUIC_PORT_OFFSET } else { 0 };
let addr = SocketAddr::new(addr.ip(), addr.port() + port_offset);
let mut lock_timing_ms = get_connection_map_lock_measure.as_ms(); let mut lock_timing_ms = get_connection_map_lock_measure.as_ms();
let report_stats = self let report_stats = self
@ -244,57 +374,35 @@ impl ConnectionCache {
.should_update(CONNECTION_STAT_SUBMISSION_INTERVAL); .should_update(CONNECTION_STAT_SUBMISSION_INTERVAL);
let mut get_connection_map_measure = Measure::start("get_connection_hit_measure"); let mut get_connection_map_measure = Measure::start("get_connection_hit_measure");
let (connection, cache_hit, connection_cache_stats, num_evictions, eviction_timing_ms) = let CreateConnectionResult {
match map.get(addr) { connection,
Some(connection) => (connection.clone(), true, self.stats.clone(), 0, 0), cache_hit,
None => { connection_cache_stats,
// Upgrade to write access by dropping read lock and acquire write lock num_evictions,
eviction_timing_ms,
} = match map.get(&addr) {
Some(pool) => {
if pool.need_new_connection(self.connection_pool_size) {
// create more connection and put it in the pool
drop(map); drop(map);
let mut get_connection_map_lock_measure = self.create_connection(&mut lock_timing_ms, &addr)
Measure::start("get_connection_map_lock_measure"); } else {
let mut map = self.map.write().unwrap(); let connection = pool.borrow_connection();
get_connection_map_lock_measure.stop(); CreateConnectionResult {
connection,
lock_timing_ms = cache_hit: true,
lock_timing_ms.saturating_add(get_connection_map_lock_measure.as_ms()); connection_cache_stats: self.stats.clone(),
num_evictions: 0,
// Read again, as it is possible that between read lock dropped and the write lock acquired eviction_timing_ms: 0,
// another thread could have setup the connection.
match map.get(addr) {
Some(connection) => (connection.clone(), true, self.stats.clone(), 0, 0),
None => {
let connection: Connection = if self.use_quic.load(Ordering::Relaxed) {
QuicTpuConnection::new(*addr, self.stats.clone()).into()
} else {
UdpTpuConnection::new(*addr, self.stats.clone()).into()
};
let connection = Arc::new(connection);
// evict a connection if the cache is reaching upper bounds
let mut num_evictions = 0;
let mut get_connection_cache_eviction_measure =
Measure::start("get_connection_cache_eviction_measure");
while map.len() >= MAX_CONNECTIONS {
let mut rng = thread_rng();
let n = rng.gen_range(0, MAX_CONNECTIONS);
map.swap_remove_index(n);
num_evictions += 1;
}
get_connection_cache_eviction_measure.stop();
map.insert(*addr, connection.clone());
(
connection,
false,
self.stats.clone(),
num_evictions,
get_connection_cache_eviction_measure.as_ms(),
)
}
} }
} }
}; }
None => {
// Upgrade to write access by dropping read lock and acquire write lock
drop(map);
self.create_connection(&mut lock_timing_ms, &addr)
}
};
get_connection_map_measure.stop(); get_connection_map_measure.stop();
GetConnectionResult { GetConnectionResult {
@ -359,13 +467,15 @@ impl ConnectionCache {
connection connection
} }
} }
impl Default for ConnectionCache { impl Default for ConnectionCache {
fn default() -> Self { fn default() -> Self {
Self { Self {
map: RwLock::new(IndexMap::with_capacity(MAX_CONNECTIONS)), map: RwLock::new(IndexMap::with_capacity(MAX_CONNECTIONS)),
stats: Arc::new(ConnectionCacheStats::default()), stats: Arc::new(ConnectionCacheStats::default()),
last_stats: AtomicInterval::default(), last_stats: AtomicInterval::default(),
use_quic: AtomicBool::new(DEFAULT_TPU_USE_QUIC), use_quic: DEFAULT_TPU_USE_QUIC,
connection_pool_size: DEFAULT_TPU_CONNECTION_POOL_SIZE,
} }
} }
} }
@ -381,6 +491,14 @@ struct GetConnectionResult {
eviction_timing_ms: u64, eviction_timing_ms: u64,
} }
struct CreateConnectionResult {
connection: Arc<Connection>,
cache_hit: bool,
connection_cache_stats: Arc<ConnectionCacheStats>,
num_evictions: u64,
eviction_timing_ms: u64,
}
#[cfg(test)] #[cfg(test)]
mod tests { mod tests {
use { use {
@ -432,7 +550,7 @@ mod tests {
let map = connection_cache.map.read().unwrap(); let map = connection_cache.map.read().unwrap();
assert!(map.len() == MAX_CONNECTIONS); assert!(map.len() == MAX_CONNECTIONS);
addrs.iter().for_each(|a| { addrs.iter().for_each(|a| {
let conn = map.get(a).expect("Address not found"); let conn = &map.get(a).expect("Address not found").connections[0];
assert!(a.ip() == conn.tpu_addr().ip()); assert!(a.ip() == conn.tpu_addr().ip());
}); });
} }

View File

@ -9,6 +9,7 @@ use {
async_mutex::Mutex, async_mutex::Mutex,
futures::future::join_all, futures::future::join_all,
itertools::Itertools, itertools::Itertools,
log::*,
quinn::{ quinn::{
ClientConfig, Endpoint, EndpointConfig, IdleTimeout, NewConnection, VarInt, WriteError, ClientConfig, Endpoint, EndpointConfig, IdleTimeout, NewConnection, VarInt, WriteError,
}, },
@ -18,8 +19,10 @@ use {
std::{ std::{
net::{IpAddr, Ipv4Addr, SocketAddr, UdpSocket}, net::{IpAddr, Ipv4Addr, SocketAddr, UdpSocket},
sync::{atomic::Ordering, Arc}, sync::{atomic::Ordering, Arc},
thread,
time::Duration, time::Duration,
}, },
tokio::sync::RwLock,
}; };
struct SkipServerVerification; struct SkipServerVerification;
@ -44,18 +47,19 @@ impl rustls::client::ServerCertVerifier for SkipServerVerification {
} }
} }
/// A wrapper over NewConnection with additional capability to create the endpoint as part /// A lazy-initialized Quic Endpoint
/// of creating a new connection. pub struct QuicLazyInitializedEndpoint {
#[derive(Clone)] endpoint: RwLock<Option<Arc<Endpoint>>>,
struct QuicNewConnection {
endpoint: Endpoint,
connection: Arc<NewConnection>,
} }
impl QuicNewConnection { impl QuicLazyInitializedEndpoint {
/// Create a QuicNewConnection given the remote address 'addr'. pub fn new() -> Self {
async fn make_connection(addr: SocketAddr, stats: &ClientStats) -> Result<Self, WriteError> { Self {
let mut make_connection_measure = Measure::start("make_connection_measure"); endpoint: RwLock::new(None),
}
}
fn create_endpoint() -> Endpoint {
let (_, client_socket) = solana_net_utils::bind_in_range( let (_, client_socket) = solana_net_utils::bind_in_range(
IpAddr::V4(Ipv4Addr::new(0, 0, 0, 0)), IpAddr::V4(Ipv4Addr::new(0, 0, 0, 0)),
VALIDATOR_PORT_RANGE, VALIDATOR_PORT_RANGE,
@ -78,6 +82,56 @@ impl QuicNewConnection {
transport_config.keep_alive_interval(Some(Duration::from_millis(QUIC_KEEP_ALIVE_MS))); transport_config.keep_alive_interval(Some(Duration::from_millis(QUIC_KEEP_ALIVE_MS)));
endpoint.set_default_client_config(config); endpoint.set_default_client_config(config);
endpoint
}
async fn get_endpoint(&self) -> Arc<Endpoint> {
let lock = self.endpoint.read().await;
let endpoint = lock.as_ref();
match endpoint {
Some(endpoint) => endpoint.clone(),
None => {
drop(lock);
let mut lock = self.endpoint.write().await;
let endpoint = lock.as_ref();
match endpoint {
Some(endpoint) => endpoint.clone(),
None => {
let connection = Arc::new(Self::create_endpoint());
*lock = Some(connection.clone());
connection
}
}
}
}
}
}
impl Default for QuicLazyInitializedEndpoint {
fn default() -> Self {
Self::new()
}
}
/// A wrapper over NewConnection with additional capability to create the endpoint as part
/// of creating a new connection.
#[derive(Clone)]
struct QuicNewConnection {
endpoint: Arc<Endpoint>,
connection: Arc<NewConnection>,
}
impl QuicNewConnection {
/// Create a QuicNewConnection given the remote address 'addr'.
async fn make_connection(
endpoint: Arc<QuicLazyInitializedEndpoint>,
addr: SocketAddr,
stats: &ClientStats,
) -> Result<Self, WriteError> {
let mut make_connection_measure = Measure::start("make_connection_measure");
let endpoint = endpoint.get_endpoint().await;
let connecting = endpoint.connect(addr, "connect").unwrap(); let connecting = endpoint.connect(addr, "connect").unwrap();
stats.total_connections.fetch_add(1, Ordering::Relaxed); stats.total_connections.fetch_add(1, Ordering::Relaxed);
@ -132,14 +186,16 @@ impl QuicNewConnection {
} }
pub struct QuicClient { pub struct QuicClient {
endpoint: Arc<QuicLazyInitializedEndpoint>,
connection: Arc<Mutex<Option<QuicNewConnection>>>, connection: Arc<Mutex<Option<QuicNewConnection>>>,
addr: SocketAddr, addr: SocketAddr,
stats: Arc<ClientStats>, stats: Arc<ClientStats>,
} }
impl QuicClient { impl QuicClient {
pub fn new(addr: SocketAddr) -> Self { pub fn new(endpoint: Arc<QuicLazyInitializedEndpoint>, addr: SocketAddr) -> Self {
Self { Self {
endpoint,
connection: Arc::new(Mutex::new(None)), connection: Arc::new(Mutex::new(None)),
addr, addr,
stats: Arc::new(ClientStats::default()), stats: Arc::new(ClientStats::default()),
@ -165,63 +221,131 @@ impl QuicClient {
stats: &ClientStats, stats: &ClientStats,
connection_stats: Arc<ConnectionCacheStats>, connection_stats: Arc<ConnectionCacheStats>,
) -> Result<Arc<NewConnection>, WriteError> { ) -> Result<Arc<NewConnection>, WriteError> {
let connection = { let mut connection_try_count = 0;
let mut conn_guard = self.connection.lock().await; let mut last_connection_id = 0;
let mut last_error = None;
let maybe_conn = conn_guard.clone(); while connection_try_count < 2 {
match maybe_conn { let connection = {
Some(conn) => { let mut conn_guard = self.connection.lock().await;
stats.connection_reuse.fetch_add(1, Ordering::Relaxed);
conn.connection.clone() let maybe_conn = conn_guard.as_mut();
match maybe_conn {
Some(conn) => {
if conn.connection.connection.stable_id() == last_connection_id {
// this is the problematic connection we had used before, create a new one
let conn = conn.make_connection_0rtt(self.addr, stats).await;
match conn {
Ok(conn) => {
info!(
"Made 0rtt connection to {} with id {} try_count {}, last_connection_id: {}, last_error: {:?}",
self.addr,
conn.connection.stable_id(),
connection_try_count,
last_connection_id,
last_error,
);
connection_try_count += 1;
conn
}
Err(err) => {
info!(
"Cannot make 0rtt connection to {}, error {:}",
self.addr, err
);
return Err(err);
}
}
} else {
stats.connection_reuse.fetch_add(1, Ordering::Relaxed);
conn.connection.clone()
}
}
None => {
let conn = QuicNewConnection::make_connection(
self.endpoint.clone(),
self.addr,
stats,
)
.await;
match conn {
Ok(conn) => {
*conn_guard = Some(conn.clone());
info!(
"Made connection to {} id {} try_count {}",
self.addr,
conn.connection.connection.stable_id(),
connection_try_count
);
connection_try_count += 1;
conn.connection.clone()
}
Err(err) => {
info!("Cannot make connection to {}, error {:}", self.addr, err);
return Err(err);
}
}
}
} }
None => { };
let conn = QuicNewConnection::make_connection(self.addr, stats).await?;
*conn_guard = Some(conn.clone()); let new_stats = connection.connection.stats();
conn.connection.clone()
connection_stats
.total_client_stats
.congestion_events
.update_stat(
&self.stats.congestion_events,
new_stats.path.congestion_events,
);
connection_stats
.total_client_stats
.tx_streams_blocked_uni
.update_stat(
&self.stats.tx_streams_blocked_uni,
new_stats.frame_tx.streams_blocked_uni,
);
connection_stats
.total_client_stats
.tx_data_blocked
.update_stat(&self.stats.tx_data_blocked, new_stats.frame_tx.data_blocked);
connection_stats
.total_client_stats
.tx_acks
.update_stat(&self.stats.tx_acks, new_stats.frame_tx.acks);
last_connection_id = connection.connection.stable_id();
match Self::_send_buffer_using_conn(data, &connection).await {
Ok(()) => {
return Ok(connection);
} }
} Err(err) => match err {
}; WriteError::ConnectionLost(_) => {
last_error = Some(err);
let new_stats = connection.connection.stats(); }
_ => {
connection_stats info!(
.total_client_stats "Error sending to {} with id {}, error {:?} thread: {:?}",
.congestion_events self.addr,
.update_stat( connection.connection.stable_id(),
&self.stats.congestion_events, err,
new_stats.path.congestion_events, thread::current().id(),
); );
return Err(err);
connection_stats }
.total_client_stats },
.tx_streams_blocked_uni
.update_stat(
&self.stats.tx_streams_blocked_uni,
new_stats.frame_tx.streams_blocked_uni,
);
connection_stats
.total_client_stats
.tx_data_blocked
.update_stat(&self.stats.tx_data_blocked, new_stats.frame_tx.data_blocked);
connection_stats
.total_client_stats
.tx_acks
.update_stat(&self.stats.tx_acks, new_stats.frame_tx.acks);
match Self::_send_buffer_using_conn(data, &connection).await {
Ok(()) => Ok(connection),
_ => {
let connection = {
let mut conn_guard = self.connection.lock().await;
let conn = conn_guard.as_mut().unwrap();
conn.make_connection_0rtt(self.addr, stats).await?
};
Self::_send_buffer_using_conn(data, &connection).await?;
Ok(connection)
} }
} }
// if we come here, that means we have exhausted maximum retries, return the error
info!(
"Ran into an error sending transactions {:?}, exhausted retries to {}",
last_error, self.addr
);
Err(last_error.unwrap())
} }
pub async fn send_buffer<T>( pub async fn send_buffer<T>(

View File

@ -4,12 +4,12 @@
use { use {
crate::{ crate::{
connection_cache::ConnectionCacheStats, connection_cache::ConnectionCacheStats,
nonblocking::quic_client::QuicClient, nonblocking::quic_client::{QuicClient, QuicLazyInitializedEndpoint},
tpu_connection::{ClientStats, TpuConnection}, tpu_connection::{ClientStats, TpuConnection},
}, },
lazy_static::lazy_static, lazy_static::lazy_static,
log::*, log::*,
solana_sdk::{quic::QUIC_PORT_OFFSET, transport::Result as TransportResult}, solana_sdk::transport::Result as TransportResult,
std::{net::SocketAddr, sync::Arc}, std::{net::SocketAddr, sync::Arc},
tokio::runtime::Runtime, tokio::runtime::Runtime,
}; };
@ -31,9 +31,12 @@ impl QuicTpuConnection {
self.client.stats() self.client.stats()
} }
pub fn new(tpu_addr: SocketAddr, connection_stats: Arc<ConnectionCacheStats>) -> Self { pub fn new(
let tpu_addr = SocketAddr::new(tpu_addr.ip(), tpu_addr.port() + QUIC_PORT_OFFSET); endpoint: Arc<QuicLazyInitializedEndpoint>,
let client = Arc::new(QuicClient::new(tpu_addr)); tpu_addr: SocketAddr,
connection_stats: Arc<ConnectionCacheStats>,
) -> Self {
let client = Arc::new(QuicClient::new(endpoint, tpu_addr));
Self { Self {
client, client,
@ -74,7 +77,11 @@ impl TpuConnection for QuicTpuConnection {
let send_buffer = let send_buffer =
client.send_buffer(wire_transaction, &stats, connection_stats.clone()); client.send_buffer(wire_transaction, &stats, connection_stats.clone());
if let Err(e) = send_buffer.await { if let Err(e) = send_buffer.await {
warn!("Failed to send transaction async to {:?}", e); warn!(
"Failed to send transaction async to {}, error: {:?} ",
client.tpu_addr(),
e
);
datapoint_warn!("send-wire-async", ("failure", 1, i64),); datapoint_warn!("send-wire-async", ("failure", 1, i64),);
connection_stats.add_client_stats(&stats, 1, false); connection_stats.add_client_stats(&stats, 1, false);
} else { } else {

View File

@ -3,10 +3,11 @@ mod tests {
use { use {
crossbeam_channel::unbounded, crossbeam_channel::unbounded,
solana_client::{ solana_client::{
connection_cache::ConnectionCacheStats, quic_client::QuicTpuConnection, connection_cache::ConnectionCacheStats,
nonblocking::quic_client::QuicLazyInitializedEndpoint, quic_client::QuicTpuConnection,
tpu_connection::TpuConnection, tpu_connection::TpuConnection,
}, },
solana_sdk::{packet::PACKET_DATA_SIZE, quic::QUIC_PORT_OFFSET, signature::Keypair}, solana_sdk::{packet::PACKET_DATA_SIZE, signature::Keypair},
solana_streamer::quic::{spawn_server, StreamStats}, solana_streamer::quic::{spawn_server, StreamStats},
std::{ std::{
collections::HashMap, collections::HashMap,
@ -44,10 +45,14 @@ mod tests {
.unwrap(); .unwrap();
let addr = s.local_addr().unwrap().ip(); let addr = s.local_addr().unwrap().ip();
let port = s.local_addr().unwrap().port() - QUIC_PORT_OFFSET; let port = s.local_addr().unwrap().port();
let tpu_addr = SocketAddr::new(addr, port); let tpu_addr = SocketAddr::new(addr, port);
let connection_cache_stats = Arc::new(ConnectionCacheStats::default()); let connection_cache_stats = Arc::new(ConnectionCacheStats::default());
let client = QuicTpuConnection::new(tpu_addr, connection_cache_stats); let client = QuicTpuConnection::new(
Arc::new(QuicLazyInitializedEndpoint::default()),
tpu_addr,
connection_cache_stats,
);
// Send a full size packet with single byte writes. // Send a full size packet with single byte writes.
let num_bytes = PACKET_DATA_SIZE; let num_bytes = PACKET_DATA_SIZE;

View File

@ -381,6 +381,7 @@ impl Validator {
start_progress: Arc<RwLock<ValidatorStartProgress>>, start_progress: Arc<RwLock<ValidatorStartProgress>>,
socket_addr_space: SocketAddrSpace, socket_addr_space: SocketAddrSpace,
use_quic: bool, use_quic: bool,
tpu_connection_pool_size: usize,
) -> Self { ) -> Self {
let id = identity_keypair.pubkey(); let id = identity_keypair.pubkey();
assert_eq!(id, node.info.id); assert_eq!(id, node.info.id);
@ -748,7 +749,7 @@ impl Validator {
}; };
let poh_recorder = Arc::new(Mutex::new(poh_recorder)); let poh_recorder = Arc::new(Mutex::new(poh_recorder));
let connection_cache = Arc::new(ConnectionCache::new(use_quic)); let connection_cache = Arc::new(ConnectionCache::new(use_quic, tpu_connection_pool_size));
let rpc_override_health_check = Arc::new(AtomicBool::new(false)); let rpc_override_health_check = Arc::new(AtomicBool::new(false));
let ( let (
@ -2047,7 +2048,7 @@ mod tests {
use { use {
super::*, super::*,
crossbeam_channel::{bounded, RecvTimeoutError}, crossbeam_channel::{bounded, RecvTimeoutError},
solana_client::connection_cache::DEFAULT_TPU_USE_QUIC, solana_client::connection_cache::{DEFAULT_TPU_CONNECTION_POOL_SIZE, DEFAULT_TPU_USE_QUIC},
solana_ledger::{create_new_tmp_ledger, genesis_utils::create_genesis_config_with_leader}, solana_ledger::{create_new_tmp_ledger, genesis_utils::create_genesis_config_with_leader},
solana_sdk::{genesis_config::create_genesis_config, poh_config::PohConfig}, solana_sdk::{genesis_config::create_genesis_config, poh_config::PohConfig},
std::{fs::remove_dir_all, thread, time::Duration}, std::{fs::remove_dir_all, thread, time::Duration},
@ -2084,6 +2085,7 @@ mod tests {
start_progress.clone(), start_progress.clone(),
SocketAddrSpace::Unspecified, SocketAddrSpace::Unspecified,
DEFAULT_TPU_USE_QUIC, DEFAULT_TPU_USE_QUIC,
DEFAULT_TPU_CONNECTION_POOL_SIZE,
); );
assert_eq!( assert_eq!(
*start_progress.read().unwrap(), *start_progress.read().unwrap(),
@ -2179,6 +2181,7 @@ mod tests {
Arc::new(RwLock::new(ValidatorStartProgress::default())), Arc::new(RwLock::new(ValidatorStartProgress::default())),
SocketAddrSpace::Unspecified, SocketAddrSpace::Unspecified,
DEFAULT_TPU_USE_QUIC, DEFAULT_TPU_USE_QUIC,
DEFAULT_TPU_CONNECTION_POOL_SIZE,
) )
}) })
.collect(); .collect();

View File

@ -44,7 +44,10 @@ use {
log::*, log::*,
rand::{thread_rng, Rng}, rand::{thread_rng, Rng},
solana_bench_tps::{bench::generate_and_fund_keypairs, bench_tps_client::BenchTpsClient}, solana_bench_tps::{bench::generate_and_fund_keypairs, bench_tps_client::BenchTpsClient},
solana_client::{connection_cache::ConnectionCache, rpc_client::RpcClient}, solana_client::{
connection_cache::{ConnectionCache, DEFAULT_TPU_CONNECTION_POOL_SIZE},
rpc_client::RpcClient,
},
solana_core::serve_repair::RepairProtocol, solana_core::serve_repair::RepairProtocol,
solana_dos::cli::*, solana_dos::cli::*,
solana_gossip::{ solana_gossip::{
@ -598,7 +601,10 @@ fn main() {
exit(1); exit(1);
}); });
let connection_cache = Arc::new(ConnectionCache::new(cmd_params.tpu_use_quic)); let connection_cache = Arc::new(ConnectionCache::new(
cmd_params.tpu_use_quic,
DEFAULT_TPU_CONNECTION_POOL_SIZE,
));
let (client, num_clients) = let (client, num_clients) =
get_multi_client(&validators, &SocketAddrSpace::Unspecified, connection_cache); get_multi_client(&validators, &SocketAddrSpace::Unspecified, connection_cache);
if validators.len() < num_clients { if validators.len() < num_clients {

View File

@ -7,7 +7,9 @@ use {
itertools::izip, itertools::izip,
log::*, log::*,
solana_client::{ solana_client::{
connection_cache::{ConnectionCache, DEFAULT_TPU_USE_QUIC}, connection_cache::{
ConnectionCache, DEFAULT_TPU_CONNECTION_POOL_SIZE, DEFAULT_TPU_USE_QUIC,
},
thin_client::ThinClient, thin_client::ThinClient,
}, },
solana_core::{ solana_core::{
@ -80,6 +82,7 @@ pub struct ClusterConfig {
pub poh_config: PohConfig, pub poh_config: PohConfig,
pub additional_accounts: Vec<(Pubkey, AccountSharedData)>, pub additional_accounts: Vec<(Pubkey, AccountSharedData)>,
pub tpu_use_quic: bool, pub tpu_use_quic: bool,
pub tpu_connection_pool_size: usize,
} }
impl Default for ClusterConfig { impl Default for ClusterConfig {
@ -100,6 +103,7 @@ impl Default for ClusterConfig {
skip_warmup_slots: false, skip_warmup_slots: false,
additional_accounts: vec![], additional_accounts: vec![],
tpu_use_quic: DEFAULT_TPU_USE_QUIC, tpu_use_quic: DEFAULT_TPU_USE_QUIC,
tpu_connection_pool_size: DEFAULT_TPU_CONNECTION_POOL_SIZE,
} }
} }
} }
@ -255,6 +259,7 @@ impl LocalCluster {
Arc::new(RwLock::new(ValidatorStartProgress::default())), Arc::new(RwLock::new(ValidatorStartProgress::default())),
socket_addr_space, socket_addr_space,
DEFAULT_TPU_USE_QUIC, DEFAULT_TPU_USE_QUIC,
DEFAULT_TPU_CONNECTION_POOL_SIZE,
); );
let mut validators = HashMap::new(); let mut validators = HashMap::new();
@ -277,7 +282,10 @@ impl LocalCluster {
entry_point_info: leader_contact_info, entry_point_info: leader_contact_info,
validators, validators,
genesis_config, genesis_config,
connection_cache: Arc::new(ConnectionCache::new(config.tpu_use_quic)), connection_cache: Arc::new(ConnectionCache::new(
config.tpu_use_quic,
config.tpu_connection_pool_size,
)),
}; };
let node_pubkey_to_vote_key: HashMap<Pubkey, Arc<Keypair>> = keys_in_genesis let node_pubkey_to_vote_key: HashMap<Pubkey, Arc<Keypair>> = keys_in_genesis
@ -450,6 +458,7 @@ impl LocalCluster {
Arc::new(RwLock::new(ValidatorStartProgress::default())), Arc::new(RwLock::new(ValidatorStartProgress::default())),
socket_addr_space, socket_addr_space,
DEFAULT_TPU_USE_QUIC, DEFAULT_TPU_USE_QUIC,
DEFAULT_TPU_CONNECTION_POOL_SIZE,
); );
let validator_pubkey = validator_keypair.pubkey(); let validator_pubkey = validator_keypair.pubkey();
@ -797,6 +806,7 @@ impl Cluster for LocalCluster {
Arc::new(RwLock::new(ValidatorStartProgress::default())), Arc::new(RwLock::new(ValidatorStartProgress::default())),
socket_addr_space, socket_addr_space,
DEFAULT_TPU_USE_QUIC, DEFAULT_TPU_USE_QUIC,
DEFAULT_TPU_CONNECTION_POOL_SIZE,
); );
cluster_validator_info.validator = Some(restarted_node); cluster_validator_info.validator = Some(restarted_node);
cluster_validator_info cluster_validator_info

View File

@ -8,7 +8,7 @@ use {
solana_account_decoder::UiAccount, solana_account_decoder::UiAccount,
solana_client::{ solana_client::{
client_error::{ClientErrorKind, Result as ClientResult}, client_error::{ClientErrorKind, Result as ClientResult},
connection_cache::ConnectionCache, connection_cache::{ConnectionCache, DEFAULT_TPU_CONNECTION_POOL_SIZE},
nonblocking::pubsub_client::PubsubClient, nonblocking::pubsub_client::PubsubClient,
rpc_client::RpcClient, rpc_client::RpcClient,
rpc_config::{RpcAccountInfoConfig, RpcSignatureSubscribeConfig}, rpc_config::{RpcAccountInfoConfig, RpcSignatureSubscribeConfig},
@ -420,7 +420,10 @@ fn run_tpu_send_transaction(tpu_use_quic: bool) {
test_validator.rpc_url(), test_validator.rpc_url(),
CommitmentConfig::processed(), CommitmentConfig::processed(),
)); ));
let connection_cache = Arc::new(ConnectionCache::new(tpu_use_quic)); let connection_cache = Arc::new(ConnectionCache::new(
tpu_use_quic,
DEFAULT_TPU_CONNECTION_POOL_SIZE,
));
let tpu_client = TpuClient::new_with_connection_cache( let tpu_client = TpuClient::new_with_connection_cache(
rpc_client.clone(), rpc_client.clone(),
&test_validator.rpc_pubsub_url(), &test_validator.rpc_pubsub_url(),

View File

@ -2,7 +2,11 @@
use { use {
log::*, log::*,
solana_cli_output::CliAccount, solana_cli_output::CliAccount,
solana_client::{connection_cache::DEFAULT_TPU_USE_QUIC, nonblocking, rpc_client::RpcClient}, solana_client::{
connection_cache::{DEFAULT_TPU_CONNECTION_POOL_SIZE, DEFAULT_TPU_USE_QUIC},
nonblocking,
rpc_client::RpcClient,
},
solana_core::{ solana_core::{
tower_storage::TowerStorage, tower_storage::TowerStorage,
validator::{Validator, ValidatorConfig, ValidatorStartProgress}, validator::{Validator, ValidatorConfig, ValidatorStartProgress},
@ -749,6 +753,7 @@ impl TestValidator {
config.start_progress.clone(), config.start_progress.clone(),
socket_addr_space, socket_addr_space,
DEFAULT_TPU_USE_QUIC, DEFAULT_TPU_USE_QUIC,
DEFAULT_TPU_CONNECTION_POOL_SIZE,
)); ));
// Needed to avoid panics in `solana-responder-gossip` in tests that create a number of // Needed to avoid panics in `solana-responder-gossip` in tests that create a number of

View File

@ -19,8 +19,8 @@ use {
keypair::SKIP_SEED_PHRASE_VALIDATION_ARG, keypair::SKIP_SEED_PHRASE_VALIDATION_ARG,
}, },
solana_client::{ solana_client::{
rpc_client::RpcClient, rpc_config::RpcLeaderScheduleConfig, connection_cache::DEFAULT_TPU_CONNECTION_POOL_SIZE, rpc_client::RpcClient,
rpc_request::MAX_MULTIPLE_ACCOUNTS, rpc_config::RpcLeaderScheduleConfig, rpc_request::MAX_MULTIPLE_ACCOUNTS,
}, },
solana_core::{ solana_core::{
ledger_cleanup_service::{DEFAULT_MAX_LEDGER_SHREDS, DEFAULT_MIN_MAX_LEDGER_SHREDS}, ledger_cleanup_service::{DEFAULT_MAX_LEDGER_SHREDS, DEFAULT_MIN_MAX_LEDGER_SHREDS},
@ -468,6 +468,7 @@ pub fn main() {
let default_accounts_shrink_ratio = &DEFAULT_ACCOUNTS_SHRINK_RATIO.to_string(); let default_accounts_shrink_ratio = &DEFAULT_ACCOUNTS_SHRINK_RATIO.to_string();
let default_rocksdb_fifo_shred_storage_size = let default_rocksdb_fifo_shred_storage_size =
&DEFAULT_ROCKS_FIFO_SHRED_STORAGE_SIZE_BYTES.to_string(); &DEFAULT_ROCKS_FIFO_SHRED_STORAGE_SIZE_BYTES.to_string();
let default_tpu_connection_pool_size = &DEFAULT_TPU_CONNECTION_POOL_SIZE.to_string();
let matches = App::new(crate_name!()).about(crate_description!()) let matches = App::new(crate_name!()).about(crate_description!())
.version(solana_version::version!()) .version(solana_version::version!())
@ -1209,6 +1210,14 @@ pub fn main() {
.takes_value(false) .takes_value(false)
.help("Use QUIC to send transactions."), .help("Use QUIC to send transactions."),
) )
.arg(
Arg::with_name("tpu_connection_pool_size")
.long("tpu-connection-pool-size")
.takes_value(true)
.default_value(default_tpu_connection_pool_size)
.validator(is_parsable::<usize>)
.help("Controls the TPU connection pool size per remote addresss"),
)
.arg( .arg(
Arg::with_name("rocksdb_max_compaction_jitter") Arg::with_name("rocksdb_max_compaction_jitter")
.long("rocksdb-max-compaction-jitter-slots") .long("rocksdb-max-compaction-jitter-slots")
@ -2214,6 +2223,7 @@ pub fn main() {
let accounts_shrink_optimize_total_space = let accounts_shrink_optimize_total_space =
value_t_or_exit!(matches, "accounts_shrink_optimize_total_space", bool); value_t_or_exit!(matches, "accounts_shrink_optimize_total_space", bool);
let tpu_use_quic = matches.is_present("tpu_use_quic"); let tpu_use_quic = matches.is_present("tpu_use_quic");
let tpu_connection_pool_size = value_t_or_exit!(matches, "tpu_connection_pool_size", usize);
let shrink_ratio = value_t_or_exit!(matches, "accounts_shrink_ratio", f64); let shrink_ratio = value_t_or_exit!(matches, "accounts_shrink_ratio", f64);
if !(0.0..=1.0).contains(&shrink_ratio) { if !(0.0..=1.0).contains(&shrink_ratio) {
@ -2973,6 +2983,7 @@ pub fn main() {
start_progress, start_progress,
socket_addr_space, socket_addr_space,
tpu_use_quic, tpu_use_quic,
tpu_connection_pool_size,
); );
*admin_service_post_init.write().unwrap() = *admin_service_post_init.write().unwrap() =
Some(admin_rpc_service::AdminRpcRequestMetadataPostInit { Some(admin_rpc_service::AdminRpcRequestMetadataPostInit {