Data plane verification (#4639)
* Add signature to blob * Change Signable trait to support returning references to signable data * Add signing to broadcast * Verify signatures in window_service * Add testing for signatures to erasure * Add RPC for getting current slot, consume RPC call in test_repairman_catchup for more deterministic results
This commit is contained in:
@ -4,7 +4,7 @@ use crate::blocktree::Blocktree;
|
||||
use crate::cluster_info::{ClusterInfo, ClusterInfoError};
|
||||
use crate::entry::EntrySlice;
|
||||
use crate::erasure::CodingGenerator;
|
||||
use crate::packet::index_blobs_with_genesis;
|
||||
use crate::packet::index_blobs;
|
||||
use crate::poh_recorder::WorkingBankEntries;
|
||||
use crate::result::{Error, Result};
|
||||
use crate::service::Service;
|
||||
@ -15,6 +15,7 @@ use solana_metrics::{
|
||||
datapoint, inc_new_counter_debug, inc_new_counter_error, inc_new_counter_info,
|
||||
};
|
||||
use solana_sdk::pubkey::Pubkey;
|
||||
use solana_sdk::signature::Signable;
|
||||
use solana_sdk::timing::duration_as_ms;
|
||||
use std::net::UdpSocket;
|
||||
use std::sync::atomic::{AtomicBool, Ordering};
|
||||
@ -104,7 +105,7 @@ impl Broadcast {
|
||||
.map(|meta| meta.consumed)
|
||||
.unwrap_or(0);
|
||||
|
||||
index_blobs_with_genesis(
|
||||
index_blobs(
|
||||
&blobs,
|
||||
&self.id,
|
||||
blob_index,
|
||||
@ -116,10 +117,27 @@ impl Broadcast {
|
||||
blobs.last().unwrap().write().unwrap().set_is_last_in_slot();
|
||||
}
|
||||
|
||||
// Make sure not to modify the blob header or data after signing it here
|
||||
self.thread_pool.install(|| {
|
||||
blobs.par_iter().for_each(|b| {
|
||||
b.write()
|
||||
.unwrap()
|
||||
.sign(&cluster_info.read().unwrap().keypair);
|
||||
})
|
||||
});
|
||||
|
||||
blocktree.write_shared_blobs(&blobs)?;
|
||||
|
||||
let coding = self.coding_generator.next(&blobs);
|
||||
|
||||
self.thread_pool.install(|| {
|
||||
coding.par_iter().for_each(|c| {
|
||||
c.write()
|
||||
.unwrap()
|
||||
.sign(&cluster_info.read().unwrap().keypair);
|
||||
})
|
||||
});
|
||||
|
||||
let to_blobs_elapsed = duration_as_ms(&to_blobs_start.elapsed());
|
||||
|
||||
let broadcast_start = Instant::now();
|
||||
|
@ -133,7 +133,7 @@ mod tests {
|
||||
hasher.hash(&buf[..size]);
|
||||
|
||||
// golden needs to be updated if blob stuff changes....
|
||||
let golden: Hash = "HZJWPVZcLtdQg34ov1vq9fjeqbgagHyhn4weLcvFsFnY"
|
||||
let golden: Hash = "E2HZjSC6VgH4nmEiTbMDATTeBcFjwSYz7QYvU7doGNhD"
|
||||
.parse()
|
||||
.unwrap();
|
||||
|
||||
|
@ -1,6 +1,8 @@
|
||||
use solana_client::thin_client::ThinClient;
|
||||
use solana_sdk::pubkey::Pubkey;
|
||||
|
||||
pub trait Cluster {
|
||||
fn get_node_pubkeys(&self) -> Vec<Pubkey>;
|
||||
fn get_validator_client(&self, pubkey: &Pubkey) -> Option<ThinClient>;
|
||||
fn restart_node(&mut self, pubkey: Pubkey);
|
||||
}
|
||||
|
@ -44,6 +44,7 @@ use solana_sdk::pubkey::Pubkey;
|
||||
use solana_sdk::signature::{Keypair, KeypairUtil, Signable, Signature};
|
||||
use solana_sdk::timing::{duration_as_ms, timestamp};
|
||||
use solana_sdk::transaction::Transaction;
|
||||
use std::borrow::Cow;
|
||||
use std::cmp::min;
|
||||
use std::collections::{BTreeSet, HashMap};
|
||||
use std::fmt;
|
||||
@ -126,7 +127,7 @@ impl Signable for PruneData {
|
||||
self.pubkey
|
||||
}
|
||||
|
||||
fn signable_data(&self) -> Vec<u8> {
|
||||
fn signable_data(&self) -> Cow<[u8]> {
|
||||
#[derive(Serialize)]
|
||||
struct SignData {
|
||||
pubkey: Pubkey,
|
||||
@ -140,7 +141,7 @@ impl Signable for PruneData {
|
||||
destination: self.destination,
|
||||
wallclock: self.wallclock,
|
||||
};
|
||||
serialize(&data).expect("serialize PruneData")
|
||||
Cow::Owned(serialize(&data).expect("serialize PruneData"))
|
||||
}
|
||||
|
||||
fn get_signature(&self) -> Signature {
|
||||
|
@ -6,6 +6,7 @@ use solana_sdk::rpc_port;
|
||||
use solana_sdk::signature::{Keypair, KeypairUtil};
|
||||
use solana_sdk::signature::{Signable, Signature};
|
||||
use solana_sdk::timing::timestamp;
|
||||
use std::borrow::Cow;
|
||||
use std::cmp::{Ord, Ordering, PartialEq, PartialOrd};
|
||||
use std::net::{IpAddr, SocketAddr};
|
||||
|
||||
@ -225,7 +226,7 @@ impl Signable for ContactInfo {
|
||||
self.id
|
||||
}
|
||||
|
||||
fn signable_data(&self) -> Vec<u8> {
|
||||
fn signable_data(&self) -> Cow<[u8]> {
|
||||
#[derive(Serialize)]
|
||||
struct SignData {
|
||||
id: Pubkey,
|
||||
@ -251,7 +252,7 @@ impl Signable for ContactInfo {
|
||||
rpc_pubsub: me.rpc_pubsub,
|
||||
wallclock: me.wallclock,
|
||||
};
|
||||
serialize(&data).expect("failed to serialize ContactInfo")
|
||||
Cow::Owned(serialize(&data).expect("failed to serialize ContactInfo"))
|
||||
}
|
||||
|
||||
fn get_signature(&self) -> Signature {
|
||||
|
@ -3,6 +3,7 @@ use bincode::serialize;
|
||||
use solana_sdk::pubkey::Pubkey;
|
||||
use solana_sdk::signature::{Keypair, Signable, Signature};
|
||||
use solana_sdk::transaction::Transaction;
|
||||
use std::borrow::Cow;
|
||||
use std::collections::BTreeSet;
|
||||
use std::fmt;
|
||||
|
||||
@ -43,7 +44,7 @@ impl Signable for EpochSlots {
|
||||
self.from
|
||||
}
|
||||
|
||||
fn signable_data(&self) -> Vec<u8> {
|
||||
fn signable_data(&self) -> Cow<[u8]> {
|
||||
#[derive(Serialize)]
|
||||
struct SignData<'a> {
|
||||
root: u64,
|
||||
@ -55,7 +56,7 @@ impl Signable for EpochSlots {
|
||||
slots: &self.slots,
|
||||
wallclock: self.wallclock,
|
||||
};
|
||||
serialize(&data).expect("unable to serialize EpochSlots")
|
||||
Cow::Owned(serialize(&data).expect("unable to serialize EpochSlots"))
|
||||
}
|
||||
|
||||
fn get_signature(&self) -> Signature {
|
||||
@ -91,7 +92,7 @@ impl Signable for Vote {
|
||||
self.from
|
||||
}
|
||||
|
||||
fn signable_data(&self) -> Vec<u8> {
|
||||
fn signable_data(&self) -> Cow<[u8]> {
|
||||
#[derive(Serialize)]
|
||||
struct SignData<'a> {
|
||||
transaction: &'a Transaction,
|
||||
@ -101,7 +102,7 @@ impl Signable for Vote {
|
||||
transaction: &self.transaction,
|
||||
wallclock: self.wallclock,
|
||||
};
|
||||
serialize(&data).expect("unable to serialize Vote")
|
||||
Cow::Owned(serialize(&data).expect("unable to serialize Vote"))
|
||||
}
|
||||
|
||||
fn get_signature(&self) -> Signature {
|
||||
@ -215,7 +216,7 @@ impl Signable for CrdsValue {
|
||||
}
|
||||
}
|
||||
|
||||
fn signable_data(&self) -> Vec<u8> {
|
||||
fn signable_data(&self) -> Cow<[u8]> {
|
||||
unimplemented!()
|
||||
}
|
||||
|
||||
|
@ -41,7 +41,7 @@ pub type EntryReceiver = Receiver<Vec<Entry>>;
|
||||
/// a Verifiable Delay Function (VDF) and a Proof of Work (not to be confused with Proof of
|
||||
/// Work consensus!)
|
||||
|
||||
#[derive(Serialize, Deserialize, Debug, PartialEq, Eq, Clone)]
|
||||
#[derive(Serialize, Deserialize, Debug, Default, PartialEq, Eq, Clone)]
|
||||
pub struct Entry {
|
||||
/// The number of hashes since the previous Entry ID.
|
||||
pub num_hashes: u64,
|
||||
|
@ -281,6 +281,7 @@ pub mod test {
|
||||
use crate::blocktree::Blocktree;
|
||||
use crate::packet::{index_blobs, SharedBlob, BLOB_DATA_SIZE, BLOB_HEADER_SIZE};
|
||||
use solana_sdk::pubkey::Pubkey;
|
||||
use solana_sdk::signature::Signable;
|
||||
use solana_sdk::signature::{Keypair, KeypairUtil};
|
||||
use std::borrow::Borrow;
|
||||
|
||||
@ -739,6 +740,7 @@ pub mod test {
|
||||
let mut blob = Blob::default();
|
||||
blob.data_mut()[..data.len()].copy_from_slice(&data);
|
||||
blob.set_size(data.len());
|
||||
blob.sign(&Keypair::new());
|
||||
Arc::new(RwLock::new(blob))
|
||||
})
|
||||
.collect();
|
||||
|
@ -34,6 +34,7 @@ pub struct ValidatorInfo {
|
||||
pub voting_keypair: Arc<Keypair>,
|
||||
pub storage_keypair: Arc<Keypair>,
|
||||
pub ledger_path: String,
|
||||
pub contact_info: ContactInfo,
|
||||
}
|
||||
|
||||
pub struct ReplicatorInfo {
|
||||
@ -173,6 +174,7 @@ impl LocalCluster {
|
||||
voting_keypair: leader_voting_keypair,
|
||||
storage_keypair: leader_storage_keypair,
|
||||
ledger_path: leader_ledger_path,
|
||||
contact_info: leader_contact_info.clone(),
|
||||
},
|
||||
);
|
||||
|
||||
@ -247,6 +249,7 @@ impl LocalCluster {
|
||||
let storage_keypair = Arc::new(Keypair::new());
|
||||
let validator_pubkey = validator_keypair.pubkey();
|
||||
let validator_node = Node::new_localhost_with_pubkey(&validator_keypair.pubkey());
|
||||
let contact_info = validator_node.info.clone();
|
||||
let ledger_path = tmp_copy_blocktree!(&self.genesis_ledger_path);
|
||||
|
||||
if validator_config.voting_disabled {
|
||||
@ -299,6 +302,7 @@ impl LocalCluster {
|
||||
voting_keypair,
|
||||
storage_keypair,
|
||||
ledger_path,
|
||||
contact_info,
|
||||
},
|
||||
);
|
||||
} else {
|
||||
@ -309,6 +313,7 @@ impl LocalCluster {
|
||||
voting_keypair,
|
||||
storage_keypair,
|
||||
ledger_path,
|
||||
contact_info,
|
||||
},
|
||||
);
|
||||
}
|
||||
@ -513,6 +518,12 @@ impl Cluster for LocalCluster {
|
||||
self.fullnodes.keys().cloned().collect()
|
||||
}
|
||||
|
||||
fn get_validator_client(&self, pubkey: &Pubkey) -> Option<ThinClient> {
|
||||
self.fullnode_infos
|
||||
.get(pubkey)
|
||||
.map(|f| create_client(f.contact_info.client_facing_addr(), FULLNODE_PORT_RANGE))
|
||||
}
|
||||
|
||||
fn restart_node(&mut self, pubkey: Pubkey) {
|
||||
// Shut down the fullnode
|
||||
let node = self.fullnodes.remove(&pubkey).unwrap();
|
||||
|
@ -7,7 +7,10 @@ use serde::Serialize;
|
||||
use solana_metrics::inc_new_counter_debug;
|
||||
pub use solana_sdk::packet::PACKET_DATA_SIZE;
|
||||
use solana_sdk::pubkey::Pubkey;
|
||||
use solana_sdk::signature::Signable;
|
||||
use solana_sdk::signature::Signature;
|
||||
use std::borrow::Borrow;
|
||||
use std::borrow::Cow;
|
||||
use std::cmp;
|
||||
use std::fmt;
|
||||
use std::io;
|
||||
@ -335,12 +338,13 @@ macro_rules! range {
|
||||
};
|
||||
}
|
||||
|
||||
const PARENT_RANGE: std::ops::Range<usize> = range!(0, u64);
|
||||
const SIGNATURE_RANGE: std::ops::Range<usize> = range!(0, Signature);
|
||||
const FORWARDED_RANGE: std::ops::Range<usize> = range!(SIGNATURE_RANGE.end, bool);
|
||||
const PARENT_RANGE: std::ops::Range<usize> = range!(FORWARDED_RANGE.end, u64);
|
||||
const SLOT_RANGE: std::ops::Range<usize> = range!(PARENT_RANGE.end, u64);
|
||||
const INDEX_RANGE: std::ops::Range<usize> = range!(SLOT_RANGE.end, u64);
|
||||
const ID_RANGE: std::ops::Range<usize> = range!(INDEX_RANGE.end, Pubkey);
|
||||
const FORWARDED_RANGE: std::ops::Range<usize> = range!(ID_RANGE.end, bool);
|
||||
const FLAGS_RANGE: std::ops::Range<usize> = range!(FORWARDED_RANGE.end, u32);
|
||||
const FLAGS_RANGE: std::ops::Range<usize> = range!(ID_RANGE.end, u32);
|
||||
const SIZE_RANGE: std::ops::Range<usize> = range!(FLAGS_RANGE.end, u64);
|
||||
|
||||
macro_rules! align {
|
||||
@ -350,6 +354,7 @@ macro_rules! align {
|
||||
}
|
||||
|
||||
pub const BLOB_HEADER_SIZE: usize = align!(SIZE_RANGE.end, BLOB_DATA_ALIGN); // make sure data() is safe for erasure
|
||||
pub const SIGNABLE_START: usize = PARENT_RANGE.start;
|
||||
|
||||
pub const BLOB_FLAG_IS_LAST_IN_SLOT: u32 = 0x2;
|
||||
|
||||
@ -593,21 +598,29 @@ impl Blob {
|
||||
}
|
||||
}
|
||||
|
||||
pub fn index_blobs(blobs: &[SharedBlob], id: &Pubkey, blob_index: u64, slot: u64, parent: u64) {
|
||||
index_blobs_with_genesis(blobs, id, blob_index, slot, parent)
|
||||
impl Signable for Blob {
|
||||
fn pubkey(&self) -> Pubkey {
|
||||
self.id()
|
||||
}
|
||||
|
||||
fn signable_data(&self) -> Cow<[u8]> {
|
||||
let end = cmp::max(SIGNABLE_START, self.data_size() as usize);
|
||||
Cow::Borrowed(&self.data[SIGNABLE_START..end])
|
||||
}
|
||||
|
||||
fn get_signature(&self) -> Signature {
|
||||
Signature::new(&self.data[SIGNATURE_RANGE])
|
||||
}
|
||||
|
||||
fn set_signature(&mut self, signature: Signature) {
|
||||
self.data[SIGNATURE_RANGE].copy_from_slice(signature.as_ref())
|
||||
}
|
||||
}
|
||||
|
||||
pub fn index_blobs_with_genesis(
|
||||
blobs: &[SharedBlob],
|
||||
id: &Pubkey,
|
||||
mut blob_index: u64,
|
||||
slot: u64,
|
||||
parent: u64,
|
||||
) {
|
||||
pub fn index_blobs(blobs: &[SharedBlob], id: &Pubkey, mut blob_index: u64, slot: u64, parent: u64) {
|
||||
// enumerate all the blobs, those are the indices
|
||||
for blob in blobs.iter() {
|
||||
let mut blob = blob.write().unwrap();
|
||||
|
||||
blob.set_index(blob_index);
|
||||
blob.set_slot(slot);
|
||||
blob.set_parent(parent);
|
||||
@ -828,4 +841,19 @@ mod tests {
|
||||
p2.data[1] = 4;
|
||||
assert!(p1 != p2);
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_sign_blob() {
|
||||
let mut b = Blob::default();
|
||||
let k = Keypair::new();
|
||||
let p = k.pubkey();
|
||||
b.set_id(&p);
|
||||
b.sign(&k);
|
||||
assert!(b.verify());
|
||||
|
||||
// Set a bigger chunk of data to sign
|
||||
b.set_size(80);
|
||||
b.sign(&k);
|
||||
assert!(b.verify());
|
||||
}
|
||||
}
|
||||
|
@ -96,6 +96,10 @@ impl JsonRpcRequestProcessor {
|
||||
self.bank().get_signature_confirmation_status(&signature)
|
||||
}
|
||||
|
||||
fn get_slot(&self) -> Result<u64> {
|
||||
Ok(self.bank().slot())
|
||||
}
|
||||
|
||||
fn get_transaction_count(&self) -> Result<u64> {
|
||||
Ok(self.bank().transaction_count() as u64)
|
||||
}
|
||||
@ -218,6 +222,9 @@ pub trait RpcSol {
|
||||
_: String,
|
||||
) -> Result<Option<transaction::Result<()>>>;
|
||||
|
||||
#[rpc(meta, name = "getSlot")]
|
||||
fn get_slot(&self, _: Self::Metadata) -> Result<u64>;
|
||||
|
||||
#[rpc(meta, name = "getTransactionCount")]
|
||||
fn get_transaction_count(&self, _: Self::Metadata) -> Result<u64>;
|
||||
|
||||
@ -334,6 +341,10 @@ impl RpcSol for RpcSolImpl {
|
||||
.map(|res| res.map(|x| x.1))
|
||||
}
|
||||
|
||||
fn get_slot(&self, meta: Self::Metadata) -> Result<u64> {
|
||||
meta.request_processor.read().unwrap().get_slot()
|
||||
}
|
||||
|
||||
fn get_num_blocks_since_signature_confirmation(
|
||||
&self,
|
||||
meta: Self::Metadata,
|
||||
|
@ -9,9 +9,12 @@ use crate::repair_service::{RepairService, RepairStrategy};
|
||||
use crate::result::{Error, Result};
|
||||
use crate::service::Service;
|
||||
use crate::streamer::{BlobReceiver, BlobSender};
|
||||
use rayon::prelude::*;
|
||||
use rayon::ThreadPool;
|
||||
use solana_metrics::{inc_new_counter_debug, inc_new_counter_error};
|
||||
use solana_runtime::bank::Bank;
|
||||
use solana_sdk::pubkey::Pubkey;
|
||||
use solana_sdk::signature::Signable;
|
||||
use solana_sdk::timing::duration_as_ms;
|
||||
use std::net::UdpSocket;
|
||||
use std::sync::atomic::{AtomicBool, Ordering};
|
||||
@ -20,6 +23,8 @@ use std::sync::{Arc, RwLock};
|
||||
use std::thread::{self, Builder, JoinHandle};
|
||||
use std::time::{Duration, Instant};
|
||||
|
||||
pub const NUM_THREADS: u32 = 10;
|
||||
|
||||
fn retransmit_blobs(blobs: &[SharedBlob], retransmit: &BlobSender, id: &Pubkey) -> Result<()> {
|
||||
let mut retransmit_queue: Vec<SharedBlob> = Vec::new();
|
||||
for blob in blobs {
|
||||
@ -86,7 +91,10 @@ pub fn should_retransmit_and_persist(
|
||||
Some(bank) => leader_schedule_cache.slot_leader_at(blob.slot(), Some(&bank)),
|
||||
};
|
||||
|
||||
if blob.id() == *my_pubkey {
|
||||
if !blob.verify() {
|
||||
inc_new_counter_debug!("streamer-recv_window-invalid_signature", 1);
|
||||
false
|
||||
} else if blob.id() == *my_pubkey {
|
||||
inc_new_counter_debug!("streamer-recv_window-circular_transmission", 1);
|
||||
false
|
||||
} else if slot_leader_pubkey == None {
|
||||
@ -108,9 +116,11 @@ fn recv_window<F>(
|
||||
r: &BlobReceiver,
|
||||
retransmit: &BlobSender,
|
||||
blob_filter: F,
|
||||
thread_pool: &ThreadPool,
|
||||
) -> Result<()>
|
||||
where
|
||||
F: Fn(&Blob) -> bool,
|
||||
F: Sync,
|
||||
{
|
||||
let timer = Duration::from_millis(200);
|
||||
let mut blobs = r.recv_timeout(timer)?;
|
||||
@ -121,7 +131,12 @@ where
|
||||
let now = Instant::now();
|
||||
inc_new_counter_debug!("streamer-recv_window-recv", blobs.len(), 0, 1000);
|
||||
|
||||
blobs.retain(|blob| blob_filter(&blob.read().unwrap()));
|
||||
let blobs: Vec<_> = thread_pool.install(|| {
|
||||
blobs
|
||||
.into_par_iter()
|
||||
.filter(|b| blob_filter(&b.read().unwrap()))
|
||||
.collect()
|
||||
});
|
||||
|
||||
retransmit_blobs(&blobs, retransmit, my_pubkey)?;
|
||||
|
||||
@ -200,20 +215,31 @@ impl WindowService {
|
||||
let _exit = Finalizer::new(exit.clone());
|
||||
let id = cluster_info.read().unwrap().id();
|
||||
trace!("{}: RECV_WINDOW started", id);
|
||||
let thread_pool = rayon::ThreadPoolBuilder::new()
|
||||
.num_threads(sys_info::cpu_num().unwrap_or(NUM_THREADS) as usize)
|
||||
.build()
|
||||
.unwrap();
|
||||
loop {
|
||||
if exit.load(Ordering::Relaxed) {
|
||||
break;
|
||||
}
|
||||
|
||||
if let Err(e) = recv_window(&blocktree, &id, &r, &retransmit, |blob| {
|
||||
blob_filter(
|
||||
&id,
|
||||
blob,
|
||||
bank_forks
|
||||
.as_ref()
|
||||
.map(|bank_forks| bank_forks.read().unwrap().working_bank()),
|
||||
)
|
||||
}) {
|
||||
if let Err(e) = recv_window(
|
||||
&blocktree,
|
||||
&id,
|
||||
&r,
|
||||
&retransmit,
|
||||
|blob| {
|
||||
blob_filter(
|
||||
&id,
|
||||
blob,
|
||||
bank_forks
|
||||
.as_ref()
|
||||
.map(|bank_forks| bank_forks.read().unwrap().working_bank()),
|
||||
)
|
||||
},
|
||||
&thread_pool,
|
||||
) {
|
||||
match e {
|
||||
Error::RecvTimeoutError(RecvTimeoutError::Disconnected) => break,
|
||||
Error::RecvTimeoutError(RecvTimeoutError::Timeout) => (),
|
||||
@ -249,13 +275,14 @@ mod test {
|
||||
use crate::bank_forks::BankForks;
|
||||
use crate::blocktree::{get_tmp_ledger_path, Blocktree};
|
||||
use crate::cluster_info::{ClusterInfo, Node};
|
||||
use crate::entry::{make_consecutive_blobs, make_tiny_test_entries, EntrySlice};
|
||||
use crate::entry::{make_consecutive_blobs, make_tiny_test_entries, Entry, EntrySlice};
|
||||
use crate::genesis_utils::create_genesis_block_with_leader;
|
||||
use crate::packet::{index_blobs, Blob};
|
||||
use crate::packet::index_blobs;
|
||||
use crate::service::Service;
|
||||
use crate::streamer::{blob_receiver, responder};
|
||||
use solana_runtime::epoch_schedule::MINIMUM_SLOT_LENGTH;
|
||||
use solana_sdk::hash::Hash;
|
||||
use solana_sdk::signature::{Keypair, KeypairUtil};
|
||||
use std::fs::remove_dir_all;
|
||||
use std::net::UdpSocket;
|
||||
use std::sync::atomic::{AtomicBool, Ordering};
|
||||
@ -289,14 +316,17 @@ mod test {
|
||||
#[test]
|
||||
fn test_should_retransmit_and_persist() {
|
||||
let me_id = Pubkey::new_rand();
|
||||
let leader_pubkey = Pubkey::new_rand();
|
||||
let leader_keypair = Keypair::new();
|
||||
let leader_pubkey = leader_keypair.pubkey();
|
||||
let bank = Arc::new(Bank::new(
|
||||
&create_genesis_block_with_leader(100, &leader_pubkey, 10).genesis_block,
|
||||
));
|
||||
let cache = Arc::new(LeaderScheduleCache::new_from_bank(&bank));
|
||||
|
||||
let mut blob = Blob::default();
|
||||
let entry = Entry::default();
|
||||
let mut blob = entry.to_blob();
|
||||
blob.set_id(&leader_pubkey);
|
||||
blob.sign(&leader_keypair);
|
||||
|
||||
// without a Bank and blobs not from me, blob gets thrown out
|
||||
assert_eq!(
|
||||
|
Reference in New Issue
Block a user