patches crds vote-index assignment bug (#14438)

If tower is full, old votes are evicted from the front of the deque:
https://github.com/solana-labs/solana/blob/2074e407c/programs/vote/src/vote_state/mod.rs#L367-L373
whereas recent votes if expire are evicted from the back:
https://github.com/solana-labs/solana/blob/2074e407c/programs/vote/src/vote_state/mod.rs#L529-L537

As a result, from a single tower_index scalar, we cannot infer which crds-vote
should be overwritten:
https://github.com/solana-labs/solana/blob/2074e407c/core/src/crds_value.rs#L576

In addition there is an off by one bug in the existing code. tower_index is
bounded by MAX_LOCKOUT_HISTORY - 1:
https://github.com/solana-labs/solana/blob/2074e407c/core/src/consensus.rs#L382
So, it is at most 30, whereas MAX_VOTES is 32:
https://github.com/solana-labs/solana/blob/2074e407c/core/src/crds_value.rs#L29
Which means that this branch is never taken:
https://github.com/solana-labs/solana/blob/2074e407c/core/src/crds_value.rs#L590-L593
so crds table alwasys keeps 29 **oldest** votes by wallclock, and then
only overrides the 30st one each time. (i.e a tally of only two most
recent votes).
This commit is contained in:
behzad nouri 2021-01-21 13:08:07 +00:00 committed by GitHub
parent fcd72f309a
commit 8e581601d6
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
7 changed files with 289 additions and 132 deletions

View File

@ -63,6 +63,7 @@ use solana_sdk::{
};
use solana_streamer::sendmmsg::multicast;
use solana_streamer::streamer::{PacketReceiver, PacketSender};
use solana_vote_program::vote_state::MAX_LOCKOUT_HISTORY;
use std::{
borrow::Cow,
collections::{hash_map::Entry, HashMap, HashSet, VecDeque},
@ -1076,22 +1077,60 @@ impl ClusterInfo {
self.push_message(CrdsValue::new_signed(message, &self.keypair));
}
pub fn push_vote(&self, tower_index: usize, vote: Transaction) {
pub fn push_vote(&self, tower: &[Slot], vote: Transaction) {
debug_assert!(tower.iter().tuple_windows().all(|(a, b)| a < b));
let now = timestamp();
let vote = Vote::new(&self.id(), vote, now);
let vote_ix = {
let r_gossip =
self.time_gossip_read_lock("gossip_read_push_vote", &self.stats.push_vote_read);
let current_votes: Vec<_> = (0..crds_value::MAX_VOTES)
.filter_map(|ix| r_gossip.crds.lookup(&CrdsValueLabel::Vote(ix, self.id())))
.collect();
CrdsValue::compute_vote_index(tower_index, current_votes)
// Find a crds vote which is evicted from the tower, and recycle its
// vote-index. This can be either an old vote which is popped off the
// deque, or recent vote which has expired before getting enough
// confirmations.
// If all votes are still in the tower, add a new vote-index. If more
// than one vote is evicted, the oldest one by wallclock is returned in
// order to allow more recent votes more time to propagate through
// gossip.
// TODO: When there are more than one vote evicted from the tower, only
// one crds vote is overwritten here. Decide what to do with the rest.
let mut num_crds_votes = 0;
let self_pubkey = self.id();
// Returns true if the tower does not contain the vote.slot.
let should_evict_vote = |vote: &Vote| -> bool {
match vote.slot() {
Some(slot) => !tower.contains(&slot),
None => {
error!("crds vote with no slots!");
true
}
}
};
let entry = CrdsValue::new_signed(CrdsData::Vote(vote_ix, vote), &self.keypair);
self.local_message_pending_push_queue
let vote_index = {
let gossip =
self.time_gossip_read_lock("gossip_read_push_vote", &self.stats.push_vote_read);
(0..MAX_LOCKOUT_HISTORY as u8)
.filter_map(|ix| {
let vote = CrdsValueLabel::Vote(ix, self_pubkey);
let vote = gossip.crds.lookup(&vote)?;
num_crds_votes += 1;
match &vote.data {
CrdsData::Vote(_, vote) if should_evict_vote(vote) => {
Some((vote.wallclock, ix))
}
CrdsData::Vote(_, _) => None,
_ => panic!("this should not happen!"),
}
})
.min() // Boot the oldest evicted vote by wallclock.
.map(|(_ /*wallclock*/, ix)| ix)
};
let vote_index = vote_index.unwrap_or(num_crds_votes);
assert!((vote_index as usize) < MAX_LOCKOUT_HISTORY);
let vote = Vote::new(self_pubkey, vote, now);
debug_assert_eq!(vote.slot().unwrap(), *tower.last().unwrap());
let vote = CrdsData::Vote(vote_index, vote);
let vote = CrdsValue::new_signed(vote, &self.keypair);
self.gossip
.write()
.unwrap()
.push((entry, now));
.process_push_message(&self_pubkey, vec![vote], now);
}
pub fn send_vote(&self, vote: &Transaction) -> Result<()> {
@ -1116,7 +1155,7 @@ impl ClusterInfo {
.map(|vote| {
max_ts = std::cmp::max(vote.insert_timestamp, max_ts);
let transaction = match &vote.value.data {
CrdsData::Vote(_, vote) => vote.transaction.clone(),
CrdsData::Vote(_, vote) => vote.transaction().clone(),
_ => panic!("this should not happen!"),
};
(vote.value.label(), transaction)
@ -3146,7 +3185,6 @@ mod tests {
use crate::crds_value::{CrdsValue, CrdsValueLabel, Vote as CrdsVote};
use itertools::izip;
use rand::seq::SliceRandom;
use solana_perf::test_tx::test_tx;
use solana_sdk::signature::{Keypair, Signer};
use solana_vote_program::{vote_instruction, vote_state::Vote};
use std::iter::repeat_with;
@ -3632,6 +3670,7 @@ mod tests {
#[test]
fn test_push_vote() {
let mut rng = rand::thread_rng();
let keys = Keypair::new();
let contact_info = ContactInfo::new_localhost(&keys.pubkey(), 0);
let cluster_info = ClusterInfo::new_with_invalid_keypair(contact_info);
@ -3643,9 +3682,21 @@ mod tests {
assert_eq!(max_ts, now);
// add a vote
let tx = test_tx();
let index = 1;
cluster_info.push_vote(index, tx.clone());
let vote = Vote::new(
vec![1, 3, 7], // slots
solana_sdk::hash::new_rand(&mut rng),
);
let ix = vote_instruction::vote(
&Pubkey::new_unique(), // vote_pubkey
&Pubkey::new_unique(), // authorized_voter_pubkey
vote,
);
let tx = Transaction::new_with_payer(
&[ix], // instructions
None, // payer
);
let tower = vec![7]; // Last slot in the vote.
cluster_info.push_vote(&tower, tx.clone());
cluster_info.flush_push_queue();
// -1 to make sure that the clock is strictly lower then when insert occurred
@ -3667,6 +3718,81 @@ mod tests {
assert_eq!(max_ts, new_max_ts);
}
fn new_vote_transaction<R: Rng>(rng: &mut R, slots: Vec<Slot>) -> Transaction {
let vote = Vote::new(slots, solana_sdk::hash::new_rand(rng));
let ix = vote_instruction::vote(
&Pubkey::new_unique(), // vote_pubkey
&Pubkey::new_unique(), // authorized_voter_pubkey
vote,
);
Transaction::new_with_payer(
&[ix], // instructions
None, // payer
)
}
#[test]
fn test_push_votes_with_tower() {
let get_vote_slots = |cluster_info: &ClusterInfo, now| -> Vec<Slot> {
let (labels, _, _) = cluster_info.get_votes(now);
let gossip = cluster_info.gossip.read().unwrap();
let mut vote_slots = HashSet::new();
for label in labels {
match &gossip.crds.lookup(&label).unwrap().data {
CrdsData::Vote(_, vote) => {
assert!(vote_slots.insert(vote.slot().unwrap()));
}
_ => panic!("this should not happen!"),
}
}
vote_slots.into_iter().collect()
};
let mut rng = rand::thread_rng();
let now = timestamp();
let keys = Keypair::new();
let contact_info = ContactInfo::new_localhost(&keys.pubkey(), 0);
let cluster_info = ClusterInfo::new_with_invalid_keypair(contact_info);
let mut tower = Vec::new();
for k in 0..MAX_LOCKOUT_HISTORY {
let slot = k as Slot;
tower.push(slot);
let vote = new_vote_transaction(&mut rng, vec![slot]);
cluster_info.push_vote(&tower, vote);
}
let vote_slots = get_vote_slots(&cluster_info, now);
assert_eq!(vote_slots.len(), MAX_LOCKOUT_HISTORY);
for vote_slot in vote_slots {
assert!(vote_slot < MAX_LOCKOUT_HISTORY as u64);
}
// Push a new vote evicting one.
let slot = MAX_LOCKOUT_HISTORY as Slot;
tower.push(slot);
tower.remove(23);
let vote = new_vote_transaction(&mut rng, vec![slot]);
cluster_info.push_vote(&tower, vote);
let vote_slots = get_vote_slots(&cluster_info, now);
assert_eq!(vote_slots.len(), MAX_LOCKOUT_HISTORY);
for vote_slot in vote_slots {
assert!(vote_slot <= slot);
assert!(vote_slot != 23);
}
// Push a new vote evicting two.
// Older one should be evicted from the crds table.
let slot = slot + 1;
tower.push(slot);
tower.remove(17);
tower.remove(5);
let vote = new_vote_transaction(&mut rng, vec![slot]);
cluster_info.push_vote(&tower, vote);
let vote_slots = get_vote_slots(&cluster_info, now);
assert_eq!(vote_slots.len(), MAX_LOCKOUT_HISTORY);
for vote_slot in vote_slots {
assert!(vote_slot <= slot);
assert!(vote_slot != 23);
assert!(vote_slot != 5);
}
}
#[test]
fn test_push_epoch_slots() {
let keys = Keypair::new();
@ -4029,11 +4155,11 @@ mod tests {
vote_tx.partial_sign(&[keypair.as_ref()], Hash::default());
vote_tx.partial_sign(&[keypair.as_ref()], Hash::default());
let vote = CrdsVote {
from: keypair.pubkey(),
transaction: vote_tx,
wallclock: 0,
};
let vote = CrdsVote::new(
keypair.pubkey(),
vote_tx,
0, // wallclock
);
let vote = CrdsValue::new_signed(CrdsData::Vote(1, vote), &Keypair::new());
assert!(bincode::serialized_size(&vote).unwrap() <= PUSH_MESSAGE_MAX_PAYLOAD_SIZE as u64);
}

View File

@ -359,7 +359,7 @@ impl Tower {
slot: Slot,
hash: Hash,
last_voted_slot_in_bank: Option<Slot>,
) -> (Vote, usize) {
) -> (Vote, Vec<Slot> /*VoteState.tower*/) {
let mut local_vote_state = local_vote_state.clone();
let vote = Vote::new(vec![slot], hash);
local_vote_state.process_vote_unchecked(&vote);
@ -379,7 +379,7 @@ impl Tower {
slots,
local_vote_state.votes
);
(Vote::new(slots, hash), local_vote_state.votes.len() - 1)
(Vote::new(slots, hash), local_vote_state.tower())
}
fn last_voted_slot_in_bank(bank: &Bank, vote_account_pubkey: &Pubkey) -> Option<Slot> {
@ -388,7 +388,11 @@ impl Tower {
slot
}
pub fn new_vote_from_bank(&self, bank: &Bank, vote_account_pubkey: &Pubkey) -> (Vote, usize) {
pub fn new_vote_from_bank(
&self,
bank: &Bank,
vote_account_pubkey: &Pubkey,
) -> (Vote, Vec<Slot> /*VoteState.tower*/) {
let voted_slot = Self::last_voted_slot_in_bank(bank, vote_account_pubkey);
Self::new_vote(&self.lockouts, bank.slot(), bank.hash(), voted_slot)
}
@ -2268,10 +2272,10 @@ pub mod test {
#[test]
fn test_new_vote() {
let local = VoteState::default();
let vote = Tower::new_vote(&local, 0, Hash::default(), None);
let (vote, tower_slots) = Tower::new_vote(&local, 0, Hash::default(), None);
assert_eq!(local.votes.len(), 0);
assert_eq!(vote.0.slots, vec![0]);
assert_eq!(vote.1, 0);
assert_eq!(vote.slots, vec![0]);
assert_eq!(tower_slots, vec![0]);
}
#[test]
@ -2291,9 +2295,9 @@ pub mod test {
};
local.process_vote_unchecked(&vote);
assert_eq!(local.votes.len(), 1);
let vote = Tower::new_vote(&local, 1, Hash::default(), Some(0));
assert_eq!(vote.0.slots, vec![1]);
assert_eq!(vote.1, 1);
let (vote, tower_slots) = Tower::new_vote(&local, 1, Hash::default(), Some(0));
assert_eq!(vote.slots, vec![1]);
assert_eq!(tower_slots, vec![0, 1]);
}
#[test]
@ -2306,10 +2310,10 @@ pub mod test {
};
local.process_vote_unchecked(&vote);
assert_eq!(local.votes.len(), 1);
let vote = Tower::new_vote(&local, 3, Hash::default(), Some(0));
//first vote expired, so index should be 0
assert_eq!(vote.0.slots, vec![3]);
assert_eq!(vote.1, 0);
let (vote, tower_slots) = Tower::new_vote(&local, 3, Hash::default(), Some(0));
assert_eq!(vote.slots, vec![3]);
// First vote expired, so should be evicted from tower.
assert_eq!(tower_slots, vec![3]);
}
#[test]

View File

@ -1428,7 +1428,7 @@ mod test {
// construct something that's not a contact info
let peer_vote =
CrdsValue::new_unsigned(CrdsData::Vote(0, Vote::new(&peer_pubkey, test_tx(), 0)));
CrdsValue::new_unsigned(CrdsData::Vote(0, Vote::new(peer_pubkey, test_tx(), 0)));
// check that older CrdsValues (non-ContactInfos) infos pass even if are too old,
// but a recent contact info (inserted above) exists
assert_eq!(

View File

@ -7,6 +7,7 @@ use crate::{
};
use bincode::{serialize, serialized_size};
use rand::{CryptoRng, Rng};
use serde::de::{Deserialize, Deserializer};
use solana_sdk::sanitize::{Sanitize, SanitizeError};
use solana_sdk::timing::timestamp;
use solana_sdk::{
@ -16,9 +17,10 @@ use solana_sdk::{
signature::{Keypair, Signable, Signature, Signer},
transaction::Transaction,
};
use solana_vote_program::vote_transaction::parse_vote_transaction;
use std::{
borrow::{Borrow, Cow},
collections::{hash_map::Entry, BTreeSet, HashMap, HashSet},
collections::{hash_map::Entry, BTreeSet, HashMap},
fmt,
};
@ -26,6 +28,8 @@ pub const MAX_WALLCLOCK: u64 = 1_000_000_000_000_000;
pub const MAX_SLOT: u64 = 1_000_000_000_000_000;
pub type VoteIndex = u8;
// TODO: Remove this in favor of vote_state::MAX_LOCKOUT_HISTORY once
// the fleet is updated to the new ClusterInfo::push_vote code.
pub const MAX_VOTES: VoteIndex = 32;
pub type EpochSlotsIndex = u8;
@ -241,11 +245,13 @@ impl Sanitize for LowestSlot {
}
}
#[derive(Serialize, Deserialize, Clone, Debug, PartialEq, AbiExample)]
#[derive(Clone, Debug, PartialEq, AbiExample, Serialize)]
pub struct Vote {
pub from: Pubkey,
pub transaction: Transaction,
pub wallclock: u64,
pub(crate) from: Pubkey,
transaction: Transaction,
pub(crate) wallclock: u64,
#[serde(skip_serializing)]
slot: Option<Slot>,
}
impl Sanitize for Vote {
@ -257,11 +263,14 @@ impl Sanitize for Vote {
}
impl Vote {
pub fn new(from: &Pubkey, transaction: Transaction, wallclock: u64) -> Self {
pub fn new(from: Pubkey, transaction: Transaction, wallclock: u64) -> Self {
let slot = parse_vote_transaction(&transaction)
.and_then(|(_, vote, _)| vote.slots.last().copied());
Self {
from: *from,
from,
transaction,
wallclock,
slot,
}
}
@ -271,8 +280,42 @@ impl Vote {
from: pubkey.unwrap_or_else(pubkey::new_rand),
transaction: Transaction::default(),
wallclock: new_rand_timestamp(rng),
slot: None,
}
}
pub(crate) fn transaction(&self) -> &Transaction {
&self.transaction
}
pub(crate) fn slot(&self) -> Option<Slot> {
self.slot
}
}
impl<'de> Deserialize<'de> for Vote {
fn deserialize<D>(deserializer: D) -> Result<Self, D::Error>
where
D: Deserializer<'de>,
{
#[derive(Deserialize)]
struct Vote {
from: Pubkey,
transaction: Transaction,
wallclock: u64,
};
let vote = Vote::deserialize(deserializer)?;
let vote = match vote.transaction.sanitize() {
Ok(_) => Self::new(vote.from, vote.transaction, vote.wallclock),
Err(_) => Self {
from: vote.from,
transaction: vote.transaction,
wallclock: vote.wallclock,
slot: None,
},
};
Ok(vote)
}
}
#[derive(Serialize, Deserialize, Clone, Debug, PartialEq, AbiExample)]
@ -529,20 +572,15 @@ impl CrdsValue {
_ => None,
}
}
pub fn vote(&self) -> Option<&Vote> {
#[cfg(test)]
fn vote(&self) -> Option<&Vote> {
match &self.data {
CrdsData::Vote(_, vote) => Some(vote),
_ => None,
}
}
pub fn vote_index(&self) -> Option<VoteIndex> {
match &self.data {
CrdsData::Vote(ix, _) => Some(*ix),
_ => None,
}
}
pub fn lowest_slot(&self) -> Option<&LowestSlot> {
match &self.data {
CrdsData::LowestSlot(_, slots) => Some(slots),
@ -590,33 +628,6 @@ impl CrdsValue {
serialized_size(&self).expect("unable to serialize contact info")
}
pub fn compute_vote_index(tower_index: usize, mut votes: Vec<&CrdsValue>) -> VoteIndex {
let mut available: HashSet<VoteIndex> = (0..MAX_VOTES).collect();
votes.iter().filter_map(|v| v.vote_index()).for_each(|ix| {
available.remove(&ix);
});
// free index
if !available.is_empty() {
return *available.iter().next().unwrap();
}
assert!(votes.len() == MAX_VOTES as usize);
votes.sort_by_key(|v| v.vote().expect("all values must be votes").wallclock);
// If Tower is full, oldest removed first
if tower_index + 1 == MAX_VOTES as usize {
return votes[0].vote_index().expect("all values must be votes");
}
// If Tower is not full, the early votes have expired
assert!(tower_index < MAX_VOTES as usize);
votes[tower_index]
.vote_index()
.expect("all values must be votes")
}
/// Returns true if, regardless of prunes, this crds-value
/// should be pushed to the receiving node.
pub fn should_force_push(&self, peer: &Pubkey) -> bool {
@ -663,10 +674,11 @@ pub(crate) fn sanitize_wallclock(wallclock: u64) -> Result<(), SanitizeError> {
mod test {
use super::*;
use crate::contact_info::ContactInfo;
use bincode::deserialize;
use bincode::{deserialize, Options};
use solana_perf::test_tx::test_tx;
use solana_sdk::signature::{Keypair, Signer};
use solana_sdk::timing::timestamp;
use solana_vote_program::{vote_instruction, vote_state};
use std::cmp::Ordering;
use std::iter::repeat_with;
@ -679,7 +691,7 @@ mod test {
let v = CrdsValue::new_unsigned(CrdsData::Vote(
0,
Vote::new(&Pubkey::default(), test_tx(), 0),
Vote::new(Pubkey::default(), test_tx(), 0),
));
assert_eq!(v.wallclock(), 0);
let key = v.vote().unwrap().from;
@ -731,7 +743,7 @@ mod test {
verify_signatures(&mut v, &keypair, &wrong_keypair);
v = CrdsValue::new_unsigned(CrdsData::Vote(
0,
Vote::new(&keypair.pubkey(), test_tx(), timestamp()),
Vote::new(keypair.pubkey(), test_tx(), timestamp()),
));
verify_signatures(&mut v, &keypair, &wrong_keypair);
v = CrdsValue::new_unsigned(CrdsData::LowestSlot(
@ -747,13 +759,45 @@ mod test {
let vote = CrdsValue::new_signed(
CrdsData::Vote(
MAX_VOTES,
Vote::new(&keypair.pubkey(), test_tx(), timestamp()),
Vote::new(keypair.pubkey(), test_tx(), timestamp()),
),
&keypair,
);
assert!(vote.sanitize().is_err());
}
#[test]
fn test_vote_round_trip() {
let mut rng = rand::thread_rng();
let vote = vote_state::Vote::new(
vec![1, 3, 7], // slots
solana_sdk::hash::new_rand(&mut rng),
);
let ix = vote_instruction::vote(
&Pubkey::new_unique(), // vote_pubkey
&Pubkey::new_unique(), // authorized_voter_pubkey
vote,
);
let tx = Transaction::new_with_payer(
&[ix], // instructions
Some(&Pubkey::new_unique()), // payer
);
let vote = Vote::new(
Pubkey::new_unique(), // from
tx,
rng.gen(), // wallclock
);
assert_eq!(vote.slot, Some(7));
let bytes = bincode::serialize(&vote).unwrap();
let other = bincode::deserialize(&bytes[..]).unwrap();
assert_eq!(vote, other);
assert_eq!(other.slot, Some(7));
let bytes = bincode::options().serialize(&vote).unwrap();
let other = bincode::options().deserialize(&bytes[..]).unwrap();
assert_eq!(vote, other);
assert_eq!(other.slot, Some(7));
}
#[test]
fn test_max_epoch_slots_index() {
let keypair = Keypair::new();
@ -766,49 +810,6 @@ mod test {
);
assert_eq!(item.sanitize(), Err(SanitizeError::ValueOutOfBounds));
}
#[test]
fn test_compute_vote_index_empty() {
for i in 0..MAX_VOTES {
let votes = vec![];
assert!(CrdsValue::compute_vote_index(i as usize, votes) < MAX_VOTES);
}
}
#[test]
fn test_compute_vote_index_one() {
let keypair = Keypair::new();
let vote = CrdsValue::new_unsigned(CrdsData::Vote(
0,
Vote::new(&keypair.pubkey(), test_tx(), 0),
));
for i in 0..MAX_VOTES {
let votes = vec![&vote];
assert!(CrdsValue::compute_vote_index(i as usize, votes) > 0);
let votes = vec![&vote];
assert!(CrdsValue::compute_vote_index(i as usize, votes) < MAX_VOTES);
}
}
#[test]
fn test_compute_vote_index_full() {
let keypair = Keypair::new();
let votes: Vec<_> = (0..MAX_VOTES)
.map(|x| {
CrdsValue::new_unsigned(CrdsData::Vote(
x,
Vote::new(&keypair.pubkey(), test_tx(), x as u64),
))
})
.collect();
let vote_refs = votes.iter().collect();
//pick the oldest vote when full
assert_eq!(CrdsValue::compute_vote_index(31, vote_refs), 0);
//pick the index
let vote_refs = votes.iter().collect();
assert_eq!(CrdsValue::compute_vote_index(0, vote_refs), 0);
let vote_refs = votes.iter().collect();
assert_eq!(CrdsValue::compute_vote_index(30, vote_refs), 30);
}
fn serialize_deserialize_value(value: &mut CrdsValue, keypair: &Keypair) {
let num_tries = 10;

View File

@ -1088,7 +1088,7 @@ impl ReplayStage {
inc_new_counter_info!("replay_stage-voted_empty_bank", 1);
}
trace!("handle votable bank {}", bank.slot());
let (vote, tower_index) = tower.new_vote_from_bank(bank, vote_account_pubkey);
let (vote, tower_slots) = tower.new_vote_from_bank(bank, vote_account_pubkey);
let new_root = tower.record_bank_vote(vote);
let last_vote = tower.last_vote_and_timestamp();
@ -1163,7 +1163,7 @@ impl ReplayStage {
vote_account_pubkey,
authorized_voter_keypairs,
last_vote,
tower_index,
&tower_slots,
switch_fork_decision,
);
}
@ -1174,7 +1174,7 @@ impl ReplayStage {
vote_account_pubkey: &Pubkey,
authorized_voter_keypairs: &[Arc<Keypair>],
vote: Vote,
tower_index: usize,
tower: &[Slot],
switch_fork_decision: &SwitchForkDecision,
) {
if authorized_voter_keypairs.is_empty() {
@ -1249,7 +1249,7 @@ impl ReplayStage {
vote_tx.partial_sign(&[node_keypair.as_ref()], blockhash);
vote_tx.partial_sign(&[authorized_voter_keypair.as_ref()], blockhash);
let _ = cluster_info.send_vote(&vote_tx);
cluster_info.push_vote(tower_index, vote_tx);
cluster_info.push_vote(tower, vote_tx);
}
fn update_commitment_cache(

View File

@ -7,8 +7,14 @@ use solana_core::gossip_service::GossipService;
use solana_runtime::bank_forks::BankForks;
use solana_perf::packet::Packet;
use solana_sdk::signature::{Keypair, Signer};
use solana_sdk::timing::timestamp;
use solana_sdk::{
hash::Hash,
pubkey::Pubkey,
signature::{Keypair, Signer},
timing::timestamp,
transaction::Transaction,
};
use solana_vote_program::{vote_instruction, vote_state::Vote};
use std::net::UdpSocket;
use std::sync::atomic::{AtomicBool, Ordering};
use std::sync::{Arc, RwLock};
@ -269,7 +275,21 @@ pub fn cluster_info_scale() {
let mut time = Measure::start("votes");
let tx = test_tx();
warn!("tx.message.account_keys: {:?}", tx.message.account_keys);
nodes[0].0.push_vote(0, tx.clone());
let vote = Vote::new(
vec![1, 3, num_votes + 5], // slots
Hash::default(),
);
let ix = vote_instruction::vote(
&Pubkey::new_unique(), // vote_pubkey
&Pubkey::new_unique(), // authorized_voter_pubkey
vote,
);
let tx = Transaction::new_with_payer(
&[ix], // instructions
None, // payer
);
let tower = vec![num_votes + 5];
nodes[0].0.push_vote(&tower, tx.clone());
let mut success = false;
for _ in 0..(30 * 5) {
let mut not_done = 0;

View File

@ -429,6 +429,12 @@ impl VoteState {
self.last_lockout().map(|v| v.slot)
}
// Upto MAX_LOCKOUT_HISTORY many recent unexpired
// vote slots pushed onto the stack.
pub fn tower(&self) -> Vec<Slot> {
self.votes.iter().map(|v| v.slot).collect()
}
fn current_epoch(&self) -> Epoch {
if self.epoch_credits.is_empty() {
0