Ingest votes from gossip into fork choice (#16560)

This commit is contained in:
carllin
2021-04-21 14:40:35 -07:00
committed by GitHub
parent 63957f0677
commit 4c94f8933f
10 changed files with 1156 additions and 336 deletions

View File

@ -1,154 +0,0 @@
use crate::{
consensus::{ComputedBankState, Tower},
fork_choice::ForkChoice,
progress_map::{ForkStats, ProgressMap},
};
use solana_runtime::{bank::Bank, bank_forks::BankForks};
use solana_sdk::{clock::Slot, timing};
use std::time::Instant;
use std::{
collections::{HashMap, HashSet},
sync::{Arc, RwLock},
};
#[derive(Default)]
pub struct BankWeightForkChoice {}
impl ForkChoice for BankWeightForkChoice {
fn compute_bank_stats(
&mut self,
bank: &Bank,
_tower: &Tower,
progress: &mut ProgressMap,
computed_bank_state: &ComputedBankState,
) {
let bank_slot = bank.slot();
// Only time progress map should be missing a bank slot
// is if this node was the leader for this slot as those banks
// are not replayed in replay_active_banks()
let parent_weight = bank
.parent()
.and_then(|b| progress.get(&b.slot()))
.map(|x| x.fork_stats.fork_weight)
.unwrap_or(0);
let stats = progress
.get_fork_stats_mut(bank_slot)
.expect("All frozen banks must exist in the Progress map");
let ComputedBankState { bank_weight, .. } = computed_bank_state;
stats.weight = *bank_weight;
stats.fork_weight = stats.weight + parent_weight;
}
// Returns:
// 1) The heaviest overall bank
// 2) The heaviest bank on the same fork as the last vote (doesn't require a
// switching proof to vote for)
fn select_forks(
&self,
frozen_banks: &[Arc<Bank>],
tower: &Tower,
progress: &ProgressMap,
ancestors: &HashMap<u64, HashSet<u64>>,
_bank_forks: &RwLock<BankForks>,
) -> (Arc<Bank>, Option<Arc<Bank>>) {
let tower_start = Instant::now();
assert!(!frozen_banks.is_empty());
let num_frozen_banks = frozen_banks.len();
trace!("frozen_banks {}", frozen_banks.len());
let num_old_banks = frozen_banks
.iter()
.filter(|b| b.slot() < tower.root())
.count();
let last_voted_slot = tower.last_voted_slot();
let mut heaviest_bank_on_same_fork = None;
let mut heaviest_same_fork_weight = 0;
let stats: Vec<&ForkStats> = frozen_banks
.iter()
.map(|bank| {
// Only time progress map should be missing a bank slot
// is if this node was the leader for this slot as those banks
// are not replayed in replay_active_banks()
let stats = progress
.get_fork_stats(bank.slot())
.expect("All frozen banks must exist in the Progress map");
if let Some(last_voted_slot) = last_voted_slot {
if ancestors
.get(&bank.slot())
.expect("Entry in frozen banks must exist in ancestors")
.contains(&last_voted_slot)
{
// Descendant of last vote cannot be locked out
assert!(!stats.is_locked_out);
// ancestors(slot) should not contain the slot itself,
// so we should never get the same bank as the last vote
assert_ne!(bank.slot(), last_voted_slot);
// highest weight, lowest slot first. frozen_banks is sorted
// from least slot to greatest slot, so if two banks have
// the same fork weight, the lower slot will be picked
if stats.fork_weight > heaviest_same_fork_weight {
heaviest_bank_on_same_fork = Some(bank.clone());
heaviest_same_fork_weight = stats.fork_weight;
}
}
}
stats
})
.collect();
let num_not_recent = stats.iter().filter(|s| !s.is_recent).count();
let num_has_voted = stats.iter().filter(|s| s.has_voted).count();
let num_empty = stats.iter().filter(|s| s.is_empty).count();
let num_threshold_failure = stats.iter().filter(|s| !s.vote_threshold).count();
let num_votable_threshold_failure = stats
.iter()
.filter(|s| s.is_recent && !s.has_voted && !s.vote_threshold)
.count();
let mut candidates: Vec<_> = frozen_banks.iter().zip(stats.iter()).collect();
//highest weight, lowest slot first
candidates.sort_by_key(|b| (b.1.fork_weight, 0i64 - b.0.slot() as i64));
let rv = candidates
.last()
.expect("frozen banks was nonempty so candidates must also be nonempty");
let ms = timing::duration_as_ms(&tower_start.elapsed());
let weights: Vec<(u128, u64, u64)> = candidates
.iter()
.map(|x| (x.1.weight, x.0.slot(), x.1.block_height))
.collect();
debug!(
"@{:?} tower duration: {:?} len: {}/{} weights: {:?}",
timing::timestamp(),
ms,
candidates.len(),
stats.iter().filter(|s| !s.has_voted).count(),
weights,
);
datapoint_debug!(
"replay_stage-select_forks",
("frozen_banks", num_frozen_banks as i64, i64),
("not_recent", num_not_recent as i64, i64),
("has_voted", num_has_voted as i64, i64),
("old_banks", num_old_banks as i64, i64),
("empty_banks", num_empty as i64, i64),
("threshold_failure", num_threshold_failure as i64, i64),
(
"votable_threshold_failure",
num_votable_threshold_failure as i64,
i64
),
("tower_duration", ms as i64, i64),
);
(rv.0.clone(), heaviest_bank_on_same_fork)
}
fn mark_fork_invalid_candidate(&mut self, _invalid_slot: Slot) {}
fn mark_fork_valid_candidate(&mut self, _valid_slots: &[Slot]) {}
}

View File

@ -5,7 +5,7 @@ use crate::{
use solana_sdk::{clock::Slot, hash::Hash};
use std::collections::{BTreeMap, HashMap, HashSet};
pub type GossipDuplicateConfirmedSlots = BTreeMap<Slot, Hash>;
pub(crate) type GossipDuplicateConfirmedSlots = BTreeMap<Slot, Hash>;
type SlotStateHandler = fn(Slot, &Hash, Option<&Hash>, bool, bool) -> Vec<ResultingStateChange>;
#[derive(PartialEq, Debug)]

View File

@ -1,4 +1,7 @@
use crate::progress_map::{LockoutIntervals, ProgressMap};
use crate::{
latest_validator_votes_for_frozen_banks::LatestValidatorVotesForFrozenBanks,
progress_map::{LockoutIntervals, ProgressMap},
};
use chrono::prelude::*;
use solana_ledger::{ancestor_iterator::AncestorIterator, blockstore::Blockstore, blockstore_db};
use solana_measure::measure::Measure;
@ -95,7 +98,6 @@ pub(crate) struct ComputedBankState {
// Tree of intervals of lockouts of the form [slot, slot + slot.lockout],
// keyed by end of the range
pub lockout_intervals: LockoutIntervals,
pub pubkey_votes: Arc<PubkeyVotes>,
}
#[frozen_abi(digest = "Eay84NBbJqiMBfE7HHH2o6e51wcvoU79g8zCi5sw6uj3")]
@ -219,6 +221,8 @@ impl Tower {
bank_slot: Slot,
vote_accounts: F,
ancestors: &HashMap<Slot, HashSet<Slot>>,
get_frozen_hash: impl Fn(Slot) -> Option<Hash>,
latest_validator_votes_for_frozen_banks: &mut LatestValidatorVotesForFrozenBanks,
) -> ComputedBankState
where
F: IntoIterator<Item = (Pubkey, (u64, ArcVoteAccount))>,
@ -230,7 +234,6 @@ impl Tower {
// Tree of intervals of lockouts of the form [slot, slot + slot.lockout],
// keyed by end of the range
let mut lockout_intervals = LockoutIntervals::new();
let mut pubkey_votes = vec![];
for (key, (voted_stake, account)) in vote_accounts {
if voted_stake == 0 {
continue;
@ -277,8 +280,12 @@ impl Tower {
let start_root = vote_state.root_slot;
// Add the last vote to update the `heaviest_subtree_fork_choice`
if let Some(last_voted_slot) = vote_state.last_voted_slot() {
pubkey_votes.push((key, last_voted_slot));
if let Some(last_landed_voted_slot) = vote_state.last_voted_slot() {
latest_validator_votes_for_frozen_banks.check_add_vote(
key,
last_landed_voted_slot,
get_frozen_hash(last_landed_voted_slot),
);
}
vote_state.process_slot_vote_unchecked(bank_slot);
@ -341,7 +348,6 @@ impl Tower {
total_stake,
bank_weight,
lockout_intervals,
pubkey_votes: Arc::new(pubkey_votes),
}
}
@ -1272,11 +1278,13 @@ pub mod test {
use super::*;
use crate::{
cluster_info_vote_listener::VoteTracker,
cluster_slot_state_verifier::GossipDuplicateConfirmedSlots,
cluster_slots::ClusterSlots,
fork_choice::SelectVoteAndResetForkResult,
heaviest_subtree_fork_choice::HeaviestSubtreeForkChoice,
heaviest_subtree_fork_choice::{HeaviestSubtreeForkChoice, SlotHashKey},
progress_map::{DuplicateStats, ForkProgress},
replay_stage::{HeaviestForkFailures, ReplayStage},
unfrozen_gossip_verified_vote_hashes::UnfrozenGossipVerifiedVoteHashes,
};
use solana_ledger::{blockstore::make_slot_entries, get_tmp_ledger_path};
use solana_runtime::{
@ -1300,7 +1308,7 @@ pub mod test {
vote_transaction,
};
use std::{
collections::{BTreeMap, HashMap},
collections::HashMap,
fs::{remove_file, OpenOptions},
io::{Read, Seek, SeekFrom, Write},
sync::RwLock,
@ -1315,6 +1323,7 @@ pub mod test {
pub bank_forks: RwLock<BankForks>,
pub progress: ProgressMap,
pub heaviest_subtree_fork_choice: HeaviestSubtreeForkChoice,
pub latest_validator_votes_for_frozen_banks: LatestValidatorVotesForFrozenBanks,
}
impl VoteSimulator {
@ -1334,6 +1343,8 @@ pub mod test {
bank_forks: RwLock::new(bank_forks),
progress,
heaviest_subtree_fork_choice,
latest_validator_votes_for_frozen_banks:
LatestValidatorVotesForFrozenBanks::default(),
}
}
pub(crate) fn fill_bank_forks(
@ -1415,6 +1426,7 @@ pub mod test {
&ClusterSlots::default(),
&self.bank_forks,
&mut self.heaviest_subtree_fork_choice,
&mut self.latest_validator_votes_for_frozen_banks,
);
let vote_bank = self
@ -1461,8 +1473,8 @@ pub mod test {
&AbsRequestSender::default(),
None,
&mut self.heaviest_subtree_fork_choice,
&mut BTreeMap::new(),
&mut BTreeMap::new(),
&mut GossipDuplicateConfirmedSlots::default(),
&mut UnfrozenGossipVerifiedVoteHashes::default(),
&mut true,
&mut Vec::new(),
)
@ -2130,24 +2142,34 @@ pub mod test {
//two accounts voting for slot 0 with 1 token staked
let mut accounts = gen_stakes(&[(1, &[0]), (1, &[0])]);
accounts.sort_by_key(|(pk, _)| *pk);
let account_latest_votes: PubkeyVotes =
accounts.iter().map(|(pubkey, _)| (*pubkey, 0)).collect();
let account_latest_votes: Vec<(Pubkey, SlotHashKey)> = accounts
.iter()
.map(|(pubkey, _)| (*pubkey, (0, Hash::default())))
.collect();
let ancestors = vec![(1, vec![0].into_iter().collect()), (0, HashSet::new())]
.into_iter()
.collect();
let mut latest_validator_votes_for_frozen_banks =
LatestValidatorVotesForFrozenBanks::default();
let ComputedBankState {
voted_stakes,
total_stake,
bank_weight,
pubkey_votes,
..
} = Tower::collect_vote_lockouts(&Pubkey::default(), 1, accounts.into_iter(), &ancestors);
} = Tower::collect_vote_lockouts(
&Pubkey::default(),
1,
accounts.into_iter(),
&ancestors,
|_| Some(Hash::default()),
&mut latest_validator_votes_for_frozen_banks,
);
assert_eq!(voted_stakes[&0], 2);
assert_eq!(total_stake, 2);
let mut pubkey_votes = Arc::try_unwrap(pubkey_votes).unwrap();
pubkey_votes.sort();
assert_eq!(pubkey_votes, account_latest_votes);
let mut new_votes = latest_validator_votes_for_frozen_banks.take_votes_dirty_set(0);
new_votes.sort();
assert_eq!(new_votes, account_latest_votes);
// Each account has 1 vote in it. After simulating a vote in collect_vote_lockouts,
// the account will have 2 votes, with lockout 2 + 4 = 6. So expected weight for
@ -2160,9 +2182,14 @@ pub mod test {
//two accounts voting for slots 0..MAX_LOCKOUT_HISTORY with 1 token staked
let mut accounts = gen_stakes(&[(1, &votes), (1, &votes)]);
accounts.sort_by_key(|(pk, _)| *pk);
let account_latest_votes: PubkeyVotes = accounts
let account_latest_votes: Vec<(Pubkey, SlotHashKey)> = accounts
.iter()
.map(|(pubkey, _)| (*pubkey, (MAX_LOCKOUT_HISTORY - 1) as Slot))
.map(|(pubkey, _)| {
(
*pubkey,
((MAX_LOCKOUT_HISTORY - 1) as Slot, Hash::default()),
)
})
.collect();
let mut tower = Tower::new_for_tests(0, 0.67);
let mut ancestors = HashMap::new();
@ -2184,16 +2211,19 @@ pub mod test {
+ root_weight;
let expected_bank_weight = 2 * vote_account_expected_weight;
assert_eq!(tower.lockouts.root_slot, Some(0));
let mut latest_validator_votes_for_frozen_banks =
LatestValidatorVotesForFrozenBanks::default();
let ComputedBankState {
voted_stakes,
bank_weight,
pubkey_votes,
..
} = Tower::collect_vote_lockouts(
&Pubkey::default(),
MAX_LOCKOUT_HISTORY as u64,
accounts.into_iter(),
&ancestors,
|_| Some(Hash::default()),
&mut latest_validator_votes_for_frozen_banks,
);
for i in 0..MAX_LOCKOUT_HISTORY {
assert_eq!(voted_stakes[&(i as u64)], 2);
@ -2201,9 +2231,9 @@ pub mod test {
// should be the sum of all the weights for root
assert_eq!(bank_weight, expected_bank_weight);
let mut pubkey_votes = Arc::try_unwrap(pubkey_votes).unwrap();
pubkey_votes.sort();
assert_eq!(pubkey_votes, account_latest_votes);
let mut new_votes = latest_validator_votes_for_frozen_banks.take_votes_dirty_set(root.slot);
new_votes.sort();
assert_eq!(new_votes, account_latest_votes);
}
#[test]
@ -2500,6 +2530,8 @@ pub mod test {
vote_to_evaluate,
accounts.clone().into_iter(),
&ancestors,
|_| None,
&mut LatestValidatorVotesForFrozenBanks::default(),
);
assert!(tower.check_vote_stake_threshold(vote_to_evaluate, &voted_stakes, total_stake,));
@ -2516,6 +2548,8 @@ pub mod test {
vote_to_evaluate,
accounts.into_iter(),
&ancestors,
|_| None,
&mut LatestValidatorVotesForFrozenBanks::default(),
);
assert!(!tower.check_vote_stake_threshold(vote_to_evaluate, &voted_stakes, total_stake,));
}

View File

@ -1,5 +1,6 @@
use crate::{
consensus::{ComputedBankState, SwitchForkDecision, Tower},
consensus::{SwitchForkDecision, Tower},
latest_validator_votes_for_frozen_banks::LatestValidatorVotesForFrozenBanks,
progress_map::ProgressMap,
replay_stage::HeaviestForkFailures,
};
@ -21,8 +22,7 @@ pub(crate) trait ForkChoice {
&mut self,
bank: &Bank,
tower: &Tower,
progress: &mut ProgressMap,
computed_bank_state: &ComputedBankState,
latest_validator_votes_for_frozen_banks: &mut LatestValidatorVotesForFrozenBanks,
);
// Returns:

View File

@ -1,8 +1,7 @@
use crate::{
consensus::{ComputedBankState, Tower},
fork_choice::ForkChoice,
progress_map::ProgressMap,
tree_diff::TreeDiff,
consensus::Tower, fork_choice::ForkChoice,
latest_validator_votes_for_frozen_banks::LatestValidatorVotesForFrozenBanks,
progress_map::ProgressMap, tree_diff::TreeDiff,
};
use solana_measure::measure::Measure;
use solana_runtime::{bank::Bank, bank_forks::BankForks, epoch_stakes::EpochStakes};
@ -22,7 +21,7 @@ use std::{
use trees::{Tree, TreeWalk};
pub type ForkWeight = u64;
type SlotHashKey = (Slot, Hash);
pub type SlotHashKey = (Slot, Hash);
type UpdateOperations = BTreeMap<(SlotHashKey, UpdateLabel), UpdateOperation>;
const MAX_ROOT_PRINT_SECONDS: u64 = 30;
@ -163,6 +162,10 @@ impl HeaviestSubtreeForkChoice {
heaviest_subtree_fork_choice
}
pub fn contains_block(&self, key: &SlotHashKey) -> bool {
self.fork_infos.contains_key(key)
}
pub fn best_slot(&self, key: &SlotHashKey) -> Option<SlotHashKey> {
self.fork_infos
.get(key)
@ -529,6 +532,15 @@ impl HeaviestSubtreeForkChoice {
let (pubkey, new_vote_slot_hash) = pubkey_vote.borrow();
let (new_vote_slot, new_vote_hash) = *new_vote_slot_hash;
if new_vote_slot < self.root.0 {
// If the new vote is less than the root we can ignore it. This is because there
// are two cases. Either:
// 1) The validator's latest vote was bigger than the new vote, so we can ignore it
// 2) The validator's latest vote was less than the new vote, then the validator's latest
// vote was also less than root. This means either every node in the current tree has the
// validators stake counted toward it (if the latest vote was an ancestor of the current root),
// OR every node doesn't have this validator's vote counting toward it (if the latest vote
// was not an ancestor of the current root). Thus this validator is essentially a no-op
// and won't affect fork choice.
continue;
}
@ -744,29 +756,14 @@ impl ForkChoice for HeaviestSubtreeForkChoice {
&mut self,
bank: &Bank,
_tower: &Tower,
progress: &mut ProgressMap,
computed_bank_state: &ComputedBankState,
latest_validator_votes_for_frozen_banks: &mut LatestValidatorVotesForFrozenBanks,
) {
let ComputedBankState { pubkey_votes, .. } = computed_bank_state;
let mut start = Measure::start("compute_bank_stats_time");
// Update `heaviest_subtree_fork_choice` to find the best fork to build on
let root = self.root.0;
let new_votes = latest_validator_votes_for_frozen_banks.take_votes_dirty_set(root);
let (best_overall_slot, best_overall_hash) = self.add_votes(
pubkey_votes.iter().filter_map(|(pubkey, slot)| {
if *slot >= root {
Some((
*pubkey,
(
*slot,
progress
.get_hash(*slot)
.expect("Votes for ancestors must exist in progress map"),
),
))
} else {
None
}
}),
new_votes.into_iter(),
bank.epoch_stakes_map(),
bank.epoch_schedule(),
);

View File

@ -0,0 +1,351 @@
use crate::heaviest_subtree_fork_choice::SlotHashKey;
use solana_sdk::{clock::Slot, hash::Hash, pubkey::Pubkey};
use std::collections::{hash_map::Entry, HashMap};
#[derive(Default)]
pub(crate) struct LatestValidatorVotesForFrozenBanks {
// TODO: Clean outdated/unstaked pubkeys from this list.
max_frozen_votes: HashMap<Pubkey, (Slot, Vec<Hash>)>,
// Pubkeys that had their `max_frozen_votes` updated since the last
// fork choice update
fork_choice_dirty_set: HashMap<Pubkey, (Slot, Vec<Hash>)>,
}
impl LatestValidatorVotesForFrozenBanks {
// `frozen_hash.is_some()` if the bank with slot == `vote_slot` is frozen
// Returns whether the vote was actually added, and the latest voted frozen slot
pub(crate) fn check_add_vote(
&mut self,
vote_pubkey: Pubkey,
vote_slot: Slot,
frozen_hash: Option<Hash>,
) -> (bool, Option<Slot>) {
let max_frozen_votes_entry = self.max_frozen_votes.entry(vote_pubkey);
if let Some(frozen_hash) = frozen_hash {
match max_frozen_votes_entry {
Entry::Occupied(mut occupied_entry) => {
let (latest_frozen_vote_slot, latest_frozen_vote_hashes) =
occupied_entry.get_mut();
if vote_slot > *latest_frozen_vote_slot {
self.fork_choice_dirty_set
.insert(vote_pubkey, (vote_slot, vec![frozen_hash]));
*latest_frozen_vote_slot = vote_slot;
*latest_frozen_vote_hashes = vec![frozen_hash];
return (true, Some(vote_slot));
} else if vote_slot == *latest_frozen_vote_slot
&& !latest_frozen_vote_hashes.contains(&frozen_hash)
{
let (_, dirty_frozen_hashes) =
self.fork_choice_dirty_set.entry(vote_pubkey).or_default();
assert!(!dirty_frozen_hashes.contains(&frozen_hash));
dirty_frozen_hashes.push(frozen_hash);
latest_frozen_vote_hashes.push(frozen_hash);
return (true, Some(vote_slot));
} else {
// We have newer votes for this validator, we don't care about this vote
return (false, Some(*latest_frozen_vote_slot));
}
}
Entry::Vacant(vacant_entry) => {
vacant_entry.insert((vote_slot, vec![frozen_hash]));
self.fork_choice_dirty_set
.insert(vote_pubkey, (vote_slot, vec![frozen_hash]));
return (true, Some(vote_slot));
}
}
}
// Non-frozen banks are not inserted because we only track frozen votes in this
// struct
(
false,
match max_frozen_votes_entry {
Entry::Occupied(occupied_entry) => Some(occupied_entry.get().0),
Entry::Vacant(_) => None,
},
)
}
pub(crate) fn take_votes_dirty_set(&mut self, root: Slot) -> Vec<(Pubkey, SlotHashKey)> {
let new_votes = std::mem::take(&mut self.fork_choice_dirty_set);
new_votes
.into_iter()
.filter(|(_, (slot, _))| *slot >= root)
.flat_map(|(pk, (slot, hashes))| {
hashes
.into_iter()
.map(|hash| (pk, (slot, hash)))
.collect::<Vec<(Pubkey, SlotHashKey)>>()
})
.collect()
}
}
#[cfg(test)]
mod tests {
use super::*;
#[test]
fn test_latest_validator_votes_for_frozen_banks_check_add_vote() {
let mut latest_validator_votes_for_frozen_banks =
LatestValidatorVotesForFrozenBanks::default();
// Case 1: Non-frozen banks shouldn't be added
let vote_pubkey = Pubkey::new_unique();
let mut vote_slot = 1;
let frozen_hash = None;
assert_eq!(
latest_validator_votes_for_frozen_banks.check_add_vote(
vote_pubkey,
vote_slot,
frozen_hash,
),
// Non-frozen bank isn't inserted, so should return None for
// the highest voted frozen slot
(false, None)
);
assert!(latest_validator_votes_for_frozen_banks
.max_frozen_votes
.is_empty());
assert!(latest_validator_votes_for_frozen_banks
.fork_choice_dirty_set
.is_empty());
// Case 2: Frozen vote should be added, but the same vote added again
// shouldn't update state
let num_repeated_iterations = 3;
let frozen_hash = Some(Hash::new_unique());
for i in 0..num_repeated_iterations {
let expected_result = if i == 0 {
(true, Some(vote_slot))
} else {
(false, Some(vote_slot))
};
assert_eq!(
latest_validator_votes_for_frozen_banks.check_add_vote(
vote_pubkey,
vote_slot,
frozen_hash,
),
expected_result
);
assert_eq!(
*latest_validator_votes_for_frozen_banks
.max_frozen_votes
.get(&vote_pubkey)
.unwrap(),
(vote_slot, vec![frozen_hash.unwrap()])
);
assert_eq!(
*latest_validator_votes_for_frozen_banks
.fork_choice_dirty_set
.get(&vote_pubkey)
.unwrap(),
(vote_slot, vec![frozen_hash.unwrap()])
);
}
// Case 3: Adding duplicate vote for same slot should update the state
let duplicate_frozen_hash = Some(Hash::new_unique());
let all_frozen_hashes = vec![frozen_hash.unwrap(), duplicate_frozen_hash.unwrap()];
assert_eq!(
latest_validator_votes_for_frozen_banks.check_add_vote(
vote_pubkey,
vote_slot,
duplicate_frozen_hash,
),
(true, Some(vote_slot))
);
assert_eq!(
*latest_validator_votes_for_frozen_banks
.max_frozen_votes
.get(&vote_pubkey)
.unwrap(),
(vote_slot, all_frozen_hashes.clone())
);
assert_eq!(
*latest_validator_votes_for_frozen_banks
.fork_choice_dirty_set
.get(&vote_pubkey)
.unwrap(),
(vote_slot, all_frozen_hashes.clone())
);
// Case 4: Adding duplicate vote that is not frozen should not update the state
let frozen_hash = None;
assert_eq!(
latest_validator_votes_for_frozen_banks.check_add_vote(
vote_pubkey,
vote_slot,
frozen_hash,
),
(false, Some(vote_slot))
);
assert_eq!(
*latest_validator_votes_for_frozen_banks
.max_frozen_votes
.get(&vote_pubkey)
.unwrap(),
(vote_slot, all_frozen_hashes.clone())
);
assert_eq!(
*latest_validator_votes_for_frozen_banks
.fork_choice_dirty_set
.get(&vote_pubkey)
.unwrap(),
(vote_slot, all_frozen_hashes.clone())
);
// Case 5: Adding a vote for a new higher slot that is not yet frozen
// should not update the state
let frozen_hash = None;
let old_vote_slot = vote_slot;
vote_slot += 1;
assert_eq!(
latest_validator_votes_for_frozen_banks.check_add_vote(
vote_pubkey,
vote_slot,
frozen_hash,
),
(false, Some(old_vote_slot))
);
assert_eq!(
*latest_validator_votes_for_frozen_banks
.max_frozen_votes
.get(&vote_pubkey)
.unwrap(),
(old_vote_slot, all_frozen_hashes.clone())
);
assert_eq!(
*latest_validator_votes_for_frozen_banks
.fork_choice_dirty_set
.get(&vote_pubkey)
.unwrap(),
(old_vote_slot, all_frozen_hashes)
);
// Case 6: Adding a vote for a new higher slot that *is* frozen
// should upate the state
let frozen_hash = Some(Hash::new_unique());
assert_eq!(
latest_validator_votes_for_frozen_banks.check_add_vote(
vote_pubkey,
vote_slot,
frozen_hash,
),
(true, Some(vote_slot))
);
assert_eq!(
*latest_validator_votes_for_frozen_banks
.max_frozen_votes
.get(&vote_pubkey)
.unwrap(),
(vote_slot, vec![frozen_hash.unwrap()])
);
assert_eq!(
*latest_validator_votes_for_frozen_banks
.fork_choice_dirty_set
.get(&vote_pubkey)
.unwrap(),
(vote_slot, vec![frozen_hash.unwrap()])
);
// Case 7: Adding a vote for a new pubkey should also update the state
vote_slot += 1;
let frozen_hash = Some(Hash::new_unique());
let vote_pubkey = Pubkey::new_unique();
assert_eq!(
latest_validator_votes_for_frozen_banks.check_add_vote(
vote_pubkey,
vote_slot,
frozen_hash,
),
(true, Some(vote_slot))
);
assert_eq!(
*latest_validator_votes_for_frozen_banks
.max_frozen_votes
.get(&vote_pubkey)
.unwrap(),
(vote_slot, vec![frozen_hash.unwrap()])
);
assert_eq!(
*latest_validator_votes_for_frozen_banks
.fork_choice_dirty_set
.get(&vote_pubkey)
.unwrap(),
(vote_slot, vec![frozen_hash.unwrap()])
);
}
#[test]
fn test_latest_validator_votes_for_frozen_banks_take_votes_dirty_set() {
let mut latest_validator_votes_for_frozen_banks =
LatestValidatorVotesForFrozenBanks::default();
let num_validators = 10;
let setup_dirty_set =
|latest_validator_votes_for_frozen_banks: &mut LatestValidatorVotesForFrozenBanks| {
(0..num_validators)
.flat_map(|vote_slot| {
let vote_pubkey = Pubkey::new_unique();
let frozen_hash1 = Hash::new_unique();
assert_eq!(
latest_validator_votes_for_frozen_banks.check_add_vote(
vote_pubkey,
vote_slot,
Some(frozen_hash1),
),
// This vote slot was frozen, and is the highest slot inserted thus far,
// so the highest vote should be Some(vote_slot)
(true, Some(vote_slot))
);
// Add a duplicate
let frozen_hash2 = Hash::new_unique();
assert_eq!(
latest_validator_votes_for_frozen_banks.check_add_vote(
vote_pubkey,
vote_slot,
Some(frozen_hash2),
),
// This vote slot was frozen, and is for a duplicate version of the highest slot
// inserted thus far, so the highest vote should be Some(vote_slot).
(true, Some(vote_slot))
);
vec![
(vote_pubkey, (vote_slot, frozen_hash1)),
(vote_pubkey, (vote_slot, frozen_hash2)),
]
})
.collect()
};
// Taking all the dirty votes >= 0 will return everything
let root = 0;
let mut expected_dirty_set: Vec<(Pubkey, SlotHashKey)> =
setup_dirty_set(&mut latest_validator_votes_for_frozen_banks);
let mut votes_dirty_set_output =
latest_validator_votes_for_frozen_banks.take_votes_dirty_set(root);
votes_dirty_set_output.sort();
expected_dirty_set.sort();
assert_eq!(votes_dirty_set_output, expected_dirty_set);
assert!(latest_validator_votes_for_frozen_banks
.take_votes_dirty_set(0)
.is_empty());
// Taking all the firty votes >= num_validators - 1 will only return the last vote
let root = num_validators - 1;
let dirty_set = setup_dirty_set(&mut latest_validator_votes_for_frozen_banks);
let mut expected_dirty_set: Vec<(Pubkey, SlotHashKey)> =
dirty_set[dirty_set.len() - 2..dirty_set.len()].to_vec();
let mut votes_dirty_set_output =
latest_validator_votes_for_frozen_banks.take_votes_dirty_set(root);
votes_dirty_set_output.sort();
expected_dirty_set.sort();
assert_eq!(votes_dirty_set_output, expected_dirty_set);
assert!(latest_validator_votes_for_frozen_banks
.take_votes_dirty_set(0)
.is_empty());
}
}

View File

@ -41,6 +41,7 @@ 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 non_circulating_supply;
pub mod optimistic_confirmation_verifier;
@ -79,6 +80,7 @@ pub mod tpu;
pub mod transaction_status_service;
pub mod tree_diff;
pub mod tvu;
pub mod unfrozen_gossip_verified_vote_hashes;
pub mod validator;
pub mod verified_vote_packets;
pub mod vote_stake_tracker;

View File

@ -15,6 +15,7 @@ use crate::{
},
fork_choice::{ForkChoice, SelectVoteAndResetForkResult},
heaviest_subtree_fork_choice::HeaviestSubtreeForkChoice,
latest_validator_votes_for_frozen_banks::LatestValidatorVotesForFrozenBanks,
optimistically_confirmed_bank_tracker::{BankNotification, BankNotificationSender},
poh_recorder::{PohRecorder, GRACE_TICKS_FACTOR, MAX_GRACE_SLOTS},
progress_map::{DuplicateStats, ForkProgress, ProgressMap, PropagatedStats},
@ -22,6 +23,7 @@ use crate::{
result::Result,
rewards_recorder_service::RewardsRecorderSender,
rpc_subscriptions::RpcSubscriptions,
unfrozen_gossip_verified_vote_hashes::UnfrozenGossipVerifiedVoteHashes,
window_service::DuplicateSlotReceiver,
};
use solana_client::rpc_response::SlotUpdate;
@ -68,8 +70,6 @@ pub const DUPLICATE_LIVENESS_THRESHOLD: f64 = 0.1;
pub const DUPLICATE_THRESHOLD: f64 = 1.0 - SWITCH_FORK_THRESHOLD - DUPLICATE_LIVENESS_THRESHOLD;
const MAX_VOTE_SIGNATURES: usize = 200;
pub type GossipVerifiedVoteHashes = BTreeMap<Slot, HashMap<Hash, Vec<Pubkey>>>;
#[derive(PartialEq, Debug)]
pub(crate) enum HeaviestForkFailures {
LockedOut(u64),
@ -138,7 +138,7 @@ pub struct ReplayTiming {
bank_count: u64,
process_gossip_duplicate_confirmed_slots_elapsed: u64,
process_duplicate_slots_elapsed: u64,
process_gossip_verified_vote_hashes_elapsed: u64,
process_unfrozen_gossip_verified_vote_hashes_elapsed: u64,
}
impl ReplayTiming {
#[allow(clippy::too_many_arguments)]
@ -158,7 +158,7 @@ impl ReplayTiming {
heaviest_fork_failures_elapsed: u64,
bank_count: u64,
process_gossip_duplicate_confirmed_slots_elapsed: u64,
process_gossip_verified_vote_hashes_elapsed: u64,
process_unfrozen_gossip_verified_vote_hashes_elapsed: u64,
process_duplicate_slots_elapsed: u64,
) {
self.collect_frozen_banks_elapsed += collect_frozen_banks_elapsed;
@ -176,8 +176,8 @@ impl ReplayTiming {
self.bank_count += bank_count;
self.process_gossip_duplicate_confirmed_slots_elapsed +=
process_gossip_duplicate_confirmed_slots_elapsed;
self.process_gossip_verified_vote_hashes_elapsed +=
process_gossip_verified_vote_hashes_elapsed;
self.process_unfrozen_gossip_verified_vote_hashes_elapsed +=
process_unfrozen_gossip_verified_vote_hashes_elapsed;
self.process_duplicate_slots_elapsed += process_duplicate_slots_elapsed;
let now = timestamp();
let elapsed_ms = now - self.last_print;
@ -233,8 +233,8 @@ impl ReplayTiming {
i64
),
(
"process_gossip_verified_vote_hashes_elapsed",
self.process_gossip_verified_vote_hashes_elapsed as i64,
"process_unfrozen_gossip_verified_vote_hashes_elapsed",
self.process_unfrozen_gossip_verified_vote_hashes_elapsed as i64,
i64
),
(
@ -329,8 +329,9 @@ impl ReplayStage {
let mut partition_exists = false;
let mut skipped_slots_info = SkippedSlotsInfo::default();
let mut replay_timing = ReplayTiming::default();
let mut gossip_duplicate_confirmed_slots: GossipDuplicateConfirmedSlots = BTreeMap::new();
let mut gossip_verified_vote_hashes: GossipVerifiedVoteHashes = BTreeMap::new();
let mut gossip_duplicate_confirmed_slots: GossipDuplicateConfirmedSlots = GossipDuplicateConfirmedSlots::default();
let mut unfrozen_gossip_verified_vote_hashes: UnfrozenGossipVerifiedVoteHashes = UnfrozenGossipVerifiedVoteHashes::default();
let mut latest_validator_votes_for_frozen_banks: LatestValidatorVotesForFrozenBanks = LatestValidatorVotesForFrozenBanks::default();
let mut voted_signatures = Vec::new();
let mut has_new_vote_been_rooted = !wait_for_vote_to_start_leader;
loop {
@ -371,6 +372,8 @@ impl ReplayStage {
&gossip_duplicate_confirmed_slots,
&ancestors,
&descendants,
&mut unfrozen_gossip_verified_vote_hashes,
&mut latest_validator_votes_for_frozen_banks,
);
replay_active_banks_time.stop();
@ -406,12 +409,15 @@ impl ReplayStage {
// and switching proofs because these may be votes that haven't yet been
// included in a block, so we may not have yet observed these votes just
// by replaying blocks.
let mut process_gossip_verified_vote_hashes_time = Measure::start("process_gossip_duplicate_confirmed_slots");
Self::process_gossip_verified_vote_hashes(
let mut process_unfrozen_gossip_verified_vote_hashes_time = Measure::start("process_gossip_duplicate_confirmed_slots");
/*Self::process_gossip_verified_vote_hashes(
&gossip_verified_vote_hash_receiver,
&mut gossip_verified_vote_hashes,
);
process_gossip_verified_vote_hashes_time.stop();
&mut unfrozen_gossip_verified_vote_hashes,
&heaviest_subtree_fork_choice,
&mut latest_validator_votes_for_frozen_banks,
);*/
for _ in gossip_verified_vote_hash_receiver.try_iter() {}
process_unfrozen_gossip_verified_vote_hashes_time.stop();
// Check to remove any duplicated slots from fork choice
let mut process_duplicate_slots_time = Measure::start("process_duplicate_slots");
@ -450,6 +456,7 @@ impl ReplayStage {
&cluster_slots,
&bank_forks,
&mut heaviest_subtree_fork_choice,
&mut latest_validator_votes_for_frozen_banks,
);
compute_bank_stats_time.stop();
@ -545,7 +552,7 @@ impl ReplayStage {
&cache_block_time_sender,
&bank_notification_sender,
&mut gossip_duplicate_confirmed_slots,
&mut gossip_verified_vote_hashes,
&mut unfrozen_gossip_verified_vote_hashes,
&mut voted_signatures,
&mut has_new_vote_been_rooted,
);
@ -677,7 +684,7 @@ impl ReplayStage {
heaviest_fork_failures_time.as_us(),
if did_complete_bank {1} else {0},
process_gossip_duplicate_confirmed_slots_time.as_us(),
process_gossip_verified_vote_hashes_time.as_us(),
process_unfrozen_gossip_verified_vote_hashes_time.as_us(),
process_duplicate_slots_time.as_us(),
);
}
@ -912,18 +919,23 @@ impl ReplayStage {
}
}
#[cfg(test)]
fn process_gossip_verified_vote_hashes(
gossip_verified_vote_hash_receiver: &GossipVerifiedVoteHashReceiver,
gossip_verified_vote_hashes: &mut GossipVerifiedVoteHashes,
unfrozen_gossip_verified_vote_hashes: &mut UnfrozenGossipVerifiedVoteHashes,
heaviest_subtree_fork_choice: &HeaviestSubtreeForkChoice,
latest_validator_votes_for_frozen_banks: &mut LatestValidatorVotesForFrozenBanks,
) {
for (pubkey, slot, hash) in gossip_verified_vote_hash_receiver.try_iter() {
let is_frozen = heaviest_subtree_fork_choice.contains_block(&(slot, hash));
// cluster_info_vote_listener will ensure it doesn't push duplicates
gossip_verified_vote_hashes
.entry(slot)
.or_default()
.entry(hash)
.or_default()
.push(pubkey);
unfrozen_gossip_verified_vote_hashes.add_vote(
pubkey,
slot,
hash,
is_frozen,
latest_validator_votes_for_frozen_banks,
)
}
}
@ -1270,7 +1282,7 @@ impl ReplayStage {
cache_block_time_sender: &Option<CacheBlockTimeSender>,
bank_notification_sender: &Option<BankNotificationSender>,
gossip_duplicate_confirmed_slots: &mut GossipDuplicateConfirmedSlots,
gossip_verified_vote_hashes: &mut GossipVerifiedVoteHashes,
unfrozen_gossip_verified_vote_hashes: &mut UnfrozenGossipVerifiedVoteHashes,
vote_signatures: &mut Vec<Signature>,
has_new_vote_been_rooted: &mut bool,
) {
@ -1325,7 +1337,7 @@ impl ReplayStage {
highest_confirmed_root,
heaviest_subtree_fork_choice,
gossip_duplicate_confirmed_slots,
gossip_verified_vote_hashes,
unfrozen_gossip_verified_vote_hashes,
has_new_vote_been_rooted,
vote_signatures,
);
@ -1528,6 +1540,8 @@ impl ReplayStage {
gossip_duplicate_confirmed_slots: &GossipDuplicateConfirmedSlots,
ancestors: &HashMap<Slot, HashSet<Slot>>,
descendants: &HashMap<Slot, HashSet<Slot>>,
unfrozen_gossip_verified_vote_hashes: &mut UnfrozenGossipVerifiedVoteHashes,
latest_validator_votes_for_frozen_banks: &mut LatestValidatorVotesForFrozenBanks,
) -> bool {
let mut did_complete_bank = false;
let mut tx_count = 0;
@ -1642,6 +1656,18 @@ impl ReplayStage {
.unwrap_or_else(|err| warn!("bank_notification_sender failed: {:?}", err));
}
let bank_hash = bank.hash();
if let Some(new_frozen_voters) =
unfrozen_gossip_verified_vote_hashes.remove_slot_hash(bank.slot(), &bank_hash)
{
for pubkey in new_frozen_voters {
latest_validator_votes_for_frozen_banks.check_add_vote(
pubkey,
bank.slot(),
Some(bank_hash),
);
}
}
Self::record_rewards(&bank, &rewards_recorder_sender);
} else {
trace!(
@ -1667,6 +1693,7 @@ impl ReplayStage {
cluster_slots: &ClusterSlots,
bank_forks: &RwLock<BankForks>,
heaviest_subtree_fork_choice: &mut HeaviestSubtreeForkChoice,
latest_validator_votes_for_frozen_banks: &mut LatestValidatorVotesForFrozenBanks,
) -> Vec<Slot> {
frozen_banks.sort_by_key(|bank| bank.slot());
let mut new_stats = vec![];
@ -1686,14 +1713,15 @@ impl ReplayStage {
bank_slot,
bank.vote_accounts().into_iter(),
&ancestors,
|slot| progress.get_hash(slot),
latest_validator_votes_for_frozen_banks,
);
// Notify any listeners of the votes found in this newly computed
// bank
heaviest_subtree_fork_choice.compute_bank_stats(
&bank,
tower,
progress,
&computed_bank_state,
latest_validator_votes_for_frozen_banks,
);
let ComputedBankState {
voted_stakes,
@ -2194,7 +2222,7 @@ impl ReplayStage {
highest_confirmed_root: Option<Slot>,
heaviest_subtree_fork_choice: &mut HeaviestSubtreeForkChoice,
gossip_duplicate_confirmed_slots: &mut GossipDuplicateConfirmedSlots,
gossip_verified_vote_hashes: &mut GossipVerifiedVoteHashes,
unfrozen_gossip_verified_vote_hashes: &mut UnfrozenGossipVerifiedVoteHashes,
has_new_vote_been_rooted: &mut bool,
voted_signatures: &mut Vec<Signature>,
) {
@ -2222,9 +2250,7 @@ impl ReplayStage {
// gossip_confirmed_slots now only contains entries >= `new_root`
std::mem::swap(gossip_duplicate_confirmed_slots, &mut slots_ge_root);
let mut slots_ge_root = gossip_verified_vote_hashes.split_off(&new_root);
// gossip_verified_vote_hashes now only contains entries >= `new_root`
std::mem::swap(gossip_verified_vote_hashes, &mut slots_ge_root);
unfrozen_gossip_verified_vote_hashes.set_root(new_root);
}
fn generate_new_bank_forks(
@ -2419,7 +2445,7 @@ pub(crate) mod tests {
#[test]
fn test_is_partition_detected() {
let (bank_forks, _) = setup_forks();
let VoteSimulator { bank_forks, .. } = setup_forks();
let ancestors = bank_forks.read().unwrap().ancestors();
// Last vote 1 is an ancestor of the heaviest slot 3, no partition
assert!(!ReplayStage::is_partition_detected(&ancestors, 1, 3));
@ -2638,11 +2664,13 @@ pub(crate) mod tests {
.into_iter()
.map(|s| (s, Hash::default()))
.collect();
let mut gossip_verified_vote_hashes: GossipVerifiedVoteHashes =
vec![root - 1, root, root + 1]
.into_iter()
.map(|s| (s, HashMap::new()))
.collect();
let mut unfrozen_gossip_verified_vote_hashes: UnfrozenGossipVerifiedVoteHashes =
UnfrozenGossipVerifiedVoteHashes {
votes_per_slot: vec![root - 1, root, root + 1]
.into_iter()
.map(|s| (s, HashMap::new()))
.collect(),
};
ReplayStage::handle_new_root(
root,
&bank_forks,
@ -2651,7 +2679,7 @@ pub(crate) mod tests {
None,
&mut heaviest_subtree_fork_choice,
&mut gossip_duplicate_confirmed_slots,
&mut gossip_verified_vote_hashes,
&mut unfrozen_gossip_verified_vote_hashes,
&mut true,
&mut Vec::new(),
);
@ -2667,7 +2695,8 @@ pub(crate) mod tests {
vec![root, root + 1]
);
assert_eq!(
gossip_verified_vote_hashes
unfrozen_gossip_verified_vote_hashes
.votes_per_slot
.keys()
.cloned()
.collect::<Vec<Slot>>(),
@ -2718,8 +2747,8 @@ pub(crate) mod tests {
&AbsRequestSender::default(),
Some(confirmed_root),
&mut heaviest_subtree_fork_choice,
&mut BTreeMap::new(),
&mut BTreeMap::new(),
&mut GossipDuplicateConfirmedSlots::default(),
&mut UnfrozenGossipVerifiedVoteHashes::default(),
&mut true,
&mut Vec::new(),
);
@ -3262,6 +3291,8 @@ pub(crate) mod tests {
let (bank_forks, mut progress, mut heaviest_subtree_fork_choice) =
initialize_state(&keypairs, 10_000);
let mut latest_validator_votes_for_frozen_banks =
LatestValidatorVotesForFrozenBanks::default();
let bank0 = bank_forks.get(0).unwrap().clone();
let my_keypairs = keypairs.get(&node_pubkey).unwrap();
let vote_tx = vote_transaction::new_vote_transaction(
@ -3299,6 +3330,7 @@ pub(crate) mod tests {
&ClusterSlots::default(),
&bank_forks,
&mut heaviest_subtree_fork_choice,
&mut latest_validator_votes_for_frozen_banks,
);
// bank 0 has no votes, should not send any votes on the channel
@ -3349,6 +3381,7 @@ pub(crate) mod tests {
&ClusterSlots::default(),
&bank_forks,
&mut heaviest_subtree_fork_choice,
&mut latest_validator_votes_for_frozen_banks,
);
// Bank 1 had one vote
@ -3384,6 +3417,7 @@ pub(crate) mod tests {
&ClusterSlots::default(),
&bank_forks,
&mut heaviest_subtree_fork_choice,
&mut latest_validator_votes_for_frozen_banks,
);
// No new stats should have been computed
assert!(newly_computed.is_empty());
@ -3408,7 +3442,8 @@ pub(crate) mod tests {
.cloned()
.collect();
let mut heaviest_subtree_fork_choice = &mut vote_simulator.heaviest_subtree_fork_choice;
let mut latest_validator_votes_for_frozen_banks =
LatestValidatorVotesForFrozenBanks::default();
let ancestors = vote_simulator.bank_forks.read().unwrap().ancestors();
ReplayStage::compute_bank_stats(
&node_pubkey,
@ -3420,6 +3455,7 @@ pub(crate) mod tests {
&ClusterSlots::default(),
&vote_simulator.bank_forks,
&mut heaviest_subtree_fork_choice,
&mut latest_validator_votes_for_frozen_banks,
);
let bank1 = vote_simulator
@ -3499,6 +3535,7 @@ pub(crate) mod tests {
&ClusterSlots::default(),
&vote_simulator.bank_forks,
&mut vote_simulator.heaviest_subtree_fork_choice,
&mut vote_simulator.latest_validator_votes_for_frozen_banks,
);
frozen_banks.sort_by_key(|bank| bank.slot());
@ -4178,7 +4215,11 @@ pub(crate) mod tests {
#[test]
fn test_purge_unconfirmed_duplicate_slot() {
let (bank_forks, mut progress) = setup_forks();
let VoteSimulator {
bank_forks,
mut progress,
..
} = setup_forks();
let mut descendants = bank_forks.read().unwrap().descendants().clone();
let mut ancestors = bank_forks.read().unwrap().ancestors();
@ -4238,7 +4279,7 @@ pub(crate) mod tests {
#[test]
fn test_purge_ancestors_descendants() {
let (bank_forks, _) = setup_forks();
let VoteSimulator { bank_forks, .. } = setup_forks();
// Purge branch rooted at slot 2
let mut descendants = bank_forks.read().unwrap().descendants().clone();
@ -4344,6 +4385,7 @@ pub(crate) mod tests {
&ClusterSlots::default(),
&bank_forks,
&mut HeaviestSubtreeForkChoice::new_from_bank_forks(&bank_forks.read().unwrap()),
&mut LatestValidatorVotesForFrozenBanks::default(),
);
// Check status is true
@ -4386,8 +4428,6 @@ pub(crate) mod tests {
Blockstore::open(&ledger_path).expect("Expected to be able to open database ledger"),
);
let mut tower = Tower::new_for_tests(8, 0.67);
let mut heaviest_subtree_fork_choice =
HeaviestSubtreeForkChoice::new_from_bank_forks(&bank_forks.read().unwrap());
// All forks have same weight so heaviest bank to vote/reset on should be the tip of
// the fork with the lower slot
@ -4395,7 +4435,8 @@ pub(crate) mod tests {
&bank_forks,
&mut progress,
&mut tower,
&mut heaviest_subtree_fork_choice,
&mut vote_simulator.heaviest_subtree_fork_choice,
&mut vote_simulator.latest_validator_votes_for_frozen_banks,
);
assert_eq!(vote_fork.unwrap(), 4);
assert_eq!(reset_fork.unwrap(), 4);
@ -4422,7 +4463,7 @@ pub(crate) mod tests {
&ancestors,
&descendants,
&mut progress,
&mut heaviest_subtree_fork_choice,
&mut vote_simulator.heaviest_subtree_fork_choice,
SlotStateUpdate::Duplicate,
);
@ -4430,7 +4471,8 @@ pub(crate) mod tests {
&bank_forks,
&mut progress,
&mut tower,
&mut heaviest_subtree_fork_choice,
&mut vote_simulator.heaviest_subtree_fork_choice,
&mut vote_simulator.latest_validator_votes_for_frozen_banks,
);
assert!(vote_fork.is_none());
assert_eq!(reset_fork.unwrap(), 3);
@ -4449,7 +4491,7 @@ pub(crate) mod tests {
&ancestors,
&descendants,
&mut progress,
&mut heaviest_subtree_fork_choice,
&mut vote_simulator.heaviest_subtree_fork_choice,
SlotStateUpdate::Duplicate,
);
@ -4457,7 +4499,8 @@ pub(crate) mod tests {
&bank_forks,
&mut progress,
&mut tower,
&mut heaviest_subtree_fork_choice,
&mut vote_simulator.heaviest_subtree_fork_choice,
&mut vote_simulator.latest_validator_votes_for_frozen_banks,
);
// Should now pick the next heaviest fork that is not a descendant of 2, which is 6.
@ -4476,14 +4519,15 @@ pub(crate) mod tests {
&ancestors,
&descendants,
&mut progress,
&mut heaviest_subtree_fork_choice,
&mut vote_simulator.heaviest_subtree_fork_choice,
SlotStateUpdate::DuplicateConfirmed,
);
let (vote_fork, reset_fork) = run_compute_and_select_forks(
&bank_forks,
&mut progress,
&mut tower,
&mut heaviest_subtree_fork_choice,
&mut vote_simulator.heaviest_subtree_fork_choice,
&mut vote_simulator.latest_validator_votes_for_frozen_banks,
);
// Should now pick the heaviest fork 4 again, but lockouts apply so fork 4
// is not votable, which avoids voting for 4 again.
@ -4491,11 +4535,123 @@ pub(crate) mod tests {
assert_eq!(reset_fork.unwrap(), 4);
}
#[test]
fn test_gossip_vote_for_unrooted_slot() {
let VoteSimulator {
bank_forks,
mut heaviest_subtree_fork_choice,
mut latest_validator_votes_for_frozen_banks,
mut progress,
vote_pubkeys,
..
} = setup_forks();
let vote_pubkey = vote_pubkeys[0];
let mut unfrozen_gossip_verified_vote_hashes = UnfrozenGossipVerifiedVoteHashes::default();
let (gossip_verified_vote_hash_sender, gossip_verified_vote_hash_receiver) = unbounded();
// Cast a vote for slot 3 on one fork
let vote_slot = 3;
let vote_bank = bank_forks.read().unwrap().get(vote_slot).unwrap().clone();
gossip_verified_vote_hash_sender
.send((vote_pubkey, vote_slot, vote_bank.hash()))
.expect("Send should succeed");
ReplayStage::process_gossip_verified_vote_hashes(
&gossip_verified_vote_hash_receiver,
&mut unfrozen_gossip_verified_vote_hashes,
&heaviest_subtree_fork_choice,
&mut latest_validator_votes_for_frozen_banks,
);
// Pick the best fork
heaviest_subtree_fork_choice.compute_bank_stats(
&vote_bank,
&Tower::default(),
&mut latest_validator_votes_for_frozen_banks,
);
assert_eq!(heaviest_subtree_fork_choice.best_overall_slot().0, 6);
// Now send another vote for a frozen bank on the other fork, where the new vote
// is bigger than the last vote
let bigger_vote_slot = 4;
let bigger_vote_bank = bank_forks
.read()
.unwrap()
.get(bigger_vote_slot)
.unwrap()
.clone();
assert!(heaviest_subtree_fork_choice
.contains_block(&(bigger_vote_slot, bigger_vote_bank.hash())));
gossip_verified_vote_hash_sender
.send((vote_pubkey, bigger_vote_slot, bigger_vote_bank.hash()))
.expect("Send should succeed");
ReplayStage::process_gossip_verified_vote_hashes(
&gossip_verified_vote_hash_receiver,
&mut unfrozen_gossip_verified_vote_hashes,
&heaviest_subtree_fork_choice,
&mut latest_validator_votes_for_frozen_banks,
);
// Now set a root for a slot on the previously voted fork thats smaller than the new vote
let new_root = 3;
ReplayStage::handle_new_root(
new_root,
&bank_forks,
&mut progress,
&AbsRequestSender::default(),
None,
&mut heaviest_subtree_fork_choice,
&mut GossipDuplicateConfirmedSlots::default(),
&mut unfrozen_gossip_verified_vote_hashes,
&mut true,
&mut vec![],
);
// Add a new bank, freeze it
let parent_bank = bank_forks.read().unwrap().get(6).unwrap().clone();
let new_bank = Bank::new_from_parent(&parent_bank, &Pubkey::default(), 7);
bank_forks.write().unwrap().insert(new_bank);
let new_bank = bank_forks.read().unwrap().get(7).unwrap().clone();
new_bank.freeze();
heaviest_subtree_fork_choice.add_new_leaf_slot(
(new_bank.slot(), new_bank.hash()),
Some((parent_bank.slot(), parent_bank.hash())),
);
// Compute bank stats on new slot
heaviest_subtree_fork_choice.compute_bank_stats(
&new_bank,
&Tower::default(),
&mut latest_validator_votes_for_frozen_banks,
);
// Even though the `bigger_vote_slot` no longer exists in the fork choice tree,
// this vote should remove the previous vote's weight because we know there
// was a later vote
let old_vote_node = (vote_slot, vote_bank.hash());
assert_eq!(
heaviest_subtree_fork_choice
.stake_voted_at(&old_vote_node)
.unwrap(),
0
);
assert_eq!(
heaviest_subtree_fork_choice
.stake_voted_subtree(&old_vote_node)
.unwrap(),
0
);
assert_eq!(
heaviest_subtree_fork_choice.best_overall_slot(),
(new_bank.slot(), new_bank.hash())
);
}
fn run_compute_and_select_forks(
bank_forks: &RwLock<BankForks>,
progress: &mut ProgressMap,
tower: &mut Tower,
heaviest_subtree_fork_choice: &mut HeaviestSubtreeForkChoice,
latest_validator_votes_for_frozen_banks: &mut LatestValidatorVotesForFrozenBanks,
) -> (Option<Slot>, Option<Slot>) {
let mut frozen_banks: Vec<_> = bank_forks
.read()
@ -4516,6 +4672,7 @@ pub(crate) mod tests {
&ClusterSlots::default(),
&bank_forks,
heaviest_subtree_fork_choice,
latest_validator_votes_for_frozen_banks,
);
let (heaviest_bank, heaviest_bank_on_same_fork) = heaviest_subtree_fork_choice
.select_forks(&frozen_banks, &tower, &progress, &ancestors, bank_forks);
@ -4538,7 +4695,7 @@ pub(crate) mod tests {
)
}
fn setup_forks() -> (RwLock<BankForks>, ProgressMap) {
fn setup_forks() -> VoteSimulator {
/*
Build fork structure:
@ -4558,7 +4715,7 @@ pub(crate) mod tests {
let mut vote_simulator = VoteSimulator::new(1);
vote_simulator.fill_bank_forks(forks, &HashMap::new());
(vote_simulator.bank_forks, vote_simulator.progress)
vote_simulator
}
fn check_map_eq<K: Eq + std::hash::Hash + std::fmt::Debug, T: PartialEq + std::fmt::Debug>(

View File

@ -0,0 +1,132 @@
use crate::latest_validator_votes_for_frozen_banks::LatestValidatorVotesForFrozenBanks;
use solana_sdk::{clock::Slot, hash::Hash, pubkey::Pubkey};
use std::collections::{BTreeMap, HashMap};
#[derive(Default)]
pub(crate) struct UnfrozenGossipVerifiedVoteHashes {
pub votes_per_slot: BTreeMap<Slot, HashMap<Hash, Vec<Pubkey>>>,
}
impl UnfrozenGossipVerifiedVoteHashes {
// Update `latest_validator_votes_for_frozen_banks` if gossip has seen a newer vote
// for a frozen bank.
#[allow(dead_code)]
pub(crate) fn add_vote(
&mut self,
pubkey: Pubkey,
vote_slot: Slot,
hash: Hash,
is_frozen: bool,
latest_validator_votes_for_frozen_banks: &mut LatestValidatorVotesForFrozenBanks,
) {
// If this is a frozen bank, then we need to update the `latest_validator_votes_for_frozen_banks`
let frozen_hash = if is_frozen { Some(hash) } else { None };
let (was_added, latest_frozen_vote_slot) =
latest_validator_votes_for_frozen_banks.check_add_vote(pubkey, vote_slot, frozen_hash);
if !was_added
&& latest_frozen_vote_slot
.map(|latest_frozen_vote_slot| vote_slot >= latest_frozen_vote_slot)
// If there's no latest frozen vote slot yet, then we should also insert
.unwrap_or(true)
{
// At this point it must be that:
// 1) `vote_slot` was not yet frozen
// 2) and `vote_slot` >= than the latest frozen vote slot.
// Thus we want to record this vote for later, in case a slot with this `vote_slot` + hash gets
// frozen later
self.votes_per_slot
.entry(vote_slot)
.or_default()
.entry(hash)
.or_default()
.push(pubkey);
}
}
// Cleanup `votes_per_slot` based on new roots
pub(crate) fn set_root(&mut self, new_root: Slot) {
let mut slots_ge_root = self.votes_per_slot.split_off(&new_root);
// `self.votes_per_slot` now only contains entries >= `new_root`
std::mem::swap(&mut self.votes_per_slot, &mut slots_ge_root);
}
pub(crate) fn remove_slot_hash(&mut self, slot: Slot, hash: &Hash) -> Option<Vec<Pubkey>> {
self.votes_per_slot.get_mut(&slot).and_then(|slot_hashes| {
slot_hashes.remove(hash)
// If `slot_hashes` becomes empty, it'll be removed by `set_root()` later
})
}
}
#[cfg(test)]
mod tests {
use super::*;
#[test]
fn test_unfrozen_gossip_verified_vote_hashes_add_vote() {
let mut unfrozen_gossip_verified_vote_hashes = UnfrozenGossipVerifiedVoteHashes::default();
let mut latest_validator_votes_for_frozen_banks =
LatestValidatorVotesForFrozenBanks::default();
let num_validators = 10;
let validator_keys: Vec<Pubkey> = std::iter::repeat_with(Pubkey::new_unique)
.take(num_validators)
.collect();
// Case 1: Frozen banks shouldn't be added
let frozen_vote_slot = 1;
let num_repeated_iterations = 10;
for _ in 0..num_repeated_iterations {
let hash = Hash::new_unique();
let is_frozen = true;
for vote_pubkey in validator_keys.iter() {
unfrozen_gossip_verified_vote_hashes.add_vote(
*vote_pubkey,
frozen_vote_slot,
hash,
is_frozen,
&mut latest_validator_votes_for_frozen_banks,
);
}
assert!(unfrozen_gossip_verified_vote_hashes
.votes_per_slot
.is_empty());
}
// Case 2: Other >= non-frozen banks should be added in case they're frozen later
for unfrozen_vote_slot in &[frozen_vote_slot - 1, frozen_vote_slot, frozen_vote_slot + 1] {
// If the vote slot is smaller than the latest known frozen `vote_slot`
// for each pubkey (which was added above), then they shouldn't be added
let num_duplicate_hashes = 10;
for _ in 0..num_duplicate_hashes {
let hash = Hash::new_unique();
let is_frozen = false;
for vote_pubkey in validator_keys.iter() {
unfrozen_gossip_verified_vote_hashes.add_vote(
*vote_pubkey,
*unfrozen_vote_slot,
hash,
is_frozen,
&mut latest_validator_votes_for_frozen_banks,
);
}
}
if *unfrozen_vote_slot >= frozen_vote_slot {
let vote_hashes_map = unfrozen_gossip_verified_vote_hashes
.votes_per_slot
.get(&unfrozen_vote_slot)
.unwrap();
assert_eq!(vote_hashes_map.len(), num_duplicate_hashes);
for pubkey_votes in vote_hashes_map.values() {
assert_eq!(*pubkey_votes, validator_keys);
}
} else {
assert!(unfrozen_gossip_verified_vote_hashes
.votes_per_slot
.is_empty());
}
}
}
}