From 025651e0d475764d5f27f6588be2d9bcc2850ad3 Mon Sep 17 00:00:00 2001 From: Ashwin Sekar Date: Wed, 30 Aug 2023 17:00:19 -0700 Subject: [PATCH] =?UTF-8?q?ff=20cleanup:=20allow=5Fvotes=5Fto=5Fdirectly?= =?UTF-8?q?=5Fupdate=5Fvote=5Fstate=20and=20compact=5Fvot=E2=80=A6=20(#329?= =?UTF-8?q?67)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ff cleanup: allow_votes_to_directly_update_vote_state and compact_vote_state_updates --- core/src/banking_stage.rs | 75 +++++--------- core/src/cluster_info_vote_listener.rs | 20 ---- core/src/consensus.rs | 129 +++---------------------- core/src/replay_stage.rs | 38 +++----- core/src/verified_vote_packets.rs | 76 +++++---------- core/src/vote_simulator.rs | 4 +- programs/vote/src/vote_processor.rs | 61 +++++------- 7 files changed, 92 insertions(+), 311 deletions(-) diff --git a/core/src/banking_stage.rs b/core/src/banking_stage.rs index b74b4d56f8..398dad86d0 100644 --- a/core/src/banking_stage.rs +++ b/core/src/banking_stage.rs @@ -28,7 +28,7 @@ use { bank_forks::BankForks, prioritization_fee_cache::PrioritizationFeeCache, vote_sender_types::ReplayVoteSender, }, - solana_sdk::{feature_set::allow_votes_to_directly_update_vote_state, timing::AtomicInterval}, + solana_sdk::timing::AtomicInterval, std::{ cmp, env, sync::{ @@ -359,55 +359,32 @@ impl BankingStage { TOTAL_BUFFERED_PACKETS / ((num_threads - NUM_VOTE_PROCESSING_THREADS) as usize); // Keeps track of extraneous vote transactions for the vote threads let latest_unprocessed_votes = Arc::new(LatestUnprocessedVotes::new()); - let should_split_voting_threads = bank_forks - .read() - .map(|bank_forks| { - let bank = bank_forks.root_bank(); - bank.feature_set - .is_active(&allow_votes_to_directly_update_vote_state::id()) - }) - .unwrap_or(false); // Many banks that process transactions in parallel. let bank_thread_hdls: Vec> = (0..num_threads) .map(|id| { - let (packet_receiver, unprocessed_transaction_storage) = - match (id, should_split_voting_threads) { - (0, false) => ( - gossip_vote_receiver.clone(), - UnprocessedTransactionStorage::new_transaction_storage( - UnprocessedPacketBatches::with_capacity(batch_limit), - ThreadType::Voting(VoteSource::Gossip), - ), + let (packet_receiver, unprocessed_transaction_storage) = match id { + 0 => ( + gossip_vote_receiver.clone(), + UnprocessedTransactionStorage::new_vote_storage( + latest_unprocessed_votes.clone(), + VoteSource::Gossip, ), - (0, true) => ( - gossip_vote_receiver.clone(), - UnprocessedTransactionStorage::new_vote_storage( - latest_unprocessed_votes.clone(), - VoteSource::Gossip, - ), + ), + 1 => ( + tpu_vote_receiver.clone(), + UnprocessedTransactionStorage::new_vote_storage( + latest_unprocessed_votes.clone(), + VoteSource::Tpu, ), - (1, false) => ( - tpu_vote_receiver.clone(), - UnprocessedTransactionStorage::new_transaction_storage( - UnprocessedPacketBatches::with_capacity(batch_limit), - ThreadType::Voting(VoteSource::Tpu), - ), + ), + _ => ( + non_vote_receiver.clone(), + UnprocessedTransactionStorage::new_transaction_storage( + UnprocessedPacketBatches::with_capacity(batch_limit), + ThreadType::Transactions, ), - (1, true) => ( - tpu_vote_receiver.clone(), - UnprocessedTransactionStorage::new_vote_storage( - latest_unprocessed_votes.clone(), - VoteSource::Tpu, - ), - ), - _ => ( - non_vote_receiver.clone(), - UnprocessedTransactionStorage::new_transaction_storage( - UnprocessedPacketBatches::with_capacity(batch_limit), - ThreadType::Transactions, - ), - ), - }; + ), + }; let mut packet_receiver = PacketReceiver::new(id, packet_receiver, bank_forks.clone()); @@ -609,9 +586,7 @@ mod tests { poh_service::PohService, }, solana_runtime::{ - bank::Bank, - bank_forks::BankForks, - genesis_utils::{activate_feature, bootstrap_validator_stake_lamports}, + bank::Bank, bank_forks::BankForks, genesis_utils::bootstrap_validator_stake_lamports, }, solana_sdk::{ hash::Hash, @@ -1126,14 +1101,10 @@ mod tests { fn test_unprocessed_transaction_storage_full_send() { solana_logger::setup(); let GenesisConfigInfo { - mut genesis_config, + genesis_config, mint_keypair, .. } = create_slow_genesis_config(10000); - activate_feature( - &mut genesis_config, - allow_votes_to_directly_update_vote_state::id(), - ); let bank = Bank::new_no_wallclock_throttle_for_tests(&genesis_config); let bank_forks = Arc::new(RwLock::new(BankForks::new(bank))); let bank = bank_forks.read().unwrap().get(0).unwrap(); diff --git a/core/src/cluster_info_vote_listener.rs b/core/src/cluster_info_vote_listener.rs index d1f4107f4a..4a851946fa 100644 --- a/core/src/cluster_info_vote_listener.rs +++ b/core/src/cluster_info_vote_listener.rs @@ -36,7 +36,6 @@ use { }, solana_sdk::{ clock::{Slot, DEFAULT_MS_PER_SLOT, DEFAULT_TICKS_PER_SLOT}, - feature_set::allow_votes_to_directly_update_vote_state, hash::Hash, pubkey::Pubkey, signature::Signature, @@ -265,7 +264,6 @@ impl ClusterInfoVoteListener { }) .unwrap() }; - let bank_forks_clone = bank_forks.clone(); let bank_send_thread = { let exit = exit.clone(); Builder::new() @@ -276,7 +274,6 @@ impl ClusterInfoVoteListener { verified_vote_label_packets_receiver, poh_recorder, &verified_packets_sender, - bank_forks_clone, ); }) .unwrap() @@ -382,17 +379,10 @@ impl ClusterInfoVoteListener { verified_vote_label_packets_receiver: VerifiedLabelVotePacketsReceiver, poh_recorder: Arc>, verified_packets_sender: &BankingPacketSender, - bank_forks: Arc>, ) -> Result<()> { let mut verified_vote_packets = VerifiedVotePackets::default(); let mut time_since_lock = Instant::now(); let mut bank_vote_sender_state_option: Option = None; - let mut is_tower_full_vote_enabled = bank_forks - .read() - .unwrap() - .root_bank() - .feature_set - .is_active(&allow_votes_to_directly_update_vote_state::id()); loop { if exit.load(Ordering::Relaxed) { @@ -407,7 +397,6 @@ impl ClusterInfoVoteListener { if let Err(e) = verified_vote_packets.receive_and_process_vote_packets( &verified_vote_label_packets_receiver, would_be_leader, - is_tower_full_vote_enabled, ) { match e { Error::RecvTimeout(RecvTimeoutError::Disconnected) @@ -428,15 +417,6 @@ impl ClusterInfoVoteListener { verified_packets_sender, &verified_vote_packets, )?; - // Check if we've crossed the feature boundary - if !is_tower_full_vote_enabled { - is_tower_full_vote_enabled = bank_forks - .read() - .unwrap() - .root_bank() - .feature_set - .is_active(&allow_votes_to_directly_update_vote_state::id()); - } } } } diff --git a/core/src/consensus.rs b/core/src/consensus.rs index d0e7959f74..9223b49445 100644 --- a/core/src/consensus.rs +++ b/core/src/consensus.rs @@ -25,7 +25,6 @@ use { }, solana_sdk::{ clock::{Slot, UnixTimestamp}, - feature_set, hash::Hash, instruction::Instruction, pubkey::Pubkey, @@ -508,50 +507,16 @@ impl Tower { self.last_vote_tx_blockhash = Some(new_vote_tx_blockhash); } - // Returns true if we have switched the new vote instruction that directly sets vote state - pub(crate) fn is_direct_vote_state_update_enabled(bank: &Bank) -> bool { - bank.feature_set - .is_active(&feature_set::allow_votes_to_directly_update_vote_state::id()) - } - - fn apply_vote_and_generate_vote_diff( - local_vote_state: &mut VoteState, - slot: Slot, - hash: Hash, - last_voted_slot_in_bank: Option, - ) -> VoteTransaction { - let vote = Vote::new(vec![slot], hash); - let _ignored = process_vote_unchecked(local_vote_state, vote); - let slots = if let Some(last_voted_slot) = last_voted_slot_in_bank { - local_vote_state - .votes - .iter() - .map(|v| v.slot()) - .skip_while(|s| *s <= last_voted_slot) - .collect() - } else { - local_vote_state.votes.iter().map(|v| v.slot()).collect() - }; - VoteTransaction::from(Vote::new(slots, hash)) - } - pub fn last_voted_slot_in_bank(bank: &Bank, vote_account_pubkey: &Pubkey) -> Option { let vote_account = bank.get_vote_account(vote_account_pubkey)?; let vote_state = vote_account.vote_state(); vote_state.as_ref().ok()?.last_voted_slot() } - pub fn record_bank_vote(&mut self, bank: &Bank, vote_account_pubkey: &Pubkey) -> Option { - let last_voted_slot_in_bank = Self::last_voted_slot_in_bank(bank, vote_account_pubkey); - + pub fn record_bank_vote(&mut self, bank: &Bank) -> Option { // Returns the new root if one is made after applying a vote for the given bank to // `self.vote_state` - self.record_bank_vote_and_update_lockouts( - bank.slot(), - bank.hash(), - last_voted_slot_in_bank, - Self::is_direct_vote_state_update_enabled(bank), - ) + self.record_bank_vote_and_update_lockouts(bank.slot(), bank.hash()) } /// If we've recently updated the vote state by applying a new vote @@ -575,34 +540,19 @@ impl Tower { &mut self, vote_slot: Slot, vote_hash: Hash, - last_voted_slot_in_bank: Option, - is_direct_vote_state_update_enabled: bool, ) -> Option { trace!("{} record_vote for {}", self.node_pubkey, vote_slot); let old_root = self.root(); - if is_direct_vote_state_update_enabled { - let vote = Vote::new(vec![vote_slot], vote_hash); - let result = process_vote_unchecked(&mut self.vote_state, vote); - if result.is_err() { - error!( - "Error while recording vote {} {} in local tower {:?}", - vote_slot, vote_hash, result - ); - } - self.update_last_vote_from_vote_state(vote_hash); - } else { - let mut new_vote = Self::apply_vote_and_generate_vote_diff( - &mut self.vote_state, - vote_slot, - vote_hash, - last_voted_slot_in_bank, + let vote = Vote::new(vec![vote_slot], vote_hash); + let result = process_vote_unchecked(&mut self.vote_state, vote); + if result.is_err() { + error!( + "Error while recording vote {} {} in local tower {:?}", + vote_slot, vote_hash, result ); - - new_vote - .set_timestamp(self.maybe_timestamp(self.last_voted_slot().unwrap_or_default())); - self.last_vote = new_vote; - }; + } + self.update_last_vote_from_vote_state(vote_hash); let new_root = self.root(); @@ -620,7 +570,7 @@ impl Tower { #[cfg(test)] pub fn record_vote(&mut self, slot: Slot, hash: Hash) -> Option { - self.record_bank_vote_and_update_lockouts(slot, hash, self.last_voted_slot(), true) + self.record_bank_vote_and_update_lockouts(slot, hash) } /// Used for tests @@ -1579,7 +1529,7 @@ pub mod test { signature::Signer, slot_history::SlotHistory, }, - solana_vote_program::vote_state::{self, Vote, VoteStateVersions, MAX_LOCKOUT_HISTORY}, + solana_vote_program::vote_state::{Vote, VoteStateVersions, MAX_LOCKOUT_HISTORY}, std::{ collections::{HashMap, VecDeque}, fs::{remove_file, OpenOptions}, @@ -2530,61 +2480,6 @@ pub mod test { assert_eq!(voted_stakes[&2], 1); } - #[test] - fn test_apply_vote_and_generate_vote_diff() { - let mut local = VoteState::default(); - let vote = Tower::apply_vote_and_generate_vote_diff(&mut local, 0, Hash::default(), None); - assert_eq!(local.votes.len(), 1); - assert_eq!(vote.slots(), vec![0]); - assert_eq!(local.tower(), vec![0]); - } - - #[test] - fn test_apply_vote_and_generate_vote_diff_dup_vote() { - let mut local = VoteState::default(); - // If `latest_voted_slot_in_bank == Some(0)`, then we already have a vote for 0. Adding - // another vote for slot 0 should return an empty vote as the diff. - let vote = - Tower::apply_vote_and_generate_vote_diff(&mut local, 0, Hash::default(), Some(0)); - assert!(vote.is_empty()); - } - - #[test] - fn test_apply_vote_and_generate_vote_diff_next_vote() { - let mut local = VoteState::default(); - let vote = Vote { - slots: vec![0], - hash: Hash::default(), - timestamp: None, - }; - let _ = vote_state::process_vote_unchecked(&mut local, vote); - assert_eq!(local.votes.len(), 1); - let vote = - Tower::apply_vote_and_generate_vote_diff(&mut local, 1, Hash::default(), Some(0)); - assert_eq!(vote.slots(), vec![1]); - assert_eq!(local.tower(), vec![0, 1]); - } - - #[test] - fn test_apply_vote_and_generate_vote_diff_next_after_expired_vote() { - let mut local = VoteState::default(); - let vote = Vote { - slots: vec![0], - hash: Hash::default(), - timestamp: None, - }; - let _ = vote_state::process_vote_unchecked(&mut local, vote); - assert_eq!(local.votes.len(), 1); - - // First vote expired, so should be evicted from tower. Thus even with - // `latest_voted_slot_in_bank == Some(0)`, the first vote slot won't be - // observable in any of the results. - let vote = - Tower::apply_vote_and_generate_vote_diff(&mut local, 3, Hash::default(), Some(0)); - assert_eq!(vote.slots(), vec![3]); - assert_eq!(local.tower(), vec![3]); - } - #[test] fn test_check_vote_threshold_forks() { // Create the ancestor relationships diff --git a/core/src/replay_stage.rs b/core/src/replay_stage.rs index b62c0da708..58e57cacf2 100644 --- a/core/src/replay_stage.rs +++ b/core/src/replay_stage.rs @@ -67,7 +67,6 @@ use { }, solana_sdk::{ clock::{BankId, Slot, MAX_PROCESSING_AGE, NUM_CONSECUTIVE_LEADER_SLOTS}, - feature_set, genesis_config::ClusterType, hash::Hash, pubkey::Pubkey, @@ -2123,7 +2122,7 @@ impl ReplayStage { datapoint_info!("replay_stage-voted_empty_bank", ("slot", bank.slot(), i64)); } trace!("handle votable bank {}", bank.slot()); - let new_root = tower.record_bank_vote(bank, vote_account_pubkey); + let new_root = tower.record_bank_vote(bank); if let Some(new_root) = new_root { // get the root bank before squash @@ -2303,14 +2302,11 @@ impl ReplayStage { // Send our last few votes along with the new one // Compact the vote state update before sending - let should_compact = bank - .feature_set - .is_active(&feature_set::compact_vote_state_updates::id()); - let vote = match (should_compact, vote) { - (true, VoteTransaction::VoteStateUpdate(vote_state_update)) => { + let vote = match vote { + VoteTransaction::VoteStateUpdate(vote_state_update) => { VoteTransaction::CompactVoteStateUpdate(vote_state_update) } - (_, vote) => vote, + vote => vote, }; let vote_ix = switch_fork_decision .to_vote_instruction( @@ -3024,10 +3020,7 @@ impl ReplayStage { if !is_computed { // Check if our tower is behind, if so (and the feature migration flag is in use) // overwrite with the newer bank. - if let (true, Some(vote_account)) = ( - Tower::is_direct_vote_state_update_enabled(bank), - bank.get_vote_account(my_vote_pubkey), - ) { + if let Some(vote_account) = bank.get_vote_account(my_vote_pubkey) { if let Ok(mut bank_vote_state) = vote_account.vote_state().cloned() { if bank_vote_state.last_voted_slot() > tower.vote_state.last_voted_slot() @@ -6297,10 +6290,7 @@ pub(crate) mod tests { assert_eq!(reset_fork.unwrap(), 4); // Record the vote for 5 which is not on the heaviest fork. - tower.record_bank_vote( - &bank_forks.read().unwrap().get(5).unwrap(), - &Pubkey::default(), - ); + tower.record_bank_vote(&bank_forks.read().unwrap().get(5).unwrap()); // 4 should be the heaviest slot, but should not be votable // because of lockout. 5 is the heaviest slot on the same fork as the last vote. @@ -6447,10 +6437,7 @@ pub(crate) mod tests { assert_eq!(reset_fork.unwrap(), 4); // Record the vote for 4 - tower.record_bank_vote( - &bank_forks.read().unwrap().get(4).unwrap(), - &Pubkey::default(), - ); + tower.record_bank_vote(&bank_forks.read().unwrap().get(4).unwrap()); // Mark 4 as duplicate, 3 should be the heaviest slot, but should not be votable // because of lockout @@ -6684,10 +6671,7 @@ pub(crate) mod tests { .. } = vote_simulator; - tower.record_bank_vote( - &bank_forks.read().unwrap().get(first_vote).unwrap(), - &Pubkey::default(), - ); + tower.record_bank_vote(&bank_forks.read().unwrap().get(first_vote).unwrap()); // Simulate another version of slot 2 was duplicate confirmed let our_bank2_hash = bank_forks.read().unwrap().bank_hash(2).unwrap(); @@ -7079,7 +7063,7 @@ pub(crate) mod tests { // Simulate landing a vote for slot 0 landing in slot 1 let bank1 = Arc::new(Bank::new_from_parent(bank0.clone(), &Pubkey::default(), 1)); bank1.fill_bank_with_ticks_for_tests(); - tower.record_bank_vote(&bank0, &my_vote_pubkey); + tower.record_bank_vote(&bank0); ReplayStage::push_vote( &bank0, &my_vote_pubkey, @@ -7149,7 +7133,7 @@ pub(crate) mod tests { // Simulate submitting a new vote for bank 1 to the network, but the vote // not landing - tower.record_bank_vote(&bank1, &my_vote_pubkey); + tower.record_bank_vote(&bank1); ReplayStage::push_vote( &bank1, &my_vote_pubkey, @@ -7344,7 +7328,7 @@ pub(crate) mod tests { progress: &mut ProgressMap, ) -> Arc { let my_vote_pubkey = &my_vote_keypair[0].pubkey(); - tower.record_bank_vote(&parent_bank, my_vote_pubkey); + tower.record_bank_vote(&parent_bank); ReplayStage::push_vote( &parent_bank, my_vote_pubkey, diff --git a/core/src/verified_vote_packets.rs b/core/src/verified_vote_packets.rs index 102d0ef121..fa486062cf 100644 --- a/core/src/verified_vote_packets.rs +++ b/core/src/verified_vote_packets.rs @@ -209,7 +209,6 @@ impl VerifiedVotePackets { &mut self, vote_packets_receiver: &VerifiedLabelVotePacketsReceiver, would_be_leader: bool, - is_full_tower_vote_enabled: bool, ) -> Result<()> { use SingleValidatorVotes::*; const RECV_TIMEOUT: Duration = Duration::from_millis(200); @@ -233,8 +232,8 @@ impl VerifiedVotePackets { let hash = vote.hash(); let timestamp = vote.timestamp(); - match (vote, is_full_tower_vote_enabled) { - (VoteStateUpdate(_), true) => { + match vote { + VoteStateUpdate(_) => { let (latest_gossip_slot, latest_timestamp) = self.0.get(&vote_account_key).map_or((0, None), |vote| { (vote.get_latest_gossip_slot(), vote.get_latest_timestamp()) @@ -265,7 +264,7 @@ impl VerifiedVotePackets { if let Some(FullTowerVote(gossip_vote)) = self.0.get_mut(&vote_account_key) { - if slot > gossip_vote.slot && is_full_tower_vote_enabled { + if slot > gossip_vote.slot { warn!( "Originally {} submitted full tower votes, but now has reverted to incremental votes. Converting back to old format.", vote_account_key @@ -341,7 +340,7 @@ mod tests { }]) .unwrap(); verified_vote_packets - .receive_and_process_vote_packets(&r, true, false) + .receive_and_process_vote_packets(&r, true) .unwrap(); assert_eq!( verified_vote_packets @@ -361,7 +360,7 @@ mod tests { }]) .unwrap(); verified_vote_packets - .receive_and_process_vote_packets(&r, true, false) + .receive_and_process_vote_packets(&r, true) .unwrap(); assert_eq!( verified_vote_packets @@ -383,7 +382,7 @@ mod tests { }]) .unwrap(); verified_vote_packets - .receive_and_process_vote_packets(&r, true, false) + .receive_and_process_vote_packets(&r, true) .unwrap(); assert_eq!( verified_vote_packets @@ -406,7 +405,7 @@ mod tests { }]) .unwrap(); verified_vote_packets - .receive_and_process_vote_packets(&r, true, false) + .receive_and_process_vote_packets(&r, true) .unwrap(); assert_eq!( verified_vote_packets @@ -419,7 +418,7 @@ mod tests { // No new messages, should time out assert_matches!( - verified_vote_packets.receive_and_process_vote_packets(&r, true, false), + verified_vote_packets.receive_and_process_vote_packets(&r, true), Err(Error::RecvTimeout(_)) ); } @@ -448,7 +447,7 @@ mod tests { // At most `MAX_VOTES_PER_VALIDATOR` should be stored per validator verified_vote_packets - .receive_and_process_vote_packets(&r, true, false) + .receive_and_process_vote_packets(&r, true) .unwrap(); assert_eq!( verified_vote_packets @@ -486,7 +485,7 @@ mod tests { // Ingest the votes into the buffer let mut verified_vote_packets = VerifiedVotePackets(HashMap::new()); verified_vote_packets - .receive_and_process_vote_packets(&r, true, false) + .receive_and_process_vote_packets(&r, true) .unwrap(); // Create tracker for previously sent bank votes @@ -542,7 +541,7 @@ mod tests { // Ingest the votes into the buffer let mut verified_vote_packets = VerifiedVotePackets(HashMap::new()); verified_vote_packets - .receive_and_process_vote_packets(&r, true, false) + .receive_and_process_vote_packets(&r, true) .unwrap(); // One batch of vote packets per validator @@ -603,7 +602,7 @@ mod tests { .unwrap(); // Ingest the votes into the buffer verified_vote_packets - .receive_and_process_vote_packets(&r, true, false) + .receive_and_process_vote_packets(&r, true) .unwrap(); let mut gossip_votes_iterator = ValidatorGossipVotesIterator::new( my_leader_bank, @@ -624,7 +623,7 @@ mod tests { let second_vote = VoteStateUpdate::from(vec![(2, 4), (4, 3), (11, 1)]); let third_vote = VoteStateUpdate::from(vec![(2, 5), (4, 4), (11, 3), (12, 2), (13, 1)]); - for vote in [second_vote.clone(), first_vote.clone()] { + for vote in [second_vote, first_vote] { s.send(vec![VerifiedVoteMetadata { vote_account_key, vote: VoteTransaction::from(vote), @@ -636,7 +635,7 @@ mod tests { let mut verified_vote_packets = VerifiedVotePackets(HashMap::new()); verified_vote_packets - .receive_and_process_vote_packets(&r, true, true) + .receive_and_process_vote_packets(&r, true) .unwrap(); // second_vote should be kept and first_vote ignored @@ -650,14 +649,14 @@ mod tests { // Now send the third_vote, it should overwrite second_vote s.send(vec![VerifiedVoteMetadata { vote_account_key, - vote: VoteTransaction::from(third_vote.clone()), + vote: VoteTransaction::from(third_vote), packet_batch: PacketBatch::default(), signature: Signature::from([1u8; 64]), }]) .unwrap(); verified_vote_packets - .receive_and_process_vote_packets(&r, true, true) + .receive_and_process_vote_packets(&r, true) .unwrap(); let slot = verified_vote_packets .0 @@ -665,30 +664,6 @@ mod tests { .unwrap() .get_latest_gossip_slot(); assert_eq!(13, slot); - - // Now send all three, but keep the feature off - for vote in vec![second_vote, first_vote, third_vote] { - s.send(vec![VerifiedVoteMetadata { - vote_account_key, - vote: VoteTransaction::from(vote), - packet_batch: PacketBatch::default(), - signature: Signature::from([1u8; 64]), - }]) - .unwrap(); - } - let mut verified_vote_packets = VerifiedVotePackets(HashMap::new()); - verified_vote_packets - .receive_and_process_vote_packets(&r, true, false) - .unwrap(); - - assert_eq!( - 3, - verified_vote_packets - .0 - .get(&vote_account_key) - .unwrap() - .len() - ); } fn send_vote_state_update_and_process( @@ -696,7 +671,6 @@ mod tests { r: &Receiver>, vote: VoteStateUpdate, vote_account_key: Pubkey, - is_tower_full_vote_enabled: bool, verified_vote_packets: &mut VerifiedVotePackets, ) -> GossipVote { s.send(vec![VerifiedVoteMetadata { @@ -707,7 +681,7 @@ mod tests { }]) .unwrap(); verified_vote_packets - .receive_and_process_vote_packets(r, true, is_tower_full_vote_enabled) + .receive_and_process_vote_packets(r, true) .unwrap(); match verified_vote_packets.0.get(&vote_account_key).unwrap() { SingleValidatorVotes::FullTowerVote(gossip_vote) => gossip_vote.clone(), @@ -743,7 +717,6 @@ mod tests { &r, vote.clone(), vote_account_key, - true, &mut verified_vote_packets, ); assert_eq!(slot, vote.last_voted_slot().unwrap()); @@ -757,7 +730,6 @@ mod tests { &r, vote_later_ts.clone(), vote_account_key, - true, &mut verified_vote_packets, ); assert_eq!(slot, vote_later_ts.last_voted_slot().unwrap()); @@ -771,7 +743,6 @@ mod tests { &r, vote_earlier_ts, vote_account_key, - true, &mut verified_vote_packets, ); assert_eq!(slot, vote_later_ts.last_voted_slot().unwrap()); @@ -785,7 +756,6 @@ mod tests { &r, vote_no_ts, vote_account_key, - true, &mut verified_vote_packets, ); assert_eq!(slot, vote_later_ts.last_voted_slot().unwrap()); @@ -812,7 +782,7 @@ mod tests { let mut verified_vote_packets = VerifiedVotePackets(HashMap::new()); // Receive votes without the feature active verified_vote_packets - .receive_and_process_vote_packets(&r, true, false) + .receive_and_process_vote_packets(&r, true) .unwrap(); assert_eq!( 100, @@ -846,7 +816,7 @@ mod tests { // Receive votes with the feature active verified_vote_packets - .receive_and_process_vote_packets(&r, true, true) + .receive_and_process_vote_packets(&r, true) .unwrap(); if let FullTowerVote(vote) = verified_vote_packets.0.get(&vote_account_key).unwrap() { assert_eq!(200, vote.slot); @@ -873,7 +843,7 @@ mod tests { // Receive incremental votes with the feature active verified_vote_packets - .receive_and_process_vote_packets(&r, true, true) + .receive_and_process_vote_packets(&r, true) .unwrap(); // Should still store as incremental votes @@ -902,7 +872,7 @@ mod tests { // Receive full votes verified_vote_packets - .receive_and_process_vote_packets(&r, true, true) + .receive_and_process_vote_packets(&r, true) .unwrap(); assert_eq!( 42, @@ -925,7 +895,7 @@ mod tests { // Try to receive nothing should happen verified_vote_packets - .receive_and_process_vote_packets(&r, true, true) + .receive_and_process_vote_packets(&r, true) .unwrap(); if let FullTowerVote(vote) = verified_vote_packets.0.get(&vote_account_key).unwrap() { assert_eq!(42, vote.slot); @@ -946,7 +916,7 @@ mod tests { // Try to receive and vote lands as well as the conversion back to incremental votes verified_vote_packets - .receive_and_process_vote_packets(&r, true, true) + .receive_and_process_vote_packets(&r, true) .unwrap(); if let IncrementalVotes(votes) = verified_vote_packets.0.get(&vote_account_key).unwrap() { assert!(votes.contains_key(&(42, hash_42))); diff --git a/core/src/vote_simulator.rs b/core/src/vote_simulator.rs index fc0444d410..79c418bcc7 100644 --- a/core/src/vote_simulator.rs +++ b/core/src/vote_simulator.rs @@ -143,8 +143,6 @@ impl VoteSimulator { tower: &mut Tower, ) -> Vec { // Try to simulate the vote - let my_keypairs = self.validator_keypairs.get(my_pubkey).unwrap(); - let my_vote_pubkey = my_keypairs.vote_keypair.pubkey(); let ancestors = self.bank_forks.read().unwrap().ancestors(); let mut frozen_banks: Vec<_> = self .bank_forks @@ -197,7 +195,7 @@ impl VoteSimulator { return heaviest_fork_failures; } - let new_root = tower.record_bank_vote(&vote_bank, &my_vote_pubkey); + let new_root = tower.record_bank_vote(&vote_bank); if let Some(new_root) = new_root { self.set_root(new_root); } diff --git a/programs/vote/src/vote_processor.rs b/programs/vote/src/vote_processor.rs index c1a122c22b..423193f5d3 100644 --- a/programs/vote/src/vote_processor.rs +++ b/programs/vote/src/vote_processor.rs @@ -180,48 +180,31 @@ declare_process_instruction!( } VoteInstruction::UpdateVoteState(vote_state_update) | VoteInstruction::UpdateVoteStateSwitch(vote_state_update, _) => { - if invoke_context - .feature_set - .is_active(&feature_set::allow_votes_to_directly_update_vote_state::id()) - { - let sysvar_cache = invoke_context.get_sysvar_cache(); - let slot_hashes = sysvar_cache.get_slot_hashes()?; - let clock = sysvar_cache.get_clock()?; - vote_state::process_vote_state_update( - &mut me, - slot_hashes.slot_hashes(), - &clock, - vote_state_update, - &signers, - &invoke_context.feature_set, - ) - } else { - Err(InstructionError::InvalidInstructionData) - } + let sysvar_cache = invoke_context.get_sysvar_cache(); + let slot_hashes = sysvar_cache.get_slot_hashes()?; + let clock = sysvar_cache.get_clock()?; + vote_state::process_vote_state_update( + &mut me, + slot_hashes.slot_hashes(), + &clock, + vote_state_update, + &signers, + &invoke_context.feature_set, + ) } VoteInstruction::CompactUpdateVoteState(vote_state_update) | VoteInstruction::CompactUpdateVoteStateSwitch(vote_state_update, _) => { - if invoke_context - .feature_set - .is_active(&feature_set::allow_votes_to_directly_update_vote_state::id()) - && invoke_context - .feature_set - .is_active(&feature_set::compact_vote_state_updates::id()) - { - let sysvar_cache = invoke_context.get_sysvar_cache(); - let slot_hashes = sysvar_cache.get_slot_hashes()?; - let clock = sysvar_cache.get_clock()?; - vote_state::process_vote_state_update( - &mut me, - slot_hashes.slot_hashes(), - &clock, - vote_state_update, - &signers, - &invoke_context.feature_set, - ) - } else { - Err(InstructionError::InvalidInstructionData) - } + let sysvar_cache = invoke_context.get_sysvar_cache(); + let slot_hashes = sysvar_cache.get_slot_hashes()?; + let clock = sysvar_cache.get_clock()?; + vote_state::process_vote_state_update( + &mut me, + slot_hashes.slot_hashes(), + &clock, + vote_state_update, + &signers, + &invoke_context.feature_set, + ) } VoteInstruction::Withdraw(lamports) => {