Add updated duplicate broadcast test (#18506)

This commit is contained in:
carllin
2021-07-10 22:22:07 -07:00
committed by GitHub
parent 899b09872b
commit 175083c4c1
6 changed files with 484 additions and 310 deletions

View File

@ -12,16 +12,17 @@ use solana_client::{
thin_client::{create_client, ThinClient},
};
use solana_core::{
broadcast_stage::{BroadcastDuplicatesConfig, BroadcastStageType},
broadcast_stage::{broadcast_duplicates_run::BroadcastDuplicatesConfig, BroadcastStageType},
consensus::{Tower, SWITCH_FORK_THRESHOLD, VOTE_THRESHOLD_DEPTH},
optimistic_confirmation_verifier::OptimisticConfirmationVerifier,
replay_stage::DUPLICATE_THRESHOLD,
validator::ValidatorConfig,
};
use solana_download_utils::download_snapshot;
use solana_gossip::{
cluster_info::{self, VALIDATOR_PORT_RANGE},
crds_value::{self, CrdsData, CrdsValue},
gossip_service::discover_cluster,
cluster_info::VALIDATOR_PORT_RANGE,
crds::Cursor,
gossip_service::{self, discover_cluster},
};
use solana_ledger::{
ancestor_iterator::AncestorIterator,
@ -52,13 +53,8 @@ use solana_sdk::{
pubkey::Pubkey,
signature::{Keypair, Signer},
system_program, system_transaction,
timing::timestamp,
transaction::Transaction,
};
use solana_vote_program::{
vote_instruction,
vote_state::{Vote, MAX_LOCKOUT_HISTORY},
};
use solana_vote_program::{vote_state::MAX_LOCKOUT_HISTORY, vote_transaction};
use std::{
collections::{BTreeSet, HashMap, HashSet},
fs,
@ -844,42 +840,19 @@ fn test_switch_threshold_uses_gossip_votes() {
.info
.keypair
.clone();
let vote_ix = vote_instruction::vote(
&vote_keypair.pubkey(),
&vote_keypair.pubkey(),
Vote::new(
vec![heavier_validator_latest_vote],
heavier_validator_latest_vote_hash,
),
);
let mut vote_tx = Transaction::new_with_payer(&[vote_ix], Some(&node_keypair.pubkey()));
// Make the vote transaction with a random blockhash. Thus, the vote only lives in gossip but
// never makes it into a block
let blockhash = Hash::new_unique();
vote_tx.partial_sign(&[node_keypair.as_ref()], blockhash);
vote_tx.partial_sign(&[vote_keypair.as_ref()], blockhash);
let heavier_node_gossip = cluster
.get_contact_info(&context.heaviest_validator_key)
.unwrap()
.gossip;
cluster_info::push_messages_to_peer(
vec![CrdsValue::new_signed(
CrdsData::Vote(
0,
crds_value::Vote::new(node_keypair.pubkey(), vote_tx, timestamp()),
),
node_keypair,
)],
context
.dead_validator_info
.as_ref()
cluster_tests::submit_vote_to_cluster_gossip(
node_keypair,
vote_keypair,
heavier_validator_latest_vote,
heavier_validator_latest_vote_hash,
// Make the vote transaction with a random blockhash. Thus, the vote only lives in gossip but
// never makes it into a block
Hash::new_unique(),
cluster
.get_contact_info(&context.heaviest_validator_key)
.unwrap()
.info
.keypair
.pubkey(),
heavier_node_gossip,
.gossip,
)
.unwrap();
@ -1966,7 +1939,9 @@ fn test_snapshots_restart_validity() {
#[allow(unused_attributes)]
#[ignore]
fn test_fail_entry_verification_leader() {
test_faulty_node(BroadcastStageType::FailEntryVerification);
let (cluster, _) =
test_faulty_node(BroadcastStageType::FailEntryVerification, vec![60, 50, 60]);
cluster.check_for_new_roots(16, "test_fail_entry_verification_leader");
}
#[test]
@ -1974,7 +1949,9 @@ fn test_fail_entry_verification_leader() {
#[ignore]
#[allow(unused_attributes)]
fn test_fake_shreds_broadcast_leader() {
test_faulty_node(BroadcastStageType::BroadcastFakeShreds);
let node_stakes = vec![300, 100];
let (cluster, _) = test_faulty_node(BroadcastStageType::BroadcastFakeShreds, node_stakes);
cluster.check_for_new_roots(16, "test_fake_shreds_broadcast_leader");
}
#[test]
@ -1982,30 +1959,212 @@ fn test_fake_shreds_broadcast_leader() {
#[ignore]
#[allow(unused_attributes)]
fn test_duplicate_shreds_broadcast_leader() {
test_faulty_node(BroadcastStageType::BroadcastDuplicates(
BroadcastDuplicatesConfig {
stake_partition: 50,
duplicate_send_delay: 1,
},
));
// Create 4 nodes:
// 1) Bad leader sending different versions of shreds to both of the other nodes
// 2) 1 node who's voting behavior in gossip
// 3) 1 validator gets the same version as the leader, will see duplicate confirmation
// 4) 1 validator will not get the same version as the leader. For each of these
// duplicate slots `S` either:
// a) The leader's version of `S` gets > DUPLICATE_THRESHOLD of votes in gossip and so this
// node will repair that correct version
// b) A descendant `D` of some version of `S` gets > DUPLICATE_THRESHOLD votes in gossip,
// but no version of `S` does. Then the node will not know to repair the right version
// by just looking at gossip, but will instead have to use EpochSlots repair after
// detecting that a descendant does not chain to its version of `S`, and marks that descendant
// dead.
// Scenarios a) or b) are triggered by our node in 2) who's voting behavior we control.
// Critical that bad_leader_stake + good_node_stake < DUPLICATE_THRESHOLD and that
// bad_leader_stake + good_node_stake + our_node_stake > DUPLICATE_THRESHOLD so that
// our vote is the determining factor
let bad_leader_stake = 10000000000;
// Ensure that the good_node_stake is always on the critical path, and the partition node
// should never be on the critical path. This way, none of the bad shreds sent to the partition
// node corrupt the good node.
let good_node_stake = 500000;
let our_node_stake = 10000000000;
let partition_node_stake = 1;
let node_stakes = vec![
bad_leader_stake,
partition_node_stake,
good_node_stake,
// Needs to be last in the vector, so that we can
// find the id of this node. See call to `test_faulty_node`
// below for more details.
our_node_stake,
];
assert_eq!(*node_stakes.last().unwrap(), our_node_stake);
let total_stake: u64 = node_stakes.iter().sum();
assert!(
((bad_leader_stake + good_node_stake) as f64 / total_stake as f64) < DUPLICATE_THRESHOLD
);
assert!(
(bad_leader_stake + good_node_stake + our_node_stake) as f64 / total_stake as f64
> DUPLICATE_THRESHOLD
);
// Important that the partition node stake is the smallest so that it gets selected
// for the partition.
assert!(partition_node_stake < our_node_stake && partition_node_stake < good_node_stake);
// 1) Set up the cluster
let (mut cluster, validator_keys) = test_faulty_node(
BroadcastStageType::BroadcastDuplicates(BroadcastDuplicatesConfig {
stake_partition: partition_node_stake,
}),
node_stakes,
);
// This is why it's important our node was last in `node_stakes`
let our_id = validator_keys.last().unwrap().pubkey();
// 2) Kill our node and start up a thread to simulate votes to control our voting behavior
let our_info = cluster.exit_node(&our_id);
let node_keypair = our_info.info.keypair;
let vote_keypair = our_info.info.voting_keypair;
let bad_leader_id = cluster.entry_point_info.id;
let bad_leader_ledger_path = cluster.validators[&bad_leader_id].info.ledger_path.clone();
info!("our node id: {}", node_keypair.pubkey());
// 3) Start up a spy to listen for votes
let exit = Arc::new(AtomicBool::new(false));
let (gossip_service, _tcp_listener, cluster_info) = gossip_service::make_gossip_node(
// Need to use our validator's keypair to gossip EpochSlots and votes for our
// node later.
Keypair::from_bytes(&node_keypair.to_bytes()).unwrap(),
Some(&cluster.entry_point_info.gossip),
&exit,
None,
0,
false,
);
let t_voter = {
let exit = exit.clone();
std::thread::spawn(move || {
let mut cursor = Cursor::default();
let mut max_vote_slot = 0;
let mut gossip_vote_index = 0;
loop {
if exit.load(Ordering::Relaxed) {
return;
}
let (labels, votes) = cluster_info.get_votes(&mut cursor);
let mut parsed_vote_iter: Vec<_> = labels
.into_iter()
.zip(votes.into_iter())
.filter_map(|(label, leader_vote_tx)| {
// Filter out votes not from the bad leader
if label.pubkey() == bad_leader_id {
let vote = vote_transaction::parse_vote_transaction(&leader_vote_tx)
.map(|(_, vote, _)| vote)
.unwrap();
// Filter out empty votes
if !vote.slots.is_empty() {
Some((vote, leader_vote_tx))
} else {
None
}
} else {
None
}
})
.collect();
parsed_vote_iter.sort_by(|(vote, _), (vote2, _)| {
vote.slots.last().unwrap().cmp(vote2.slots.last().unwrap())
});
for (parsed_vote, leader_vote_tx) in parsed_vote_iter {
if let Some(latest_vote_slot) = parsed_vote.slots.last() {
info!("received vote for {}", latest_vote_slot);
// Add to EpochSlots. Mark all slots frozen between slot..=max_vote_slot.
if *latest_vote_slot > max_vote_slot {
let new_epoch_slots: Vec<Slot> =
(max_vote_slot + 1..latest_vote_slot + 1).collect();
info!(
"Simulating epoch slots from our node: {:?}",
new_epoch_slots
);
cluster_info.push_epoch_slots(&new_epoch_slots);
max_vote_slot = *latest_vote_slot;
}
// Only vote on even slots. Note this may violate lockouts if the
// validator started voting on a different fork before we could exit
// it above.
let vote_hash = parsed_vote.hash;
if latest_vote_slot % 2 == 0 {
info!(
"Simulating vote from our node on slot {}, hash {}",
latest_vote_slot, vote_hash
);
// Add all recent vote slots on this fork to allow cluster to pass
// vote threshold checks in replay. Note this will instantly force a
// root by this validator, but we're not concerned with lockout violations
// by this validator so it's fine.
let leader_blockstore = open_blockstore(&bad_leader_ledger_path);
let mut vote_slots: Vec<Slot> = AncestorIterator::new_inclusive(
*latest_vote_slot,
&leader_blockstore,
)
.take(MAX_LOCKOUT_HISTORY)
.collect();
vote_slots.reverse();
let vote_tx = vote_transaction::new_vote_transaction(
vote_slots,
vote_hash,
leader_vote_tx.message.recent_blockhash,
&node_keypair,
&vote_keypair,
&vote_keypair,
None,
);
gossip_vote_index += 1;
gossip_vote_index %= MAX_LOCKOUT_HISTORY;
cluster_info.push_vote_at_index(vote_tx, gossip_vote_index as u8)
}
}
// Give vote some time to propagate
sleep(Duration::from_millis(100));
}
}
})
};
// 4) Check that the cluster is making progress
cluster.check_for_new_roots(16, "test_duplicate_shreds_broadcast_leader");
// Clean up threads
exit.store(true, Ordering::Relaxed);
t_voter.join().unwrap();
gossip_service.join().unwrap();
}
fn test_faulty_node(faulty_node_type: BroadcastStageType) {
fn test_faulty_node(
faulty_node_type: BroadcastStageType,
node_stakes: Vec<u64>,
) -> (LocalCluster, Vec<Arc<Keypair>>) {
solana_logger::setup_with_default("solana_local_cluster=info");
let num_nodes = 3;
let num_nodes = node_stakes.len();
let error_validator_config = ValidatorConfig {
broadcast_stage_type: faulty_node_type,
..ValidatorConfig::default()
};
let mut validator_configs = Vec::with_capacity(num_nodes);
validator_configs.resize_with(num_nodes - 1, ValidatorConfig::default);
// First validator is the bootstrap leader with the malicious broadcast logic.
validator_configs.push(error_validator_config);
validator_configs.resize_with(num_nodes, ValidatorConfig::default);
let mut validator_keys = Vec::with_capacity(num_nodes);
validator_keys.resize_with(num_nodes, || (Arc::new(Keypair::new()), true));
let node_stakes = vec![60, 50, 60];
assert_eq!(node_stakes.len(), num_nodes);
assert_eq!(validator_keys.len(), num_nodes);
@ -2013,16 +2172,18 @@ fn test_faulty_node(faulty_node_type: BroadcastStageType) {
cluster_lamports: 10_000,
node_stakes,
validator_configs,
validator_keys: Some(validator_keys),
slots_per_epoch: MINIMUM_SLOTS_PER_EPOCH * 2u64,
stakers_slot_offset: MINIMUM_SLOTS_PER_EPOCH * 2u64,
validator_keys: Some(validator_keys.clone()),
skip_warmup_slots: true,
..ClusterConfig::default()
};
let cluster = LocalCluster::new(&mut cluster_config);
let validator_keys: Vec<Arc<Keypair>> = validator_keys
.into_iter()
.map(|(keypair, _)| keypair)
.collect();
// Check for new roots
cluster.check_for_new_roots(16, "test_faulty_node");
(cluster, validator_keys)
}
#[test]