Refactor vote signer code (#2368)
* Refactor vote signer code * fixed test compilation errors * address clippy errors * fix missing macro_use * move macro use * review comments
This commit is contained in:
@@ -157,11 +157,11 @@ impl Service for ComputeLeaderConfirmationService {
|
||||
pub mod tests {
|
||||
use crate::bank::Bank;
|
||||
use crate::compute_leader_confirmation_service::ComputeLeaderConfirmationService;
|
||||
use crate::create_vote_account::*;
|
||||
use crate::vote_signer_proxy::VoteSignerProxy;
|
||||
|
||||
use crate::local_vote_signer_service::LocalVoteSignerService;
|
||||
use crate::mint::Mint;
|
||||
use crate::rpc_request::RpcClient;
|
||||
use crate::vote_stage::create_new_signed_vote_transaction;
|
||||
use crate::service::Service;
|
||||
use bincode::serialize;
|
||||
use solana_sdk::hash::hash;
|
||||
use solana_sdk::signature::{Keypair, KeypairUtil};
|
||||
@@ -187,35 +187,28 @@ pub mod tests {
|
||||
})
|
||||
.collect();
|
||||
|
||||
let (signer, t_signer, signer_exit) = local_vote_signer_service().unwrap();
|
||||
let rpc_client = RpcClient::new_from_socket(signer);
|
||||
let (signer_service, addr) = LocalVoteSignerService::new();
|
||||
// Create a total of 10 vote accounts, each will have a balance of 1 (after giving 1 to
|
||||
// their vote account), for a total staking pool of 10 tokens.
|
||||
let vote_accounts: Vec<_> = (0..10)
|
||||
.map(|i| {
|
||||
// Create new validator to vote
|
||||
let validator_keypair = Keypair::new();
|
||||
let validator_keypair = Arc::new(Keypair::new());
|
||||
let last_id = ids[i];
|
||||
let vote_signer = VoteSignerProxy::new(&validator_keypair, addr.clone());
|
||||
|
||||
// Give the validator some tokens
|
||||
bank.transfer(2, &mint.keypair(), validator_keypair.pubkey(), last_id)
|
||||
.unwrap();
|
||||
let vote_account =
|
||||
create_vote_account(&validator_keypair, &bank, 1, last_id, &rpc_client)
|
||||
.expect("Expected successful creation of account");
|
||||
vote_signer
|
||||
.new_vote_account(&bank, 1, last_id)
|
||||
.expect("Expected successful creation of account");
|
||||
|
||||
let validator_keypair = Arc::new(validator_keypair);
|
||||
if i < 6 {
|
||||
let vote_tx = create_new_signed_vote_transaction(
|
||||
&last_id,
|
||||
&validator_keypair,
|
||||
(i + 1) as u64,
|
||||
&vote_account,
|
||||
&rpc_client,
|
||||
);
|
||||
let vote_tx = vote_signer.new_signed_vote_transaction(&last_id, (i + 1) as u64);
|
||||
bank.process_transaction(&vote_tx).unwrap();
|
||||
}
|
||||
(vote_account, validator_keypair)
|
||||
(vote_signer, validator_keypair)
|
||||
})
|
||||
.collect();
|
||||
|
||||
@@ -229,14 +222,8 @@ pub mod tests {
|
||||
assert_eq!(bank.confirmation_time(), std::usize::MAX);
|
||||
|
||||
// Get another validator to vote, so we now have 2/3 consensus
|
||||
let vote_account = &vote_accounts[7].0;
|
||||
let vote_tx = create_new_signed_vote_transaction(
|
||||
&ids[6],
|
||||
&vote_accounts[7].1,
|
||||
7,
|
||||
&vote_account,
|
||||
&rpc_client,
|
||||
);
|
||||
let vote_signer = &vote_accounts[7].0;
|
||||
let vote_tx = vote_signer.new_signed_vote_transaction(&ids[6], 7);
|
||||
bank.process_transaction(&vote_tx).unwrap();
|
||||
|
||||
ComputeLeaderConfirmationService::compute_confirmation(
|
||||
@@ -246,6 +233,6 @@ pub mod tests {
|
||||
);
|
||||
assert!(bank.confirmation_time() != std::usize::MAX);
|
||||
assert!(last_confirmation_time > 0);
|
||||
stop_local_vote_signer_service(t_signer, &signer_exit);
|
||||
signer_service.join().unwrap();
|
||||
}
|
||||
}
|
||||
|
||||
@@ -1,61 +0,0 @@
|
||||
use crate::bank::Bank;
|
||||
use crate::cluster_info::FULLNODE_PORT_RANGE;
|
||||
use crate::result::Result;
|
||||
use crate::rpc_request::{RpcClient, RpcRequest};
|
||||
use solana_netutil::find_available_port_in_range;
|
||||
use solana_sdk::hash::Hash;
|
||||
use solana_sdk::pubkey::Pubkey;
|
||||
use solana_sdk::signature::{Keypair, KeypairUtil, Signature};
|
||||
use solana_sdk::transaction::Transaction;
|
||||
use solana_sdk::vote_transaction::*;
|
||||
use solana_vote_signer::rpc::VoteSignerRpcService;
|
||||
use std::io;
|
||||
use std::net::{IpAddr, Ipv4Addr, SocketAddr};
|
||||
use std::sync::atomic::{AtomicBool, Ordering};
|
||||
use std::sync::Arc;
|
||||
use std::thread::{Builder, JoinHandle};
|
||||
|
||||
pub fn local_vote_signer_service() -> io::Result<(SocketAddr, JoinHandle<()>, Arc<AtomicBool>)> {
|
||||
let addr = match find_available_port_in_range(FULLNODE_PORT_RANGE) {
|
||||
Ok(port) => SocketAddr::new(IpAddr::V4(Ipv4Addr::new(0, 0, 0, 0)), port),
|
||||
Err(e) => return Err(e),
|
||||
};
|
||||
let service_addr = addr;
|
||||
let exit = Arc::new(AtomicBool::new(false));
|
||||
let thread_exit = exit.clone();
|
||||
let thread = Builder::new()
|
||||
.name("solana-vote-signer".to_string())
|
||||
.spawn(move || {
|
||||
let service = VoteSignerRpcService::new(service_addr, thread_exit);
|
||||
service.join().unwrap();
|
||||
})
|
||||
.unwrap();
|
||||
Ok((addr, thread, exit))
|
||||
}
|
||||
|
||||
pub fn stop_local_vote_signer_service(t: JoinHandle<()>, exit: &Arc<AtomicBool>) {
|
||||
exit.store(true, Ordering::Relaxed);
|
||||
t.join().unwrap();
|
||||
}
|
||||
|
||||
pub fn create_vote_account(
|
||||
node_keypair: &Keypair,
|
||||
bank: &Bank,
|
||||
num_tokens: u64,
|
||||
last_id: Hash,
|
||||
rpc_client: &RpcClient,
|
||||
) -> Result<Pubkey> {
|
||||
let msg = "Registering a new node";
|
||||
let sig = Signature::new(&node_keypair.sign(msg.as_bytes()).as_ref());
|
||||
let params = json!([node_keypair.pubkey(), sig, msg.as_bytes()]);
|
||||
let resp = RpcRequest::RegisterNode
|
||||
.make_rpc_request(&rpc_client, 1, Some(params))
|
||||
.unwrap();
|
||||
let new_vote_account: Pubkey = serde_json::from_value(resp).unwrap();
|
||||
|
||||
// Create and register the new vote account
|
||||
let tx = Transaction::vote_account_new(node_keypair, new_vote_account, last_id, num_tokens, 0);
|
||||
bank.process_transaction(&tx)?;
|
||||
|
||||
Ok(new_vote_account)
|
||||
}
|
||||
@@ -652,18 +652,17 @@ impl Service for Fullnode {
|
||||
mod tests {
|
||||
use crate::bank::Bank;
|
||||
use crate::cluster_info::Node;
|
||||
use crate::create_vote_account::*;
|
||||
use crate::db_ledger::*;
|
||||
use crate::entry::make_consecutive_blobs;
|
||||
use crate::fullnode::{Fullnode, FullnodeReturnType, NodeRole, TvuReturnType};
|
||||
use crate::leader_scheduler::{
|
||||
make_active_set_entries, LeaderScheduler, LeaderSchedulerConfig,
|
||||
};
|
||||
use crate::rpc_request::{RpcClient, RpcRequest};
|
||||
use crate::local_vote_signer_service::LocalVoteSignerService;
|
||||
use crate::service::Service;
|
||||
use crate::streamer::responder;
|
||||
use solana_sdk::pubkey::Pubkey;
|
||||
use solana_sdk::signature::{Keypair, KeypairUtil, Signature};
|
||||
use crate::vote_signer_proxy::VoteSignerProxy;
|
||||
use solana_sdk::signature::{Keypair, KeypairUtil};
|
||||
use std::cmp;
|
||||
use std::fs::remove_dir_all;
|
||||
use std::net::UdpSocket;
|
||||
@@ -861,7 +860,7 @@ mod tests {
|
||||
|
||||
// Write the entries to the ledger that will cause leader rotation
|
||||
// after the bootstrap height
|
||||
let (signer, t_signer, signer_exit) = local_vote_signer_service().unwrap();
|
||||
let (signer_service, signer) = LocalVoteSignerService::new();
|
||||
let (active_set_entries, validator_vote_account_id) = make_active_set_entries(
|
||||
&validator_keypair,
|
||||
signer,
|
||||
@@ -913,7 +912,8 @@ mod tests {
|
||||
|
||||
{
|
||||
// Test that a node knows to transition to a validator based on parsing the ledger
|
||||
let leader_vote_id = register_node(signer, bootstrap_leader_keypair.clone());
|
||||
let vote_signer = VoteSignerProxy::new(&bootstrap_leader_keypair, signer);
|
||||
let leader_vote_id = vote_signer.vote_account.clone();
|
||||
let bootstrap_leader = Fullnode::new(
|
||||
bootstrap_leader_node,
|
||||
&bootstrap_leader_ledger_path,
|
||||
@@ -962,21 +962,7 @@ mod tests {
|
||||
DbLedger::destroy(&path).expect("Expected successful database destruction");
|
||||
let _ignored = remove_dir_all(&path);
|
||||
}
|
||||
stop_local_vote_signer_service(t_signer, &signer_exit);
|
||||
}
|
||||
|
||||
fn register_node(signer: SocketAddr, keypair: Arc<Keypair>) -> Pubkey {
|
||||
let rpc_client = RpcClient::new_from_socket(signer);
|
||||
|
||||
let msg = "Registering a new node";
|
||||
let sig = Signature::new(&keypair.sign(msg.as_bytes()).as_ref());
|
||||
|
||||
let params = json!([keypair.pubkey(), sig, msg.as_bytes()]);
|
||||
let resp = RpcRequest::RegisterNode
|
||||
.make_rpc_request(&rpc_client, 1, Some(params))
|
||||
.unwrap();
|
||||
let vote_account_id: Pubkey = serde_json::from_value(resp).unwrap();
|
||||
vote_account_id
|
||||
signer_service.join().unwrap();
|
||||
}
|
||||
|
||||
#[test]
|
||||
@@ -1013,7 +999,7 @@ mod tests {
|
||||
// after the bootstrap height
|
||||
//
|
||||
// 2) A vote from the validator
|
||||
let (signer, t_signer, signer_exit) = local_vote_signer_service().unwrap();
|
||||
let (signer_service, signer) = LocalVoteSignerService::new();
|
||||
let (active_set_entries, _validator_vote_account_id) = make_active_set_entries(
|
||||
&validator_keypair,
|
||||
signer,
|
||||
@@ -1056,7 +1042,8 @@ mod tests {
|
||||
);
|
||||
|
||||
let validator_keypair = Arc::new(validator_keypair);
|
||||
let vote_id = register_node(signer, validator_keypair.clone());
|
||||
let vote_signer = VoteSignerProxy::new(&validator_keypair, signer);
|
||||
let vote_id = vote_signer.vote_account.clone();
|
||||
// Start the validator
|
||||
let mut validator = Fullnode::new(
|
||||
validator_node,
|
||||
@@ -1132,7 +1119,7 @@ mod tests {
|
||||
);
|
||||
|
||||
// Shut down
|
||||
stop_local_vote_signer_service(t_signer, &signer_exit);
|
||||
signer_service.join().unwrap();
|
||||
t_responder.join().expect("responder thread join");
|
||||
validator.close().unwrap();
|
||||
DbLedger::destroy(&validator_ledger_path)
|
||||
|
||||
@@ -536,14 +536,14 @@ pub fn make_active_set_entries(
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
use crate::bank::Bank;
|
||||
use crate::create_vote_account::*;
|
||||
use crate::leader_scheduler::{
|
||||
LeaderScheduler, LeaderSchedulerConfig, DEFAULT_BOOTSTRAP_HEIGHT,
|
||||
DEFAULT_LEADER_ROTATION_INTERVAL, DEFAULT_SEED_ROTATION_INTERVAL,
|
||||
};
|
||||
use crate::local_vote_signer_service::LocalVoteSignerService;
|
||||
use crate::mint::Mint;
|
||||
use crate::rpc_request::RpcClient;
|
||||
use crate::vote_stage::create_new_signed_vote_transaction;
|
||||
use crate::service::Service;
|
||||
use crate::vote_signer_proxy::VoteSignerProxy;
|
||||
use hashbrown::HashSet;
|
||||
use solana_sdk::hash::Hash;
|
||||
use solana_sdk::pubkey::Pubkey;
|
||||
@@ -559,16 +559,8 @@ mod tests {
|
||||
HashSet::from_iter(slice.iter().cloned())
|
||||
}
|
||||
|
||||
fn push_vote(
|
||||
keypair: &Arc<Keypair>,
|
||||
vote_account: &Pubkey,
|
||||
bank: &Bank,
|
||||
height: u64,
|
||||
last_id: Hash,
|
||||
rpc_client: &RpcClient,
|
||||
) {
|
||||
let new_vote_tx =
|
||||
create_new_signed_vote_transaction(&last_id, keypair, height, vote_account, rpc_client);
|
||||
fn push_vote(vote_signer: &VoteSignerProxy, bank: &Bank, height: u64, last_id: Hash) {
|
||||
let new_vote_tx = vote_signer.new_signed_vote_transaction(&last_id, height);
|
||||
|
||||
bank.process_transaction(&new_vote_tx).unwrap();
|
||||
}
|
||||
@@ -607,11 +599,11 @@ mod tests {
|
||||
.last()
|
||||
.expect("Mint should not create empty genesis entries")
|
||||
.id;
|
||||
let (signer, t_signer, signer_exit) = local_vote_signer_service().unwrap();
|
||||
let rpc_client = RpcClient::new_from_socket(signer);
|
||||
let (signer_service, signer) = LocalVoteSignerService::new();
|
||||
for i in 0..num_validators {
|
||||
let new_validator = Keypair::new();
|
||||
let new_pubkey = new_validator.pubkey();
|
||||
let vote_signer = VoteSignerProxy::new(&Arc::new(new_validator), signer.clone());
|
||||
validators.push(new_pubkey);
|
||||
// Give the validator some tokens
|
||||
bank.transfer(
|
||||
@@ -623,24 +615,12 @@ mod tests {
|
||||
.unwrap();
|
||||
|
||||
// Create a vote account
|
||||
let new_vote_account = create_vote_account(
|
||||
&new_validator,
|
||||
&bank,
|
||||
num_vote_account_tokens as u64,
|
||||
mint.last_id(),
|
||||
&rpc_client,
|
||||
)
|
||||
.unwrap();
|
||||
vote_signer
|
||||
.new_vote_account(&bank, num_vote_account_tokens as u64, mint.last_id())
|
||||
.unwrap();
|
||||
// Vote to make the validator part of the active set for the entire test
|
||||
// (we made the active_window_length large enough at the beginning of the test)
|
||||
push_vote(
|
||||
&Arc::new(new_validator),
|
||||
&new_vote_account,
|
||||
&bank,
|
||||
1,
|
||||
mint.last_id(),
|
||||
&rpc_client,
|
||||
);
|
||||
push_vote(&vote_signer, &bank, 1, mint.last_id());
|
||||
}
|
||||
|
||||
// The scheduled leader during the bootstrapping period (assuming a seed + schedule
|
||||
@@ -717,7 +697,7 @@ mod tests {
|
||||
Some((current_leader, slot))
|
||||
);
|
||||
}
|
||||
stop_local_vote_signer_service(t_signer, &signer_exit);
|
||||
signer_service.join().unwrap();
|
||||
}
|
||||
|
||||
#[test]
|
||||
@@ -737,8 +717,7 @@ mod tests {
|
||||
let start_height = 3;
|
||||
let num_old_ids = 20;
|
||||
let mut old_ids = HashSet::new();
|
||||
let (signer, t_signer, signer_exit) = local_vote_signer_service().unwrap();
|
||||
let rpc_client = RpcClient::new_from_socket(signer);
|
||||
let (signer_service, signer) = LocalVoteSignerService::new();
|
||||
for _ in 0..num_old_ids {
|
||||
let new_keypair = Keypair::new();
|
||||
let pk = new_keypair.pubkey();
|
||||
@@ -749,18 +728,13 @@ mod tests {
|
||||
.unwrap();
|
||||
|
||||
// Create a vote account
|
||||
let new_vote_account =
|
||||
create_vote_account(&new_keypair, &bank, 1, mint.last_id(), &rpc_client).unwrap();
|
||||
let vote_signer = VoteSignerProxy::new(&Arc::new(new_keypair), signer);
|
||||
vote_signer
|
||||
.new_vote_account(&bank, 1 as u64, mint.last_id())
|
||||
.unwrap();
|
||||
|
||||
// Push a vote for the account
|
||||
push_vote(
|
||||
&Arc::new(new_keypair),
|
||||
&new_vote_account,
|
||||
&bank,
|
||||
start_height,
|
||||
mint.last_id(),
|
||||
&rpc_client,
|
||||
);
|
||||
push_vote(&vote_signer, &bank, start_height, mint.last_id());
|
||||
}
|
||||
|
||||
// Insert a bunch of votes at height "start_height + active_window_length"
|
||||
@@ -775,16 +749,16 @@ mod tests {
|
||||
.unwrap();
|
||||
|
||||
// Create a vote account
|
||||
let new_vote_account =
|
||||
create_vote_account(&new_keypair, &bank, 1, mint.last_id(), &rpc_client).unwrap();
|
||||
let vote_signer = VoteSignerProxy::new(&Arc::new(new_keypair), signer);
|
||||
vote_signer
|
||||
.new_vote_account(&bank, 1 as u64, mint.last_id())
|
||||
.unwrap();
|
||||
|
||||
push_vote(
|
||||
&Arc::new(new_keypair),
|
||||
&new_vote_account,
|
||||
&vote_signer,
|
||||
&bank,
|
||||
start_height + active_window_length,
|
||||
mint.last_id(),
|
||||
&rpc_client,
|
||||
);
|
||||
}
|
||||
|
||||
@@ -803,7 +777,7 @@ mod tests {
|
||||
let result =
|
||||
leader_scheduler.get_active_set(2 * active_window_length + start_height, &bank);
|
||||
assert!(result.is_empty());
|
||||
stop_local_vote_signer_service(t_signer, &signer_exit);
|
||||
signer_service.join().unwrap();
|
||||
}
|
||||
|
||||
#[test]
|
||||
@@ -1046,11 +1020,11 @@ mod tests {
|
||||
.last()
|
||||
.expect("Mint should not create empty genesis entries")
|
||||
.id;
|
||||
let (signer, t_signer, signer_exit) = local_vote_signer_service().unwrap();
|
||||
let rpc_client = RpcClient::new_from_socket(signer);
|
||||
let (signer_service, signer) = LocalVoteSignerService::new();
|
||||
for i in 0..num_validators {
|
||||
let new_validator = Keypair::new();
|
||||
let new_pubkey = new_validator.pubkey();
|
||||
let vote_signer = VoteSignerProxy::new(&Arc::new(new_validator), signer);
|
||||
validators.push(new_pubkey);
|
||||
// Give the validator some tokens
|
||||
bank.transfer(
|
||||
@@ -1062,23 +1036,16 @@ mod tests {
|
||||
.unwrap();
|
||||
|
||||
// Create a vote account
|
||||
let new_vote_account = create_vote_account(
|
||||
&new_validator,
|
||||
&bank,
|
||||
num_vote_account_tokens as u64,
|
||||
mint.last_id(),
|
||||
&rpc_client,
|
||||
)
|
||||
.unwrap();
|
||||
vote_signer
|
||||
.new_vote_account(&bank, num_vote_account_tokens as u64, mint.last_id())
|
||||
.unwrap();
|
||||
|
||||
// Vote at height i * active_window_length for validator i
|
||||
push_vote(
|
||||
&Arc::new(new_validator),
|
||||
&new_vote_account,
|
||||
&vote_signer,
|
||||
&bank,
|
||||
i * active_window_length + bootstrap_height,
|
||||
mint.last_id(),
|
||||
&rpc_client,
|
||||
);
|
||||
}
|
||||
|
||||
@@ -1096,7 +1063,7 @@ mod tests {
|
||||
|
||||
assert_eq!(vec![expected], *result);
|
||||
}
|
||||
stop_local_vote_signer_service(t_signer, &signer_exit);
|
||||
signer_service.join().unwrap();
|
||||
}
|
||||
|
||||
#[test]
|
||||
@@ -1117,29 +1084,16 @@ mod tests {
|
||||
// window
|
||||
let initial_vote_height = 1;
|
||||
|
||||
let (signer, t_signer, signer_exit) = local_vote_signer_service().unwrap();
|
||||
let rpc_client = RpcClient::new_from_socket(signer);
|
||||
let (signer_service, signer) = LocalVoteSignerService::new();
|
||||
let vote_signer = VoteSignerProxy::new(&Arc::new(leader_keypair), signer);
|
||||
// Create a vote account
|
||||
let new_vote_account =
|
||||
create_vote_account(&leader_keypair, &bank, 1, mint.last_id(), &rpc_client).unwrap();
|
||||
let leader_keypair = Arc::new(leader_keypair);
|
||||
vote_signer
|
||||
.new_vote_account(&bank, 1 as u64, mint.last_id())
|
||||
.unwrap();
|
||||
|
||||
// Vote twice
|
||||
push_vote(
|
||||
&leader_keypair,
|
||||
&new_vote_account,
|
||||
&bank,
|
||||
initial_vote_height,
|
||||
mint.last_id(),
|
||||
&rpc_client,
|
||||
);
|
||||
push_vote(
|
||||
&leader_keypair,
|
||||
&new_vote_account,
|
||||
&bank,
|
||||
initial_vote_height + 1,
|
||||
mint.last_id(),
|
||||
&rpc_client,
|
||||
);
|
||||
push_vote(&vote_signer, &bank, initial_vote_height, mint.last_id());
|
||||
push_vote(&vote_signer, &bank, initial_vote_height + 1, mint.last_id());
|
||||
|
||||
let result =
|
||||
leader_scheduler.get_active_set(initial_vote_height + active_window_length, &bank);
|
||||
@@ -1147,7 +1101,7 @@ mod tests {
|
||||
let result =
|
||||
leader_scheduler.get_active_set(initial_vote_height + active_window_length + 1, &bank);
|
||||
assert!(result.is_empty());
|
||||
stop_local_vote_signer_service(t_signer, &signer_exit);
|
||||
signer_service.join().unwrap();
|
||||
}
|
||||
|
||||
#[test]
|
||||
@@ -1268,23 +1222,17 @@ mod tests {
|
||||
// Create and add validator to the active set
|
||||
let validator_keypair = Keypair::new();
|
||||
let validator_id = validator_keypair.pubkey();
|
||||
let (signer, t_signer, signer_exit) = local_vote_signer_service().unwrap();
|
||||
let rpc_client = RpcClient::new_from_socket(signer);
|
||||
let (signer_service, signer) = LocalVoteSignerService::new();
|
||||
if add_validator {
|
||||
bank.transfer(5, &mint.keypair(), validator_id, last_id)
|
||||
.unwrap();
|
||||
// Create a vote account
|
||||
let new_vote_account =
|
||||
create_vote_account(&validator_keypair, &bank, 1, mint.last_id(), &rpc_client)
|
||||
.unwrap();
|
||||
push_vote(
|
||||
&Arc::new(validator_keypair),
|
||||
&new_vote_account,
|
||||
&bank,
|
||||
initial_vote_height,
|
||||
mint.last_id(),
|
||||
&rpc_client,
|
||||
);
|
||||
let vote_signer = VoteSignerProxy::new(&Arc::new(validator_keypair), signer);
|
||||
vote_signer
|
||||
.new_vote_account(&bank, 1 as u64, mint.last_id())
|
||||
.unwrap();
|
||||
|
||||
push_vote(&vote_signer, &bank, initial_vote_height, mint.last_id());
|
||||
}
|
||||
|
||||
// Make sure the bootstrap leader, not the validator, is picked again on next slot
|
||||
@@ -1311,24 +1259,13 @@ mod tests {
|
||||
.unwrap();
|
||||
|
||||
// Create a vote account
|
||||
let new_vote_account = create_vote_account(
|
||||
&bootstrap_leader_keypair,
|
||||
&bank,
|
||||
vote_account_tokens,
|
||||
mint.last_id(),
|
||||
&rpc_client,
|
||||
)
|
||||
.unwrap();
|
||||
let vote_signer = VoteSignerProxy::new(&Arc::new(bootstrap_leader_keypair), signer);
|
||||
vote_signer
|
||||
.new_vote_account(&bank, vote_account_tokens as u64, mint.last_id())
|
||||
.unwrap();
|
||||
|
||||
// Add leader to the active set
|
||||
push_vote(
|
||||
&Arc::new(bootstrap_leader_keypair),
|
||||
&new_vote_account,
|
||||
&bank,
|
||||
initial_vote_height,
|
||||
mint.last_id(),
|
||||
&rpc_client,
|
||||
);
|
||||
push_vote(&vote_signer, &bank, initial_vote_height, mint.last_id());
|
||||
|
||||
leader_scheduler.generate_schedule(bootstrap_height, &bank);
|
||||
|
||||
@@ -1339,7 +1276,7 @@ mod tests {
|
||||
} else {
|
||||
assert!(leader_scheduler.leader_schedule[0] == bootstrap_leader_id);
|
||||
}
|
||||
stop_local_vote_signer_service(t_signer, &signer_exit);
|
||||
signer_service.join().unwrap();
|
||||
}
|
||||
|
||||
#[test]
|
||||
@@ -1444,40 +1381,23 @@ mod tests {
|
||||
// Create a vote account for the validator
|
||||
bank.transfer(5, &mint.keypair(), validator_id, last_id)
|
||||
.unwrap();
|
||||
let (signer, t_signer, signer_exit) = local_vote_signer_service().unwrap();
|
||||
let rpc_client = RpcClient::new_from_socket(signer);
|
||||
let new_validator_vote_account =
|
||||
create_vote_account(&validator_keypair, &bank, 1, mint.last_id(), &rpc_client).unwrap();
|
||||
push_vote(
|
||||
&Arc::new(validator_keypair),
|
||||
&new_validator_vote_account,
|
||||
&bank,
|
||||
initial_vote_height,
|
||||
mint.last_id(),
|
||||
&rpc_client,
|
||||
);
|
||||
let (signer_service, signer) = LocalVoteSignerService::new();
|
||||
let vote_signer = VoteSignerProxy::new(&Arc::new(validator_keypair), signer);
|
||||
vote_signer
|
||||
.new_vote_account(&bank, 1 as u64, mint.last_id())
|
||||
.unwrap();
|
||||
push_vote(&vote_signer, &bank, initial_vote_height, mint.last_id());
|
||||
|
||||
// Create a vote account for the leader
|
||||
bank.transfer(5, &mint.keypair(), bootstrap_leader_id, last_id)
|
||||
.unwrap();
|
||||
let new_leader_vote_account = create_vote_account(
|
||||
&bootstrap_leader_keypair,
|
||||
&bank,
|
||||
1,
|
||||
mint.last_id(),
|
||||
&rpc_client,
|
||||
)
|
||||
.unwrap();
|
||||
let vote_signer = VoteSignerProxy::new(&Arc::new(bootstrap_leader_keypair), signer);
|
||||
vote_signer
|
||||
.new_vote_account(&bank, 1 as u64, mint.last_id())
|
||||
.unwrap();
|
||||
|
||||
// Add leader to the active set
|
||||
push_vote(
|
||||
&Arc::new(bootstrap_leader_keypair),
|
||||
&new_leader_vote_account,
|
||||
&bank,
|
||||
initial_vote_height,
|
||||
mint.last_id(),
|
||||
&rpc_client,
|
||||
);
|
||||
push_vote(&vote_signer, &bank, initial_vote_height, mint.last_id());
|
||||
|
||||
// Generate the schedule
|
||||
leader_scheduler.generate_schedule(bootstrap_height, &bank);
|
||||
@@ -1535,6 +1455,6 @@ mod tests {
|
||||
.max_height_for_leader(bootstrap_height + 2 * seed_rotation_interval + 1),
|
||||
None
|
||||
);
|
||||
stop_local_vote_signer_service(t_signer, &signer_exit);
|
||||
signer_service.join().unwrap();
|
||||
}
|
||||
}
|
||||
|
||||
@@ -28,7 +28,6 @@ pub mod crds_gossip_pull;
|
||||
pub mod crds_gossip_push;
|
||||
pub mod crds_traits_impls;
|
||||
pub mod crds_value;
|
||||
pub mod create_vote_account;
|
||||
#[macro_use]
|
||||
pub mod contact_info;
|
||||
pub mod cluster_info;
|
||||
@@ -42,6 +41,7 @@ pub mod fetch_stage;
|
||||
pub mod fullnode;
|
||||
pub mod gossip_service;
|
||||
pub mod leader_scheduler;
|
||||
pub mod local_vote_signer_service;
|
||||
pub mod mint;
|
||||
pub mod packet;
|
||||
pub mod poh;
|
||||
@@ -68,7 +68,7 @@ pub mod thin_client;
|
||||
pub mod tpu;
|
||||
pub mod tpu_forwarder;
|
||||
pub mod tvu;
|
||||
pub mod vote_stage;
|
||||
pub mod vote_signer_proxy;
|
||||
pub mod window;
|
||||
pub mod window_service;
|
||||
|
||||
|
||||
44
src/local_vote_signer_service.rs
Normal file
44
src/local_vote_signer_service.rs
Normal file
@@ -0,0 +1,44 @@
|
||||
//! The `local_vote_signer_service` can be started locally to sign fullnode votes
|
||||
|
||||
use crate::cluster_info::FULLNODE_PORT_RANGE;
|
||||
use crate::service::Service;
|
||||
use solana_vote_signer::rpc::VoteSignerRpcService;
|
||||
use std::net::{IpAddr, Ipv4Addr, SocketAddr};
|
||||
use std::sync::atomic::{AtomicBool, Ordering};
|
||||
use std::sync::Arc;
|
||||
use std::thread::{self, Builder, JoinHandle};
|
||||
|
||||
pub struct LocalVoteSignerService {
|
||||
thread: JoinHandle<()>,
|
||||
exit: Arc<AtomicBool>,
|
||||
}
|
||||
|
||||
impl Service for LocalVoteSignerService {
|
||||
type JoinReturnType = ();
|
||||
|
||||
fn join(self) -> thread::Result<()> {
|
||||
self.exit.store(true, Ordering::Relaxed);
|
||||
self.thread.join()
|
||||
}
|
||||
}
|
||||
|
||||
impl LocalVoteSignerService {
|
||||
#[allow(clippy::new_ret_no_self)]
|
||||
pub fn new() -> (Self, SocketAddr) {
|
||||
let addr = match solana_netutil::find_available_port_in_range(FULLNODE_PORT_RANGE) {
|
||||
Ok(port) => SocketAddr::new(IpAddr::V4(Ipv4Addr::new(0, 0, 0, 0)), port),
|
||||
Err(_e) => panic!("Failed to find an available port for local vote signer service"),
|
||||
};
|
||||
let exit = Arc::new(AtomicBool::new(false));
|
||||
let thread_exit = exit.clone();
|
||||
let thread = Builder::new()
|
||||
.name("solana-vote-signer".to_string())
|
||||
.spawn(move || {
|
||||
let service = VoteSignerRpcService::new(addr, thread_exit);
|
||||
service.join().unwrap();
|
||||
})
|
||||
.unwrap();
|
||||
|
||||
(Self { thread, exit }, addr)
|
||||
}
|
||||
}
|
||||
@@ -9,10 +9,9 @@ use solana_sdk::hash::Hash;
|
||||
use crate::entry::EntrySlice;
|
||||
use crate::packet::BlobError;
|
||||
use crate::result::{Error, Result};
|
||||
use crate::rpc_request::RpcClient;
|
||||
use crate::service::Service;
|
||||
use crate::streamer::{responder, BlobSender};
|
||||
use crate::vote_stage::send_validator_vote;
|
||||
use crate::vote_signer_proxy::VoteSignerProxy;
|
||||
use log::Level;
|
||||
use solana_metrics::{influxdb, submit};
|
||||
use solana_sdk::pubkey::Pubkey;
|
||||
@@ -66,8 +65,8 @@ impl ReplayStage {
|
||||
cluster_info: &Arc<RwLock<ClusterInfo>>,
|
||||
window_receiver: &EntryReceiver,
|
||||
keypair: &Arc<Keypair>,
|
||||
vote_account_id: &Pubkey,
|
||||
vote_signer_rpc: &RpcClient,
|
||||
_vote_account_id: &Pubkey,
|
||||
vote_signer: &VoteSignerProxy,
|
||||
vote_blob_sender: Option<&BlobSender>,
|
||||
ledger_entry_sender: &EntrySender,
|
||||
entry_height: &mut u64,
|
||||
@@ -141,15 +140,9 @@ impl ReplayStage {
|
||||
|
||||
if 0 == num_ticks_to_next_vote {
|
||||
if let Some(sender) = vote_blob_sender {
|
||||
send_validator_vote(
|
||||
bank,
|
||||
&keypair,
|
||||
&vote_account_id,
|
||||
vote_signer_rpc,
|
||||
&cluster_info,
|
||||
sender,
|
||||
)
|
||||
.unwrap();
|
||||
vote_signer
|
||||
.send_validator_vote(bank, &cluster_info, sender)
|
||||
.unwrap();
|
||||
}
|
||||
}
|
||||
let (scheduled_leader, _) = bank
|
||||
@@ -219,14 +212,14 @@ impl ReplayStage {
|
||||
|
||||
let keypair = Arc::new(keypair);
|
||||
let vote_account_id = *vote_account_id;
|
||||
|
||||
let rpc_client = RpcClient::new_from_socket(*vote_signer_addr);
|
||||
let vote_signer_addr = *vote_signer_addr;
|
||||
let t_replay = Builder::new()
|
||||
.name("solana-replay-stage".to_string())
|
||||
.spawn(move || {
|
||||
let _exit = Finalizer::new(exit);
|
||||
let mut entry_height_ = entry_height;
|
||||
let mut last_entry_id = last_entry_id;
|
||||
let vote_signer = VoteSignerProxy::new(&keypair, vote_signer_addr);
|
||||
loop {
|
||||
let (leader_id, _) = bank
|
||||
.get_current_leader()
|
||||
@@ -250,7 +243,7 @@ impl ReplayStage {
|
||||
&window_receiver,
|
||||
&keypair,
|
||||
&vote_account_id,
|
||||
&rpc_client,
|
||||
&vote_signer,
|
||||
Some(&vote_blob_sender),
|
||||
&ledger_entry_sender,
|
||||
&mut entry_height_,
|
||||
@@ -299,17 +292,15 @@ mod test {
|
||||
make_active_set_entries, LeaderScheduler, LeaderSchedulerConfig,
|
||||
};
|
||||
|
||||
use crate::create_vote_account::*;
|
||||
use crate::local_vote_signer_service::LocalVoteSignerService;
|
||||
use crate::packet::BlobError;
|
||||
use crate::replay_stage::{ReplayStage, ReplayStageReturnType};
|
||||
use crate::result::Error;
|
||||
use crate::rpc_request::RpcClient;
|
||||
use crate::service::Service;
|
||||
use crate::vote_stage::send_validator_vote;
|
||||
use crate::vote_signer_proxy::VoteSignerProxy;
|
||||
use solana_sdk::hash::Hash;
|
||||
use solana_sdk::signature::{Keypair, KeypairUtil};
|
||||
use std::fs::remove_dir_all;
|
||||
use std::net::{IpAddr, Ipv4Addr, SocketAddr};
|
||||
use std::sync::atomic::{AtomicBool, Ordering};
|
||||
use std::sync::mpsc::channel;
|
||||
use std::sync::{Arc, RwLock};
|
||||
@@ -344,7 +335,7 @@ mod test {
|
||||
// Write two entries to the ledger so that the validator is in the active set:
|
||||
// 1) Give the validator a nonzero number of tokens 2) A vote from the validator .
|
||||
// This will cause leader rotation after the bootstrap height
|
||||
let (signer, t_signer, signer_exit) = local_vote_signer_service().unwrap();
|
||||
let (signer_service, signer) = LocalVoteSignerService::new();
|
||||
let (active_set_entries, vote_account_id) =
|
||||
make_active_set_entries(&my_keypair, signer, &mint.keypair(), &last_id, &last_id, 0);
|
||||
last_id = active_set_entries.last().unwrap().id;
|
||||
@@ -443,7 +434,7 @@ mod test {
|
||||
&entries_to_send[..leader_rotation_index + 1]
|
||||
);
|
||||
|
||||
stop_local_vote_signer_service(t_signer, &signer_exit);
|
||||
signer_service.join().unwrap();
|
||||
|
||||
assert_eq!(exit.load(Ordering::Relaxed), true);
|
||||
|
||||
@@ -480,12 +471,13 @@ mod test {
|
||||
let cluster_info_me = Arc::new(RwLock::new(ClusterInfo::new(my_node.info.clone())));
|
||||
|
||||
// Set up the replay stage
|
||||
let vote_account_id = Keypair::new().pubkey();
|
||||
let (signer_service, signer) = LocalVoteSignerService::new();
|
||||
let bank = Arc::new(bank);
|
||||
let (entry_sender, entry_receiver) = channel();
|
||||
let exit = Arc::new(AtomicBool::new(false));
|
||||
let signer = SocketAddr::new(IpAddr::V4(Ipv4Addr::new(0, 0, 0, 0)), 0);
|
||||
let my_keypair = Arc::new(my_keypair);
|
||||
let vote_signer = VoteSignerProxy::new(&my_keypair, signer);
|
||||
let vote_account_id = vote_signer.vote_account.clone();
|
||||
let (replay_stage, ledger_writer_recv) = ReplayStage::new(
|
||||
my_keypair.clone(),
|
||||
&vote_account_id,
|
||||
@@ -501,14 +493,7 @@ mod test {
|
||||
// Vote sender should error because no leader contact info is found in the
|
||||
// ClusterInfo
|
||||
let (mock_sender, _mock_receiver) = channel();
|
||||
let _vote_err = send_validator_vote(
|
||||
&bank,
|
||||
&my_keypair,
|
||||
&vote_account_id,
|
||||
&RpcClient::new_from_socket(signer),
|
||||
&cluster_info_me,
|
||||
&mock_sender,
|
||||
);
|
||||
let _vote_err = vote_signer.send_validator_vote(&bank, &cluster_info_me, &mock_sender);
|
||||
|
||||
// Send ReplayStage an entry, should see it on the ledger writer receiver
|
||||
let next_tick = create_ticks(
|
||||
@@ -527,6 +512,7 @@ mod test {
|
||||
|
||||
assert_eq!(next_tick, received_tick);
|
||||
drop(entry_sender);
|
||||
signer_service.join().unwrap();
|
||||
replay_stage
|
||||
.join()
|
||||
.expect("Expect successful ReplayStage exit");
|
||||
@@ -560,7 +546,7 @@ mod test {
|
||||
// Write two entries to the ledger so that the validator is in the active set:
|
||||
// 1) Give the validator a nonzero number of tokens 2) A vote from the validator.
|
||||
// This will cause leader rotation after the bootstrap height
|
||||
let (signer, t_signer, signer_exit) = local_vote_signer_service().unwrap();
|
||||
let (signer_service, signer) = LocalVoteSignerService::new();
|
||||
let (active_set_entries, vote_account_id) =
|
||||
make_active_set_entries(&my_keypair, signer, &mint.keypair(), &last_id, &last_id, 0);
|
||||
last_id = active_set_entries.last().unwrap().id;
|
||||
@@ -606,11 +592,12 @@ mod test {
|
||||
let cluster_info_me = Arc::new(RwLock::new(ClusterInfo::new(my_node.info.clone())));
|
||||
|
||||
// Set up the replay stage
|
||||
let my_keypair = Arc::new(my_keypair);
|
||||
let signer_proxy = VoteSignerProxy::new(&my_keypair, signer);
|
||||
let vote_account_id = Arc::new(vote_account_id);
|
||||
let bank = Arc::new(bank);
|
||||
let (entry_sender, entry_receiver) = channel();
|
||||
let exit = Arc::new(AtomicBool::new(false));
|
||||
let my_keypair = Arc::new(my_keypair);
|
||||
let (replay_stage, ledger_writer_recv) = ReplayStage::new(
|
||||
my_keypair.clone(),
|
||||
&vote_account_id,
|
||||
@@ -626,14 +613,7 @@ mod test {
|
||||
// Vote sender should error because no leader contact info is found in the
|
||||
// ClusterInfo
|
||||
let (mock_sender, _mock_receiver) = channel();
|
||||
let _vote_err = send_validator_vote(
|
||||
&bank,
|
||||
&my_keypair,
|
||||
&vote_account_id,
|
||||
&RpcClient::new_from_socket(signer),
|
||||
&cluster_info_me,
|
||||
&mock_sender,
|
||||
);
|
||||
let _vote_err = signer_proxy.send_validator_vote(&bank, &cluster_info_me, &mock_sender);
|
||||
|
||||
// Send enough ticks to trigger leader rotation
|
||||
let total_entries_to_send = (bootstrap_height - initial_tick_height) as usize;
|
||||
@@ -673,7 +653,7 @@ mod test {
|
||||
)),
|
||||
replay_stage.join().expect("replay stage join")
|
||||
);
|
||||
stop_local_vote_signer_service(t_signer, &signer_exit);
|
||||
signer_service.join().unwrap();
|
||||
assert_eq!(exit.load(Ordering::Relaxed), true);
|
||||
let _ignored = remove_dir_all(&my_ledger_path);
|
||||
}
|
||||
@@ -713,16 +693,16 @@ mod test {
|
||||
.send(entries.clone())
|
||||
.expect("Expected to err out");
|
||||
|
||||
let signer = SocketAddr::new(IpAddr::V4(Ipv4Addr::new(0, 0, 0, 0)), 0);
|
||||
let rpc_client = RpcClient::new_from_socket(signer);
|
||||
let my_keypair = Arc::new(my_keypair);
|
||||
let (signer_service, signer) = LocalVoteSignerService::new();
|
||||
let vote_signer = VoteSignerProxy::new(&my_keypair, signer);
|
||||
let res = ReplayStage::process_entries(
|
||||
&Arc::new(Bank::default()),
|
||||
&cluster_info_me,
|
||||
&entry_receiver,
|
||||
&my_keypair,
|
||||
&vote_keypair.pubkey(),
|
||||
&rpc_client,
|
||||
&vote_signer,
|
||||
None,
|
||||
&ledger_entry_sender,
|
||||
&mut entry_height,
|
||||
@@ -749,7 +729,7 @@ mod test {
|
||||
&entry_receiver,
|
||||
&Arc::new(Keypair::new()),
|
||||
&Keypair::new().pubkey(),
|
||||
&rpc_client,
|
||||
&vote_signer,
|
||||
None,
|
||||
&ledger_entry_sender,
|
||||
&mut entry_height,
|
||||
@@ -765,7 +745,7 @@ mod test {
|
||||
e
|
||||
),
|
||||
}
|
||||
|
||||
signer_service.join().unwrap();
|
||||
let _ignored = remove_dir_all(&my_ledger_path);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -7,7 +7,7 @@ use crate::db_ledger;
|
||||
use crate::erasure;
|
||||
use crate::packet;
|
||||
use crate::poh_recorder;
|
||||
use crate::vote_stage;
|
||||
use crate::vote_signer_proxy;
|
||||
use bincode;
|
||||
use serde_json;
|
||||
use std;
|
||||
@@ -29,7 +29,7 @@ pub enum Error {
|
||||
ErasureError(erasure::ErasureError),
|
||||
SendError,
|
||||
PohRecorderError(poh_recorder::PohRecorderError),
|
||||
VoteError(vote_stage::VoteError),
|
||||
VoteError(vote_signer_proxy::VoteError),
|
||||
DbLedgerError(db_ledger::DbLedgerError),
|
||||
}
|
||||
|
||||
@@ -104,8 +104,8 @@ impl std::convert::From<poh_recorder::PohRecorderError> for Error {
|
||||
Error::PohRecorderError(e)
|
||||
}
|
||||
}
|
||||
impl std::convert::From<vote_stage::VoteError> for Error {
|
||||
fn from(e: vote_stage::VoteError) -> Error {
|
||||
impl std::convert::From<vote_signer_proxy::VoteError> for Error {
|
||||
fn from(e: vote_signer_proxy::VoteError) -> Error {
|
||||
Error::VoteError(e)
|
||||
}
|
||||
}
|
||||
|
||||
143
src/vote_signer_proxy.rs
Normal file
143
src/vote_signer_proxy.rs
Normal file
@@ -0,0 +1,143 @@
|
||||
//! The `vote_signer_proxy` votes on the `last_id` of the bank at a regular cadence
|
||||
|
||||
use crate::bank::Bank;
|
||||
use crate::cluster_info::ClusterInfo;
|
||||
use crate::counter::Counter;
|
||||
use crate::packet::SharedBlob;
|
||||
use crate::result::{Error, Result};
|
||||
use crate::rpc_request::{RpcClient, RpcRequest};
|
||||
use crate::streamer::BlobSender;
|
||||
use bincode::serialize;
|
||||
use log::Level;
|
||||
use solana_sdk::hash::Hash;
|
||||
use solana_sdk::pubkey::Pubkey;
|
||||
use solana_sdk::signature::{Keypair, KeypairUtil, Signature};
|
||||
use solana_sdk::transaction::Transaction;
|
||||
use solana_sdk::vote_program::Vote;
|
||||
use solana_sdk::vote_transaction::VoteTransaction;
|
||||
use std::net::SocketAddr;
|
||||
use std::sync::atomic::AtomicUsize;
|
||||
use std::sync::{Arc, RwLock};
|
||||
|
||||
#[derive(Debug, PartialEq, Eq)]
|
||||
pub enum VoteError {
|
||||
NoValidSupermajority,
|
||||
NoLeader,
|
||||
LeaderInfoNotFound,
|
||||
}
|
||||
|
||||
pub struct VoteSignerProxy {
|
||||
rpc_client: RpcClient,
|
||||
keypair: Arc<Keypair>,
|
||||
pub vote_account: Pubkey,
|
||||
}
|
||||
|
||||
impl VoteSignerProxy {
|
||||
pub fn new(keypair: &Arc<Keypair>, signer: SocketAddr) -> Self {
|
||||
let rpc_client = RpcClient::new_from_socket(signer);
|
||||
|
||||
let msg = "Registering a new node";
|
||||
let sig = Signature::new(&keypair.sign(msg.as_bytes()).as_ref());
|
||||
let params = json!([keypair.pubkey(), sig, msg.as_bytes()]);
|
||||
let resp = RpcRequest::RegisterNode
|
||||
.retry_make_rpc_request(&rpc_client, 1, Some(params), 5)
|
||||
.unwrap();
|
||||
let vote_account: Pubkey = serde_json::from_value(resp).unwrap();
|
||||
|
||||
Self {
|
||||
rpc_client,
|
||||
keypair: keypair.clone(),
|
||||
vote_account,
|
||||
}
|
||||
}
|
||||
|
||||
pub fn new_vote_account(&self, bank: &Bank, num_tokens: u64, last_id: Hash) -> Result<()> {
|
||||
// Create and register the new vote account
|
||||
let tx =
|
||||
Transaction::vote_account_new(&self.keypair, self.vote_account, last_id, num_tokens, 0);
|
||||
bank.process_transaction(&tx)?;
|
||||
Ok(())
|
||||
}
|
||||
|
||||
pub fn send_validator_vote(
|
||||
&self,
|
||||
bank: &Arc<Bank>,
|
||||
cluster_info: &Arc<RwLock<ClusterInfo>>,
|
||||
vote_blob_sender: &BlobSender,
|
||||
) -> Result<()> {
|
||||
let last_id = bank.last_id();
|
||||
|
||||
if let Ok(shared_blob) = self.new_signed_vote_blob(&last_id, bank, cluster_info) {
|
||||
inc_new_counter_info!("validator-vote_sent", 1);
|
||||
vote_blob_sender.send(vec![shared_blob])?;
|
||||
}
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
pub fn new_signed_vote_transaction(&self, last_id: &Hash, tick_height: u64) -> Transaction {
|
||||
let vote = Vote { tick_height };
|
||||
let tx = Transaction::vote_new(&self.vote_account, vote, *last_id, 0);
|
||||
|
||||
let msg = tx.get_sign_data();
|
||||
let sig = Signature::new(&self.keypair.sign(&msg).as_ref());
|
||||
|
||||
let keypair = self.keypair.clone();
|
||||
let params = json!([keypair.pubkey(), sig, &msg]);
|
||||
let resp = RpcRequest::SignVote
|
||||
.make_rpc_request(&self.rpc_client, 1, Some(params))
|
||||
.unwrap();
|
||||
let vote_signature: Signature = serde_json::from_value(resp).unwrap();
|
||||
|
||||
Transaction {
|
||||
signatures: vec![vote_signature],
|
||||
account_keys: tx.account_keys,
|
||||
last_id: tx.last_id,
|
||||
fee: tx.fee,
|
||||
program_ids: tx.program_ids,
|
||||
instructions: tx.instructions,
|
||||
}
|
||||
}
|
||||
|
||||
// TODO: Change voting to be on fixed tick intervals based on bank state
|
||||
fn new_signed_vote_blob(
|
||||
&self,
|
||||
last_id: &Hash,
|
||||
bank: &Arc<Bank>,
|
||||
cluster_info: &Arc<RwLock<ClusterInfo>>,
|
||||
) -> Result<SharedBlob> {
|
||||
let shared_blob = SharedBlob::default();
|
||||
let tick_height = bank.tick_height();
|
||||
|
||||
let leader_tpu = VoteSignerProxy::get_leader_tpu(&bank, cluster_info)?;
|
||||
//TODO: doesn't seem like there is a synchronous call to get height and id
|
||||
debug!("voting on {:?}", &last_id.as_ref()[..8]);
|
||||
let tx = self.new_signed_vote_transaction(last_id, tick_height);
|
||||
|
||||
{
|
||||
let mut blob = shared_blob.write().unwrap();
|
||||
let bytes = serialize(&tx)?;
|
||||
let len = bytes.len();
|
||||
blob.data[..len].copy_from_slice(&bytes);
|
||||
blob.meta.set_addr(&leader_tpu);
|
||||
blob.meta.size = len;
|
||||
};
|
||||
|
||||
Ok(shared_blob)
|
||||
}
|
||||
|
||||
fn get_leader_tpu(bank: &Bank, cluster_info: &Arc<RwLock<ClusterInfo>>) -> Result<SocketAddr> {
|
||||
let leader_id = match bank.get_current_leader() {
|
||||
Some((leader_id, _)) => leader_id,
|
||||
None => return Err(Error::VoteError(VoteError::NoLeader)),
|
||||
};
|
||||
|
||||
let rcluster_info = cluster_info.read().unwrap();
|
||||
let leader_tpu = rcluster_info.lookup(leader_id).map(|leader| leader.tpu);
|
||||
if let Some(leader_tpu) = leader_tpu {
|
||||
Ok(leader_tpu)
|
||||
} else {
|
||||
Err(Error::VoteError(VoteError::LeaderInfoNotFound))
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -1,126 +0,0 @@
|
||||
//! The `vote_stage` votes on the `last_id` of the bank at a regular cadence
|
||||
|
||||
use crate::bank::Bank;
|
||||
use crate::cluster_info::ClusterInfo;
|
||||
use crate::counter::Counter;
|
||||
use crate::packet::SharedBlob;
|
||||
use crate::result::{Error, Result};
|
||||
use crate::rpc_request::{RpcClient, RpcRequest};
|
||||
use crate::streamer::BlobSender;
|
||||
use bincode::serialize;
|
||||
use log::Level;
|
||||
use solana_sdk::hash::Hash;
|
||||
use solana_sdk::pubkey::Pubkey;
|
||||
use solana_sdk::signature::{Keypair, KeypairUtil, Signature};
|
||||
use solana_sdk::transaction::Transaction;
|
||||
use solana_sdk::vote_program::Vote;
|
||||
use solana_sdk::vote_transaction::VoteTransaction;
|
||||
use std::net::SocketAddr;
|
||||
use std::sync::atomic::AtomicUsize;
|
||||
use std::sync::{Arc, RwLock};
|
||||
|
||||
#[derive(Debug, PartialEq, Eq)]
|
||||
pub enum VoteError {
|
||||
NoValidSupermajority,
|
||||
NoLeader,
|
||||
LeaderInfoNotFound,
|
||||
}
|
||||
|
||||
pub fn create_new_signed_vote_transaction(
|
||||
last_id: &Hash,
|
||||
keypair: &Arc<Keypair>,
|
||||
tick_height: u64,
|
||||
vote_account: &Pubkey,
|
||||
rpc_client: &RpcClient,
|
||||
) -> Transaction {
|
||||
let vote = Vote { tick_height };
|
||||
let tx = Transaction::vote_new(&vote_account, vote, *last_id, 0);
|
||||
|
||||
let msg = tx.get_sign_data();
|
||||
let sig = Signature::new(&keypair.sign(&msg).as_ref());
|
||||
|
||||
let params = json!([keypair.pubkey(), sig, &msg]);
|
||||
let resp = RpcRequest::SignVote
|
||||
.make_rpc_request(&rpc_client, 1, Some(params))
|
||||
.unwrap();
|
||||
let vote_signature: Signature = serde_json::from_value(resp).unwrap();
|
||||
|
||||
Transaction {
|
||||
signatures: vec![vote_signature],
|
||||
account_keys: tx.account_keys,
|
||||
last_id: tx.last_id,
|
||||
fee: tx.fee,
|
||||
program_ids: tx.program_ids,
|
||||
instructions: tx.instructions,
|
||||
}
|
||||
}
|
||||
|
||||
// TODO: Change voting to be on fixed tick intervals based on bank state
|
||||
pub fn create_new_signed_vote_blob(
|
||||
last_id: &Hash,
|
||||
keypair: &Arc<Keypair>,
|
||||
vote_account: &Pubkey,
|
||||
rpc_client: &RpcClient,
|
||||
bank: &Arc<Bank>,
|
||||
cluster_info: &Arc<RwLock<ClusterInfo>>,
|
||||
) -> Result<SharedBlob> {
|
||||
let shared_blob = SharedBlob::default();
|
||||
let tick_height = bank.tick_height();
|
||||
|
||||
let leader_tpu = get_leader_tpu(&bank, cluster_info)?;
|
||||
//TODO: doesn't seem like there is a synchronous call to get height and id
|
||||
debug!("voting on {:?}", &last_id.as_ref()[..8]);
|
||||
let tx =
|
||||
create_new_signed_vote_transaction(last_id, keypair, tick_height, vote_account, rpc_client);
|
||||
|
||||
{
|
||||
let mut blob = shared_blob.write().unwrap();
|
||||
let bytes = serialize(&tx)?;
|
||||
let len = bytes.len();
|
||||
blob.data[..len].copy_from_slice(&bytes);
|
||||
blob.meta.set_addr(&leader_tpu);
|
||||
blob.meta.size = len;
|
||||
};
|
||||
|
||||
Ok(shared_blob)
|
||||
}
|
||||
|
||||
fn get_leader_tpu(bank: &Bank, cluster_info: &Arc<RwLock<ClusterInfo>>) -> Result<SocketAddr> {
|
||||
let leader_id = match bank.get_current_leader() {
|
||||
Some((leader_id, _)) => leader_id,
|
||||
None => return Err(Error::VoteError(VoteError::NoLeader)),
|
||||
};
|
||||
|
||||
let rcluster_info = cluster_info.read().unwrap();
|
||||
let leader_tpu = rcluster_info.lookup(leader_id).map(|leader| leader.tpu);
|
||||
if let Some(leader_tpu) = leader_tpu {
|
||||
Ok(leader_tpu)
|
||||
} else {
|
||||
Err(Error::VoteError(VoteError::LeaderInfoNotFound))
|
||||
}
|
||||
}
|
||||
|
||||
pub fn send_validator_vote(
|
||||
bank: &Arc<Bank>,
|
||||
keypair: &Arc<Keypair>,
|
||||
vote_account: &Pubkey,
|
||||
vote_signer_rpc: &RpcClient,
|
||||
cluster_info: &Arc<RwLock<ClusterInfo>>,
|
||||
vote_blob_sender: &BlobSender,
|
||||
) -> Result<()> {
|
||||
let last_id = bank.last_id();
|
||||
|
||||
if let Ok(shared_blob) = create_new_signed_vote_blob(
|
||||
&last_id,
|
||||
keypair,
|
||||
vote_account,
|
||||
vote_signer_rpc,
|
||||
bank,
|
||||
cluster_info,
|
||||
) {
|
||||
inc_new_counter_info!("validator-vote_sent", 1);
|
||||
vote_blob_sender.send(vec![shared_blob])?;
|
||||
}
|
||||
|
||||
Ok(())
|
||||
}
|
||||
Reference in New Issue
Block a user