Move gossip modules into solana-gossip crate (#17352)

* Move gossip modules to solana-gossip

* Update Protocol abi digest due to move

* Move gossip benches and hook up CI

* Remove unneeded Result entries

* Single use statements
This commit is contained in:
Tyera Eulberg
2021-05-26 09:15:46 -06:00
committed by GitHub
parent 752cd7d15e
commit 9a5330b7eb
80 changed files with 846 additions and 702 deletions

View File

@ -4,11 +4,9 @@
// hash on gossip. Monitor gossip for messages from validators in the --trusted-validators
// set and halt the node if a mismatch is detected.
use crate::{
cluster_info::{ClusterInfo, MAX_SNAPSHOT_HASHES},
snapshot_packager_service::PendingSnapshotPackage,
};
use crate::snapshot_packager_service::PendingSnapshotPackage;
use rayon::ThreadPool;
use solana_gossip::cluster_info::{ClusterInfo, MAX_SNAPSHOT_HASHES};
use solana_runtime::{
accounts_db,
snapshot_package::{AccountsPackage, AccountsPackagePre, AccountsPackageReceiver},
@ -218,8 +216,7 @@ impl AccountsHashVerifier {
#[cfg(test)]
mod tests {
use super::*;
use crate::cluster_info::make_accounts_hashes_message;
use crate::contact_info::ContactInfo;
use solana_gossip::{cluster_info::make_accounts_hashes_message, contact_info::ContactInfo};
use solana_runtime::bank_forks::ArchiveFormat;
use solana_runtime::snapshot_utils::SnapshotVersion;
use solana_sdk::{

View File

@ -2,7 +2,6 @@
//! to contruct a software pipeline. The stage uses all available CPU cores and
//! can do its processing in parallel with signature verification on the GPU.
use crate::{
cluster_info::ClusterInfo,
packet_hasher::PacketHasher,
poh_recorder::{PohRecorder, PohRecorderError, TransactionRecorder, WorkingBankEntry},
poh_service::{self, PohService},
@ -11,6 +10,7 @@ use crossbeam_channel::{Receiver as CrossbeamReceiver, RecvTimeoutError};
use itertools::Itertools;
use lru::LruCache;
use retain_mut::RetainMut;
use solana_gossip::cluster_info::ClusterInfo;
use solana_ledger::{
blockstore::Blockstore, blockstore_processor::TransactionStatusSender,
entry::hash_transactions, leader_schedule_cache::LeaderScheduleCache,
@ -1435,11 +1435,12 @@ pub fn create_test_recorder(
mod tests {
use super::*;
use crate::{
cluster_info::Node, poh_recorder::Record, poh_recorder::WorkingBank,
poh_recorder::Record, poh_recorder::WorkingBank,
transaction_status_service::TransactionStatusService,
};
use crossbeam_channel::unbounded;
use itertools::Itertools;
use solana_gossip::cluster_info::Node;
use solana_ledger::{
blockstore::entries_to_test_shreds,
entry::{next_entry, Entry, EntrySlice},

View File

@ -5,11 +5,7 @@ use self::{
fail_entry_verification_broadcast_run::FailEntryVerificationBroadcastRun,
standard_broadcast_run::StandardBroadcastRun,
};
use crate::contact_info::ContactInfo;
use crate::crds_gossip_pull::CRDS_GOSSIP_PULL_CRDS_TIMEOUT_MS;
use crate::weighted_shuffle::weighted_best;
use crate::{
cluster_info::{ClusterInfo, ClusterInfoError},
poh_recorder::WorkingBankEntry,
result::{Error, Result},
};
@ -17,6 +13,12 @@ use crossbeam_channel::{
Receiver as CrossbeamReceiver, RecvTimeoutError as CrossbeamRecvTimeoutError,
Sender as CrossbeamSender,
};
use solana_gossip::{
cluster_info::{self, ClusterInfo, ClusterInfoError},
contact_info::ContactInfo,
crds_gossip_pull::CRDS_GOSSIP_PULL_CRDS_TIMEOUT_MS,
weighted_shuffle::weighted_best,
};
use solana_ledger::{blockstore::Blockstore, shred::Shred};
use solana_measure::measure::Measure;
use solana_metrics::{inc_new_counter_error, inc_new_counter_info};
@ -363,7 +365,6 @@ pub fn get_broadcast_peers(
cluster_info: &ClusterInfo,
stakes: Option<&HashMap<Pubkey, u64>>,
) -> (Vec<ContactInfo>, Vec<(u64, usize)>) {
use crate::cluster_info;
let mut peers = cluster_info.tvu_peers();
let peers_and_stakes = cluster_info::stake_weight_peers(&mut peers, stakes);
(peers, peers_and_stakes)
@ -440,8 +441,8 @@ fn num_live_peers(peers: &[ContactInfo]) -> i64 {
#[cfg(test)]
pub mod test {
use super::*;
use crate::cluster_info::{ClusterInfo, Node};
use crossbeam_channel::unbounded;
use solana_gossip::cluster_info::{ClusterInfo, Node};
use solana_ledger::{
blockstore::{make_slot_entries, Blockstore},
entry::create_ticks,

View File

@ -138,7 +138,7 @@ impl BroadcastRun for BroadcastFakeShredsRun {
#[cfg(test)]
mod tests {
use super::*;
use crate::contact_info::ContactInfo;
use solana_gossip::contact_info::ContactInfo;
use std::net::{IpAddr, Ipv4Addr, SocketAddr};
#[test]

View File

@ -494,7 +494,7 @@ impl BroadcastRun for StandardBroadcastRun {
#[cfg(test)]
mod test {
use super::*;
use crate::cluster_info::{ClusterInfo, Node};
use solana_gossip::cluster_info::{ClusterInfo, Node};
use solana_ledger::genesis_utils::create_genesis_config;
use solana_ledger::{
blockstore::Blockstore, entry::create_ticks, get_tmp_ledger_path,

File diff suppressed because it is too large Load Diff

View File

@ -1,404 +0,0 @@
use crate::crds_gossip::CrdsGossip;
use solana_measure::measure::Measure;
use solana_sdk::pubkey::Pubkey;
use std::{
collections::HashMap,
sync::{
atomic::{AtomicU64, Ordering},
RwLock,
},
time::Instant,
};
#[derive(Default)]
pub(crate) struct Counter(AtomicU64);
impl Counter {
pub(crate) fn add_measure(&self, x: &mut Measure) {
x.stop();
self.0.fetch_add(x.as_us(), Ordering::Relaxed);
}
pub(crate) fn add_relaxed(&self, x: u64) {
self.0.fetch_add(x, Ordering::Relaxed);
}
fn clear(&self) -> u64 {
self.0.swap(0, Ordering::Relaxed)
}
}
pub(crate) struct ScopedTimer<'a> {
clock: Instant,
metric: &'a AtomicU64,
}
impl<'a> From<&'a Counter> for ScopedTimer<'a> {
// Output should be assigned to a *named* variable, otherwise it is
// immediately dropped.
#[must_use]
fn from(counter: &'a Counter) -> Self {
Self {
clock: Instant::now(),
metric: &counter.0,
}
}
}
impl Drop for ScopedTimer<'_> {
fn drop(&mut self) {
let micros = self.clock.elapsed().as_micros();
self.metric.fetch_add(micros as u64, Ordering::Relaxed);
}
}
#[derive(Default)]
pub(crate) struct GossipStats {
pub(crate) all_tvu_peers: Counter,
pub(crate) entrypoint2: Counter,
pub(crate) entrypoint: Counter,
pub(crate) epoch_slots_lookup: Counter,
pub(crate) filter_pull_response: Counter,
pub(crate) generate_pull_responses: Counter,
pub(crate) get_accounts_hash: Counter,
pub(crate) get_votes: Counter,
pub(crate) gossip_packets_dropped_count: Counter,
pub(crate) handle_batch_ping_messages_time: Counter,
pub(crate) handle_batch_pong_messages_time: Counter,
pub(crate) handle_batch_prune_messages_time: Counter,
pub(crate) handle_batch_pull_requests_time: Counter,
pub(crate) handle_batch_pull_responses_time: Counter,
pub(crate) handle_batch_push_messages_time: Counter,
pub(crate) mark_pull_request: Counter,
pub(crate) new_pull_requests: Counter,
pub(crate) new_pull_requests_count: Counter,
pub(crate) new_pull_requests_pings_count: Counter,
pub(crate) new_push_requests2: Counter,
pub(crate) new_push_requests: Counter,
pub(crate) new_push_requests_num: Counter,
pub(crate) packets_received_count: Counter,
pub(crate) packets_received_prune_messages_count: Counter,
pub(crate) packets_received_pull_requests_count: Counter,
pub(crate) packets_received_pull_responses_count: Counter,
pub(crate) packets_received_push_messages_count: Counter,
pub(crate) packets_received_verified_count: Counter,
pub(crate) packets_sent_gossip_requests_count: Counter,
pub(crate) packets_sent_prune_messages_count: Counter,
pub(crate) packets_sent_pull_requests_count: Counter,
pub(crate) packets_sent_pull_responses_count: Counter,
pub(crate) packets_sent_push_messages_count: Counter,
pub(crate) process_gossip_packets_time: Counter,
pub(crate) process_prune: Counter,
pub(crate) process_pull_requests: Counter,
pub(crate) process_pull_response: Counter,
pub(crate) process_pull_response_count: Counter,
pub(crate) process_pull_response_fail_insert: Counter,
pub(crate) process_pull_response_fail_timeout: Counter,
pub(crate) process_pull_response_len: Counter,
pub(crate) process_pull_response_success: Counter,
pub(crate) process_pull_response_timeout: Counter,
pub(crate) process_push_message: Counter,
pub(crate) prune_message_count: Counter,
pub(crate) prune_message_len: Counter,
pub(crate) prune_received_cache: Counter,
pub(crate) pull_from_entrypoint_count: Counter,
pub(crate) pull_request_ping_pong_check_failed_count: Counter,
pub(crate) pull_requests_count: Counter,
pub(crate) purge: Counter,
pub(crate) push_message_count: Counter,
pub(crate) push_message_value_count: Counter,
pub(crate) push_response_count: Counter,
pub(crate) push_vote_read: Counter,
pub(crate) repair_peers: Counter,
pub(crate) require_stake_for_gossip_unknown_feature_set: Counter,
pub(crate) require_stake_for_gossip_unknown_stakes: Counter,
pub(crate) skip_pull_response_shred_version: Counter,
pub(crate) skip_pull_shred_version: Counter,
pub(crate) skip_push_message_shred_version: Counter,
pub(crate) trim_crds_table_failed: Counter,
pub(crate) trim_crds_table_purged_values_count: Counter,
pub(crate) tvu_peers: Counter,
}
pub(crate) fn submit_gossip_stats(
stats: &GossipStats,
gossip: &RwLock<CrdsGossip>,
stakes: &HashMap<Pubkey, u64>,
) {
let (table_size, num_nodes, purged_values_size, failed_inserts_size) = {
let gossip = gossip.read().unwrap();
(
gossip.crds.len(),
gossip.crds.num_nodes(),
gossip.crds.num_purged(),
gossip.pull.failed_inserts.len(),
)
};
let num_nodes_staked = stakes.values().filter(|stake| **stake > 0).count();
datapoint_info!(
"cluster_info_stats",
("entrypoint", stats.entrypoint.clear(), i64),
("entrypoint2", stats.entrypoint2.clear(), i64),
("push_vote_read", stats.push_vote_read.clear(), i64),
("get_votes", stats.get_votes.clear(), i64),
("get_accounts_hash", stats.get_accounts_hash.clear(), i64),
("all_tvu_peers", stats.all_tvu_peers.clear(), i64),
("tvu_peers", stats.tvu_peers.clear(), i64),
(
"new_push_requests_num",
stats.new_push_requests_num.clear(),
i64
),
("table_size", table_size as i64, i64),
("purged_values_size", purged_values_size as i64, i64),
("failed_inserts_size", failed_inserts_size as i64, i64),
("num_nodes", num_nodes as i64, i64),
("num_nodes_staked", num_nodes_staked as i64, i64),
);
datapoint_info!(
"cluster_info_stats2",
(
"gossip_packets_dropped_count",
stats.gossip_packets_dropped_count.clear(),
i64
),
("repair_peers", stats.repair_peers.clear(), i64),
("new_push_requests", stats.new_push_requests.clear(), i64),
("new_push_requests2", stats.new_push_requests2.clear(), i64),
("purge", stats.purge.clear(), i64),
(
"process_gossip_packets_time",
stats.process_gossip_packets_time.clear(),
i64
),
(
"handle_batch_ping_messages_time",
stats.handle_batch_ping_messages_time.clear(),
i64
),
(
"handle_batch_pong_messages_time",
stats.handle_batch_pong_messages_time.clear(),
i64
),
(
"handle_batch_prune_messages_time",
stats.handle_batch_prune_messages_time.clear(),
i64
),
(
"handle_batch_pull_requests_time",
stats.handle_batch_pull_requests_time.clear(),
i64
),
(
"handle_batch_pull_responses_time",
stats.handle_batch_pull_responses_time.clear(),
i64
),
(
"handle_batch_push_messages_time",
stats.handle_batch_push_messages_time.clear(),
i64
),
(
"process_pull_resp",
stats.process_pull_response.clear(),
i64
),
("filter_pull_resp", stats.filter_pull_response.clear(), i64),
(
"process_pull_resp_count",
stats.process_pull_response_count.clear(),
i64
),
(
"pull_response_fail_insert",
stats.process_pull_response_fail_insert.clear(),
i64
),
(
"pull_response_fail_timeout",
stats.process_pull_response_fail_timeout.clear(),
i64
),
(
"pull_response_success",
stats.process_pull_response_success.clear(),
i64
),
(
"process_pull_resp_timeout",
stats.process_pull_response_timeout.clear(),
i64
),
(
"push_response_count",
stats.push_response_count.clear(),
i64
),
);
datapoint_info!(
"cluster_info_stats3",
(
"process_pull_resp_len",
stats.process_pull_response_len.clear(),
i64
),
(
"process_pull_requests",
stats.process_pull_requests.clear(),
i64
),
(
"pull_request_ping_pong_check_failed_count",
stats.pull_request_ping_pong_check_failed_count.clear(),
i64
),
(
"new_pull_requests_pings_count",
stats.new_pull_requests_pings_count.clear(),
i64
),
(
"generate_pull_responses",
stats.generate_pull_responses.clear(),
i64
),
("process_prune", stats.process_prune.clear(), i64),
(
"process_push_message",
stats.process_push_message.clear(),
i64
),
(
"prune_received_cache",
stats.prune_received_cache.clear(),
i64
),
("epoch_slots_lookup", stats.epoch_slots_lookup.clear(), i64),
("new_pull_requests", stats.new_pull_requests.clear(), i64),
("mark_pull_request", stats.mark_pull_request.clear(), i64),
);
datapoint_info!(
"cluster_info_stats4",
(
"skip_push_message_shred_version",
stats.skip_push_message_shred_version.clear(),
i64
),
(
"skip_pull_response_shred_version",
stats.skip_pull_response_shred_version.clear(),
i64
),
(
"skip_pull_shred_version",
stats.skip_pull_shred_version.clear(),
i64
),
("push_message_count", stats.push_message_count.clear(), i64),
(
"push_message_value_count",
stats.push_message_value_count.clear(),
i64
),
(
"new_pull_requests_count",
stats.new_pull_requests_count.clear(),
i64
),
(
"pull_from_entrypoint_count",
stats.pull_from_entrypoint_count.clear(),
i64
),
(
"prune_message_count",
stats.prune_message_count.clear(),
i64
),
("prune_message_len", stats.prune_message_len.clear(), i64),
);
datapoint_info!(
"cluster_info_stats5",
(
"pull_requests_count",
stats.pull_requests_count.clear(),
i64
),
(
"packets_received_count",
stats.packets_received_count.clear(),
i64
),
(
"packets_received_prune_messages_count",
stats.packets_received_prune_messages_count.clear(),
i64
),
(
"packets_received_pull_requests_count",
stats.packets_received_pull_requests_count.clear(),
i64
),
(
"packets_received_pull_responses_count",
stats.packets_received_pull_responses_count.clear(),
i64
),
(
"packets_received_push_messages_count",
stats.packets_received_push_messages_count.clear(),
i64
),
(
"packets_received_verified_count",
stats.packets_received_verified_count.clear(),
i64
),
(
"packets_sent_gossip_requests_count",
stats.packets_sent_gossip_requests_count.clear(),
i64
),
(
"packets_sent_prune_messages_count",
stats.packets_sent_prune_messages_count.clear(),
i64
),
(
"packets_sent_pull_requests_count",
stats.packets_sent_pull_requests_count.clear(),
i64
),
(
"packets_sent_pull_responses_count",
stats.packets_sent_pull_responses_count.clear(),
i64
),
(
"packets_sent_push_messages_count",
stats.packets_sent_push_messages_count.clear(),
i64
),
(
"require_stake_for_gossip_unknown_feature_set",
stats.require_stake_for_gossip_unknown_feature_set.clear(),
i64
),
(
"require_stake_for_gossip_unknown_stakes",
stats.require_stake_for_gossip_unknown_stakes.clear(),
i64
),
(
"trim_crds_table_failed",
stats.trim_crds_table_failed.clear(),
i64
),
(
"trim_crds_table_purged_values_count",
stats.trim_crds_table_purged_values_count.clear(),
i64
),
);
}

View File

@ -1,7 +1,4 @@
use crate::{
cluster_info::{ClusterInfo, GOSSIP_SLEEP_MILLIS},
crds::Cursor,
crds_value::CrdsValueLabel,
optimistic_confirmation_verifier::OptimisticConfirmationVerifier,
poh_recorder::PohRecorder,
replay_stage::DUPLICATE_THRESHOLD,
@ -15,6 +12,11 @@ use crossbeam_channel::{
};
use itertools::izip;
use log::*;
use solana_gossip::{
cluster_info::{ClusterInfo, GOSSIP_SLEEP_MILLIS},
crds::Cursor,
crds_value::CrdsValueLabel,
};
use solana_ledger::blockstore::Blockstore;
use solana_metrics::inc_new_counter_debug;
use solana_perf::packet::{self, Packets};

View File

@ -1,8 +1,8 @@
use crate::{
cluster_info::ClusterInfo, contact_info::ContactInfo, crds::Cursor, epoch_slots::EpochSlots,
serve_repair::RepairType,
};
use crate::serve_repair::RepairType;
use itertools::Itertools;
use solana_gossip::{
cluster_info::ClusterInfo, contact_info::ContactInfo, crds::Cursor, epoch_slots::EpochSlots,
};
use solana_runtime::{bank_forks::BankForks, epoch_stakes::NodeIdToVoteAccounts};
use solana_sdk::{clock::Slot, pubkey::Pubkey};
use std::{

View File

@ -1,4 +1,5 @@
use crate::{cluster_info::ClusterInfo, cluster_slots::ClusterSlots};
use crate::cluster_slots::ClusterSlots;
use solana_gossip::cluster_info::ClusterInfo;
use solana_ledger::blockstore::{Blockstore, CompletedSlotsReceiver};
use solana_measure::measure::Measure;
use solana_runtime::bank_forks::BankForks;
@ -185,7 +186,7 @@ impl ClusterSlotsService {
#[cfg(test)]
mod test {
use super::*;
use crate::{cluster_info::Node, crds_value::CrdsValueLabel};
use solana_gossip::{cluster_info::Node, crds_value::CrdsValueLabel};
#[test]
pub fn test_update_lowest_slot() {

View File

@ -1,321 +0,0 @@
use crate::crds_value::MAX_WALLCLOCK;
use solana_sdk::pubkey::Pubkey;
#[cfg(test)]
use solana_sdk::rpc_port;
use solana_sdk::sanitize::{Sanitize, SanitizeError};
#[cfg(test)]
use solana_sdk::signature::{Keypair, Signer};
use solana_sdk::timing::timestamp;
use std::net::{IpAddr, SocketAddr};
/// Structure representing a node on the network
#[derive(Clone, Debug, Eq, Ord, PartialEq, PartialOrd, AbiExample, Deserialize, Serialize)]
pub struct ContactInfo {
pub id: Pubkey,
/// gossip address
pub gossip: SocketAddr,
/// address to connect to for replication
pub tvu: SocketAddr,
/// address to forward shreds to
pub tvu_forwards: SocketAddr,
/// address to send repair responses to
pub repair: SocketAddr,
/// transactions address
pub tpu: SocketAddr,
/// address to forward unprocessed transactions to
pub tpu_forwards: SocketAddr,
/// address to which to send bank state requests
pub unused: SocketAddr,
/// address to which to send JSON-RPC requests
pub rpc: SocketAddr,
/// websocket for JSON-RPC push notifications
pub rpc_pubsub: SocketAddr,
/// address to send repair requests to
pub serve_repair: SocketAddr,
/// latest wallclock picked
pub wallclock: u64,
/// node shred version
pub shred_version: u16,
}
impl Sanitize for ContactInfo {
fn sanitize(&self) -> std::result::Result<(), SanitizeError> {
if self.wallclock >= MAX_WALLCLOCK {
return Err(SanitizeError::ValueOutOfBounds);
}
Ok(())
}
}
#[macro_export]
macro_rules! socketaddr {
($ip:expr, $port:expr) => {
std::net::SocketAddr::from((std::net::Ipv4Addr::from($ip), $port))
};
($str:expr) => {{
$str.parse::<std::net::SocketAddr>().unwrap()
}};
}
#[macro_export]
macro_rules! socketaddr_any {
() => {
socketaddr!(0, 0)
};
}
impl Default for ContactInfo {
fn default() -> Self {
ContactInfo {
id: Pubkey::default(),
gossip: socketaddr_any!(),
tvu: socketaddr_any!(),
tvu_forwards: socketaddr_any!(),
repair: socketaddr_any!(),
tpu: socketaddr_any!(),
tpu_forwards: socketaddr_any!(),
unused: socketaddr_any!(),
rpc: socketaddr_any!(),
rpc_pubsub: socketaddr_any!(),
serve_repair: socketaddr_any!(),
wallclock: 0,
shred_version: 0,
}
}
}
impl ContactInfo {
pub fn new_localhost(id: &Pubkey, now: u64) -> Self {
Self {
id: *id,
gossip: socketaddr!("127.0.0.1:1234"),
tvu: socketaddr!("127.0.0.1:1235"),
tvu_forwards: socketaddr!("127.0.0.1:1236"),
repair: socketaddr!("127.0.0.1:1237"),
tpu: socketaddr!("127.0.0.1:1238"),
tpu_forwards: socketaddr!("127.0.0.1:1239"),
unused: socketaddr!("127.0.0.1:1240"),
rpc: socketaddr!("127.0.0.1:1241"),
rpc_pubsub: socketaddr!("127.0.0.1:1242"),
serve_repair: socketaddr!("127.0.0.1:1243"),
wallclock: now,
shred_version: 0,
}
}
/// New random ContactInfo for tests and simulations.
pub(crate) fn new_rand<R: rand::Rng>(rng: &mut R, pubkey: Option<Pubkey>) -> Self {
let delay = 10 * 60 * 1000; // 10 minutes
let now = timestamp() - delay + rng.gen_range(0, 2 * delay);
let pubkey = pubkey.unwrap_or_else(solana_sdk::pubkey::new_rand);
ContactInfo::new_localhost(&pubkey, now)
}
#[cfg(test)]
/// ContactInfo with multicast addresses for adversarial testing.
pub fn new_multicast() -> Self {
let addr = socketaddr!("224.0.1.255:1000");
assert!(addr.ip().is_multicast());
Self {
id: solana_sdk::pubkey::new_rand(),
gossip: addr,
tvu: addr,
tvu_forwards: addr,
repair: addr,
tpu: addr,
tpu_forwards: addr,
unused: addr,
rpc: addr,
rpc_pubsub: addr,
serve_repair: addr,
wallclock: 0,
shred_version: 0,
}
}
#[cfg(test)]
pub(crate) fn new_with_pubkey_socketaddr(pubkey: &Pubkey, bind_addr: &SocketAddr) -> Self {
fn next_port(addr: &SocketAddr, nxt: u16) -> SocketAddr {
let mut nxt_addr = *addr;
nxt_addr.set_port(addr.port() + nxt);
nxt_addr
}
let tpu = *bind_addr;
let gossip = next_port(&bind_addr, 1);
let tvu = next_port(&bind_addr, 2);
let tpu_forwards = next_port(&bind_addr, 3);
let tvu_forwards = next_port(&bind_addr, 4);
let repair = next_port(&bind_addr, 5);
let rpc = SocketAddr::new(bind_addr.ip(), rpc_port::DEFAULT_RPC_PORT);
let rpc_pubsub = SocketAddr::new(bind_addr.ip(), rpc_port::DEFAULT_RPC_PUBSUB_PORT);
let serve_repair = next_port(&bind_addr, 6);
Self {
id: *pubkey,
gossip,
tvu,
tvu_forwards,
repair,
tpu,
tpu_forwards,
unused: "0.0.0.0:0".parse().unwrap(),
rpc,
rpc_pubsub,
serve_repair,
wallclock: timestamp(),
shred_version: 0,
}
}
#[cfg(test)]
pub(crate) fn new_with_socketaddr(bind_addr: &SocketAddr) -> Self {
let keypair = Keypair::new();
Self::new_with_pubkey_socketaddr(&keypair.pubkey(), bind_addr)
}
// Construct a ContactInfo that's only usable for gossip
pub fn new_gossip_entry_point(gossip_addr: &SocketAddr) -> Self {
Self {
id: Pubkey::default(),
gossip: *gossip_addr,
wallclock: timestamp(),
..ContactInfo::default()
}
}
fn is_valid_ip(addr: IpAddr) -> bool {
!(addr.is_unspecified() || addr.is_multicast())
// || (addr.is_loopback() && !cfg_test))
// TODO: boot loopback in production networks
}
/// port must not be 0
/// ip must be specified and not multicast
/// loopback ip is only allowed in tests
pub fn is_valid_address(addr: &SocketAddr) -> bool {
(addr.port() != 0) && Self::is_valid_ip(addr.ip())
}
pub fn client_facing_addr(&self) -> (SocketAddr, SocketAddr) {
(self.rpc, self.tpu)
}
pub fn valid_client_facing_addr(&self) -> Option<(SocketAddr, SocketAddr)> {
if ContactInfo::is_valid_address(&self.rpc) && ContactInfo::is_valid_address(&self.tpu) {
Some((self.rpc, self.tpu))
} else {
None
}
}
}
#[cfg(test)]
mod tests {
use super::*;
#[test]
fn test_is_valid_address() {
let bad_address_port = socketaddr!("127.0.0.1:0");
assert!(!ContactInfo::is_valid_address(&bad_address_port));
let bad_address_unspecified = socketaddr!(0, 1234);
assert!(!ContactInfo::is_valid_address(&bad_address_unspecified));
let bad_address_multicast = socketaddr!([224, 254, 0, 0], 1234);
assert!(!ContactInfo::is_valid_address(&bad_address_multicast));
let loopback = socketaddr!("127.0.0.1:1234");
assert!(ContactInfo::is_valid_address(&loopback));
// assert!(!ContactInfo::is_valid_ip_internal(loopback.ip(), false));
}
#[test]
fn test_default() {
let ci = ContactInfo::default();
assert!(ci.gossip.ip().is_unspecified());
assert!(ci.tvu.ip().is_unspecified());
assert!(ci.tpu_forwards.ip().is_unspecified());
assert!(ci.rpc.ip().is_unspecified());
assert!(ci.rpc_pubsub.ip().is_unspecified());
assert!(ci.tpu.ip().is_unspecified());
assert!(ci.unused.ip().is_unspecified());
assert!(ci.serve_repair.ip().is_unspecified());
}
#[test]
fn test_multicast() {
let ci = ContactInfo::new_multicast();
assert!(ci.gossip.ip().is_multicast());
assert!(ci.tvu.ip().is_multicast());
assert!(ci.tpu_forwards.ip().is_multicast());
assert!(ci.rpc.ip().is_multicast());
assert!(ci.rpc_pubsub.ip().is_multicast());
assert!(ci.tpu.ip().is_multicast());
assert!(ci.unused.ip().is_multicast());
assert!(ci.serve_repair.ip().is_multicast());
}
#[test]
fn test_entry_point() {
let addr = socketaddr!("127.0.0.1:10");
let ci = ContactInfo::new_gossip_entry_point(&addr);
assert_eq!(ci.gossip, addr);
assert!(ci.tvu.ip().is_unspecified());
assert!(ci.tpu_forwards.ip().is_unspecified());
assert!(ci.rpc.ip().is_unspecified());
assert!(ci.rpc_pubsub.ip().is_unspecified());
assert!(ci.tpu.ip().is_unspecified());
assert!(ci.unused.ip().is_unspecified());
assert!(ci.serve_repair.ip().is_unspecified());
}
#[test]
fn test_socketaddr() {
let addr = socketaddr!("127.0.0.1:10");
let ci = ContactInfo::new_with_socketaddr(&addr);
assert_eq!(ci.tpu, addr);
assert_eq!(ci.gossip.port(), 11);
assert_eq!(ci.tvu.port(), 12);
assert_eq!(ci.tpu_forwards.port(), 13);
assert_eq!(ci.rpc.port(), rpc_port::DEFAULT_RPC_PORT);
assert_eq!(ci.rpc_pubsub.port(), rpc_port::DEFAULT_RPC_PUBSUB_PORT);
assert!(ci.unused.ip().is_unspecified());
assert_eq!(ci.serve_repair.port(), 16);
}
#[test]
fn replayed_data_new_with_socketaddr_with_pubkey() {
let keypair = Keypair::new();
let d1 = ContactInfo::new_with_pubkey_socketaddr(
&keypair.pubkey(),
&socketaddr!("127.0.0.1:1234"),
);
assert_eq!(d1.id, keypair.pubkey());
assert_eq!(d1.gossip, socketaddr!("127.0.0.1:1235"));
assert_eq!(d1.tvu, socketaddr!("127.0.0.1:1236"));
assert_eq!(d1.tpu_forwards, socketaddr!("127.0.0.1:1237"));
assert_eq!(d1.tpu, socketaddr!("127.0.0.1:1234"));
assert_eq!(
d1.rpc,
socketaddr!(format!("127.0.0.1:{}", rpc_port::DEFAULT_RPC_PORT))
);
assert_eq!(
d1.rpc_pubsub,
socketaddr!(format!("127.0.0.1:{}", rpc_port::DEFAULT_RPC_PUBSUB_PORT))
);
assert_eq!(d1.tvu_forwards, socketaddr!("127.0.0.1:1238"));
assert_eq!(d1.repair, socketaddr!("127.0.0.1:1239"));
assert_eq!(d1.serve_repair, socketaddr!("127.0.0.1:1240"));
}
#[test]
fn test_valid_client_facing() {
let mut ci = ContactInfo::default();
assert_eq!(ci.valid_client_facing_addr(), None);
ci.tpu = socketaddr!("127.0.0.1:123");
assert_eq!(ci.valid_client_facing_addr(), None);
ci.rpc = socketaddr!("127.0.0.1:234");
assert!(ci.valid_client_facing_addr().is_some());
}
#[test]
fn test_sanitize() {
let mut ci = ContactInfo::default();
assert_eq!(ci.sanitize(), Ok(()));
ci.wallclock = MAX_WALLCLOCK;
assert_eq!(ci.sanitize(), Err(SanitizeError::ValueOutOfBounds));
}
}

File diff suppressed because it is too large Load Diff

View File

@ -1,405 +0,0 @@
//! Crds Gossip
//! This module ties together Crds and the push and pull gossip overlays. The interface is
//! designed to run with a simulator or over a UDP network connection with messages up to a
//! packet::PACKET_DATA_SIZE size.
use crate::{
cluster_info::Ping,
contact_info::ContactInfo,
crds::Crds,
crds_gossip_error::CrdsGossipError,
crds_gossip_pull::{CrdsFilter, CrdsGossipPull, ProcessPullStats},
crds_gossip_push::{CrdsGossipPush, CRDS_GOSSIP_NUM_ACTIVE},
crds_value::{CrdsData, CrdsValue},
duplicate_shred::{self, DuplicateShredIndex, LeaderScheduleFn, MAX_DUPLICATE_SHREDS},
ping_pong::PingCache,
};
use itertools::Itertools;
use rayon::ThreadPool;
use solana_ledger::shred::Shred;
use solana_sdk::{
hash::Hash,
pubkey::Pubkey,
signature::{Keypair, Signer},
timing::timestamp,
};
use std::{
collections::{HashMap, HashSet},
net::SocketAddr,
sync::Mutex,
time::Duration,
};
pub struct CrdsGossip {
pub crds: Crds,
pub id: Pubkey,
pub shred_version: u16,
pub push: CrdsGossipPush,
pub pull: CrdsGossipPull,
}
impl Default for CrdsGossip {
fn default() -> Self {
CrdsGossip {
crds: Crds::default(),
id: Pubkey::default(),
shred_version: 0,
push: CrdsGossipPush::default(),
pull: CrdsGossipPull::default(),
}
}
}
impl CrdsGossip {
pub fn set_self(&mut self, id: &Pubkey) {
self.id = *id;
}
pub fn set_shred_version(&mut self, shred_version: u16) {
self.shred_version = shred_version;
}
/// process a push message to the network
/// Returns origins' pubkeys of upserted values.
pub fn process_push_message(
&mut self,
from: &Pubkey,
values: Vec<CrdsValue>,
now: u64,
) -> Vec<Pubkey> {
values
.into_iter()
.flat_map(|val| {
let origin = val.pubkey();
self.push
.process_push_message(&mut self.crds, from, val, now)
.ok()?;
Some(origin)
})
.collect()
}
/// remove redundant paths in the network
pub fn prune_received_cache<I>(
&mut self,
origins: I, // Unique pubkeys of crds values' owners.
stakes: &HashMap<Pubkey, u64>,
) -> HashMap</*gossip peer:*/ Pubkey, /*origins:*/ Vec<Pubkey>>
where
I: IntoIterator<Item = Pubkey>,
{
let self_pubkey = self.id;
origins
.into_iter()
.flat_map(|origin| {
self.push
.prune_received_cache(&self_pubkey, &origin, stakes)
.into_iter()
.zip(std::iter::repeat(origin))
})
.into_group_map()
}
pub fn new_push_messages(
&mut self,
pending_push_messages: Vec<CrdsValue>,
now: u64,
) -> HashMap<Pubkey, Vec<CrdsValue>> {
let self_pubkey = self.id;
self.process_push_message(&self_pubkey, pending_push_messages, now);
self.push.new_push_messages(&self.crds, now)
}
pub(crate) fn push_duplicate_shred(
&mut self,
keypair: &Keypair,
shred: &Shred,
other_payload: &[u8],
leader_schedule: Option<impl LeaderScheduleFn>,
// Maximum serialized size of each DuplicateShred chunk payload.
max_payload_size: usize,
) -> Result<(), duplicate_shred::Error> {
let pubkey = keypair.pubkey();
// Skip if there are already records of duplicate shreds for this slot.
let shred_slot = shred.slot();
if self
.crds
.get_records(&pubkey)
.any(|value| match &value.value.data {
CrdsData::DuplicateShred(_, value) => value.slot == shred_slot,
_ => false,
})
{
return Ok(());
}
let chunks = duplicate_shred::from_shred(
shred.clone(),
pubkey,
Vec::from(other_payload),
leader_schedule,
timestamp(),
max_payload_size,
)?;
// Find the index of oldest duplicate shred.
let mut num_dup_shreds = 0;
let offset = self
.crds
.get_records(&pubkey)
.filter_map(|value| match &value.value.data {
CrdsData::DuplicateShred(ix, value) => {
num_dup_shreds += 1;
Some((value.wallclock, *ix))
}
_ => None,
})
.min() // Override the oldest records.
.map(|(_ /*wallclock*/, ix)| ix)
.unwrap_or(0);
let offset = if num_dup_shreds < MAX_DUPLICATE_SHREDS {
num_dup_shreds
} else {
offset
};
let entries = chunks
.enumerate()
.map(|(k, chunk)| {
let index = (offset + k as DuplicateShredIndex) % MAX_DUPLICATE_SHREDS;
let data = CrdsData::DuplicateShred(index, chunk);
CrdsValue::new_signed(data, keypair)
})
.collect();
self.process_push_message(&pubkey, entries, timestamp());
Ok(())
}
/// add the `from` to the peer's filter of nodes
pub fn process_prune_msg(
&self,
peer: &Pubkey,
destination: &Pubkey,
origin: &[Pubkey],
wallclock: u64,
now: u64,
) -> Result<(), CrdsGossipError> {
let expired = now > wallclock + self.push.prune_timeout;
if expired {
return Err(CrdsGossipError::PruneMessageTimeout);
}
if self.id == *destination {
self.push.process_prune_msg(&self.id, peer, origin);
Ok(())
} else {
Err(CrdsGossipError::BadPruneDestination)
}
}
/// refresh the push active set
/// * ratio - number of actives to rotate
pub fn refresh_push_active_set(
&mut self,
stakes: &HashMap<Pubkey, u64>,
gossip_validators: Option<&HashSet<Pubkey>>,
) {
self.push.refresh_push_active_set(
&self.crds,
stakes,
gossip_validators,
&self.id,
self.shred_version,
self.crds.num_nodes(),
CRDS_GOSSIP_NUM_ACTIVE,
)
}
/// generate a random request
pub fn new_pull_request(
&self,
thread_pool: &ThreadPool,
self_keypair: &Keypair,
now: u64,
gossip_validators: Option<&HashSet<Pubkey>>,
stakes: &HashMap<Pubkey, u64>,
bloom_size: usize,
ping_cache: &Mutex<PingCache>,
pings: &mut Vec<(SocketAddr, Ping)>,
) -> Result<(ContactInfo, Vec<CrdsFilter>), CrdsGossipError> {
self.pull.new_pull_request(
thread_pool,
&self.crds,
self_keypair,
self.shred_version,
now,
gossip_validators,
stakes,
bloom_size,
ping_cache,
pings,
)
}
/// time when a request to `from` was initiated
/// This is used for weighted random selection during `new_pull_request`
/// It's important to use the local nodes request creation time as the weight
/// instead of the response received time otherwise failed nodes will increase their weight.
pub fn mark_pull_request_creation_time(&mut self, from: Pubkey, now: u64) {
self.pull.mark_pull_request_creation_time(from, now)
}
/// process a pull request and create a response
pub fn process_pull_requests<I>(&mut self, callers: I, now: u64)
where
I: IntoIterator<Item = CrdsValue>,
{
self.pull
.process_pull_requests(&mut self.crds, callers, now);
}
pub fn generate_pull_responses(
&self,
filters: &[(CrdsValue, CrdsFilter)],
output_size_limit: usize, // Limit number of crds values returned.
now: u64,
) -> Vec<Vec<CrdsValue>> {
self.pull
.generate_pull_responses(&self.crds, filters, output_size_limit, now)
}
pub fn filter_pull_responses(
&self,
timeouts: &HashMap<Pubkey, u64>,
response: Vec<CrdsValue>,
now: u64,
process_pull_stats: &mut ProcessPullStats,
) -> (
Vec<CrdsValue>, // valid responses.
Vec<CrdsValue>, // responses with expired timestamps.
Vec<Hash>, // hash of outdated values.
) {
self.pull
.filter_pull_responses(&self.crds, timeouts, response, now, process_pull_stats)
}
/// process a pull response
pub fn process_pull_responses(
&mut self,
from: &Pubkey,
responses: Vec<CrdsValue>,
responses_expired_timeout: Vec<CrdsValue>,
failed_inserts: Vec<Hash>,
now: u64,
process_pull_stats: &mut ProcessPullStats,
) {
self.pull.process_pull_responses(
&mut self.crds,
from,
responses,
responses_expired_timeout,
failed_inserts,
now,
process_pull_stats,
);
}
pub fn make_timeouts(
&self,
stakes: &HashMap<Pubkey, u64>,
epoch_duration: Duration,
) -> HashMap<Pubkey, u64> {
self.pull.make_timeouts(self.id, stakes, epoch_duration)
}
pub fn purge(
&mut self,
thread_pool: &ThreadPool,
now: u64,
timeouts: &HashMap<Pubkey, u64>,
) -> usize {
let mut rv = 0;
if now > 5 * self.push.msg_timeout {
let min = now - 5 * self.push.msg_timeout;
self.push.purge_old_received_cache(min);
}
if now > self.pull.crds_timeout {
//sanity check
assert_eq!(timeouts[&self.id], std::u64::MAX);
assert!(timeouts.contains_key(&Pubkey::default()));
rv = self
.pull
.purge_active(thread_pool, &mut self.crds, now, &timeouts);
}
self.crds
.trim_purged(now.saturating_sub(5 * self.pull.crds_timeout));
self.pull.purge_failed_inserts(now);
rv
}
// Only for tests and simulations.
pub(crate) fn mock_clone(&self) -> Self {
Self {
crds: self.crds.clone(),
push: self.push.mock_clone(),
pull: self.pull.mock_clone(),
..*self
}
}
}
/// Computes a normalized(log of actual stake) stake
pub fn get_stake<S: std::hash::BuildHasher>(id: &Pubkey, stakes: &HashMap<Pubkey, u64, S>) -> f32 {
// cap the max balance to u32 max (it should be plenty)
let bal = f64::from(u32::max_value()).min(*stakes.get(id).unwrap_or(&0) as f64);
1_f32.max((bal as f32).ln())
}
/// Computes bounded weight given some max, a time since last selected, and a stake value
/// The minimum stake is 1 and not 0 to allow 'time since last' picked to factor in.
pub fn get_weight(max_weight: f32, time_since_last_selected: u32, stake: f32) -> f32 {
let mut weight = time_since_last_selected as f32 * stake;
if weight.is_infinite() {
weight = max_weight;
}
1.0_f32.max(weight.min(max_weight))
}
#[cfg(test)]
mod test {
use super::*;
use crate::contact_info::ContactInfo;
use crate::crds_value::CrdsData;
use solana_sdk::hash::hash;
use solana_sdk::timing::timestamp;
#[test]
fn test_prune_errors() {
let mut crds_gossip = CrdsGossip {
id: Pubkey::new(&[0; 32]),
..CrdsGossip::default()
};
let id = crds_gossip.id;
let ci = ContactInfo::new_localhost(&Pubkey::new(&[1; 32]), 0);
let prune_pubkey = Pubkey::new(&[2; 32]);
crds_gossip
.crds
.insert(
CrdsValue::new_unsigned(CrdsData::ContactInfo(ci.clone())),
0,
)
.unwrap();
crds_gossip.refresh_push_active_set(&HashMap::new(), None);
let now = timestamp();
//incorrect dest
let mut res = crds_gossip.process_prune_msg(
&ci.id,
&Pubkey::new(hash(&[1; 32]).as_ref()),
&[prune_pubkey],
now,
now,
);
assert_eq!(res.err(), Some(CrdsGossipError::BadPruneDestination));
//correct dest
res = crds_gossip.process_prune_msg(&ci.id, &id, &[prune_pubkey], now, now);
res.unwrap();
//test timeout
let timeout = now + crds_gossip.push.prune_timeout * 2;
res = crds_gossip.process_prune_msg(&ci.id, &id, &[prune_pubkey], now, timeout);
assert_eq!(res.err(), Some(CrdsGossipError::PruneMessageTimeout));
}
}

View File

@ -1,8 +0,0 @@
#[derive(PartialEq, Debug)]
pub enum CrdsGossipError {
NoPeers,
PushMessageTimeout,
PushMessageOldVersion,
BadPruneDestination,
PruneMessageTimeout,
}

File diff suppressed because it is too large Load Diff

View File

@ -1,863 +0,0 @@
//! Crds Gossip Push overlay
//! This module is used to propagate recently created CrdsValues across the network
//! Eager push strategy is based on Plumtree
//! http://asc.di.fct.unl.pt/~jleitao/pdf/srds07-leitao.pdf
//!
//! Main differences are:
//! 1. There is no `max hop`. Messages are signed with a local wallclock. If they are outside of
//! the local nodes wallclock window they are dropped silently.
//! 2. The prune set is stored in a Bloom filter.
use crate::{
cluster_info::CRDS_UNIQUE_PUBKEY_CAPACITY,
contact_info::ContactInfo,
crds::{Crds, Cursor},
crds_gossip::{get_stake, get_weight},
crds_gossip_error::CrdsGossipError,
crds_value::CrdsValue,
weighted_shuffle::weighted_shuffle,
};
use bincode::serialized_size;
use indexmap::map::IndexMap;
use lru::LruCache;
use rand::{seq::SliceRandom, Rng};
use solana_runtime::bloom::{AtomicBloom, Bloom};
use solana_sdk::{packet::PACKET_DATA_SIZE, pubkey::Pubkey, timing::timestamp};
use std::{
cmp,
collections::{HashMap, HashSet},
ops::RangeBounds,
};
pub const CRDS_GOSSIP_NUM_ACTIVE: usize = 30;
pub const CRDS_GOSSIP_PUSH_FANOUT: usize = 6;
// With a fanout of 6, a 1000 node cluster should only take ~4 hops to converge.
// However since pushes are stake weighed, some trailing nodes
// might need more time to receive values. 30 seconds should be plenty.
pub const CRDS_GOSSIP_PUSH_MSG_TIMEOUT_MS: u64 = 30000;
pub const CRDS_GOSSIP_PRUNE_MSG_TIMEOUT_MS: u64 = 500;
pub const CRDS_GOSSIP_PRUNE_STAKE_THRESHOLD_PCT: f64 = 0.15;
pub const CRDS_GOSSIP_PRUNE_MIN_INGRESS_NODES: usize = 3;
// Do not push to peers which have not been updated for this long.
const PUSH_ACTIVE_TIMEOUT_MS: u64 = 60_000;
pub struct CrdsGossipPush {
/// max bytes per message
pub max_bytes: usize,
/// active set of validators for push
active_set: IndexMap<Pubkey, AtomicBloom<Pubkey>>,
/// Cursor into the crds table for values to push.
crds_cursor: Cursor,
/// Cache that tracks which validators a message was received from
/// bool indicates it has been pruned.
/// This cache represents a lagging view of which validators
/// currently have this node in their `active_set`
received_cache: HashMap<
Pubkey, // origin/owner
HashMap</*gossip peer:*/ Pubkey, (/*pruned:*/ bool, /*timestamp:*/ u64)>,
>,
last_pushed_to: LruCache<Pubkey, u64>,
pub num_active: usize,
pub push_fanout: usize,
pub msg_timeout: u64,
pub prune_timeout: u64,
pub num_total: usize,
pub num_old: usize,
pub num_pushes: usize,
}
impl Default for CrdsGossipPush {
fn default() -> Self {
Self {
// Allow upto 64 Crds Values per PUSH
max_bytes: PACKET_DATA_SIZE * 64,
active_set: IndexMap::new(),
crds_cursor: Cursor::default(),
received_cache: HashMap::new(),
last_pushed_to: LruCache::new(CRDS_UNIQUE_PUBKEY_CAPACITY),
num_active: CRDS_GOSSIP_NUM_ACTIVE,
push_fanout: CRDS_GOSSIP_PUSH_FANOUT,
msg_timeout: CRDS_GOSSIP_PUSH_MSG_TIMEOUT_MS,
prune_timeout: CRDS_GOSSIP_PRUNE_MSG_TIMEOUT_MS,
num_total: 0,
num_old: 0,
num_pushes: 0,
}
}
}
impl CrdsGossipPush {
pub fn num_pending(&self, crds: &Crds) -> usize {
let mut cursor = self.crds_cursor;
crds.get_entries(&mut cursor).count()
}
fn prune_stake_threshold(self_stake: u64, origin_stake: u64) -> u64 {
let min_path_stake = self_stake.min(origin_stake);
((CRDS_GOSSIP_PRUNE_STAKE_THRESHOLD_PCT * min_path_stake as f64).round() as u64).max(1)
}
pub fn prune_received_cache(
&mut self,
self_pubkey: &Pubkey,
origin: &Pubkey,
stakes: &HashMap<Pubkey, u64>,
) -> Vec<Pubkey> {
let origin_stake = stakes.get(origin).unwrap_or(&0);
let self_stake = stakes.get(self_pubkey).unwrap_or(&0);
let peers = match self.received_cache.get_mut(origin) {
None => return Vec::default(),
Some(peers) => peers,
};
let peer_stake_total: u64 = peers
.iter()
.filter(|(_, (pruned, _))| !pruned)
.filter_map(|(peer, _)| stakes.get(peer))
.sum();
let prune_stake_threshold = Self::prune_stake_threshold(*self_stake, *origin_stake);
if peer_stake_total < prune_stake_threshold {
return Vec::new();
}
let shuffled_staked_peers = {
let peers: Vec<_> = peers
.iter()
.filter(|(_, (pruned, _))| !pruned)
.filter_map(|(peer, _)| Some((*peer, *stakes.get(peer)?)))
.filter(|(_, stake)| *stake > 0)
.collect();
let mut seed = [0; 32];
rand::thread_rng().fill(&mut seed[..]);
let weights: Vec<_> = peers.iter().map(|(_, stake)| *stake).collect();
weighted_shuffle(&weights, seed)
.into_iter()
.map(move |i| peers[i])
};
let mut keep = HashSet::new();
let mut peer_stake_sum = 0;
keep.insert(*origin);
for (peer, stake) in shuffled_staked_peers {
if peer == *origin {
continue;
}
keep.insert(peer);
peer_stake_sum += stake;
if peer_stake_sum >= prune_stake_threshold
&& keep.len() >= CRDS_GOSSIP_PRUNE_MIN_INGRESS_NODES
{
break;
}
}
for (peer, (pruned, _)) in peers.iter_mut() {
if !*pruned && !keep.contains(peer) {
*pruned = true;
}
}
peers
.keys()
.filter(|peer| !keep.contains(peer))
.copied()
.collect()
}
fn wallclock_window(&self, now: u64) -> impl RangeBounds<u64> {
now.saturating_sub(self.msg_timeout)..=now.saturating_add(self.msg_timeout)
}
/// process a push message to the network
pub fn process_push_message(
&mut self,
crds: &mut Crds,
from: &Pubkey,
value: CrdsValue,
now: u64,
) -> Result<(), CrdsGossipError> {
self.num_total += 1;
if !self.wallclock_window(now).contains(&value.wallclock()) {
return Err(CrdsGossipError::PushMessageTimeout);
}
let origin = value.pubkey();
self.received_cache
.entry(origin)
.or_default()
.entry(*from)
.and_modify(|(_pruned, timestamp)| *timestamp = now)
.or_insert((/*pruned:*/ false, now));
crds.insert(value, now).map_err(|_| {
self.num_old += 1;
CrdsGossipError::PushMessageOldVersion
})
}
/// New push message to broadcast to peers.
/// Returns a list of Pubkeys for the selected peers and a list of values to send to all the
/// peers.
/// The list of push messages is created such that all the randomly selected peers have not
/// pruned the source addresses.
pub fn new_push_messages(&mut self, crds: &Crds, now: u64) -> HashMap<Pubkey, Vec<CrdsValue>> {
let push_fanout = self.push_fanout.min(self.active_set.len());
if push_fanout == 0 {
return HashMap::default();
}
let mut num_pushes = 0;
let mut num_values = 0;
let mut total_bytes: usize = 0;
let mut push_messages: HashMap<Pubkey, Vec<CrdsValue>> = HashMap::new();
let wallclock_window = self.wallclock_window(now);
let entries = crds
.get_entries(&mut self.crds_cursor)
.map(|entry| &entry.value)
.filter(|value| wallclock_window.contains(&value.wallclock()));
for value in entries {
let serialized_size = serialized_size(&value).unwrap();
total_bytes = total_bytes.saturating_add(serialized_size as usize);
if total_bytes > self.max_bytes {
break;
}
num_values += 1;
let origin = value.pubkey();
// Use a consistent index for the same origin so the active set
// learns the MST for that origin.
let offset = origin.as_ref()[0] as usize;
for i in offset..offset + push_fanout {
let index = i % self.active_set.len();
let (peer, filter) = self.active_set.get_index(index).unwrap();
if !filter.contains(&origin) || value.should_force_push(peer) {
trace!("new_push_messages insert {} {:?}", *peer, value);
push_messages.entry(*peer).or_default().push(value.clone());
num_pushes += 1;
}
}
}
self.num_pushes += num_pushes;
trace!("new_push_messages {} {}", num_values, self.active_set.len());
for target_pubkey in push_messages.keys().copied() {
self.last_pushed_to.put(target_pubkey, now);
}
push_messages
}
/// add the `from` to the peer's filter of nodes
pub fn process_prune_msg(&self, self_pubkey: &Pubkey, peer: &Pubkey, origins: &[Pubkey]) {
if let Some(filter) = self.active_set.get(peer) {
for origin in origins {
if origin != self_pubkey {
filter.add(origin);
}
}
}
}
fn compute_need(num_active: usize, active_set_len: usize, ratio: usize) -> usize {
let num = active_set_len / ratio;
cmp::min(num_active, (num_active - active_set_len) + num)
}
/// refresh the push active set
/// * ratio - active_set.len()/ratio is the number of actives to rotate
pub fn refresh_push_active_set(
&mut self,
crds: &Crds,
stakes: &HashMap<Pubkey, u64>,
gossip_validators: Option<&HashSet<Pubkey>>,
self_id: &Pubkey,
self_shred_version: u16,
network_size: usize,
ratio: usize,
) {
const BLOOM_FALSE_RATE: f64 = 0.1;
const BLOOM_MAX_BITS: usize = 1024 * 8 * 4;
#[cfg(debug_assertions)]
const MIN_NUM_BLOOM_ITEMS: usize = 512;
#[cfg(not(debug_assertions))]
const MIN_NUM_BLOOM_ITEMS: usize = CRDS_UNIQUE_PUBKEY_CAPACITY;
let mut rng = rand::thread_rng();
let need = Self::compute_need(self.num_active, self.active_set.len(), ratio);
let mut new_items = HashMap::new();
let (weights, peers): (Vec<_>, Vec<_>) = self
.push_options(
crds,
&self_id,
self_shred_version,
stakes,
gossip_validators,
)
.into_iter()
.unzip();
if peers.is_empty() {
return;
}
let num_bloom_items = MIN_NUM_BLOOM_ITEMS.max(network_size);
let shuffle = {
let mut seed = [0; 32];
rng.fill(&mut seed[..]);
weighted_shuffle(&weights, seed).into_iter()
};
for peer in shuffle.map(|i| peers[i].id) {
if new_items.len() >= need {
break;
}
if self.active_set.contains_key(&peer) || new_items.contains_key(&peer) {
continue;
}
let bloom = AtomicBloom::from(Bloom::random(
num_bloom_items,
BLOOM_FALSE_RATE,
BLOOM_MAX_BITS,
));
bloom.add(&peer);
new_items.insert(peer, bloom);
}
let mut keys: Vec<Pubkey> = self.active_set.keys().cloned().collect();
keys.shuffle(&mut rng);
let num = keys.len() / ratio;
for k in &keys[..num] {
self.active_set.swap_remove(k);
}
for (k, v) in new_items {
self.active_set.insert(k, v);
}
}
fn push_options<'a>(
&self,
crds: &'a Crds,
self_id: &Pubkey,
self_shred_version: u16,
stakes: &HashMap<Pubkey, u64>,
gossip_validators: Option<&HashSet<Pubkey>>,
) -> Vec<(f32, &'a ContactInfo)> {
let now = timestamp();
let mut rng = rand::thread_rng();
let max_weight = u16::MAX as f32 - 1.0;
let active_cutoff = now.saturating_sub(PUSH_ACTIVE_TIMEOUT_MS);
crds.get_nodes()
.filter_map(|value| {
let info = value.value.contact_info().unwrap();
// Stop pushing to nodes which have not been active recently.
if value.local_timestamp < active_cutoff {
// In order to mitigate eclipse attack, for staked nodes
// continue retrying periodically.
let stake = stakes.get(&info.id).unwrap_or(&0);
if *stake == 0 || !rng.gen_ratio(1, 16) {
return None;
}
}
Some(info)
})
.filter(|info| {
info.id != *self_id
&& ContactInfo::is_valid_address(&info.gossip)
&& self_shred_version == info.shred_version
&& gossip_validators.map_or(true, |gossip_validators| {
gossip_validators.contains(&info.id)
})
})
.map(|info| {
let last_pushed_to = self
.last_pushed_to
.peek(&info.id)
.copied()
.unwrap_or_default();
let since = (now.saturating_sub(last_pushed_to).min(3600 * 1000) / 1024) as u32;
let stake = get_stake(&info.id, stakes);
let weight = get_weight(max_weight, since, stake);
(weight, info)
})
.collect()
}
/// purge received push message cache
pub fn purge_old_received_cache(&mut self, min_time: u64) {
self.received_cache.retain(|_, v| {
v.retain(|_, (_, t)| *t > min_time);
!v.is_empty()
});
}
// Only for tests and simulations.
pub(crate) fn mock_clone(&self) -> Self {
let active_set = self
.active_set
.iter()
.map(|(k, v)| (*k, v.mock_clone()))
.collect();
let mut last_pushed_to = LruCache::new(self.last_pushed_to.cap());
for (k, v) in self.last_pushed_to.iter().rev() {
last_pushed_to.put(*k, *v);
}
Self {
active_set,
received_cache: self.received_cache.clone(),
last_pushed_to,
..*self
}
}
}
#[cfg(test)]
mod test {
use super::*;
use crate::contact_info::ContactInfo;
use crate::crds_value::CrdsData;
#[test]
fn test_prune() {
let mut crds = Crds::default();
let mut push = CrdsGossipPush::default();
let mut stakes = HashMap::new();
let self_id = solana_sdk::pubkey::new_rand();
let origin = solana_sdk::pubkey::new_rand();
stakes.insert(self_id, 100);
stakes.insert(origin, 100);
let value = CrdsValue::new_unsigned(CrdsData::ContactInfo(ContactInfo::new_localhost(
&origin, 0,
)));
let low_staked_peers = (0..10).map(|_| solana_sdk::pubkey::new_rand());
let mut low_staked_set = HashSet::new();
low_staked_peers.for_each(|p| {
let _ = push.process_push_message(&mut crds, &p, value.clone(), 0);
low_staked_set.insert(p);
stakes.insert(p, 1);
});
let pruned = push.prune_received_cache(&self_id, &origin, &stakes);
assert!(
pruned.is_empty(),
"should not prune if min threshold has not been reached"
);
let high_staked_peer = solana_sdk::pubkey::new_rand();
let high_stake = CrdsGossipPush::prune_stake_threshold(100, 100) + 10;
stakes.insert(high_staked_peer, high_stake);
let _ = push.process_push_message(&mut crds, &high_staked_peer, value, 0);
let pruned = push.prune_received_cache(&self_id, &origin, &stakes);
assert!(
pruned.len() < low_staked_set.len() + 1,
"should not prune all peers"
);
pruned.iter().for_each(|p| {
assert!(
low_staked_set.contains(p),
"only low staked peers should be pruned"
);
});
}
#[test]
fn test_process_push_one() {
let mut crds = Crds::default();
let mut push = CrdsGossipPush::default();
let value = CrdsValue::new_unsigned(CrdsData::ContactInfo(ContactInfo::new_localhost(
&solana_sdk::pubkey::new_rand(),
0,
)));
let label = value.label();
// push a new message
assert_eq!(
push.process_push_message(&mut crds, &Pubkey::default(), value.clone(), 0),
Ok(())
);
assert_eq!(crds.get(&label).unwrap().value, value);
// push it again
assert_matches!(
push.process_push_message(&mut crds, &Pubkey::default(), value, 0),
Err(CrdsGossipError::PushMessageOldVersion)
);
}
#[test]
fn test_process_push_old_version() {
let mut crds = Crds::default();
let mut push = CrdsGossipPush::default();
let mut ci = ContactInfo::new_localhost(&solana_sdk::pubkey::new_rand(), 0);
ci.wallclock = 1;
let value = CrdsValue::new_unsigned(CrdsData::ContactInfo(ci.clone()));
// push a new message
assert_eq!(
push.process_push_message(&mut crds, &Pubkey::default(), value, 0),
Ok(())
);
// push an old version
ci.wallclock = 0;
let value = CrdsValue::new_unsigned(CrdsData::ContactInfo(ci));
assert_eq!(
push.process_push_message(&mut crds, &Pubkey::default(), value, 0),
Err(CrdsGossipError::PushMessageOldVersion)
);
}
#[test]
fn test_process_push_timeout() {
let mut crds = Crds::default();
let mut push = CrdsGossipPush::default();
let timeout = push.msg_timeout;
let mut ci = ContactInfo::new_localhost(&solana_sdk::pubkey::new_rand(), 0);
// push a version to far in the future
ci.wallclock = timeout + 1;
let value = CrdsValue::new_unsigned(CrdsData::ContactInfo(ci.clone()));
assert_eq!(
push.process_push_message(&mut crds, &Pubkey::default(), value, 0),
Err(CrdsGossipError::PushMessageTimeout)
);
// push a version to far in the past
ci.wallclock = 0;
let value = CrdsValue::new_unsigned(CrdsData::ContactInfo(ci));
assert_eq!(
push.process_push_message(&mut crds, &Pubkey::default(), value, timeout + 1),
Err(CrdsGossipError::PushMessageTimeout)
);
}
#[test]
fn test_process_push_update() {
let mut crds = Crds::default();
let mut push = CrdsGossipPush::default();
let mut ci = ContactInfo::new_localhost(&solana_sdk::pubkey::new_rand(), 0);
ci.wallclock = 0;
let value_old = CrdsValue::new_unsigned(CrdsData::ContactInfo(ci.clone()));
// push a new message
assert_eq!(
push.process_push_message(&mut crds, &Pubkey::default(), value_old, 0),
Ok(())
);
// push an old version
ci.wallclock = 1;
let value = CrdsValue::new_unsigned(CrdsData::ContactInfo(ci));
assert_eq!(
push.process_push_message(&mut crds, &Pubkey::default(), value, 0),
Ok(())
);
}
#[test]
fn test_compute_need() {
assert_eq!(CrdsGossipPush::compute_need(30, 0, 10), 30);
assert_eq!(CrdsGossipPush::compute_need(30, 1, 10), 29);
assert_eq!(CrdsGossipPush::compute_need(30, 30, 10), 3);
assert_eq!(CrdsGossipPush::compute_need(30, 29, 10), 3);
}
#[test]
fn test_refresh_active_set() {
solana_logger::setup();
let now = timestamp();
let mut crds = Crds::default();
let mut push = CrdsGossipPush::default();
let value1 = CrdsValue::new_unsigned(CrdsData::ContactInfo(ContactInfo::new_localhost(
&solana_sdk::pubkey::new_rand(),
0,
)));
assert_eq!(crds.insert(value1.clone(), now), Ok(()));
push.refresh_push_active_set(&crds, &HashMap::new(), None, &Pubkey::default(), 0, 1, 1);
assert!(push.active_set.get(&value1.label().pubkey()).is_some());
let value2 = CrdsValue::new_unsigned(CrdsData::ContactInfo(ContactInfo::new_localhost(
&solana_sdk::pubkey::new_rand(),
0,
)));
assert!(push.active_set.get(&value2.label().pubkey()).is_none());
assert_eq!(crds.insert(value2.clone(), now), Ok(()));
for _ in 0..30 {
push.refresh_push_active_set(&crds, &HashMap::new(), None, &Pubkey::default(), 0, 1, 1);
if push.active_set.get(&value2.label().pubkey()).is_some() {
break;
}
}
assert!(push.active_set.get(&value2.label().pubkey()).is_some());
for _ in 0..push.num_active {
let value2 = CrdsValue::new_unsigned(CrdsData::ContactInfo(
ContactInfo::new_localhost(&solana_sdk::pubkey::new_rand(), 0),
));
assert_eq!(crds.insert(value2.clone(), now), Ok(()));
}
push.refresh_push_active_set(&crds, &HashMap::new(), None, &Pubkey::default(), 0, 1, 1);
assert_eq!(push.active_set.len(), push.num_active);
}
#[test]
fn test_active_set_refresh_with_bank() {
solana_logger::setup();
let time = timestamp() - 1024; //make sure there's at least a 1 second delay
let mut crds = Crds::default();
let mut push = CrdsGossipPush::default();
let mut stakes = HashMap::new();
for i in 1..=100 {
let peer = CrdsValue::new_unsigned(CrdsData::ContactInfo(ContactInfo::new_localhost(
&solana_sdk::pubkey::new_rand(),
time,
)));
let id = peer.label().pubkey();
crds.insert(peer.clone(), time).unwrap();
stakes.insert(id, i * 100);
push.last_pushed_to.put(id, time);
}
let mut options = push.push_options(&crds, &Pubkey::default(), 0, &stakes, None);
assert!(!options.is_empty());
options.sort_by(|(weight_l, _), (weight_r, _)| weight_r.partial_cmp(weight_l).unwrap());
// check that the highest stake holder is also the heaviest weighted.
assert_eq!(
*stakes.get(&options.get(0).unwrap().1.id).unwrap(),
10_000_u64
);
}
#[test]
fn test_no_pushes_to_from_different_shred_versions() {
let now = timestamp();
let mut crds = Crds::default();
let stakes = HashMap::new();
let node = CrdsGossipPush::default();
let gossip = socketaddr!("127.0.0.1:1234");
let me = CrdsValue::new_unsigned(CrdsData::ContactInfo(ContactInfo {
id: solana_sdk::pubkey::new_rand(),
shred_version: 123,
gossip,
..ContactInfo::default()
}));
let spy = CrdsValue::new_unsigned(CrdsData::ContactInfo(ContactInfo {
id: solana_sdk::pubkey::new_rand(),
shred_version: 0,
gossip,
..ContactInfo::default()
}));
let node_123 = CrdsValue::new_unsigned(CrdsData::ContactInfo(ContactInfo {
id: solana_sdk::pubkey::new_rand(),
shred_version: 123,
gossip,
..ContactInfo::default()
}));
let node_456 = CrdsValue::new_unsigned(CrdsData::ContactInfo(ContactInfo {
id: solana_sdk::pubkey::new_rand(),
shred_version: 456,
gossip,
..ContactInfo::default()
}));
crds.insert(me.clone(), now).unwrap();
crds.insert(spy.clone(), now).unwrap();
crds.insert(node_123.clone(), now).unwrap();
crds.insert(node_456, now).unwrap();
// shred version 123 should ignore nodes with versions 0 and 456
let options = node
.push_options(&crds, &me.label().pubkey(), 123, &stakes, None)
.iter()
.map(|(_, c)| c.id)
.collect::<Vec<_>>();
assert_eq!(options.len(), 1);
assert!(!options.contains(&spy.pubkey()));
assert!(options.contains(&node_123.pubkey()));
// spy nodes should not push to people on different shred versions
let options = node
.push_options(&crds, &spy.label().pubkey(), 0, &stakes, None)
.iter()
.map(|(_, c)| c.id)
.collect::<Vec<_>>();
assert!(options.is_empty());
}
#[test]
fn test_pushes_only_to_allowed() {
let now = timestamp();
let mut crds = Crds::default();
let stakes = HashMap::new();
let node = CrdsGossipPush::default();
let gossip = socketaddr!("127.0.0.1:1234");
let me = CrdsValue::new_unsigned(CrdsData::ContactInfo(ContactInfo {
id: solana_sdk::pubkey::new_rand(),
gossip,
..ContactInfo::default()
}));
let node_123 = CrdsValue::new_unsigned(CrdsData::ContactInfo(ContactInfo {
id: solana_sdk::pubkey::new_rand(),
gossip,
..ContactInfo::default()
}));
crds.insert(me.clone(), 0).unwrap();
crds.insert(node_123.clone(), now).unwrap();
// Unknown pubkey in gossip_validators -- will push to nobody
let mut gossip_validators = HashSet::new();
let options = node.push_options(
&crds,
&me.label().pubkey(),
0,
&stakes,
Some(&gossip_validators),
);
assert!(options.is_empty());
// Unknown pubkey in gossip_validators -- will push to nobody
gossip_validators.insert(solana_sdk::pubkey::new_rand());
let options = node.push_options(
&crds,
&me.label().pubkey(),
0,
&stakes,
Some(&gossip_validators),
);
assert!(options.is_empty());
// node_123 pubkey in gossip_validators -- will push to it
gossip_validators.insert(node_123.pubkey());
let options = node.push_options(
&crds,
&me.label().pubkey(),
0,
&stakes,
Some(&gossip_validators),
);
assert_eq!(options.len(), 1);
assert_eq!(options[0].1.id, node_123.pubkey());
}
#[test]
fn test_new_push_messages() {
let now = timestamp();
let mut crds = Crds::default();
let mut push = CrdsGossipPush::default();
let peer = CrdsValue::new_unsigned(CrdsData::ContactInfo(ContactInfo::new_localhost(
&solana_sdk::pubkey::new_rand(),
0,
)));
assert_eq!(crds.insert(peer.clone(), now), Ok(()));
push.refresh_push_active_set(&crds, &HashMap::new(), None, &Pubkey::default(), 0, 1, 1);
let new_msg = CrdsValue::new_unsigned(CrdsData::ContactInfo(ContactInfo::new_localhost(
&solana_sdk::pubkey::new_rand(),
0,
)));
let mut expected = HashMap::new();
expected.insert(peer.label().pubkey(), vec![new_msg.clone()]);
assert_eq!(
push.process_push_message(&mut crds, &Pubkey::default(), new_msg, 0),
Ok(())
);
assert_eq!(push.active_set.len(), 1);
assert_eq!(push.new_push_messages(&crds, 0), expected);
}
#[test]
fn test_personalized_push_messages() {
let now = timestamp();
let mut rng = rand::thread_rng();
let mut crds = Crds::default();
let mut push = CrdsGossipPush::default();
let peers: Vec<_> = vec![0, 0, now]
.into_iter()
.map(|wallclock| {
let mut peer = ContactInfo::new_rand(&mut rng, /*pubkey=*/ None);
peer.wallclock = wallclock;
CrdsValue::new_unsigned(CrdsData::ContactInfo(peer))
})
.collect();
assert_eq!(crds.insert(peers[0].clone(), now), Ok(()));
assert_eq!(crds.insert(peers[1].clone(), now), Ok(()));
assert_eq!(
push.process_push_message(&mut crds, &Pubkey::default(), peers[2].clone(), now),
Ok(())
);
push.refresh_push_active_set(&crds, &HashMap::new(), None, &Pubkey::default(), 0, 1, 1);
// push 3's contact info to 1 and 2 and 3
let expected: HashMap<_, _> = vec![
(peers[0].pubkey(), vec![peers[2].clone()]),
(peers[1].pubkey(), vec![peers[2].clone()]),
]
.into_iter()
.collect();
assert_eq!(push.active_set.len(), 3);
assert_eq!(push.new_push_messages(&crds, now), expected);
}
#[test]
fn test_process_prune() {
let mut crds = Crds::default();
let self_id = solana_sdk::pubkey::new_rand();
let mut push = CrdsGossipPush::default();
let peer = CrdsValue::new_unsigned(CrdsData::ContactInfo(ContactInfo::new_localhost(
&solana_sdk::pubkey::new_rand(),
0,
)));
assert_eq!(crds.insert(peer.clone(), 0), Ok(()));
push.refresh_push_active_set(&crds, &HashMap::new(), None, &Pubkey::default(), 0, 1, 1);
let new_msg = CrdsValue::new_unsigned(CrdsData::ContactInfo(ContactInfo::new_localhost(
&solana_sdk::pubkey::new_rand(),
0,
)));
let expected = HashMap::new();
assert_eq!(
push.process_push_message(&mut crds, &Pubkey::default(), new_msg.clone(), 0),
Ok(())
);
push.process_prune_msg(
&self_id,
&peer.label().pubkey(),
&[new_msg.label().pubkey()],
);
assert_eq!(push.new_push_messages(&crds, 0), expected);
}
#[test]
fn test_purge_old_pending_push_messages() {
let mut crds = Crds::default();
let mut push = CrdsGossipPush::default();
let peer = CrdsValue::new_unsigned(CrdsData::ContactInfo(ContactInfo::new_localhost(
&solana_sdk::pubkey::new_rand(),
0,
)));
assert_eq!(crds.insert(peer, 0), Ok(()));
push.refresh_push_active_set(&crds, &HashMap::new(), None, &Pubkey::default(), 0, 1, 1);
let mut ci = ContactInfo::new_localhost(&solana_sdk::pubkey::new_rand(), 0);
ci.wallclock = 1;
let new_msg = CrdsValue::new_unsigned(CrdsData::ContactInfo(ci));
let expected = HashMap::new();
assert_eq!(
push.process_push_message(&mut crds, &Pubkey::default(), new_msg, 1),
Ok(())
);
assert_eq!(push.new_push_messages(&crds, 0), expected);
}
#[test]
fn test_purge_old_received_cache() {
let mut crds = Crds::default();
let mut push = CrdsGossipPush::default();
let mut ci = ContactInfo::new_localhost(&solana_sdk::pubkey::new_rand(), 0);
ci.wallclock = 0;
let value = CrdsValue::new_unsigned(CrdsData::ContactInfo(ci));
let label = value.label();
// push a new message
assert_eq!(
push.process_push_message(&mut crds, &Pubkey::default(), value.clone(), 0),
Ok(())
);
assert_eq!(crds.get(&label).unwrap().value, value);
// push it again
assert_matches!(
push.process_push_message(&mut crds, &Pubkey::default(), value.clone(), 0),
Err(CrdsGossipError::PushMessageOldVersion)
);
// purge the old pushed
push.purge_old_received_cache(1);
// push it again
assert_eq!(
push.process_push_message(&mut crds, &Pubkey::default(), value, 0),
Err(CrdsGossipError::PushMessageOldVersion)
);
}
}

View File

@ -1,230 +0,0 @@
use crate::crds::VersionedCrdsValue;
use crate::crds_gossip_pull::CrdsFilter;
use indexmap::map::IndexMap;
use std::cmp::Ordering;
use std::ops::{Index, IndexMut};
#[derive(Clone)]
pub struct CrdsShards {
// shards[k] includes crds values which the first shard_bits of their hash
// value is equal to k. Each shard is a mapping from crds values indices to
// their hash value.
shards: Vec<IndexMap<usize, u64>>,
shard_bits: u32,
}
impl CrdsShards {
pub fn new(shard_bits: u32) -> Self {
CrdsShards {
shards: vec![IndexMap::new(); 1 << shard_bits],
shard_bits,
}
}
pub fn insert(&mut self, index: usize, value: &VersionedCrdsValue) -> bool {
let hash = CrdsFilter::hash_as_u64(&value.value_hash);
self.shard_mut(hash).insert(index, hash).is_none()
}
pub fn remove(&mut self, index: usize, value: &VersionedCrdsValue) -> bool {
let hash = CrdsFilter::hash_as_u64(&value.value_hash);
self.shard_mut(hash).swap_remove(&index).is_some()
}
/// Returns indices of all crds values which the first 'mask_bits' of their
/// hash value is equal to 'mask'.
pub fn find(&self, mask: u64, mask_bits: u32) -> impl Iterator<Item = usize> + '_ {
let ones = (!0u64).checked_shr(mask_bits).unwrap_or(0);
let mask = mask | ones;
match self.shard_bits.cmp(&mask_bits) {
Ordering::Less => {
let pred = move |(&index, hash)| {
if hash | ones == mask {
Some(index)
} else {
None
}
};
Iter::Less(self.shard(mask).iter().filter_map(pred))
}
Ordering::Equal => Iter::Equal(self.shard(mask).keys().cloned()),
Ordering::Greater => {
let count = 1 << (self.shard_bits - mask_bits);
let end = self.shard_index(mask) + 1;
Iter::Greater(
self.shards[end - count..end]
.iter()
.flat_map(IndexMap::keys)
.cloned(),
)
}
}
}
#[inline]
fn shard_index(&self, hash: u64) -> usize {
hash.checked_shr(64 - self.shard_bits).unwrap_or(0) as usize
}
#[inline]
fn shard(&self, hash: u64) -> &IndexMap<usize, u64> {
let shard_index = self.shard_index(hash);
self.shards.index(shard_index)
}
#[inline]
fn shard_mut(&mut self, hash: u64) -> &mut IndexMap<usize, u64> {
let shard_index = self.shard_index(hash);
self.shards.index_mut(shard_index)
}
// Checks invariants in the shards tables against the crds table.
#[cfg(test)]
pub fn check(&self, crds: &[VersionedCrdsValue]) {
let mut indices: Vec<_> = self
.shards
.iter()
.flat_map(IndexMap::keys)
.cloned()
.collect();
indices.sort_unstable();
assert_eq!(indices, (0..crds.len()).collect::<Vec<_>>());
for (shard_index, shard) in self.shards.iter().enumerate() {
for (&index, &hash) in shard {
assert_eq!(hash, CrdsFilter::hash_as_u64(&crds[index].value_hash));
assert_eq!(
shard_index as u64,
hash.checked_shr(64 - self.shard_bits).unwrap_or(0)
);
}
}
}
}
// Wrapper for 3 types of iterators we get when comparing shard_bits and
// mask_bits in find method. This is to avoid Box<dyn Iterator<Item =...>>
// which involves dynamic dispatch and is relatively slow.
enum Iter<R, S, T> {
Less(R),
Equal(S),
Greater(T),
}
impl<R, S, T> Iterator for Iter<R, S, T>
where
R: Iterator<Item = usize>,
S: Iterator<Item = usize>,
T: Iterator<Item = usize>,
{
type Item = usize;
fn next(&mut self) -> Option<Self::Item> {
match self {
Self::Greater(iter) => iter.next(),
Self::Less(iter) => iter.next(),
Self::Equal(iter) => iter.next(),
}
}
}
#[cfg(test)]
mod test {
use super::*;
use crate::{crds::Crds, crds_value::CrdsValue};
use rand::{thread_rng, Rng};
use solana_sdk::timing::timestamp;
use std::{collections::HashSet, iter::repeat_with, ops::Index};
fn new_test_crds_value<R: Rng>(rng: &mut R) -> VersionedCrdsValue {
let value = CrdsValue::new_rand(rng, None);
let label = value.label();
let mut crds = Crds::default();
crds.insert(value, timestamp()).unwrap();
crds.get(&label).cloned().unwrap()
}
// Returns true if the first mask_bits most significant bits of hash is the
// same as the given bit mask.
fn check_mask(value: &VersionedCrdsValue, mask: u64, mask_bits: u32) -> bool {
let hash = CrdsFilter::hash_as_u64(&value.value_hash);
let ones = (!0u64).checked_shr(mask_bits).unwrap_or(0u64);
(hash | ones) == (mask | ones)
}
// Manual filtering by scanning all the values.
fn filter_crds_values(
values: &[VersionedCrdsValue],
mask: u64,
mask_bits: u32,
) -> HashSet<usize> {
values
.iter()
.enumerate()
.filter_map(|(index, value)| {
if check_mask(value, mask, mask_bits) {
Some(index)
} else {
None
}
})
.collect()
}
#[test]
fn test_crds_shards_round_trip() {
let mut rng = thread_rng();
// Generate some random hash and crds value labels.
let mut values: Vec<_> = repeat_with(|| new_test_crds_value(&mut rng))
.take(4096)
.collect();
// Insert everything into the crds shards.
let mut shards = CrdsShards::new(5);
for (index, value) in values.iter().enumerate() {
assert!(shards.insert(index, value));
}
shards.check(&values);
// Remove some of the values.
for _ in 0..512 {
let index = rng.gen_range(0, values.len());
let value = values.swap_remove(index);
assert!(shards.remove(index, &value));
if index < values.len() {
let value = values.index(index);
assert!(shards.remove(values.len(), value));
assert!(shards.insert(index, value));
}
shards.check(&values);
}
// Random masks.
for _ in 0..10 {
let mask = rng.gen();
for mask_bits in 0..12 {
let mut set = filter_crds_values(&values, mask, mask_bits);
for index in shards.find(mask, mask_bits) {
assert!(set.remove(&index));
}
assert!(set.is_empty());
}
}
// Existing hash values.
for (index, value) in values.iter().enumerate() {
let mask = CrdsFilter::hash_as_u64(&value.value_hash);
let hits: Vec<_> = shards.find(mask, 64).collect();
assert_eq!(hits, vec![index]);
}
// Remove everything.
while !values.is_empty() {
let index = rng.gen_range(0, values.len());
let value = values.swap_remove(index);
assert!(shards.remove(index, &value));
if index < values.len() {
let value = values.index(index);
assert!(shards.remove(values.len(), value));
assert!(shards.insert(index, value));
}
if index % 5 == 0 {
shards.check(&values);
}
}
}
}

File diff suppressed because it is too large Load Diff

View File

@ -1,113 +0,0 @@
use std::sync::atomic::{AtomicU64, AtomicUsize, Ordering};
#[derive(Default)]
pub struct DataBudget {
// Amount of bytes we have in the budget to send.
bytes: AtomicUsize,
// Last time that we upped the bytes count, used
// to detect when to up the bytes budget again
last_timestamp_ms: AtomicU64,
}
impl DataBudget {
// If there are enough bytes in the budget, consumes from
// the budget and returns true. Otherwise returns false.
#[must_use]
pub fn take(&self, size: usize) -> bool {
let mut budget = self.bytes.load(Ordering::Acquire);
loop {
if budget < size {
return false;
}
match self.bytes.compare_exchange_weak(
budget,
budget - size,
Ordering::AcqRel,
Ordering::Acquire,
) {
Ok(_) => return true,
Err(bytes) => budget = bytes,
}
}
}
// Updates timestamp and returns true, if at least given milliseconds
// has passed since last update. Otherwise returns false.
fn can_update(&self, duration_millis: u64) -> bool {
let now = solana_sdk::timing::timestamp();
let mut last_timestamp = self.last_timestamp_ms.load(Ordering::Acquire);
loop {
if now < last_timestamp + duration_millis {
return false;
}
match self.last_timestamp_ms.compare_exchange_weak(
last_timestamp,
now,
Ordering::AcqRel,
Ordering::Acquire,
) {
Ok(_) => return true,
Err(ts) => last_timestamp = ts,
}
}
}
/// Updates the budget if at least given milliseconds has passed since last
/// update. Updater function maps current value of bytes to the new one.
/// Returns current data-budget after the update.
pub fn update<F>(&self, duration_millis: u64, updater: F) -> usize
where
F: Fn(usize) -> usize,
{
if self.can_update(duration_millis) {
let mut bytes = self.bytes.load(Ordering::Acquire);
loop {
match self.bytes.compare_exchange_weak(
bytes,
updater(bytes),
Ordering::AcqRel,
Ordering::Acquire,
) {
Ok(_) => break,
Err(b) => bytes = b,
}
}
}
self.bytes.load(Ordering::Acquire)
}
// Non-atomic clone only for tests and simulations.
pub fn clone_non_atomic(&self) -> Self {
Self {
bytes: AtomicUsize::new(self.bytes.load(Ordering::Acquire)),
last_timestamp_ms: AtomicU64::new(self.last_timestamp_ms.load(Ordering::Acquire)),
}
}
}
#[cfg(test)]
mod tests {
use super::*;
use std::time::Duration;
#[test]
fn test_data_budget() {
let budget = DataBudget::default();
assert!(!budget.take(1)); // budget = 0.
assert_eq!(budget.update(1000, |bytes| bytes + 5), 5); // budget updates to 5.
assert!(budget.take(1));
assert!(budget.take(2));
assert!(!budget.take(3)); // budget = 2, out of budget.
assert_eq!(budget.update(30, |_| 10), 2); // no update, budget = 2.
assert!(!budget.take(3)); // budget = 2, out of budget.
std::thread::sleep(Duration::from_millis(50));
assert_eq!(budget.update(30, |bytes| bytes * 2), 4); // budget updates to 4.
assert!(budget.take(3));
assert!(budget.take(1));
assert!(!budget.take(1)); // budget = 0.
}
}

View File

@ -1,21 +0,0 @@
use solana_sdk::clock::Slot;
#[derive(Serialize, Deserialize, Clone, Debug, PartialEq, AbiExample, AbiEnumVisitor)]
enum CompressionType {
Uncompressed,
GZip,
BZip2,
}
impl Default for CompressionType {
fn default() -> Self {
Self::Uncompressed
}
}
#[derive(Serialize, Deserialize, Clone, Debug, Default, PartialEq, AbiExample)]
pub(crate) struct EpochIncompleteSlots {
first: Slot,
compression: CompressionType,
compressed_list: Vec<u8>,
}

View File

@ -1,374 +0,0 @@
use crate::crds_value::sanitize_wallclock;
use itertools::Itertools;
use solana_ledger::{
blockstore_meta::DuplicateSlotProof,
shred::{Shred, ShredError, ShredType},
};
use solana_sdk::{
clock::Slot,
pubkey::Pubkey,
sanitize::{Sanitize, SanitizeError},
};
use std::{
collections::{hash_map::Entry, HashMap},
convert::TryFrom,
num::TryFromIntError,
};
use thiserror::Error;
const DUPLICATE_SHRED_HEADER_SIZE: usize = 63;
pub(crate) type DuplicateShredIndex = u16;
pub(crate) const MAX_DUPLICATE_SHREDS: DuplicateShredIndex = 512;
/// Function returning leader at a given slot.
pub trait LeaderScheduleFn: FnOnce(Slot) -> Option<Pubkey> {}
impl<F> LeaderScheduleFn for F where F: FnOnce(Slot) -> Option<Pubkey> {}
#[derive(Clone, Debug, PartialEq, AbiExample, Deserialize, Serialize)]
pub struct DuplicateShred {
pub(crate) from: Pubkey,
pub(crate) wallclock: u64,
pub(crate) slot: Slot,
shred_index: u32,
shred_type: ShredType,
// Serialized DuplicateSlotProof split into chunks.
num_chunks: u8,
chunk_index: u8,
#[serde(with = "serde_bytes")]
chunk: Vec<u8>,
}
#[derive(Debug, Error)]
pub enum Error {
#[error("data chunk mismatch")]
DataChunkMismatch,
#[error("invalid chunk index")]
InvalidChunkIndex,
#[error("invalid duplicate shreds")]
InvalidDuplicateShreds,
#[error("invalid duplicate slot proof")]
InvalidDuplicateSlotProof,
#[error("invalid signature")]
InvalidSignature,
#[error("invalid size limit")]
InvalidSizeLimit,
#[error("invalid shred")]
InvalidShred(#[from] ShredError),
#[error("number of chunks mismatch")]
NumChunksMismatch,
#[error("missing data chunk")]
MissingDataChunk,
#[error("(de)serialization error")]
SerializationError(#[from] bincode::Error),
#[error("shred index mismatch")]
ShredIndexMismatch,
#[error("shred type mismatch")]
ShredTypeMismatch,
#[error("slot mismatch")]
SlotMismatch,
#[error("type conversion error")]
TryFromIntError(#[from] TryFromIntError),
#[error("unknown slot leader")]
UnknownSlotLeader,
}
// Asserts that the two shreds can indicate duplicate proof for
// the same triplet of (slot, shred-index, and shred-type_), and
// that they have valid signatures from the slot leader.
fn check_shreds(
leader_schedule: Option<impl LeaderScheduleFn>,
shred1: &Shred,
shred2: &Shred,
) -> Result<(), Error> {
if shred1.slot() != shred2.slot() {
Err(Error::SlotMismatch)
} else if shred1.index() != shred2.index() {
Err(Error::ShredIndexMismatch)
} else if shred1.common_header.shred_type != shred2.common_header.shred_type {
Err(Error::ShredTypeMismatch)
} else if shred1.payload == shred2.payload {
Err(Error::InvalidDuplicateShreds)
} else {
if let Some(leader_schedule) = leader_schedule {
let slot_leader = leader_schedule(shred1.slot()).ok_or(Error::UnknownSlotLeader)?;
if !shred1.verify(&slot_leader) || !shred2.verify(&slot_leader) {
return Err(Error::InvalidSignature);
}
}
Ok(())
}
}
/// Splits a DuplicateSlotProof into DuplicateShred
/// chunks with a size limit on each chunk.
pub fn from_duplicate_slot_proof(
proof: &DuplicateSlotProof,
self_pubkey: Pubkey, // Pubkey of my node broadcasting crds value.
leader_schedule: Option<impl LeaderScheduleFn>,
wallclock: u64,
max_size: usize, // Maximum serialized size of each DuplicateShred.
) -> Result<impl Iterator<Item = DuplicateShred>, Error> {
if proof.shred1 == proof.shred2 {
return Err(Error::InvalidDuplicateSlotProof);
}
let shred1 = Shred::new_from_serialized_shred(proof.shred1.clone())?;
let shred2 = Shred::new_from_serialized_shred(proof.shred2.clone())?;
check_shreds(leader_schedule, &shred1, &shred2)?;
let (slot, shred_index, shred_type) = (
shred1.slot(),
shred1.index(),
shred1.common_header.shred_type,
);
let data = bincode::serialize(proof)?;
let chunk_size = if DUPLICATE_SHRED_HEADER_SIZE < max_size {
max_size - DUPLICATE_SHRED_HEADER_SIZE
} else {
return Err(Error::InvalidSizeLimit);
};
let chunks: Vec<_> = data.chunks(chunk_size).map(Vec::from).collect();
let num_chunks = u8::try_from(chunks.len())?;
let chunks = chunks
.into_iter()
.enumerate()
.map(move |(i, chunk)| DuplicateShred {
from: self_pubkey,
wallclock,
slot,
shred_index,
shred_type,
num_chunks,
chunk_index: i as u8,
chunk,
});
Ok(chunks)
}
pub(crate) fn from_shred(
shred: Shred,
self_pubkey: Pubkey, // Pubkey of my node broadcasting crds value.
other_payload: Vec<u8>,
leader_schedule: Option<impl LeaderScheduleFn>,
wallclock: u64,
max_size: usize, // Maximum serialized size of each DuplicateShred.
) -> Result<impl Iterator<Item = DuplicateShred>, Error> {
if shred.payload == other_payload {
return Err(Error::InvalidDuplicateShreds);
}
let other_shred = Shred::new_from_serialized_shred(other_payload.clone())?;
check_shreds(leader_schedule, &shred, &other_shred)?;
let (slot, shred_index, shred_type) =
(shred.slot(), shred.index(), shred.common_header.shred_type);
let proof = DuplicateSlotProof {
shred1: shred.payload,
shred2: other_payload,
};
let data = bincode::serialize(&proof)?;
let chunk_size = if DUPLICATE_SHRED_HEADER_SIZE < max_size {
max_size - DUPLICATE_SHRED_HEADER_SIZE
} else {
return Err(Error::InvalidSizeLimit);
};
let chunks: Vec<_> = data.chunks(chunk_size).map(Vec::from).collect();
let num_chunks = u8::try_from(chunks.len())?;
let chunks = chunks
.into_iter()
.enumerate()
.map(move |(i, chunk)| DuplicateShred {
from: self_pubkey,
wallclock,
slot,
shred_index,
shred_type,
num_chunks,
chunk_index: i as u8,
chunk,
});
Ok(chunks)
}
// Returns a predicate checking if a duplicate-shred chunk matches
// (slot, shred_index, shred_type) and has valid chunk_index.
fn check_chunk(
slot: Slot,
shred_index: u32,
shred_type: ShredType,
num_chunks: u8,
) -> impl Fn(&DuplicateShred) -> Result<(), Error> {
move |dup| {
if dup.slot != slot {
Err(Error::SlotMismatch)
} else if dup.shred_index != shred_index {
Err(Error::ShredIndexMismatch)
} else if dup.shred_type != shred_type {
Err(Error::ShredTypeMismatch)
} else if dup.num_chunks != num_chunks {
Err(Error::NumChunksMismatch)
} else if dup.chunk_index >= num_chunks {
Err(Error::InvalidChunkIndex)
} else {
Ok(())
}
}
}
/// Reconstructs the duplicate shreds from chunks of DuplicateShred.
pub fn into_shreds(
chunks: impl IntoIterator<Item = DuplicateShred>,
leader: impl LeaderScheduleFn,
) -> Result<(Shred, Shred), Error> {
let mut chunks = chunks.into_iter();
let DuplicateShred {
slot,
shred_index,
shred_type,
num_chunks,
chunk_index,
chunk,
..
} = chunks.next().ok_or(Error::InvalidDuplicateShreds)?;
let slot_leader = leader(slot).ok_or(Error::UnknownSlotLeader)?;
let check_chunk = check_chunk(slot, shred_index, shred_type, num_chunks);
let mut data = HashMap::new();
data.insert(chunk_index, chunk);
for chunk in chunks {
check_chunk(&chunk)?;
match data.entry(chunk.chunk_index) {
Entry::Vacant(entry) => {
entry.insert(chunk.chunk);
}
Entry::Occupied(entry) => {
if *entry.get() != chunk.chunk {
return Err(Error::DataChunkMismatch);
}
}
}
}
if data.len() != num_chunks as usize {
return Err(Error::MissingDataChunk);
}
let data = (0..num_chunks).map(|k| data.remove(&k).unwrap()).concat();
let proof: DuplicateSlotProof = bincode::deserialize(&data)?;
if proof.shred1 == proof.shred2 {
return Err(Error::InvalidDuplicateSlotProof);
}
let shred1 = Shred::new_from_serialized_shred(proof.shred1)?;
let shred2 = Shred::new_from_serialized_shred(proof.shred2)?;
if shred1.slot() != slot || shred2.slot() != slot {
Err(Error::SlotMismatch)
} else if shred1.index() != shred_index || shred2.index() != shred_index {
Err(Error::ShredIndexMismatch)
} else if shred1.common_header.shred_type != shred_type
|| shred2.common_header.shred_type != shred_type
{
Err(Error::ShredTypeMismatch)
} else if shred1.payload == shred2.payload {
Err(Error::InvalidDuplicateShreds)
} else if !shred1.verify(&slot_leader) || !shred2.verify(&slot_leader) {
Err(Error::InvalidSignature)
} else {
Ok((shred1, shred2))
}
}
impl Sanitize for DuplicateShred {
fn sanitize(&self) -> Result<(), SanitizeError> {
sanitize_wallclock(self.wallclock)?;
if self.chunk_index >= self.num_chunks {
return Err(SanitizeError::IndexOutOfBounds);
}
self.from.sanitize()
}
}
#[cfg(test)]
pub(crate) mod tests {
use super::*;
use rand::Rng;
use solana_ledger::{entry::Entry, shred::Shredder};
use solana_sdk::{hash, signature::Keypair, signature::Signer, system_transaction};
use std::sync::Arc;
#[test]
fn test_duplicate_shred_header_size() {
let dup = DuplicateShred {
from: Pubkey::new_unique(),
wallclock: u64::MAX,
slot: Slot::MAX,
shred_index: u32::MAX,
shred_type: ShredType(u8::MAX),
num_chunks: u8::MAX,
chunk_index: u8::MAX,
chunk: Vec::default(),
};
assert_eq!(
bincode::serialize(&dup).unwrap().len(),
DUPLICATE_SHRED_HEADER_SIZE
);
assert_eq!(
bincode::serialized_size(&dup).unwrap(),
DUPLICATE_SHRED_HEADER_SIZE as u64
);
}
pub fn new_rand_shred<R: Rng>(
rng: &mut R,
next_shred_index: u32,
shredder: &Shredder,
) -> Shred {
let entries: Vec<_> = std::iter::repeat_with(|| {
let tx = system_transaction::transfer(
&Keypair::new(), // from
&Pubkey::new_unique(), // to
rng.gen(), // lamports
hash::new_rand(rng), // recent blockhash
);
Entry::new(
&hash::new_rand(rng), // prev_hash
1, // num_hashes,
vec![tx], // transactions
)
})
.take(5)
.collect();
let (mut data_shreds, _coding_shreds, _last_shred_index) = shredder.entries_to_shreds(
&entries,
true, // is_last_in_slot
next_shred_index,
);
data_shreds.swap_remove(0)
}
#[test]
fn test_duplicate_shred_round_trip() {
let mut rng = rand::thread_rng();
let leader = Arc::new(Keypair::new());
let (slot, parent_slot, reference_tick, version) = (53084024, 53084023, 0, 0);
let shredder =
Shredder::new(slot, parent_slot, leader.clone(), reference_tick, version).unwrap();
let next_shred_index = rng.gen();
let shred1 = new_rand_shred(&mut rng, next_shred_index, &shredder);
let shred2 = new_rand_shred(&mut rng, next_shred_index, &shredder);
let leader_schedule = |s| {
if s == slot {
Some(leader.pubkey())
} else {
None
}
};
let chunks: Vec<_> = from_shred(
shred1.clone(),
Pubkey::new_unique(), // self_pubkey
shred2.payload.clone(),
Some(leader_schedule),
rng.gen(), // wallclock
512, // max_size
)
.unwrap()
.collect();
assert!(chunks.len() > 4);
let (shred3, shred4) = into_shreds(chunks, leader_schedule).unwrap();
assert_eq!(shred1, shred3);
assert_eq!(shred2, shred4);
}
}

View File

@ -1,549 +0,0 @@
use crate::cluster_info::MAX_CRDS_OBJECT_SIZE;
use crate::crds_value::{self, MAX_SLOT, MAX_WALLCLOCK};
use bincode::serialized_size;
use bv::BitVec;
use flate2::{Compress, Compression, Decompress, FlushCompress, FlushDecompress};
use solana_sdk::clock::Slot;
use solana_sdk::pubkey::Pubkey;
use solana_sdk::sanitize::{Sanitize, SanitizeError};
const MAX_SLOTS_PER_ENTRY: usize = 2048 * 8;
#[derive(Serialize, Deserialize, Clone, Debug, PartialEq, AbiExample)]
pub struct Uncompressed {
pub first_slot: Slot,
pub num: usize,
pub slots: BitVec<u8>,
}
impl Sanitize for Uncompressed {
fn sanitize(&self) -> std::result::Result<(), SanitizeError> {
if self.first_slot >= MAX_SLOT {
return Err(SanitizeError::ValueOutOfBounds);
}
if self.num >= MAX_SLOTS_PER_ENTRY {
return Err(SanitizeError::ValueOutOfBounds);
}
if self.slots.len() % 8 != 0 {
// Uncompressed::new() ensures the length is always a multiple of 8
return Err(SanitizeError::ValueOutOfBounds);
}
if self.slots.len() != self.slots.capacity() {
// A BitVec<u8> with a length that's a multiple of 8 will always have len() equal to
// capacity(), assuming no bit manipulation
return Err(SanitizeError::ValueOutOfBounds);
}
Ok(())
}
}
#[derive(Deserialize, Serialize, Clone, Debug, PartialEq, AbiExample)]
pub struct Flate2 {
pub first_slot: Slot,
pub num: usize,
pub compressed: Vec<u8>,
}
impl Sanitize for Flate2 {
fn sanitize(&self) -> std::result::Result<(), SanitizeError> {
if self.first_slot >= MAX_SLOT {
return Err(SanitizeError::ValueOutOfBounds);
}
if self.num >= MAX_SLOTS_PER_ENTRY {
return Err(SanitizeError::ValueOutOfBounds);
}
Ok(())
}
}
#[derive(Debug, PartialEq)]
pub enum Error {
CompressError,
DecompressError,
}
pub type Result<T> = std::result::Result<T, Error>;
impl std::convert::From<flate2::CompressError> for Error {
fn from(_e: flate2::CompressError) -> Error {
Error::CompressError
}
}
impl std::convert::From<flate2::DecompressError> for Error {
fn from(_e: flate2::DecompressError) -> Error {
Error::DecompressError
}
}
impl Flate2 {
fn deflate(mut unc: Uncompressed) -> Result<Self> {
let mut compressed = Vec::with_capacity(unc.slots.block_capacity());
let mut compressor = Compress::new(Compression::best(), false);
let first_slot = unc.first_slot;
let num = unc.num;
unc.slots.shrink_to_fit();
let bits = unc.slots.into_boxed_slice();
compressor.compress_vec(&bits, &mut compressed, FlushCompress::Finish)?;
let rv = Self {
first_slot,
num,
compressed,
};
let _ = rv.inflate()?;
Ok(rv)
}
pub fn inflate(&self) -> Result<Uncompressed> {
//add some head room for the decompressor which might spill more bits
let mut uncompressed = Vec::with_capacity(32 + (self.num + 4) / 8);
let mut decompress = Decompress::new(false);
decompress.decompress_vec(&self.compressed, &mut uncompressed, FlushDecompress::Finish)?;
Ok(Uncompressed {
first_slot: self.first_slot,
num: self.num,
slots: BitVec::from_bits(&uncompressed),
})
}
}
impl Uncompressed {
pub fn new(max_size: usize) -> Self {
Self {
num: 0,
first_slot: 0,
slots: BitVec::new_fill(false, 8 * max_size as u64),
}
}
pub fn to_slots(&self, min_slot: Slot) -> Vec<Slot> {
let mut rv = vec![];
let start = if min_slot < self.first_slot {
0
} else {
(min_slot - self.first_slot) as usize
};
for i in start..self.num {
if i >= self.slots.len() as usize {
break;
}
if self.slots.get(i as u64) {
rv.push(self.first_slot + i as Slot);
}
}
rv
}
pub fn add(&mut self, slots: &[Slot]) -> usize {
for (i, s) in slots.iter().enumerate() {
if self.num == 0 {
self.first_slot = *s;
}
if self.num >= MAX_SLOTS_PER_ENTRY {
return i;
}
if *s < self.first_slot {
return i;
}
if *s - self.first_slot >= self.slots.len() {
return i;
}
self.slots.set(*s - self.first_slot, true);
self.num = std::cmp::max(self.num, 1 + (*s - self.first_slot) as usize);
}
slots.len()
}
}
#[derive(Serialize, Deserialize, Clone, Debug, PartialEq, AbiExample, AbiEnumVisitor)]
pub enum CompressedSlots {
Flate2(Flate2),
Uncompressed(Uncompressed),
}
impl Sanitize for CompressedSlots {
fn sanitize(&self) -> std::result::Result<(), SanitizeError> {
match self {
CompressedSlots::Uncompressed(a) => a.sanitize(),
CompressedSlots::Flate2(b) => b.sanitize(),
}
}
}
impl Default for CompressedSlots {
fn default() -> Self {
CompressedSlots::new(0)
}
}
impl CompressedSlots {
fn new(max_size: usize) -> Self {
CompressedSlots::Uncompressed(Uncompressed::new(max_size))
}
pub fn first_slot(&self) -> Slot {
match self {
CompressedSlots::Uncompressed(a) => a.first_slot,
CompressedSlots::Flate2(b) => b.first_slot,
}
}
pub fn num_slots(&self) -> usize {
match self {
CompressedSlots::Uncompressed(a) => a.num,
CompressedSlots::Flate2(b) => b.num,
}
}
pub fn add(&mut self, slots: &[Slot]) -> usize {
match self {
CompressedSlots::Uncompressed(vals) => vals.add(slots),
CompressedSlots::Flate2(_) => 0,
}
}
pub fn to_slots(&self, min_slot: Slot) -> Result<Vec<Slot>> {
match self {
CompressedSlots::Uncompressed(vals) => Ok(vals.to_slots(min_slot)),
CompressedSlots::Flate2(vals) => {
let unc = vals.inflate()?;
Ok(unc.to_slots(min_slot))
}
}
}
pub fn deflate(&mut self) -> Result<()> {
match self {
CompressedSlots::Uncompressed(vals) => {
let unc = vals.clone();
let compressed = Flate2::deflate(unc)?;
let mut new = CompressedSlots::Flate2(compressed);
std::mem::swap(self, &mut new);
Ok(())
}
CompressedSlots::Flate2(_) => Ok(()),
}
}
}
#[derive(Serialize, Deserialize, Clone, Default, PartialEq, AbiExample)]
pub struct EpochSlots {
pub from: Pubkey,
pub slots: Vec<CompressedSlots>,
pub wallclock: u64,
}
impl Sanitize for EpochSlots {
fn sanitize(&self) -> std::result::Result<(), SanitizeError> {
if self.wallclock >= MAX_WALLCLOCK {
return Err(SanitizeError::ValueOutOfBounds);
}
self.from.sanitize()?;
self.slots.sanitize()
}
}
use std::fmt;
impl fmt::Debug for EpochSlots {
fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
let num_slots: usize = self.slots.iter().map(|s| s.num_slots()).sum();
let lowest_slot = self
.slots
.iter()
.map(|s| s.first_slot())
.fold(0, std::cmp::min);
write!(
f,
"EpochSlots {{ from: {} num_slots: {} lowest_slot: {} wallclock: {} }}",
self.from, num_slots, lowest_slot, self.wallclock
)
}
}
impl EpochSlots {
pub fn new(from: Pubkey, now: u64) -> Self {
Self {
from,
wallclock: now,
slots: vec![],
}
}
pub fn fill(&mut self, slots: &[Slot], now: u64) -> usize {
let mut num = 0;
self.wallclock = std::cmp::max(now, self.wallclock + 1);
while num < slots.len() {
num += self.add(&slots[num..]);
if num < slots.len() {
if self.deflate().is_err() {
return num;
}
let space = self.max_compressed_slot_size();
if space > 0 {
let cslot = CompressedSlots::new(space as usize);
self.slots.push(cslot);
} else {
return num;
}
}
}
num
}
pub fn add(&mut self, slots: &[Slot]) -> usize {
let mut num = 0;
for s in &mut self.slots {
num += s.add(&slots[num..]);
if num >= slots.len() {
break;
}
}
num
}
pub fn deflate(&mut self) -> Result<()> {
for s in self.slots.iter_mut() {
s.deflate()?;
}
Ok(())
}
pub fn max_compressed_slot_size(&self) -> isize {
let len_header = serialized_size(self).unwrap();
let len_slot = serialized_size(&CompressedSlots::default()).unwrap();
MAX_CRDS_OBJECT_SIZE as isize - (len_header + len_slot) as isize
}
pub fn first_slot(&self) -> Option<Slot> {
self.slots.iter().map(|s| s.first_slot()).min()
}
pub fn to_slots(&self, min_slot: Slot) -> Vec<Slot> {
self.slots
.iter()
.filter(|s| min_slot < s.first_slot() + s.num_slots() as u64)
.filter_map(|s| s.to_slots(min_slot).ok())
.flatten()
.collect()
}
/// New random EpochSlots for tests and simulations.
pub(crate) fn new_rand<R: rand::Rng>(rng: &mut R, pubkey: Option<Pubkey>) -> Self {
let now = crds_value::new_rand_timestamp(rng);
let pubkey = pubkey.unwrap_or_else(solana_sdk::pubkey::new_rand);
let mut epoch_slots = Self::new(pubkey, now);
let num_slots = rng.gen_range(0, 20);
let slots: Vec<_> = std::iter::repeat_with(|| 47825632 + rng.gen_range(0, 512))
.take(num_slots)
.collect();
epoch_slots.add(&slots);
epoch_slots
}
}
#[cfg(test)]
mod tests {
use super::*;
#[test]
fn test_epoch_slots_max_size() {
let epoch_slots = EpochSlots::default();
assert!(epoch_slots.max_compressed_slot_size() > 0);
}
#[test]
fn test_epoch_slots_uncompressed_add_1() {
let mut slots = Uncompressed::new(1);
assert_eq!(slots.slots.capacity(), 8);
assert_eq!(slots.add(&[1]), 1);
assert_eq!(slots.to_slots(1), vec![1]);
assert!(slots.to_slots(2).is_empty());
}
#[test]
fn test_epoch_slots_to_slots_overflow() {
let mut slots = Uncompressed::new(1);
slots.num = 100;
assert!(slots.to_slots(0).is_empty());
}
#[test]
fn test_epoch_slots_uncompressed_add_2() {
let mut slots = Uncompressed::new(1);
assert_eq!(slots.add(&[1, 2]), 2);
assert_eq!(slots.to_slots(1), vec![1, 2]);
}
#[test]
fn test_epoch_slots_uncompressed_add_3a() {
let mut slots = Uncompressed::new(1);
assert_eq!(slots.add(&[1, 3, 2]), 3);
assert_eq!(slots.to_slots(1), vec![1, 2, 3]);
}
#[test]
fn test_epoch_slots_uncompressed_add_3b() {
let mut slots = Uncompressed::new(1);
assert_eq!(slots.add(&[1, 10, 2]), 1);
assert_eq!(slots.to_slots(1), vec![1]);
}
#[test]
fn test_epoch_slots_uncompressed_add_3c() {
let mut slots = Uncompressed::new(2);
assert_eq!(slots.add(&[1, 10, 2]), 3);
assert_eq!(slots.to_slots(1), vec![1, 2, 10]);
assert_eq!(slots.to_slots(2), vec![2, 10]);
assert_eq!(slots.to_slots(3), vec![10]);
assert!(slots.to_slots(11).is_empty());
}
#[test]
fn test_epoch_slots_compressed() {
let mut slots = Uncompressed::new(100);
slots.add(&[1, 701, 2]);
assert_eq!(slots.num, 701);
let compressed = Flate2::deflate(slots).unwrap();
assert_eq!(compressed.first_slot, 1);
assert_eq!(compressed.num, 701);
assert!(compressed.compressed.len() < 32);
let slots = compressed.inflate().unwrap();
assert_eq!(slots.first_slot, 1);
assert_eq!(slots.num, 701);
assert_eq!(slots.to_slots(1), vec![1, 2, 701]);
}
#[test]
fn test_epoch_slots_sanitize() {
let mut slots = Uncompressed::new(100);
slots.add(&[1, 701, 2]);
assert_eq!(slots.num, 701);
assert!(slots.sanitize().is_ok());
let mut o = slots.clone();
o.first_slot = MAX_SLOT;
assert_eq!(o.sanitize(), Err(SanitizeError::ValueOutOfBounds));
let mut o = slots.clone();
o.num = MAX_SLOTS_PER_ENTRY;
assert_eq!(o.sanitize(), Err(SanitizeError::ValueOutOfBounds));
let mut o = slots.clone();
o.slots = BitVec::new_fill(false, 7); // Length not a multiple of 8
assert_eq!(o.sanitize(), Err(SanitizeError::ValueOutOfBounds));
let mut o = slots.clone();
o.slots = BitVec::with_capacity(8); // capacity() not equal to len()
assert_eq!(o.sanitize(), Err(SanitizeError::ValueOutOfBounds));
let compressed = Flate2::deflate(slots).unwrap();
assert!(compressed.sanitize().is_ok());
let mut o = compressed.clone();
o.first_slot = MAX_SLOT;
assert_eq!(o.sanitize(), Err(SanitizeError::ValueOutOfBounds));
let mut o = compressed;
o.num = MAX_SLOTS_PER_ENTRY;
assert_eq!(o.sanitize(), Err(SanitizeError::ValueOutOfBounds));
let mut slots = EpochSlots::default();
let range: Vec<Slot> = (0..5000).collect();
assert_eq!(slots.fill(&range, 1), 5000);
assert_eq!(slots.wallclock, 1);
assert!(slots.sanitize().is_ok());
let mut o = slots;
o.wallclock = MAX_WALLCLOCK;
assert_eq!(o.sanitize(), Err(SanitizeError::ValueOutOfBounds));
}
#[test]
fn test_epoch_slots_fill_range() {
let range: Vec<Slot> = (0..5000).collect();
let mut slots = EpochSlots::default();
assert_eq!(slots.fill(&range, 1), 5000);
assert_eq!(slots.wallclock, 1);
assert_eq!(slots.to_slots(0), range);
assert_eq!(slots.to_slots(4999), vec![4999]);
assert!(slots.to_slots(5000).is_empty());
}
#[test]
fn test_epoch_slots_fill_sparce_range() {
let range: Vec<Slot> = (0..5000).map(|x| x * 3).collect();
let mut slots = EpochSlots::default();
assert_eq!(slots.fill(&range, 2), 5000);
assert_eq!(slots.wallclock, 2);
assert_eq!(slots.slots.len(), 3);
assert_eq!(slots.slots[0].first_slot(), 0);
assert_ne!(slots.slots[0].num_slots(), 0);
let next = slots.slots[0].num_slots() as u64 + slots.slots[0].first_slot();
assert!(slots.slots[1].first_slot() >= next);
assert_ne!(slots.slots[1].num_slots(), 0);
assert_ne!(slots.slots[2].num_slots(), 0);
assert_eq!(slots.to_slots(0), range);
assert_eq!(slots.to_slots(4999 * 3), vec![4999 * 3]);
}
#[test]
fn test_epoch_slots_fill_large_sparce_range() {
let range: Vec<Slot> = (0..5000).map(|x| x * 7).collect();
let mut slots = EpochSlots::default();
assert_eq!(slots.fill(&range, 2), 5000);
assert_eq!(slots.to_slots(0), range);
}
#[test]
#[allow(clippy::same_item_push)]
fn test_epoch_slots_fill_uncompressed_random_range() {
use rand::Rng;
for _ in 0..10 {
let mut range: Vec<Slot> = vec![];
for _ in 0..5000 {
let last = *range.last().unwrap_or(&0);
range.push(last + rand::thread_rng().gen_range(1, 5));
}
let sz = EpochSlots::default().max_compressed_slot_size();
let mut slots = Uncompressed::new(sz as usize);
let sz = slots.add(&range);
let slots = slots.to_slots(0);
assert_eq!(slots.len(), sz);
assert_eq!(slots[..], range[..sz]);
}
}
#[test]
#[allow(clippy::same_item_push)]
fn test_epoch_slots_fill_compressed_random_range() {
use rand::Rng;
for _ in 0..10 {
let mut range: Vec<Slot> = vec![];
for _ in 0..5000 {
let last = *range.last().unwrap_or(&0);
range.push(last + rand::thread_rng().gen_range(1, 5));
}
let sz = EpochSlots::default().max_compressed_slot_size();
let mut slots = Uncompressed::new(sz as usize);
let sz = slots.add(&range);
let mut slots = CompressedSlots::Uncompressed(slots);
slots.deflate().unwrap();
let slots = slots.to_slots(0).unwrap();
assert_eq!(slots.len(), sz);
assert_eq!(slots[..], range[..sz]);
}
}
#[test]
#[allow(clippy::same_item_push)]
fn test_epoch_slots_fill_random_range() {
use rand::Rng;
for _ in 0..10 {
let mut range: Vec<Slot> = vec![];
for _ in 0..5000 {
let last = *range.last().unwrap_or(&0);
range.push(last + rand::thread_rng().gen_range(1, 5));
}
let mut slots = EpochSlots::default();
let sz = slots.fill(&range, 1);
let last = range[sz - 1];
assert_eq!(
last,
slots.slots.last().unwrap().first_slot()
+ slots.slots.last().unwrap().num_slots() as u64
- 1
);
for s in &slots.slots {
assert!(s.to_slots(0).is_ok());
}
let slots = slots.to_slots(0);
assert_eq!(slots[..], range[..slots.len()]);
assert_eq!(sz, slots.len())
}
}
}

View File

@ -1,404 +0,0 @@
//! The `gossip_service` module implements the network control plane.
use crate::cluster_info::{ClusterInfo, VALIDATOR_PORT_RANGE};
use crate::contact_info::ContactInfo;
use rand::{thread_rng, Rng};
use solana_client::thin_client::{create_client, ThinClient};
use solana_perf::recycler::Recycler;
use solana_runtime::bank_forks::BankForks;
use solana_sdk::{
pubkey::Pubkey,
signature::{Keypair, Signer},
};
use solana_streamer::streamer;
use std::{
collections::HashSet,
net::{IpAddr, Ipv4Addr, SocketAddr, TcpListener, UdpSocket},
sync::{
atomic::{AtomicBool, Ordering},
mpsc::channel,
{Arc, RwLock},
},
thread::{self, sleep, JoinHandle},
time::{Duration, Instant},
};
pub struct GossipService {
thread_hdls: Vec<JoinHandle<()>>,
}
impl GossipService {
pub fn new(
cluster_info: &Arc<ClusterInfo>,
bank_forks: Option<Arc<RwLock<BankForks>>>,
gossip_socket: UdpSocket,
gossip_validators: Option<HashSet<Pubkey>>,
should_check_duplicate_instance: bool,
exit: &Arc<AtomicBool>,
) -> Self {
let (request_sender, request_receiver) = channel();
let gossip_socket = Arc::new(gossip_socket);
trace!(
"GossipService: id: {}, listening on: {:?}",
&cluster_info.id(),
gossip_socket.local_addr().unwrap()
);
let t_receiver = streamer::receiver(
gossip_socket.clone(),
&exit,
request_sender,
Recycler::default(),
"gossip_receiver",
1,
);
let (response_sender, response_receiver) = channel();
let t_responder = streamer::responder("gossip", gossip_socket, response_receiver);
let t_listen = ClusterInfo::listen(
cluster_info.clone(),
bank_forks.clone(),
request_receiver,
response_sender.clone(),
should_check_duplicate_instance,
exit,
);
let t_gossip = ClusterInfo::gossip(
cluster_info.clone(),
bank_forks,
response_sender,
gossip_validators,
exit,
);
let thread_hdls = vec![t_receiver, t_responder, t_listen, t_gossip];
Self { thread_hdls }
}
pub fn join(self) -> thread::Result<()> {
for thread_hdl in self.thread_hdls {
thread_hdl.join()?;
}
Ok(())
}
}
/// Discover Validators in a cluster
pub fn discover_cluster(
entrypoint: &SocketAddr,
num_nodes: usize,
) -> std::io::Result<Vec<ContactInfo>> {
const DISCOVER_CLUSTER_TIMEOUT: Duration = Duration::from_secs(120);
let (_all_peers, validators) = discover(
None, // keypair
Some(entrypoint),
Some(num_nodes),
DISCOVER_CLUSTER_TIMEOUT,
None, // find_node_by_pubkey
None, // find_node_by_gossip_addr
None, // my_gossip_addr
0, // my_shred_version
)?;
Ok(validators)
}
pub fn discover(
keypair: Option<Arc<Keypair>>,
entrypoint: Option<&SocketAddr>,
num_nodes: Option<usize>, // num_nodes only counts validators, excludes spy nodes
timeout: Duration,
find_node_by_pubkey: Option<Pubkey>,
find_node_by_gossip_addr: Option<&SocketAddr>,
my_gossip_addr: Option<&SocketAddr>,
my_shred_version: u16,
) -> std::io::Result<(
Vec<ContactInfo>, // all gossip peers
Vec<ContactInfo>, // tvu peers (validators)
)> {
let keypair = keypair.unwrap_or_else(|| Arc::new(Keypair::new()));
let exit = Arc::new(AtomicBool::new(false));
let (gossip_service, ip_echo, spy_ref) = make_gossip_node(
keypair,
entrypoint,
&exit,
my_gossip_addr,
my_shred_version,
true, // should_check_duplicate_instance,
);
let id = spy_ref.id();
info!("Entrypoint: {:?}", entrypoint);
info!("Node Id: {:?}", id);
if let Some(my_gossip_addr) = my_gossip_addr {
info!("Gossip Address: {:?}", my_gossip_addr);
}
let _ip_echo_server = ip_echo.map(solana_net_utils::ip_echo_server);
let (met_criteria, elapsed, all_peers, tvu_peers) = spy(
spy_ref.clone(),
num_nodes,
timeout,
find_node_by_pubkey,
find_node_by_gossip_addr,
);
exit.store(true, Ordering::Relaxed);
gossip_service.join().unwrap();
if met_criteria {
info!(
"discover success in {}s...\n{}",
elapsed.as_secs(),
spy_ref.contact_info_trace()
);
return Ok((all_peers, tvu_peers));
}
if !tvu_peers.is_empty() {
info!(
"discover failed to match criteria by timeout...\n{}",
spy_ref.contact_info_trace()
);
return Ok((all_peers, tvu_peers));
}
info!("discover failed...\n{}", spy_ref.contact_info_trace());
Err(std::io::Error::new(
std::io::ErrorKind::Other,
"Discover failed",
))
}
/// Creates a ThinClient per valid node
pub fn get_clients(nodes: &[ContactInfo]) -> Vec<ThinClient> {
nodes
.iter()
.filter_map(ContactInfo::valid_client_facing_addr)
.map(|addrs| create_client(addrs, VALIDATOR_PORT_RANGE))
.collect()
}
/// Creates a ThinClient by selecting a valid node at random
pub fn get_client(nodes: &[ContactInfo]) -> ThinClient {
let nodes: Vec<_> = nodes
.iter()
.filter_map(ContactInfo::valid_client_facing_addr)
.collect();
let select = thread_rng().gen_range(0, nodes.len());
create_client(nodes[select], VALIDATOR_PORT_RANGE)
}
pub fn get_multi_client(nodes: &[ContactInfo]) -> (ThinClient, usize) {
let addrs: Vec<_> = nodes
.iter()
.filter_map(ContactInfo::valid_client_facing_addr)
.collect();
let rpc_addrs: Vec<_> = addrs.iter().map(|addr| addr.0).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();
(
ThinClient::new_from_addrs(rpc_addrs, tpu_addrs, transactions_socket),
num_nodes,
)
}
fn spy(
spy_ref: Arc<ClusterInfo>,
num_nodes: Option<usize>,
timeout: Duration,
find_node_by_pubkey: Option<Pubkey>,
find_node_by_gossip_addr: Option<&SocketAddr>,
) -> (
bool, // if found the specified nodes
Duration, // elapsed time until found the nodes or timed-out
Vec<ContactInfo>, // all gossip peers
Vec<ContactInfo>, // tvu peers (validators)
) {
let now = Instant::now();
let mut met_criteria = false;
let mut all_peers: Vec<ContactInfo> = Vec::new();
let mut tvu_peers: Vec<ContactInfo> = Vec::new();
let mut i = 1;
while !met_criteria && now.elapsed() < timeout {
all_peers = spy_ref
.all_peers()
.into_iter()
.map(|x| x.0)
.collect::<Vec<_>>();
tvu_peers = spy_ref.all_tvu_peers().into_iter().collect::<Vec<_>>();
let found_node_by_pubkey = if let Some(pubkey) = find_node_by_pubkey {
all_peers.iter().any(|x| x.id == pubkey)
} else {
false
};
let found_node_by_gossip_addr = if let Some(gossip_addr) = find_node_by_gossip_addr {
all_peers.iter().any(|x| x.gossip == *gossip_addr)
} else {
false
};
if let Some(num) = num_nodes {
// Only consider validators and archives for `num_nodes`
let mut nodes: Vec<_> = tvu_peers.iter().collect();
nodes.sort();
nodes.dedup();
if nodes.len() >= num {
if found_node_by_pubkey || found_node_by_gossip_addr {
met_criteria = true;
}
if find_node_by_pubkey.is_none() && find_node_by_gossip_addr.is_none() {
met_criteria = true;
}
}
} else if found_node_by_pubkey || found_node_by_gossip_addr {
met_criteria = true;
}
if i % 20 == 0 {
info!("discovering...\n{}", spy_ref.contact_info_trace());
}
sleep(Duration::from_millis(
crate::cluster_info::GOSSIP_SLEEP_MILLIS,
));
i += 1;
}
(met_criteria, now.elapsed(), all_peers, tvu_peers)
}
/// Makes a spy or gossip node based on whether or not a gossip_addr was passed in
/// Pass in a gossip addr to fully participate in gossip instead of relying on just pulls
fn make_gossip_node(
keypair: Arc<Keypair>,
entrypoint: Option<&SocketAddr>,
exit: &Arc<AtomicBool>,
gossip_addr: Option<&SocketAddr>,
shred_version: u16,
should_check_duplicate_instance: bool,
) -> (GossipService, Option<TcpListener>, Arc<ClusterInfo>) {
let (node, gossip_socket, ip_echo) = if let Some(gossip_addr) = gossip_addr {
ClusterInfo::gossip_node(&keypair.pubkey(), gossip_addr, shred_version)
} else {
ClusterInfo::spy_node(&keypair.pubkey(), shred_version)
};
let cluster_info = ClusterInfo::new(node, keypair);
if let Some(entrypoint) = entrypoint {
cluster_info.set_entrypoint(ContactInfo::new_gossip_entry_point(entrypoint));
}
let cluster_info = Arc::new(cluster_info);
let gossip_service = GossipService::new(
&cluster_info,
None,
gossip_socket,
None,
should_check_duplicate_instance,
&exit,
);
(gossip_service, ip_echo, cluster_info)
}
#[cfg(test)]
mod tests {
use super::*;
use crate::cluster_info::{ClusterInfo, Node};
use std::sync::atomic::AtomicBool;
use std::sync::Arc;
#[test]
#[ignore]
// test that stage will exit when flag is set
fn test_exit() {
let exit = Arc::new(AtomicBool::new(false));
let tn = Node::new_localhost();
let cluster_info = ClusterInfo::new_with_invalid_keypair(tn.info.clone());
let c = Arc::new(cluster_info);
let d = GossipService::new(
&c,
None,
tn.sockets.gossip,
None,
true, // should_check_duplicate_instance
&exit,
);
exit.store(true, Ordering::Relaxed);
d.join().unwrap();
}
#[test]
fn test_gossip_services_spy() {
const TIMEOUT: Duration = Duration::from_secs(5);
let keypair = Keypair::new();
let peer0 = solana_sdk::pubkey::new_rand();
let peer1 = solana_sdk::pubkey::new_rand();
let contact_info = ContactInfo::new_localhost(&keypair.pubkey(), 0);
let peer0_info = ContactInfo::new_localhost(&peer0, 0);
let peer1_info = ContactInfo::new_localhost(&peer1, 0);
let cluster_info = ClusterInfo::new(contact_info, Arc::new(keypair));
cluster_info.insert_info(peer0_info.clone());
cluster_info.insert_info(peer1_info);
let spy_ref = Arc::new(cluster_info);
let (met_criteria, elapsed, _, tvu_peers) = spy(spy_ref.clone(), None, TIMEOUT, None, None);
assert!(!met_criteria);
assert!((TIMEOUT..TIMEOUT + Duration::from_secs(1)).contains(&elapsed));
assert_eq!(tvu_peers, spy_ref.tvu_peers());
// Find num_nodes
let (met_criteria, _, _, _) = spy(spy_ref.clone(), Some(1), TIMEOUT, None, None);
assert!(met_criteria);
let (met_criteria, _, _, _) = spy(spy_ref.clone(), Some(2), TIMEOUT, None, None);
assert!(met_criteria);
// Find specific node by pubkey
let (met_criteria, _, _, _) = spy(spy_ref.clone(), None, TIMEOUT, Some(peer0), None);
assert!(met_criteria);
let (met_criteria, _, _, _) = spy(
spy_ref.clone(),
None,
TIMEOUT,
Some(solana_sdk::pubkey::new_rand()),
None,
);
assert!(!met_criteria);
// Find num_nodes *and* specific node by pubkey
let (met_criteria, _, _, _) = spy(spy_ref.clone(), Some(1), TIMEOUT, Some(peer0), None);
assert!(met_criteria);
let (met_criteria, _, _, _) = spy(spy_ref.clone(), Some(3), TIMEOUT, Some(peer0), None);
assert!(!met_criteria);
let (met_criteria, _, _, _) = spy(
spy_ref.clone(),
Some(1),
TIMEOUT,
Some(solana_sdk::pubkey::new_rand()),
None,
);
assert!(!met_criteria);
// Find specific node by gossip address
let (met_criteria, _, _, _) = spy(
spy_ref.clone(),
None,
TIMEOUT,
None,
Some(&peer0_info.gossip),
);
assert!(met_criteria);
let (met_criteria, _, _, _) = spy(
spy_ref,
None,
TIMEOUT,
None,
Some(&"1.1.1.1:1234".parse().unwrap()),
);
assert!(!met_criteria);
}
}

View File

@ -13,40 +13,21 @@ pub mod bigtable_upload_service;
pub mod broadcast_stage;
pub mod cache_block_time_service;
pub mod cluster_info_vote_listener;
pub mod commitment_service;
pub mod completed_data_sets_service;
mod deprecated;
pub mod sample_performance_service;
pub mod shred_fetch_stage;
#[macro_use]
pub mod contact_info;
pub mod cluster_info;
mod cluster_info_metrics;
pub mod cluster_slot_state_verifier;
pub mod cluster_slots;
pub mod cluster_slots_service;
pub mod commitment_service;
pub mod completed_data_sets_service;
pub mod consensus;
pub mod crds;
pub mod crds_gossip;
pub mod crds_gossip_error;
pub mod crds_gossip_pull;
pub mod crds_gossip_push;
pub mod crds_shards;
pub mod crds_value;
pub mod data_budget;
pub mod duplicate_shred;
pub mod epoch_slots;
pub mod fetch_stage;
pub mod fork_choice;
pub mod gen_keys;
pub mod gossip_service;
pub mod heaviest_subtree_fork_choice;
pub mod latest_validator_votes_for_frozen_banks;
pub mod ledger_cleanup_service;
pub mod optimistic_confirmation_verifier;
pub mod outstanding_requests;
pub mod packet_hasher;
pub mod ping_pong;
pub mod poh_recorder;
pub mod poh_service;
pub mod progress_map;
@ -62,9 +43,11 @@ pub mod rewards_recorder_service;
pub mod rpc;
pub mod rpc_health;
pub mod rpc_service;
pub mod sample_performance_service;
pub mod send_transaction_service;
pub mod serve_repair;
pub mod serve_repair_service;
pub mod shred_fetch_stage;
pub mod sigverify;
pub mod sigverify_shreds;
pub mod sigverify_stage;
@ -78,7 +61,6 @@ pub mod unfrozen_gossip_verified_vote_hashes;
pub mod validator;
pub mod verified_vote_packets;
pub mod vote_stake_tracker;
pub mod weighted_shuffle;
pub mod window_service;
#[macro_use]

View File

@ -1,405 +0,0 @@
use bincode::{serialize, Error};
use lru::LruCache;
use rand::{AsByteSliceMut, CryptoRng, Rng};
use serde::Serialize;
use solana_sdk::hash::{self, Hash};
use solana_sdk::pubkey::Pubkey;
use solana_sdk::sanitize::{Sanitize, SanitizeError};
use solana_sdk::signature::{Keypair, Signable, Signature, Signer};
use std::borrow::Cow;
use std::net::SocketAddr;
use std::time::{Duration, Instant};
#[derive(AbiExample, Debug, Deserialize, Serialize)]
pub struct Ping<T> {
from: Pubkey,
token: T,
signature: Signature,
}
#[derive(AbiExample, Debug, Deserialize, Serialize)]
pub struct Pong {
from: Pubkey,
hash: Hash, // Hash of received ping token.
signature: Signature,
}
/// Maintains records of remote nodes which have returned a valid response to a
/// ping message, and on-the-fly ping messages pending a pong response from the
/// remote node.
pub struct PingCache {
// Time-to-live of received pong messages.
ttl: Duration,
// Timestamp of last ping message sent to a remote node.
// Used to rate limit pings to remote nodes.
pings: LruCache<(Pubkey, SocketAddr), Instant>,
// Verified pong responses from remote nodes.
pongs: LruCache<(Pubkey, SocketAddr), Instant>,
// Hash of ping tokens sent out to remote nodes,
// pending a pong response back.
pending_cache: LruCache<Hash, (Pubkey, SocketAddr)>,
}
impl<T: Serialize> Ping<T> {
pub fn new(token: T, keypair: &Keypair) -> Result<Self, Error> {
let signature = keypair.sign_message(&serialize(&token)?);
let ping = Ping {
from: keypair.pubkey(),
token,
signature,
};
Ok(ping)
}
}
impl<T> Ping<T>
where
T: Serialize + AsByteSliceMut + Default,
{
pub fn new_rand<R>(rng: &mut R, keypair: &Keypair) -> Result<Self, Error>
where
R: Rng + CryptoRng,
{
let mut token = T::default();
rng.fill(&mut token);
Ping::new(token, keypair)
}
}
impl<T> Sanitize for Ping<T> {
fn sanitize(&self) -> Result<(), SanitizeError> {
self.from.sanitize()?;
// TODO Add self.token.sanitize()?; when rust's
// specialization feature becomes stable.
self.signature.sanitize()
}
}
impl<T: Serialize> Signable for Ping<T> {
fn pubkey(&self) -> Pubkey {
self.from
}
fn signable_data(&self) -> Cow<[u8]> {
Cow::Owned(serialize(&self.token).unwrap())
}
fn get_signature(&self) -> Signature {
self.signature
}
fn set_signature(&mut self, signature: Signature) {
self.signature = signature;
}
}
impl Pong {
pub fn new<T: Serialize>(ping: &Ping<T>, keypair: &Keypair) -> Result<Self, Error> {
let hash = hash::hash(&serialize(&ping.token)?);
let pong = Pong {
from: keypair.pubkey(),
hash,
signature: keypair.sign_message(hash.as_ref()),
};
Ok(pong)
}
}
impl Sanitize for Pong {
fn sanitize(&self) -> Result<(), SanitizeError> {
self.from.sanitize()?;
self.hash.sanitize()?;
self.signature.sanitize()
}
}
impl Signable for Pong {
fn pubkey(&self) -> Pubkey {
self.from
}
fn signable_data(&self) -> Cow<[u8]> {
Cow::Owned(self.hash.as_ref().into())
}
fn get_signature(&self) -> Signature {
self.signature
}
fn set_signature(&mut self, signature: Signature) {
self.signature = signature;
}
}
impl PingCache {
pub fn new(ttl: Duration, cap: usize) -> Self {
Self {
ttl,
pings: LruCache::new(cap),
pongs: LruCache::new(cap),
pending_cache: LruCache::new(cap),
}
}
/// Checks if the pong hash, pubkey and socket match a ping message sent
/// out previously. If so records current timestamp for the remote node and
/// returns true.
/// Note: Does not verify the signature.
pub fn add(&mut self, pong: &Pong, socket: SocketAddr, now: Instant) -> bool {
let node = (pong.pubkey(), socket);
match self.pending_cache.peek(&pong.hash) {
Some(value) if *value == node => {
self.pings.pop(&node);
self.pongs.put(node, now);
self.pending_cache.pop(&pong.hash);
true
}
_ => false,
}
}
/// Checks if the remote node has been pinged recently. If not, calls the
/// given function to generates a new ping message, records current
/// timestamp and hash of ping token, and returns the ping message.
fn maybe_ping<T, F>(
&mut self,
now: Instant,
node: (Pubkey, SocketAddr),
mut pingf: F,
) -> Option<Ping<T>>
where
T: Serialize,
F: FnMut() -> Option<Ping<T>>,
{
// Rate limit consecutive pings sent to a remote node.
let delay = self.ttl / 64;
match self.pings.peek(&node) {
Some(t) if now.saturating_duration_since(*t) < delay => None,
_ => {
let ping = pingf()?;
let hash = hash::hash(&serialize(&ping.token).ok()?);
self.pings.put(node, now);
self.pending_cache.put(hash, node);
Some(ping)
}
}
}
/// Returns true if the remote node has responded to a ping message.
/// Removes expired pong messages. In order to extend verifications before
/// expiration, if the pong message is not too recent, and the node has not
/// been pinged recently, calls the given function to generates a new ping
/// message, records current timestamp and hash of ping token, and returns
/// the ping message.
/// Caller should verify if the socket address is valid. (e.g. by using
/// ContactInfo::is_valid_address).
pub fn check<T, F>(
&mut self,
now: Instant,
node: (Pubkey, SocketAddr),
pingf: F,
) -> (bool, Option<Ping<T>>)
where
T: Serialize,
F: FnMut() -> Option<Ping<T>>,
{
let (check, should_ping) = match self.pongs.get(&node) {
None => (false, true),
Some(t) => {
let age = now.saturating_duration_since(*t);
// Pop if the pong message has expired.
if age > self.ttl {
self.pongs.pop(&node);
}
// If the pong message is not too recent, generate a new ping
// message to extend remote node verification.
(true, age > self.ttl / 8)
}
};
let ping = if should_ping {
self.maybe_ping(now, node, pingf)
} else {
None
};
(check, ping)
}
// Only for tests and simulations.
pub(crate) fn mock_clone(&self) -> Self {
let mut clone = Self {
ttl: self.ttl,
pings: LruCache::new(self.pings.cap()),
pongs: LruCache::new(self.pongs.cap()),
pending_cache: LruCache::new(self.pending_cache.cap()),
};
for (k, v) in self.pongs.iter().rev() {
clone.pings.put(*k, *v);
}
for (k, v) in self.pongs.iter().rev() {
clone.pongs.put(*k, *v);
}
for (k, v) in self.pending_cache.iter().rev() {
clone.pending_cache.put(*k, *v);
}
clone
}
/// Only for tests and simulations.
pub fn mock_pong(&mut self, node: Pubkey, socket: SocketAddr, now: Instant) {
self.pongs.put((node, socket), now);
}
}
#[cfg(test)]
mod tests {
use super::*;
use std::collections::HashSet;
use std::iter::repeat_with;
use std::net::{Ipv4Addr, SocketAddrV4};
type Token = [u8; 32];
#[test]
fn test_ping_pong() {
let mut rng = rand::thread_rng();
let keypair = Keypair::new();
let ping = Ping::<Token>::new_rand(&mut rng, &keypair).unwrap();
assert!(ping.verify());
assert!(ping.sanitize().is_ok());
let pong = Pong::new(&ping, &keypair).unwrap();
assert!(pong.verify());
assert!(pong.sanitize().is_ok());
assert_eq!(hash::hash(&ping.token), pong.hash);
}
#[test]
fn test_ping_cache() {
let now = Instant::now();
let mut rng = rand::thread_rng();
let ttl = Duration::from_millis(256);
let mut cache = PingCache::new(ttl, /*cap=*/ 1000);
let this_node = Keypair::new();
let keypairs: Vec<_> = repeat_with(Keypair::new).take(8).collect();
let sockets: Vec<_> = repeat_with(|| {
SocketAddr::V4(SocketAddrV4::new(
Ipv4Addr::new(rng.gen(), rng.gen(), rng.gen(), rng.gen()),
rng.gen(),
))
})
.take(8)
.collect();
let remote_nodes: Vec<(&Keypair, SocketAddr)> = repeat_with(|| {
let keypair = &keypairs[rng.gen_range(0, keypairs.len())];
let socket = sockets[rng.gen_range(0, sockets.len())];
(keypair, socket)
})
.take(128)
.collect();
// Initially all checks should fail. The first observation of each node
// should create a ping packet.
let mut seen_nodes = HashSet::<(Pubkey, SocketAddr)>::new();
let pings: Vec<Option<Ping<Token>>> = remote_nodes
.iter()
.map(|(keypair, socket)| {
let node = (keypair.pubkey(), *socket);
let pingf = || Ping::<Token>::new_rand(&mut rng, &this_node).ok();
let (check, ping) = cache.check(now, node, pingf);
assert!(!check);
assert_eq!(seen_nodes.insert(node), ping.is_some());
ping
})
.collect();
let now = now + Duration::from_millis(1);
let panic_ping = || -> Option<Ping<Token>> { panic!("this should not happen!") };
for ((keypair, socket), ping) in remote_nodes.iter().zip(&pings) {
match ping {
None => {
// Already have a recent ping packets for nodes, so no new
// ping packet will be generated.
let node = (keypair.pubkey(), *socket);
let (check, ping) = cache.check(now, node, panic_ping);
assert!(check);
assert!(ping.is_none());
}
Some(ping) => {
let pong = Pong::new(ping, keypair).unwrap();
assert!(cache.add(&pong, *socket, now));
}
}
}
let now = now + Duration::from_millis(1);
// All nodes now have a recent pong packet.
for (keypair, socket) in &remote_nodes {
let node = (keypair.pubkey(), *socket);
let (check, ping) = cache.check(now, node, panic_ping);
assert!(check);
assert!(ping.is_none());
}
let now = now + ttl / 8;
// All nodes still have a valid pong packet, but the cache will create
// a new ping packet to extend verification.
seen_nodes.clear();
for (keypair, socket) in &remote_nodes {
let node = (keypair.pubkey(), *socket);
let pingf = || Ping::<Token>::new_rand(&mut rng, &this_node).ok();
let (check, ping) = cache.check(now, node, pingf);
assert!(check);
assert_eq!(seen_nodes.insert(node), ping.is_some());
}
let now = now + Duration::from_millis(1);
// All nodes still have a valid pong packet, and a very recent ping
// packet pending response. So no new ping packet will be created.
for (keypair, socket) in &remote_nodes {
let node = (keypair.pubkey(), *socket);
let (check, ping) = cache.check(now, node, panic_ping);
assert!(check);
assert!(ping.is_none());
}
let now = now + ttl;
// Pong packets are still valid but expired. The first observation of
// each node will remove the pong packet from cache and create a new
// ping packet.
seen_nodes.clear();
for (keypair, socket) in &remote_nodes {
let node = (keypair.pubkey(), *socket);
let pingf = || Ping::<Token>::new_rand(&mut rng, &this_node).ok();
let (check, ping) = cache.check(now, node, pingf);
if seen_nodes.insert(node) {
assert!(check);
assert!(ping.is_some());
} else {
assert!(!check);
assert!(ping.is_none());
}
}
let now = now + Duration::from_millis(1);
// No valid pong packet in the cache. A recent ping packet already
// created, so no new one will be created.
for (keypair, socket) in &remote_nodes {
let node = (keypair.pubkey(), *socket);
let (check, ping) = cache.check(now, node, panic_ping);
assert!(!check);
assert!(ping.is_none());
}
let now = now + ttl / 64;
// No valid pong packet in the cache. Another ping packet will be
// created for the first observation of each node.
seen_nodes.clear();
for (keypair, socket) in &remote_nodes {
let node = (keypair.pubkey(), *socket);
let pingf = || Ping::<Token>::new_rand(&mut rng, &this_node).ok();
let (check, ping) = cache.check(now, node, pingf);
assert!(!check);
assert_eq!(seen_nodes.insert(node), ping.is_some());
}
}
}

View File

@ -1,7 +1,6 @@
//! The `repair_service` module implements the tools necessary to generate a thread which
//! regularly finds missing shreds in the ledger and sends repair requests for those shreds
use crate::{
cluster_info::ClusterInfo,
cluster_info_vote_listener::VerifiedVoteReceiver,
cluster_slots::ClusterSlots,
outstanding_requests::OutstandingRequests,
@ -10,6 +9,7 @@ use crate::{
serve_repair::{RepairType, ServeRepair},
};
use crossbeam_channel::{Receiver as CrossbeamReceiver, Sender as CrossbeamSender};
use solana_gossip::cluster_info::ClusterInfo;
use solana_ledger::{
blockstore::{Blockstore, SlotMeta},
shred::Nonce,
@ -668,8 +668,8 @@ impl RepairService {
#[cfg(test)]
mod test {
use super::*;
use crate::cluster_info::Node;
use crossbeam_channel::unbounded;
use solana_gossip::cluster_info::Node;
use solana_ledger::blockstore::{
make_chaining_slot_entries, make_many_slot_entries, make_slot_entries,
};

View File

@ -3,7 +3,6 @@
use crate::{
broadcast_stage::RetransmitSlotsSender,
cache_block_time_service::CacheBlockTimeSender,
cluster_info::ClusterInfo,
cluster_info_vote_listener::{
GossipDuplicateConfirmedSlotsReceiver, GossipVerifiedVoteHashReceiver, VoteTracker,
},
@ -25,6 +24,7 @@ use crate::{
window_service::DuplicateSlotReceiver,
};
use solana_client::rpc_response::SlotUpdate;
use solana_gossip::cluster_info::ClusterInfo;
use solana_ledger::{
block_error::BlockError,
blockstore::Blockstore,
@ -2473,15 +2473,14 @@ impl ReplayStage {
pub(crate) mod tests {
use super::*;
use crate::{
cluster_info::Node,
consensus::test::{initialize_state, VoteSimulator},
consensus::Tower,
crds::Cursor,
progress_map::ValidatorStakeInfo,
replay_stage::ReplayStage,
transaction_status_service::TransactionStatusService,
};
use crossbeam_channel::unbounded;
use solana_gossip::{cluster_info::Node, crds::Cursor};
use solana_ledger::{
blockstore::make_slot_entries,
blockstore::{entries_to_test_shreds, BlockstoreError},

View File

@ -1,39 +1,23 @@
//! The `result` module exposes a Result type that propagates one of many different Error types.
use crate::poh_recorder;
use crate::{cluster_info, duplicate_shred};
use solana_ledger::block_error;
use solana_gossip::{cluster_info, gossip_error::GossipError};
use solana_ledger::blockstore;
use solana_runtime::snapshot_utils;
use solana_sdk::transaction;
use std::any::Any;
#[derive(Debug)]
pub enum Error {
Io(std::io::Error),
Json(serde_json::Error),
AddrParse(std::net::AddrParseError),
JoinError(Box<dyn Any + Send + 'static>),
RecvError(std::sync::mpsc::RecvError),
TryCrossbeamRecvError(crossbeam_channel::TryRecvError),
CrossbeamRecvTimeoutError(crossbeam_channel::RecvTimeoutError),
ReadyTimeoutError,
RecvTimeoutError(std::sync::mpsc::RecvTimeoutError),
CrossbeamSendError,
TryCrossbeamSendError,
TryRecvError(std::sync::mpsc::TryRecvError),
Serialize(std::boxed::Box<bincode::ErrorKind>),
TransactionError(transaction::TransactionError),
ClusterInfoError(cluster_info::ClusterInfoError),
SendError,
PohRecorderError(poh_recorder::PohRecorderError),
BlockError(block_error::BlockError),
BlockstoreError(blockstore::BlockstoreError),
FsExtra(fs_extra::error::Error),
SnapshotError(snapshot_utils::SnapshotError),
WeightedIndexError(rand::distributions::weighted::WeightedError),
DuplicateNodeInstance,
DuplicateShredError(duplicate_shred::Error),
GossipError(GossipError),
}
pub type Result<T> = std::result::Result<T, Error>;
@ -51,16 +35,6 @@ impl std::convert::From<std::sync::mpsc::RecvError> for Error {
Error::RecvError(e)
}
}
impl std::convert::From<crossbeam_channel::TryRecvError> for Error {
fn from(e: crossbeam_channel::TryRecvError) -> Error {
Error::TryCrossbeamRecvError(e)
}
}
impl std::convert::From<std::sync::mpsc::TryRecvError> for Error {
fn from(e: std::sync::mpsc::TryRecvError) -> Error {
Error::TryRecvError(e)
}
}
impl std::convert::From<crossbeam_channel::RecvTimeoutError> for Error {
fn from(e: crossbeam_channel::RecvTimeoutError) -> Error {
Error::CrossbeamRecvTimeoutError(e)
@ -76,11 +50,6 @@ impl std::convert::From<std::sync::mpsc::RecvTimeoutError> for Error {
Error::RecvTimeoutError(e)
}
}
impl std::convert::From<transaction::TransactionError> for Error {
fn from(e: transaction::TransactionError) -> Error {
Error::TransactionError(e)
}
}
impl std::convert::From<cluster_info::ClusterInfoError> for Error {
fn from(e: cluster_info::ClusterInfoError) -> Error {
Error::ClusterInfoError(e)
@ -101,59 +70,29 @@ impl<T> std::convert::From<std::sync::mpsc::SendError<T>> for Error {
Error::SendError
}
}
impl std::convert::From<Box<dyn Any + Send + 'static>> for Error {
fn from(e: Box<dyn Any + Send + 'static>) -> Error {
Error::JoinError(e)
}
}
impl std::convert::From<std::io::Error> for Error {
fn from(e: std::io::Error) -> Error {
Error::Io(e)
}
}
impl std::convert::From<fs_extra::error::Error> for Error {
fn from(e: fs_extra::error::Error) -> Error {
Error::FsExtra(e)
}
}
impl std::convert::From<serde_json::Error> for Error {
fn from(e: serde_json::Error) -> Error {
Error::Json(e)
}
}
impl std::convert::From<std::net::AddrParseError> for Error {
fn from(e: std::net::AddrParseError) -> Error {
Error::AddrParse(e)
}
}
impl std::convert::From<std::boxed::Box<bincode::ErrorKind>> for Error {
fn from(e: std::boxed::Box<bincode::ErrorKind>) -> Error {
Error::Serialize(e)
}
}
impl std::convert::From<poh_recorder::PohRecorderError> for Error {
fn from(e: poh_recorder::PohRecorderError) -> Error {
Error::PohRecorderError(e)
}
}
impl std::convert::From<blockstore::BlockstoreError> for Error {
fn from(e: blockstore::BlockstoreError) -> Error {
Error::BlockstoreError(e)
}
}
impl std::convert::From<snapshot_utils::SnapshotError> for Error {
fn from(e: snapshot_utils::SnapshotError) -> Error {
Error::SnapshotError(e)
}
}
impl std::convert::From<rand::distributions::weighted::WeightedError> for Error {
fn from(e: rand::distributions::weighted::WeightedError) -> Error {
Error::WeightedIndexError(e)
}
}
impl std::convert::From<duplicate_shred::Error> for Error {
fn from(e: duplicate_shred::Error) -> Error {
Error::DuplicateShredError(e)
impl std::convert::From<GossipError> for Error {
fn from(e: GossipError) -> Error {
Error::GossipError(e)
}
}
@ -163,24 +102,11 @@ mod tests {
use crate::result::Result;
use std::io;
use std::io::Write;
use std::net::SocketAddr;
use std::panic;
use std::sync::mpsc::channel;
use std::sync::mpsc::RecvError;
use std::sync::mpsc::RecvTimeoutError;
use std::thread;
fn addr_parse_error() -> Result<SocketAddr> {
Ok("12fdfasfsafsadfs".parse()?)
}
fn join_error() -> Result<()> {
panic::set_hook(Box::new(|_info| {}));
Ok(thread::spawn(|| panic!("hi")).join()?)
}
fn json_error() -> Result<()> {
Ok(serde_json::from_slice(b"=342{;;;;:}")?)
}
fn send_error() -> Result<()> {
let (s, r) = channel();
drop(r);
@ -190,25 +116,20 @@ mod tests {
#[test]
fn from_test() {
assert_matches!(addr_parse_error(), Err(Error::AddrParse(_)));
assert_matches!(Error::from(RecvError {}), Error::RecvError(_));
assert_matches!(
Error::from(RecvTimeoutError::Timeout),
Error::RecvTimeoutError(_)
);
assert_matches!(send_error(), Err(Error::SendError));
assert_matches!(join_error(), Err(Error::JoinError(_)));
let ioe = io::Error::new(io::ErrorKind::NotFound, "hi");
assert_matches!(Error::from(ioe), Error::Io(_));
}
#[test]
fn fmt_test() {
write!(io::sink(), "{:?}", addr_parse_error()).unwrap();
write!(io::sink(), "{:?}", Error::from(RecvError {})).unwrap();
write!(io::sink(), "{:?}", Error::from(RecvTimeoutError::Timeout)).unwrap();
write!(io::sink(), "{:?}", send_error()).unwrap();
write!(io::sink(), "{:?}", join_error()).unwrap();
write!(io::sink(), "{:?}", json_error()).unwrap();
write!(
io::sink(),
"{:?}",

View File

@ -2,12 +2,10 @@
#![allow(clippy::rc_buffer)]
use crate::{
cluster_info::{compute_retransmit_peers, ClusterInfo, DATA_PLANE_FANOUT},
cluster_info_vote_listener::VerifiedVoteReceiver,
cluster_slots::ClusterSlots,
cluster_slots_service::ClusterSlotsService,
completed_data_sets_service::CompletedDataSetsSender,
contact_info::ContactInfo,
repair_service::DuplicateSlotsResetSender,
repair_service::RepairInfo,
result::{Error, Result},
@ -16,6 +14,10 @@ use crate::{
use crossbeam_channel::{Receiver, Sender};
use lru::LruCache;
use solana_client::rpc_response::SlotUpdate;
use solana_gossip::{
cluster_info::{compute_retransmit_peers, ClusterInfo, DATA_PLANE_FANOUT},
contact_info::ContactInfo,
};
use solana_ledger::shred::{get_shred_slot_index_type, ShredFetchStats};
use solana_ledger::{
blockstore::{Blockstore, CompletedSlotsReceiver},
@ -691,7 +693,7 @@ impl RetransmitStage {
#[cfg(test)]
mod tests {
use super::*;
use crate::contact_info::ContactInfo;
use solana_gossip::contact_info::ContactInfo;
use solana_ledger::blockstore_processor::{process_blockstore, ProcessOptions};
use solana_ledger::create_new_tmp_ledger;
use solana_ledger::genesis_utils::{create_genesis_config, GenesisConfigInfo};

View File

@ -1,8 +1,6 @@
//! The `rpc` module implements the Solana RPC interface.
use crate::{
cluster_info::ClusterInfo,
contact_info::ContactInfo,
rpc_health::*,
send_transaction_service::{SendTransactionService, TransactionInfo},
validator::ValidatorExit,
@ -32,6 +30,7 @@ use solana_client::{
rpc_response::*,
};
use solana_faucet::faucet::request_airdrop_transaction;
use solana_gossip::{cluster_info::ClusterInfo, contact_info::ContactInfo};
use solana_ledger::{
blockstore::Blockstore, blockstore_db::BlockstoreError, get_tmp_ledger_path,
leader_schedule_cache::LeaderScheduleCache,
@ -3719,14 +3718,12 @@ pub(crate) fn create_validator_exit(exit: &Arc<AtomicBool>) -> Arc<RwLock<Valida
#[cfg(test)]
pub mod tests {
use super::{rpc_full::*, rpc_minimal::*, *};
use crate::{
contact_info::ContactInfo,
replay_stage::tests::create_test_transactions_and_populate_blockstore,
};
use crate::replay_stage::tests::create_test_transactions_and_populate_blockstore;
use bincode::deserialize;
use jsonrpc_core::{futures, ErrorCode, MetaIoHandler, Output, Response, Value};
use jsonrpc_core_client::transports::local;
use solana_client::rpc_filter::{Memcmp, MemcmpEncodedBytes};
use solana_gossip::{contact_info::ContactInfo, socketaddr};
use solana_ledger::{
blockstore_meta::PerfSample,
blockstore_processor::fill_blockstore_slot_with_ticks,

View File

@ -1,5 +1,5 @@
use {
crate::cluster_info::ClusterInfo,
solana_gossip::cluster_info::ClusterInfo,
solana_sdk::{clock::Slot, pubkey::Pubkey},
std::{
collections::HashSet,

View File

@ -2,7 +2,6 @@
use crate::{
bigtable_upload_service::BigTableUploadService,
cluster_info::ClusterInfo,
poh_recorder::PohRecorder,
rpc::{rpc_deprecated_v1_7::*, rpc_full::*, rpc_minimal::*, rpc_obsolete_v1_7::*, *},
rpc_health::*,
@ -16,6 +15,7 @@ use jsonrpc_http_server::{
};
use regex::Regex;
use solana_client::rpc_cache::LargestAccountsCache;
use solana_gossip::cluster_info::ClusterInfo;
use solana_ledger::{blockstore::Blockstore, leader_schedule_cache::LeaderScheduleCache};
use solana_metrics::inc_new_counter_info;
use solana_rpc::{
@ -479,10 +479,8 @@ impl JsonRpcService {
#[cfg(test)]
mod tests {
use super::*;
use crate::{
crds_value::{CrdsData, CrdsValue, SnapshotHash},
rpc::create_validator_exit,
};
use crate::rpc::create_validator_exit;
use solana_gossip::crds_value::{CrdsData, CrdsValue, SnapshotHash};
use solana_ledger::{
genesis_utils::{create_genesis_config, GenesisConfigInfo},
get_tmp_ledger_path,

View File

@ -1,7 +1,7 @@
// TODO: Merge this implementation with the one at `banks-server/src/send_transaction_service.rs`
use crate::cluster_info::ClusterInfo;
use crate::poh_recorder::PohRecorder;
use log::*;
use solana_gossip::cluster_info::ClusterInfo;
use solana_metrics::{datapoint_warn, inc_new_counter_info};
use solana_runtime::{bank::Bank, bank_forks::BankForks};
use solana_sdk::{
@ -313,7 +313,7 @@ impl SendTransactionService {
#[cfg(test)]
mod test {
use super::*;
use crate::contact_info::ContactInfo;
use solana_gossip::contact_info::ContactInfo;
use solana_ledger::{
blockstore::Blockstore, get_tmp_ledger_path, leader_schedule_cache::LeaderScheduleCache,
};

View File

@ -1,15 +1,17 @@
use crate::{
cluster_info::{ClusterInfo, ClusterInfoError},
cluster_slots::ClusterSlots,
contact_info::ContactInfo,
repair_response,
repair_service::{OutstandingRepairs, RepairStats},
request_response::RequestResponse,
result::{Error, Result},
weighted_shuffle::weighted_best,
};
use bincode::serialize;
use rand::distributions::{Distribution, WeightedIndex};
use solana_gossip::{
cluster_info::{ClusterInfo, ClusterInfoError},
contact_info::ContactInfo,
weighted_shuffle::weighted_best,
};
use solana_ledger::{
blockstore::Blockstore,
shred::{Nonce, Shred},
@ -607,6 +609,7 @@ impl ServeRepair {
mod tests {
use super::*;
use crate::{repair_response, result::Error};
use solana_gossip::{socketaddr, socketaddr_any};
use solana_ledger::get_tmp_ledger_path;
use solana_ledger::{
blockstore::make_many_slot_entries,

View File

@ -1,4 +1,4 @@
use crate::cluster_info::{ClusterInfo, MAX_SNAPSHOT_HASHES};
use solana_gossip::cluster_info::{ClusterInfo, MAX_SNAPSHOT_HASHES};
use solana_runtime::{snapshot_package::AccountsPackage, snapshot_utils};
use solana_sdk::{clock::Slot, hash::Hash};
use std::{

View File

@ -1,11 +1,10 @@
use {
crate::{
cluster_info::Node,
gossip_service::discover_cluster,
rpc::JsonRpcConfig,
validator::{Validator, ValidatorConfig, ValidatorExit, ValidatorStartProgress},
},
solana_client::rpc_client::RpcClient,
solana_gossip::{cluster_info::Node, gossip_service::discover_cluster, socketaddr},
solana_ledger::{blockstore::create_new_ledger, create_new_tmp_ledger},
solana_net_utils::PortRange,
solana_runtime::{

View File

@ -4,7 +4,6 @@
use crate::{
banking_stage::BankingStage,
broadcast_stage::{BroadcastStage, BroadcastStageType, RetransmitSlotsReceiver},
cluster_info::ClusterInfo,
cluster_info_vote_listener::{
ClusterInfoVoteListener, GossipDuplicateConfirmedSlotsSender, GossipVerifiedVoteHashSender,
VerifiedVoteSender, VoteTracker,
@ -15,6 +14,7 @@ use crate::{
sigverify_stage::SigVerifyStage,
};
use crossbeam_channel::unbounded;
use solana_gossip::cluster_info::ClusterInfo;
use solana_ledger::{blockstore::Blockstore, blockstore_processor::TransactionStatusSender};
use solana_rpc::{
optimistically_confirmed_bank_tracker::BankNotificationSender,

View File

@ -5,7 +5,6 @@ use crate::{
accounts_hash_verifier::AccountsHashVerifier,
broadcast_stage::RetransmitSlotsSender,
cache_block_time_service::CacheBlockTimeSender,
cluster_info::ClusterInfo,
cluster_info_vote_listener::{
GossipDuplicateConfirmedSlotsReceiver, GossipVerifiedVoteHashReceiver,
VerifiedVoteReceiver, VoteTracker,
@ -24,6 +23,7 @@ use crate::{
snapshot_packager_service::PendingSnapshotPackage,
};
use crossbeam_channel::unbounded;
use solana_gossip::cluster_info::ClusterInfo;
use solana_ledger::{
blockstore::{Blockstore, CompletedSlotsReceiver},
blockstore_processor::TransactionStatusSender,
@ -338,11 +338,9 @@ impl Tvu {
#[cfg(test)]
pub mod tests {
use super::*;
use crate::{
banking_stage::create_test_recorder,
cluster_info::{ClusterInfo, Node},
};
use crate::banking_stage::create_test_recorder;
use serial_test::serial;
use solana_gossip::cluster_info::{ClusterInfo, Node};
use solana_ledger::{
blockstore::BlockstoreSignals,
create_new_tmp_ledger,

View File

@ -3,15 +3,9 @@
use crate::{
broadcast_stage::BroadcastStageType,
cache_block_time_service::{CacheBlockTimeSender, CacheBlockTimeService},
cluster_info::{
ClusterInfo, Node, DEFAULT_CONTACT_DEBUG_INTERVAL_MILLIS,
DEFAULT_CONTACT_SAVE_INTERVAL_MILLIS,
},
cluster_info_vote_listener::VoteTracker,
completed_data_sets_service::CompletedDataSetsService,
consensus::{reconcile_blockstore_roots_with_tower, Tower},
contact_info::ContactInfo,
gossip_service::GossipService,
poh_recorder::{PohRecorder, GRACE_TICKS_FACTOR, MAX_GRACE_SLOTS},
poh_service::{self, PohService},
rewards_recorder_service::{RewardsRecorderSender, RewardsRecorderService},
@ -28,6 +22,14 @@ use crate::{
};
use crossbeam_channel::{bounded, unbounded};
use rand::{thread_rng, Rng};
use solana_gossip::{
cluster_info::{
ClusterInfo, Node, DEFAULT_CONTACT_DEBUG_INTERVAL_MILLIS,
DEFAULT_CONTACT_SAVE_INTERVAL_MILLIS,
},
contact_info::ContactInfo,
gossip_service::GossipService,
};
use solana_ledger::{
bank_forks_utils,
blockstore::{Blockstore, BlockstoreSignals, CompletedSlotsReceiver, PurgeType},

View File

@ -1,7 +1,5 @@
use crate::{
cluster_info_vote_listener::VerifiedLabelVotePacketsReceiver, crds_value::CrdsValueLabel,
result::Result,
};
use crate::{cluster_info_vote_listener::VerifiedLabelVotePacketsReceiver, result::Result};
use solana_gossip::crds_value::CrdsValueLabel;
use solana_perf::packet::Packets;
use solana_sdk::clock::Slot;
use std::{

View File

@ -1,135 +0,0 @@
//! The `weighted_shuffle` module provides an iterator over shuffled weights.
use itertools::Itertools;
use num_traits::{FromPrimitive, ToPrimitive};
use rand::{Rng, SeedableRng};
use rand_chacha::ChaChaRng;
use std::iter;
use std::ops::Div;
/// Returns a list of indexes shuffled based on the input weights
/// Note - The sum of all weights must not exceed `u64::MAX`
pub fn weighted_shuffle<T>(weights: &[T], seed: [u8; 32]) -> Vec<usize>
where
T: Copy + PartialOrd + iter::Sum + Div<T, Output = T> + FromPrimitive + ToPrimitive,
{
let total_weight: T = weights.iter().copied().sum();
let mut rng = ChaChaRng::from_seed(seed);
weights
.iter()
.enumerate()
.map(|(i, v)| {
// This generates an "inverse" weight but it avoids floating point math
let x = (total_weight / *v)
.to_u64()
.expect("values > u64::max are not supported");
(
i,
// capture the u64 into u128s to prevent overflow
rng.gen_range(1, u128::from(std::u16::MAX)) * u128::from(x),
)
})
// sort in ascending order
.sorted_by(|(_, l_val), (_, r_val)| l_val.cmp(r_val))
.map(|x| x.0)
.collect()
}
/// Returns the highest index after computing a weighted shuffle.
/// Saves doing any sorting for O(n) max calculation.
// TODO: Remove in favor of rand::distributions::WeightedIndex.
pub fn weighted_best(weights_and_indexes: &[(u64, usize)], seed: [u8; 32]) -> usize {
if weights_and_indexes.is_empty() {
return 0;
}
let mut rng = ChaChaRng::from_seed(seed);
let total_weight: u64 = weights_and_indexes.iter().map(|x| x.0).sum();
let mut lowest_weight = std::u128::MAX;
let mut best_index = 0;
for v in weights_and_indexes {
// This generates an "inverse" weight but it avoids floating point math
let x = (total_weight / v.0)
.to_u64()
.expect("values > u64::max are not supported");
// capture the u64 into u128s to prevent overflow
let computed_weight = rng.gen_range(1, u128::from(std::u16::MAX)) * u128::from(x);
// The highest input weight maps to the lowest computed weight
if computed_weight < lowest_weight {
lowest_weight = computed_weight;
best_index = v.1;
}
}
best_index
}
#[cfg(test)]
mod tests {
use super::*;
#[test]
fn test_weighted_shuffle_iterator() {
let mut test_set = [0; 6];
let mut count = 0;
let shuffle = weighted_shuffle(&[50, 10, 2, 1, 1, 1], [0x5a; 32]);
shuffle.into_iter().for_each(|x| {
assert_eq!(test_set[x], 0);
test_set[x] = 1;
count += 1;
});
assert_eq!(count, 6);
}
#[test]
fn test_weighted_shuffle_iterator_large() {
let mut test_set = [0; 100];
let mut test_weights = vec![0; 100];
(0..100).for_each(|i| test_weights[i] = (i + 1) as u64);
let mut count = 0;
let shuffle = weighted_shuffle(&test_weights, [0xa5; 32]);
shuffle.into_iter().for_each(|x| {
assert_eq!(test_set[x], 0);
test_set[x] = 1;
count += 1;
});
assert_eq!(count, 100);
}
#[test]
fn test_weighted_shuffle_compare() {
let shuffle = weighted_shuffle(&[50, 10, 2, 1, 1, 1], [0x5a; 32]);
let shuffle1 = weighted_shuffle(&[50, 10, 2, 1, 1, 1], [0x5a; 32]);
shuffle1
.into_iter()
.zip(shuffle.into_iter())
.for_each(|(x, y)| {
assert_eq!(x, y);
});
}
#[test]
fn test_weighted_shuffle_imbalanced() {
let mut weights = vec![std::u32::MAX as u64; 3];
weights.push(1);
let shuffle = weighted_shuffle(&weights, [0x5a; 32]);
shuffle.into_iter().for_each(|x| {
if x == weights.len() - 1 {
assert_eq!(weights[x], 1);
} else {
assert_eq!(weights[x], std::u32::MAX as u64);
}
});
}
#[test]
fn test_weighted_best() {
let weights_and_indexes: Vec<_> = vec![100u64, 1000, 10_000, 10]
.into_iter()
.enumerate()
.map(|(i, weight)| (weight, i))
.collect();
let best_index = weighted_best(&weights_and_indexes, [0x5b; 32]);
assert_eq!(best_index, 2);
}
}

View File

@ -2,7 +2,6 @@
//! blockstore and retransmitting where required
//!
use crate::{
cluster_info::ClusterInfo,
cluster_info_vote_listener::VerifiedVoteReceiver,
cluster_slots::ClusterSlots,
completed_data_sets_service::CompletedDataSetsSender,
@ -17,6 +16,7 @@ use crossbeam_channel::{
use rayon::iter::IntoParallelRefMutIterator;
use rayon::iter::ParallelIterator;
use rayon::ThreadPool;
use solana_gossip::cluster_info::ClusterInfo;
use solana_ledger::{
blockstore::{self, Blockstore, BlockstoreInsertionMetrics, MAX_DATA_SHREDS_PER_SLOT},
leader_schedule_cache::LeaderScheduleCache,
@ -614,7 +614,7 @@ impl WindowService {
#[cfg(test)]
mod test {
use super::*;
use crate::contact_info::ContactInfo;
use solana_gossip::contact_info::ContactInfo;
use solana_ledger::{
blockstore::{make_many_slot_entries, Blockstore},
entry::{create_ticks, Entry},