Thin client quic (#23973)

Change thin-client to use connection-cache
This commit is contained in:
ryleung-solana
2022-03-31 15:47:00 -04:00
committed by GitHub
parent 31997f8251
commit 8b72200afb
13 changed files with 98 additions and 154 deletions

View File

@ -9,7 +9,6 @@ use {
solana_client::thin_client::create_client, solana_client::thin_client::create_client,
solana_core::validator::ValidatorConfig, solana_core::validator::ValidatorConfig,
solana_faucet::faucet::run_local_faucet_with_port, solana_faucet::faucet::run_local_faucet_with_port,
solana_gossip::cluster_info::VALIDATOR_PORT_RANGE,
solana_local_cluster::{ solana_local_cluster::{
local_cluster::{ClusterConfig, LocalCluster}, local_cluster::{ClusterConfig, LocalCluster},
validator_configs::make_identical_validator_configs, validator_configs::make_identical_validator_configs,
@ -45,10 +44,10 @@ fn test_bench_tps_local_cluster(config: Config) {
100_000_000, 100_000_000,
); );
let client = Arc::new(create_client( let client = Arc::new(create_client((
(cluster.entry_point_info.rpc, cluster.entry_point_info.tpu), cluster.entry_point_info.rpc,
VALIDATOR_PORT_RANGE, cluster.entry_point_info.tpu,
)); )));
let (addr_sender, addr_receiver) = unbounded(); let (addr_sender, addr_receiver) = unbounded();
run_local_faucet_with_port(faucet_keypair, addr_sender, None, 0); run_local_faucet_with_port(faucet_keypair, addr_sender, None, 0);

View File

@ -3,10 +3,11 @@ use {
quic_client::QuicTpuConnection, tpu_connection::TpuConnection, udp_client::UdpTpuConnection, quic_client::QuicTpuConnection, tpu_connection::TpuConnection, udp_client::UdpTpuConnection,
}, },
lazy_static::lazy_static, lazy_static::lazy_static,
solana_net_utils::VALIDATOR_PORT_RANGE,
solana_sdk::{transaction::VersionedTransaction, transport::TransportError}, solana_sdk::{transaction::VersionedTransaction, transport::TransportError},
std::{ std::{
collections::{hash_map::Entry, BTreeMap, HashMap}, collections::{hash_map::Entry, BTreeMap, HashMap},
net::{SocketAddr, UdpSocket}, net::{IpAddr, Ipv4Addr, SocketAddr},
sync::{Arc, Mutex}, sync::{Arc, Mutex},
}, },
}; };
@ -74,10 +75,11 @@ fn get_connection(addr: &SocketAddr) -> Connection {
(pair.0.clone(), old_ticks) (pair.0.clone(), old_ticks)
} }
Entry::Vacant(entry) => { Entry::Vacant(entry) => {
let send_socket = UdpSocket::bind("0.0.0.0:0").unwrap(); let (_, send_socket) = solana_net_utils::bind_in_range(
// TODO: see https://github.com/solana-labs/solana/issues/23659 IpAddr::V4(Ipv4Addr::new(0, 0, 0, 0)),
// make it configurable (e.g. via the command line) whether to use UDP or Quic VALIDATOR_PORT_RANGE,
)
.unwrap();
let conn = if use_quic { let conn = if use_quic {
Connection::Quic(Arc::new(QuicTpuConnection::new(send_socket, *addr))) Connection::Quic(Arc::new(QuicTpuConnection::new(send_socket, *addr)))
} else { } else {

View File

@ -5,8 +5,13 @@
use { use {
crate::{ crate::{
rpc_client::RpcClient, rpc_config::RpcProgramAccountsConfig, rpc_response::Response, connection_cache::{
tpu_connection::TpuConnection, udp_client::UdpTpuConnection, par_serialize_and_send_transaction_batch, send_wire_transaction,
serialize_and_send_transaction,
},
rpc_client::RpcClient,
rpc_config::RpcProgramAccountsConfig,
rpc_response::Response,
}, },
log::*, log::*,
solana_sdk::{ solana_sdk::{
@ -29,7 +34,7 @@ use {
}, },
std::{ std::{
io, io,
net::{IpAddr, Ipv4Addr, SocketAddr, UdpSocket}, net::SocketAddr,
sync::{ sync::{
atomic::{AtomicBool, AtomicUsize, Ordering}, atomic::{AtomicBool, AtomicUsize, Ordering},
RwLock, RwLock,
@ -118,64 +123,52 @@ impl ClientOptimizer {
} }
/// An object for querying and sending transactions to the network. /// An object for querying and sending transactions to the network.
pub struct ThinClient<C: 'static + TpuConnection> { pub struct ThinClient {
rpc_clients: Vec<RpcClient>, rpc_clients: Vec<RpcClient>,
tpu_connections: Vec<C>, tpu_addrs: Vec<SocketAddr>,
optimizer: ClientOptimizer, optimizer: ClientOptimizer,
} }
impl<C: 'static + TpuConnection> ThinClient<C> { impl ThinClient {
/// Create a new ThinClient that will interface with the Rpc at `rpc_addr` using TCP /// Create a new ThinClient that will interface with the Rpc at `rpc_addr` using TCP
/// and the Tpu at `tpu_addr` over `transactions_socket` using Quic or UDP /// and the Tpu at `tpu_addr` over `transactions_socket` using Quic or UDP
/// (currently hardcoded to UDP) /// (currently hardcoded to UDP)
pub fn new(rpc_addr: SocketAddr, tpu_addr: SocketAddr, transactions_socket: UdpSocket) -> Self { pub fn new(rpc_addr: SocketAddr, tpu_addr: SocketAddr) -> Self {
let tpu_connection = C::new(transactions_socket, tpu_addr); Self::new_from_client(RpcClient::new_socket(rpc_addr), tpu_addr)
Self::new_from_client(RpcClient::new_socket(rpc_addr), tpu_connection)
} }
pub fn new_socket_with_timeout( pub fn new_socket_with_timeout(
rpc_addr: SocketAddr, rpc_addr: SocketAddr,
tpu_addr: SocketAddr, tpu_addr: SocketAddr,
transactions_socket: UdpSocket,
timeout: Duration, timeout: Duration,
) -> Self { ) -> Self {
let rpc_client = RpcClient::new_socket_with_timeout(rpc_addr, timeout); let rpc_client = RpcClient::new_socket_with_timeout(rpc_addr, timeout);
let tpu_connection = C::new(transactions_socket, tpu_addr); Self::new_from_client(rpc_client, tpu_addr)
Self::new_from_client(rpc_client, tpu_connection)
} }
fn new_from_client(rpc_client: RpcClient, tpu_connection: C) -> Self { fn new_from_client(rpc_client: RpcClient, tpu_addr: SocketAddr) -> Self {
Self { Self {
rpc_clients: vec![rpc_client], rpc_clients: vec![rpc_client],
tpu_connections: vec![tpu_connection], tpu_addrs: vec![tpu_addr],
optimizer: ClientOptimizer::new(0), optimizer: ClientOptimizer::new(0),
} }
} }
pub fn new_from_addrs( pub fn new_from_addrs(rpc_addrs: Vec<SocketAddr>, tpu_addrs: Vec<SocketAddr>) -> Self {
rpc_addrs: Vec<SocketAddr>,
tpu_addrs: Vec<SocketAddr>,
transactions_socket: UdpSocket,
) -> Self {
assert!(!rpc_addrs.is_empty()); assert!(!rpc_addrs.is_empty());
assert_eq!(rpc_addrs.len(), tpu_addrs.len()); assert_eq!(rpc_addrs.len(), tpu_addrs.len());
let rpc_clients: Vec<_> = rpc_addrs.into_iter().map(RpcClient::new_socket).collect(); let rpc_clients: Vec<_> = rpc_addrs.into_iter().map(RpcClient::new_socket).collect();
let optimizer = ClientOptimizer::new(rpc_clients.len()); let optimizer = ClientOptimizer::new(rpc_clients.len());
let tpu_connections: Vec<_> = tpu_addrs
.into_iter()
.map(|tpu_addr| C::new(transactions_socket.try_clone().unwrap(), tpu_addr))
.collect();
Self { Self {
rpc_clients, rpc_clients,
tpu_connections, tpu_addrs,
optimizer, optimizer,
} }
} }
fn tpu_connection(&self) -> &C { fn tpu_addr(&self) -> &SocketAddr {
&self.tpu_connections[self.optimizer.best()] &self.tpu_addrs[self.optimizer.best()]
} }
fn rpc_client(&self) -> &RpcClient { fn rpc_client(&self) -> &RpcClient {
@ -220,8 +213,7 @@ impl<C: 'static + TpuConnection> ThinClient<C> {
while now.elapsed().as_secs() < wait_time as u64 { while now.elapsed().as_secs() < wait_time as u64 {
if num_confirmed == 0 { if num_confirmed == 0 {
// Send the transaction if there has been no confirmation (e.g. the first time) // Send the transaction if there has been no confirmation (e.g. the first time)
self.tpu_connection() send_wire_transaction(&wire_transaction, self.tpu_addr())?;
.send_wire_transaction(&wire_transaction)?;
} }
if let Ok(confirmed_blocks) = self.poll_for_signature_confirmation( if let Ok(confirmed_blocks) = self.poll_for_signature_confirmation(
@ -316,13 +308,13 @@ impl<C: 'static + TpuConnection> ThinClient<C> {
} }
} }
impl<C: 'static + TpuConnection> Client for ThinClient<C> { impl Client for ThinClient {
fn tpu_addr(&self) -> String { fn tpu_addr(&self) -> String {
self.tpu_connection().tpu_addr().to_string() self.tpu_addr().to_string()
} }
} }
impl<C: 'static + TpuConnection> SyncClient for ThinClient<C> { impl SyncClient for ThinClient {
fn send_and_confirm_message<T: Signers>( fn send_and_confirm_message<T: Signers>(
&self, &self,
keypairs: &T, keypairs: &T,
@ -602,18 +594,16 @@ impl<C: 'static + TpuConnection> SyncClient for ThinClient<C> {
} }
} }
impl<C: 'static + TpuConnection> AsyncClient for ThinClient<C> { impl AsyncClient for ThinClient {
fn async_send_transaction(&self, transaction: Transaction) -> TransportResult<Signature> { fn async_send_transaction(&self, transaction: Transaction) -> TransportResult<Signature> {
let transaction = VersionedTransaction::from(transaction); let transaction = VersionedTransaction::from(transaction);
self.tpu_connection() serialize_and_send_transaction(&transaction, self.tpu_addr())?;
.serialize_and_send_transaction(&transaction)?;
Ok(transaction.signatures[0]) Ok(transaction.signatures[0])
} }
fn async_send_batch(&self, transactions: Vec<Transaction>) -> TransportResult<()> { fn async_send_batch(&self, transactions: Vec<Transaction>) -> TransportResult<()> {
let batch: Vec<VersionedTransaction> = transactions.into_iter().map(Into::into).collect(); let batch: Vec<VersionedTransaction> = transactions.into_iter().map(Into::into).collect();
self.tpu_connection() par_serialize_and_send_transaction_batch(&batch[..], self.tpu_addr())?;
.par_serialize_and_send_transaction_batch(&batch[..])?;
Ok(()) Ok(())
} }
@ -648,23 +638,15 @@ impl<C: 'static + TpuConnection> AsyncClient for ThinClient<C> {
} }
} }
pub fn create_client( pub fn create_client((rpc, tpu): (SocketAddr, SocketAddr)) -> ThinClient {
(rpc, tpu): (SocketAddr, SocketAddr), ThinClient::new(rpc, tpu)
range: (u16, u16),
) -> ThinClient<UdpTpuConnection> {
let (_, transactions_socket) =
solana_net_utils::bind_in_range(IpAddr::V4(Ipv4Addr::new(0, 0, 0, 0)), range).unwrap();
ThinClient::<UdpTpuConnection>::new(rpc, tpu, transactions_socket)
} }
pub fn create_client_with_timeout( pub fn create_client_with_timeout(
(rpc, tpu): (SocketAddr, SocketAddr), (rpc, tpu): (SocketAddr, SocketAddr),
range: (u16, u16),
timeout: Duration, timeout: Duration,
) -> ThinClient<UdpTpuConnection> { ) -> ThinClient {
let (_, transactions_socket) = ThinClient::new_socket_with_timeout(rpc, tpu, timeout)
solana_net_utils::bind_in_range(IpAddr::V4(Ipv4Addr::new(0, 0, 0, 0)), range).unwrap();
ThinClient::<UdpTpuConnection>::new_socket_with_timeout(rpc, tpu, transactions_socket, timeout)
} }
#[cfg(test)] #[cfg(test)]

View File

@ -12,6 +12,13 @@
//! * layer 2 - Everyone else, if layer 1 is `2^10`, layer 2 should be able to fit `2^20` number of nodes. //! * layer 2 - Everyone else, if layer 1 is `2^10`, layer 2 should be able to fit `2^20` number of nodes.
//! //!
//! Bank needs to provide an interface for us to query the stake weight //! Bank needs to provide an interface for us to query the stake weight
#[deprecated(
since = "1.10.6",
note = "Please use `solana_net_utils::{MINIMUM_VALIDATOR_PORT_RANGE_WIDTH, VALIDATOR_PORT_RANGE}` instead"
)]
#[allow(deprecated)]
pub use solana_net_utils::{MINIMUM_VALIDATOR_PORT_RANGE_WIDTH, VALIDATOR_PORT_RANGE};
use { use {
crate::{ crate::{
cluster_info_metrics::{ cluster_info_metrics::{
@ -92,9 +99,6 @@ use {
}, },
}; };
pub const VALIDATOR_PORT_RANGE: PortRange = (8000, 10_000);
pub const MINIMUM_VALIDATOR_PORT_RANGE_WIDTH: u16 = 12; // VALIDATOR_PORT_RANGE must be at least this wide
/// The Data plane fanout size, also used as the neighborhood size /// The Data plane fanout size, also used as the neighborhood size
pub const DATA_PLANE_FANOUT: usize = 200; pub const DATA_PLANE_FANOUT: usize = 200;
/// milliseconds we sleep for between gossip requests /// milliseconds we sleep for between gossip requests
@ -3075,6 +3079,7 @@ mod tests {
rand::{seq::SliceRandom, SeedableRng}, rand::{seq::SliceRandom, SeedableRng},
rand_chacha::ChaChaRng, rand_chacha::ChaChaRng,
solana_ledger::shred::Shredder, solana_ledger::shred::Shredder,
solana_net_utils::MINIMUM_VALIDATOR_PORT_RANGE_WIDTH,
solana_sdk::signature::{Keypair, Signer}, solana_sdk::signature::{Keypair, Signer},
solana_vote_program::{vote_instruction, vote_state::Vote}, solana_vote_program::{vote_instruction, vote_state::Vote},
std::{ std::{

View File

@ -1,16 +1,10 @@
//! The `gossip_service` module implements the network control plane. //! The `gossip_service` module implements the network control plane.
use { use {
crate::{ crate::{cluster_info::ClusterInfo, contact_info::ContactInfo},
cluster_info::{ClusterInfo, VALIDATOR_PORT_RANGE},
contact_info::ContactInfo,
},
crossbeam_channel::{unbounded, Sender}, crossbeam_channel::{unbounded, Sender},
rand::{thread_rng, Rng}, rand::{thread_rng, Rng},
solana_client::{ solana_client::thin_client::{create_client, ThinClient},
thin_client::{create_client, ThinClient},
udp_client::UdpTpuConnection,
},
solana_perf::recycler::Recycler, solana_perf::recycler::Recycler,
solana_runtime::bank_forks::BankForks, solana_runtime::bank_forks::BankForks,
solana_sdk::{ solana_sdk::{
@ -20,7 +14,7 @@ use {
solana_streamer::{socket::SocketAddrSpace, streamer}, solana_streamer::{socket::SocketAddrSpace, streamer},
std::{ std::{
collections::HashSet, collections::HashSet,
net::{IpAddr, Ipv4Addr, SocketAddr, TcpListener, UdpSocket}, net::{SocketAddr, TcpListener, UdpSocket},
sync::{ sync::{
atomic::{AtomicBool, Ordering}, atomic::{AtomicBool, Ordering},
Arc, RwLock, Arc, RwLock,
@ -197,51 +191,37 @@ pub fn discover(
} }
/// Creates a ThinClient per valid node /// Creates a ThinClient per valid node
pub fn get_clients( pub fn get_clients(nodes: &[ContactInfo], socket_addr_space: &SocketAddrSpace) -> Vec<ThinClient> {
nodes: &[ContactInfo],
socket_addr_space: &SocketAddrSpace,
) -> Vec<ThinClient<UdpTpuConnection>> {
nodes nodes
.iter() .iter()
.filter_map(|node| ContactInfo::valid_client_facing_addr(node, socket_addr_space)) .filter_map(|node| ContactInfo::valid_client_facing_addr(node, socket_addr_space))
.map(|addrs| create_client(addrs, VALIDATOR_PORT_RANGE)) .map(create_client)
.collect() .collect()
} }
/// Creates a ThinClient by selecting a valid node at random /// Creates a ThinClient by selecting a valid node at random
pub fn get_client( pub fn get_client(nodes: &[ContactInfo], socket_addr_space: &SocketAddrSpace) -> ThinClient {
nodes: &[ContactInfo],
socket_addr_space: &SocketAddrSpace,
) -> ThinClient<UdpTpuConnection> {
let nodes: Vec<_> = nodes let nodes: Vec<_> = nodes
.iter() .iter()
.filter_map(|node| ContactInfo::valid_client_facing_addr(node, socket_addr_space)) .filter_map(|node| ContactInfo::valid_client_facing_addr(node, socket_addr_space))
.collect(); .collect();
let select = thread_rng().gen_range(0, nodes.len()); let select = thread_rng().gen_range(0, nodes.len());
create_client(nodes[select], VALIDATOR_PORT_RANGE) create_client(nodes[select])
} }
pub fn get_multi_client( pub fn get_multi_client(
nodes: &[ContactInfo], nodes: &[ContactInfo],
socket_addr_space: &SocketAddrSpace, socket_addr_space: &SocketAddrSpace,
) -> (ThinClient<UdpTpuConnection>, usize) { ) -> (ThinClient, usize) {
let addrs: Vec<_> = nodes let addrs: Vec<_> = nodes
.iter() .iter()
.filter_map(|node| ContactInfo::valid_client_facing_addr(node, socket_addr_space)) .filter_map(|node| ContactInfo::valid_client_facing_addr(node, socket_addr_space))
.collect(); .collect();
let rpc_addrs: Vec<_> = addrs.iter().map(|addr| addr.0).collect(); let rpc_addrs: Vec<_> = addrs.iter().map(|addr| addr.0).collect();
let tpu_addrs: Vec<_> = addrs.iter().map(|addr| addr.1).collect(); let tpu_addrs: Vec<_> = addrs.iter().map(|addr| addr.1).collect();
let (_, transactions_socket) = solana_net_utils::bind_in_range(
IpAddr::V4(Ipv4Addr::new(0, 0, 0, 0)),
VALIDATOR_PORT_RANGE,
)
.unwrap();
let num_nodes = tpu_addrs.len(); let num_nodes = tpu_addrs.len();
( (ThinClient::new_from_addrs(rpc_addrs, tpu_addrs), num_nodes)
//TODO: make it configurable whether to use quic
ThinClient::<UdpTpuConnection>::new_from_addrs(rpc_addrs, tpu_addrs, transactions_socket),
num_nodes,
)
} }
fn spy( fn spy(

View File

@ -1,5 +1,5 @@
use { use {
solana_client::{thin_client::ThinClient, udp_client::UdpTpuConnection}, solana_client::thin_client::ThinClient,
solana_core::validator::{Validator, ValidatorConfig}, solana_core::validator::{Validator, ValidatorConfig},
solana_gossip::{cluster_info::Node, contact_info::ContactInfo}, solana_gossip::{cluster_info::Node, contact_info::ContactInfo},
solana_sdk::{pubkey::Pubkey, signature::Keypair}, solana_sdk::{pubkey::Pubkey, signature::Keypair},
@ -36,7 +36,7 @@ impl ClusterValidatorInfo {
pub trait Cluster { pub trait Cluster {
fn get_node_pubkeys(&self) -> Vec<Pubkey>; fn get_node_pubkeys(&self) -> Vec<Pubkey>;
fn get_validator_client(&self, pubkey: &Pubkey) -> Option<ThinClient<UdpTpuConnection>>; fn get_validator_client(&self, pubkey: &Pubkey) -> Option<ThinClient>;
fn get_contact_info(&self, pubkey: &Pubkey) -> Option<&ContactInfo>; fn get_contact_info(&self, pubkey: &Pubkey) -> Option<&ContactInfo>;
fn exit_node(&mut self, pubkey: &Pubkey) -> ClusterValidatorInfo; fn exit_node(&mut self, pubkey: &Pubkey) -> ClusterValidatorInfo;
fn restart_node( fn restart_node(

View File

@ -10,7 +10,7 @@ use {
solana_core::consensus::VOTE_THRESHOLD_DEPTH, solana_core::consensus::VOTE_THRESHOLD_DEPTH,
solana_entry::entry::{Entry, EntrySlice}, solana_entry::entry::{Entry, EntrySlice},
solana_gossip::{ solana_gossip::{
cluster_info::{self, VALIDATOR_PORT_RANGE}, cluster_info,
contact_info::ContactInfo, contact_info::ContactInfo,
crds_value::{self, CrdsData, CrdsValue}, crds_value::{self, CrdsData, CrdsValue},
gossip_error::GossipError, gossip_error::GossipError,
@ -60,7 +60,7 @@ pub fn spend_and_verify_all_nodes<S: ::std::hash::BuildHasher + Sync + Send>(
return; return;
} }
let random_keypair = Keypair::new(); let random_keypair = Keypair::new();
let client = create_client(ingress_node.client_facing_addr(), VALIDATOR_PORT_RANGE); let client = create_client(ingress_node.client_facing_addr());
let bal = client let bal = client
.poll_get_balance_with_commitment( .poll_get_balance_with_commitment(
&funding_keypair.pubkey(), &funding_keypair.pubkey(),
@ -81,7 +81,7 @@ pub fn spend_and_verify_all_nodes<S: ::std::hash::BuildHasher + Sync + Send>(
if ignore_nodes.contains(&validator.id) { if ignore_nodes.contains(&validator.id) {
continue; continue;
} }
let client = create_client(validator.client_facing_addr(), VALIDATOR_PORT_RANGE); let client = create_client(validator.client_facing_addr());
client.poll_for_signature_confirmation(&sig, confs).unwrap(); client.poll_for_signature_confirmation(&sig, confs).unwrap();
} }
}); });
@ -91,7 +91,7 @@ pub fn verify_balances<S: ::std::hash::BuildHasher>(
expected_balances: HashMap<Pubkey, u64, S>, expected_balances: HashMap<Pubkey, u64, S>,
node: &ContactInfo, node: &ContactInfo,
) { ) {
let client = create_client(node.client_facing_addr(), VALIDATOR_PORT_RANGE); let client = create_client(node.client_facing_addr());
for (pk, b) in expected_balances { for (pk, b) in expected_balances {
let bal = client let bal = client
.poll_get_balance_with_commitment(&pk, CommitmentConfig::processed()) .poll_get_balance_with_commitment(&pk, CommitmentConfig::processed())
@ -106,7 +106,7 @@ pub fn send_many_transactions(
max_tokens_per_transfer: u64, max_tokens_per_transfer: u64,
num_txs: u64, num_txs: u64,
) -> HashMap<Pubkey, u64> { ) -> HashMap<Pubkey, u64> {
let client = create_client(node.client_facing_addr(), VALIDATOR_PORT_RANGE); let client = create_client(node.client_facing_addr());
let mut expected_balances = HashMap::new(); let mut expected_balances = HashMap::new();
for _ in 0..num_txs { for _ in 0..num_txs {
let random_keypair = Keypair::new(); let random_keypair = Keypair::new();
@ -197,7 +197,7 @@ pub fn kill_entry_and_spend_and_verify_rest(
let cluster_nodes = let cluster_nodes =
discover_cluster(&entry_point_info.gossip, nodes, socket_addr_space).unwrap(); discover_cluster(&entry_point_info.gossip, nodes, socket_addr_space).unwrap();
assert!(cluster_nodes.len() >= nodes); assert!(cluster_nodes.len() >= nodes);
let client = create_client(entry_point_info.client_facing_addr(), VALIDATOR_PORT_RANGE); let client = create_client(entry_point_info.client_facing_addr());
// sleep long enough to make sure we are in epoch 3 // sleep long enough to make sure we are in epoch 3
let first_two_epoch_slots = MINIMUM_SLOTS_PER_EPOCH * (3 + 1); let first_two_epoch_slots = MINIMUM_SLOTS_PER_EPOCH * (3 + 1);
@ -225,7 +225,7 @@ pub fn kill_entry_and_spend_and_verify_rest(
continue; continue;
} }
let client = create_client(ingress_node.client_facing_addr(), VALIDATOR_PORT_RANGE); let client = create_client(ingress_node.client_facing_addr());
let balance = client let balance = client
.poll_get_balance_with_commitment( .poll_get_balance_with_commitment(
&funding_keypair.pubkey(), &funding_keypair.pubkey(),
@ -296,7 +296,7 @@ pub fn check_for_new_roots(num_new_roots: usize, contact_infos: &[ContactInfo],
assert!(loop_start.elapsed() < loop_timeout); assert!(loop_start.elapsed() < loop_timeout);
for (i, ingress_node) in contact_infos.iter().enumerate() { for (i, ingress_node) in contact_infos.iter().enumerate() {
let client = create_client(ingress_node.client_facing_addr(), VALIDATOR_PORT_RANGE); let client = create_client(ingress_node.client_facing_addr());
let root_slot = client let root_slot = client
.get_slot_with_commitment(CommitmentConfig::finalized()) .get_slot_with_commitment(CommitmentConfig::finalized())
.unwrap_or(0); .unwrap_or(0);
@ -327,7 +327,7 @@ pub fn check_no_new_roots(
.iter() .iter()
.enumerate() .enumerate()
.map(|(i, ingress_node)| { .map(|(i, ingress_node)| {
let client = create_client(ingress_node.client_facing_addr(), VALIDATOR_PORT_RANGE); let client = create_client(ingress_node.client_facing_addr());
let initial_root = client let initial_root = client
.get_slot() .get_slot()
.unwrap_or_else(|_| panic!("get_slot for {} failed", ingress_node.id)); .unwrap_or_else(|_| panic!("get_slot for {} failed", ingress_node.id));
@ -345,7 +345,7 @@ pub fn check_no_new_roots(
let mut reached_end_slot = false; let mut reached_end_slot = false;
loop { loop {
for contact_info in contact_infos { for contact_info in contact_infos {
let client = create_client(contact_info.client_facing_addr(), VALIDATOR_PORT_RANGE); let client = create_client(contact_info.client_facing_addr());
current_slot = client current_slot = client
.get_slot_with_commitment(CommitmentConfig::processed()) .get_slot_with_commitment(CommitmentConfig::processed())
.unwrap_or_else(|_| panic!("get_slot for {} failed", contact_infos[0].id)); .unwrap_or_else(|_| panic!("get_slot for {} failed", contact_infos[0].id));
@ -367,7 +367,7 @@ pub fn check_no_new_roots(
} }
for (i, ingress_node) in contact_infos.iter().enumerate() { for (i, ingress_node) in contact_infos.iter().enumerate() {
let client = create_client(ingress_node.client_facing_addr(), VALIDATOR_PORT_RANGE); let client = create_client(ingress_node.client_facing_addr());
assert_eq!( assert_eq!(
client client
.get_slot() .get_slot()
@ -387,7 +387,7 @@ fn poll_all_nodes_for_signature(
if validator.id == entry_point_info.id { if validator.id == entry_point_info.id {
continue; continue;
} }
let client = create_client(validator.client_facing_addr(), VALIDATOR_PORT_RANGE); let client = create_client(validator.client_facing_addr());
client.poll_for_signature_confirmation(sig, confs)?; client.poll_for_signature_confirmation(sig, confs)?;
} }

View File

@ -6,18 +6,13 @@ use {
}, },
itertools::izip, itertools::izip,
log::*, log::*,
solana_client::{ solana_client::thin_client::{create_client, ThinClient},
thin_client::{create_client, ThinClient},
udp_client::UdpTpuConnection,
},
solana_core::{ solana_core::{
tower_storage::FileTowerStorage, tower_storage::FileTowerStorage,
validator::{Validator, ValidatorConfig, ValidatorStartProgress}, validator::{Validator, ValidatorConfig, ValidatorStartProgress},
}, },
solana_gossip::{ solana_gossip::{
cluster_info::{Node, VALIDATOR_PORT_RANGE}, cluster_info::Node, contact_info::ContactInfo, gossip_service::discover_cluster,
contact_info::ContactInfo,
gossip_service::discover_cluster,
}, },
solana_ledger::create_new_tmp_ledger, solana_ledger::create_new_tmp_ledger,
solana_runtime::genesis_utils::{ solana_runtime::genesis_utils::{
@ -393,10 +388,7 @@ impl LocalCluster {
mut voting_keypair: Option<Arc<Keypair>>, mut voting_keypair: Option<Arc<Keypair>>,
socket_addr_space: SocketAddrSpace, socket_addr_space: SocketAddrSpace,
) -> Pubkey { ) -> Pubkey {
let client = create_client( let client = create_client(self.entry_point_info.client_facing_addr());
self.entry_point_info.client_facing_addr(),
VALIDATOR_PORT_RANGE,
);
// Must have enough tokens to fund vote account and set delegate // Must have enough tokens to fund vote account and set delegate
let should_create_vote_pubkey = voting_keypair.is_none(); let should_create_vote_pubkey = voting_keypair.is_none();
@ -480,10 +472,7 @@ impl LocalCluster {
} }
pub fn transfer(&self, source_keypair: &Keypair, dest_pubkey: &Pubkey, lamports: u64) -> u64 { pub fn transfer(&self, source_keypair: &Keypair, dest_pubkey: &Pubkey, lamports: u64) -> u64 {
let client = create_client( let client = create_client(self.entry_point_info.client_facing_addr());
self.entry_point_info.client_facing_addr(),
VALIDATOR_PORT_RANGE,
);
Self::transfer_with_client(&client, source_keypair, dest_pubkey, lamports) Self::transfer_with_client(&client, source_keypair, dest_pubkey, lamports)
} }
@ -538,7 +527,7 @@ impl LocalCluster {
} }
fn transfer_with_client( fn transfer_with_client(
client: &ThinClient<UdpTpuConnection>, client: &ThinClient,
source_keypair: &Keypair, source_keypair: &Keypair,
dest_pubkey: &Pubkey, dest_pubkey: &Pubkey,
lamports: u64, lamports: u64,
@ -567,7 +556,7 @@ impl LocalCluster {
} }
fn setup_vote_and_stake_accounts( fn setup_vote_and_stake_accounts(
client: &ThinClient<UdpTpuConnection>, client: &ThinClient,
vote_account: &Keypair, vote_account: &Keypair,
from_account: &Arc<Keypair>, from_account: &Arc<Keypair>,
amount: u64, amount: u64,
@ -704,13 +693,10 @@ impl Cluster for LocalCluster {
self.validators.keys().cloned().collect() self.validators.keys().cloned().collect()
} }
fn get_validator_client(&self, pubkey: &Pubkey) -> Option<ThinClient<UdpTpuConnection>> { fn get_validator_client(&self, pubkey: &Pubkey) -> Option<ThinClient> {
self.validators.get(pubkey).map(|f| { self.validators
create_client( .get(pubkey)
f.info.contact_info.client_facing_addr(), .map(|f| create_client(f.info.contact_info.client_facing_addr()))
VALIDATOR_PORT_RANGE,
)
})
} }
fn exit_node(&mut self, pubkey: &Pubkey) -> ClusterValidatorInfo { fn exit_node(&mut self, pubkey: &Pubkey) -> ClusterValidatorInfo {

View File

@ -17,7 +17,6 @@ use {
rpc_config::{RpcProgramAccountsConfig, RpcSignatureSubscribeConfig}, rpc_config::{RpcProgramAccountsConfig, RpcSignatureSubscribeConfig},
rpc_response::RpcSignatureResult, rpc_response::RpcSignatureResult,
thin_client::{create_client, ThinClient}, thin_client::{create_client, ThinClient},
udp_client::UdpTpuConnection,
}, },
solana_core::{ solana_core::{
broadcast_stage::BroadcastStageType, broadcast_stage::BroadcastStageType,
@ -28,7 +27,7 @@ use {
validator::ValidatorConfig, validator::ValidatorConfig,
}, },
solana_download_utils::download_snapshot_archive, solana_download_utils::download_snapshot_archive,
solana_gossip::{cluster_info::VALIDATOR_PORT_RANGE, gossip_service::discover_cluster}, solana_gossip::gossip_service::discover_cluster,
solana_ledger::{ancestor_iterator::AncestorIterator, blockstore::Blockstore}, solana_ledger::{ancestor_iterator::AncestorIterator, blockstore::Blockstore},
solana_local_cluster::{ solana_local_cluster::{
cluster::{Cluster, ClusterValidatorInfo}, cluster::{Cluster, ClusterValidatorInfo},
@ -212,10 +211,7 @@ fn test_local_cluster_signature_subscribe() {
.unwrap(); .unwrap();
let non_bootstrap_info = cluster.get_contact_info(&non_bootstrap_id).unwrap(); let non_bootstrap_info = cluster.get_contact_info(&non_bootstrap_id).unwrap();
let tx_client = create_client( let tx_client = create_client(non_bootstrap_info.client_facing_addr());
non_bootstrap_info.client_facing_addr(),
VALIDATOR_PORT_RANGE,
);
let (blockhash, _) = tx_client let (blockhash, _) = tx_client
.get_latest_blockhash_with_commitment(CommitmentConfig::processed()) .get_latest_blockhash_with_commitment(CommitmentConfig::processed())
.unwrap(); .unwrap();
@ -520,10 +516,7 @@ fn test_mainnet_beta_cluster_type() {
.unwrap(); .unwrap();
assert_eq!(cluster_nodes.len(), 1); assert_eq!(cluster_nodes.len(), 1);
let client = create_client( let client = create_client(cluster.entry_point_info.client_facing_addr());
cluster.entry_point_info.client_facing_addr(),
VALIDATOR_PORT_RANGE,
);
// Programs that are available at epoch 0 // Programs that are available at epoch 0
for program_id in [ for program_id in [
@ -2663,8 +2656,8 @@ fn setup_transfer_scan_threads(
num_starting_accounts: usize, num_starting_accounts: usize,
exit: Arc<AtomicBool>, exit: Arc<AtomicBool>,
scan_commitment: CommitmentConfig, scan_commitment: CommitmentConfig,
update_client_receiver: Receiver<ThinClient<UdpTpuConnection>>, update_client_receiver: Receiver<ThinClient>,
scan_client_receiver: Receiver<ThinClient<UdpTpuConnection>>, scan_client_receiver: Receiver<ThinClient>,
) -> ( ) -> (
JoinHandle<()>, JoinHandle<()>,
JoinHandle<()>, JoinHandle<()>,

View File

@ -28,6 +28,9 @@ pub struct UdpSocketPair {
pub type PortRange = (u16, u16); pub type PortRange = (u16, u16);
pub const VALIDATOR_PORT_RANGE: PortRange = (8000, 10_000);
pub const MINIMUM_VALIDATOR_PORT_RANGE_WIDTH: u16 = 12; // VALIDATOR_PORT_RANGE must be at least this wide
pub(crate) const HEADER_LENGTH: usize = 4; pub(crate) const HEADER_LENGTH: usize = 4;
pub(crate) const IP_ECHO_SERVER_RESPONSE_LENGTH: usize = HEADER_LENGTH + 23; pub(crate) const IP_ECHO_SERVER_RESPONSE_LENGTH: usize = HEADER_LENGTH + 23;

View File

@ -12,10 +12,8 @@ use {
input_validators::{is_keypair_or_ask_keyword, is_parsable, is_pubkey}, input_validators::{is_keypair_or_ask_keyword, is_parsable, is_pubkey},
keypair::SKIP_SEED_PHRASE_VALIDATION_ARG, keypair::SKIP_SEED_PHRASE_VALIDATION_ARG,
}, },
solana_gossip::{ solana_gossip::{cluster_info::Node, contact_info::ContactInfo},
cluster_info::{Node, VALIDATOR_PORT_RANGE}, solana_net_utils::VALIDATOR_PORT_RANGE,
contact_info::ContactInfo,
},
solana_replica_node::{ solana_replica_node::{
replica_node::{ReplicaNode, ReplicaNodeConfig}, replica_node::{ReplicaNode, ReplicaNodeConfig},
replica_util, replica_util,

View File

@ -1,8 +1,10 @@
#![allow(clippy::integer_arithmetic)] #![allow(clippy::integer_arithmetic)]
pub use solana_test_validator as test_validator;
use { use {
console::style, console::style,
fd_lock::{RwLock, RwLockWriteGuard}, fd_lock::{RwLock, RwLockWriteGuard},
indicatif::{ProgressDrawTarget, ProgressStyle}, indicatif::{ProgressDrawTarget, ProgressStyle},
solana_net_utils::MINIMUM_VALIDATOR_PORT_RANGE_WIDTH,
std::{ std::{
borrow::Cow, borrow::Cow,
env, env,
@ -13,10 +15,6 @@ use {
thread::JoinHandle, thread::JoinHandle,
}, },
}; };
pub use {
solana_gossip::cluster_info::MINIMUM_VALIDATOR_PORT_RANGE_WIDTH,
solana_test_validator as test_validator,
};
pub mod admin_rpc_service; pub mod admin_rpc_service;
pub mod bootstrap; pub mod bootstrap;

View File

@ -28,14 +28,12 @@ use {
tpu::DEFAULT_TPU_COALESCE_MS, tpu::DEFAULT_TPU_COALESCE_MS,
validator::{is_snapshot_config_valid, Validator, ValidatorConfig, ValidatorStartProgress}, validator::{is_snapshot_config_valid, Validator, ValidatorConfig, ValidatorStartProgress},
}, },
solana_gossip::{ solana_gossip::{cluster_info::Node, contact_info::ContactInfo},
cluster_info::{Node, VALIDATOR_PORT_RANGE},
contact_info::ContactInfo,
},
solana_ledger::blockstore_db::{ solana_ledger::blockstore_db::{
BlockstoreCompressionType, BlockstoreRecoveryMode, BlockstoreRocksFifoOptions, BlockstoreCompressionType, BlockstoreRecoveryMode, BlockstoreRocksFifoOptions,
LedgerColumnOptions, ShredStorageType, DEFAULT_ROCKS_FIFO_SHRED_STORAGE_SIZE_BYTES, LedgerColumnOptions, ShredStorageType, DEFAULT_ROCKS_FIFO_SHRED_STORAGE_SIZE_BYTES,
}, },
solana_net_utils::VALIDATOR_PORT_RANGE,
solana_perf::recycler::enable_recycler_warming, solana_perf::recycler::enable_recycler_warming,
solana_poh::poh_service, solana_poh::poh_service,
solana_replica_lib::accountsdb_repl_server::AccountsDbReplServiceConfig, solana_replica_lib::accountsdb_repl_server::AccountsDbReplServiceConfig,