solana/src/accountant_skel.rs

826 lines
29 KiB
Rust
Raw Normal View History

2018-03-30 13:10:27 -07:00
//! The `accountant_skel` module is a microservice that exposes the high-level
2018-03-29 11:20:54 -07:00
//! Accountant API to the network. Its message encoding is currently
2018-03-30 10:43:38 -07:00
//! in flux. Clients should use AccountantStub to interact with it.
2018-03-29 11:20:54 -07:00
2018-02-28 09:07:54 -08:00
use accountant::Accountant;
2018-03-28 13:40:58 -07:00
use bincode::{deserialize, serialize};
use ecdsa;
use entry::Entry;
use event::Event;
use hash::Hash;
2018-04-02 20:15:21 -07:00
use historian::Historian;
2018-03-26 21:07:11 -07:00
use packet;
use packet::SharedPackets;
2018-04-02 20:15:21 -07:00
use rayon::prelude::*;
use recorder::Signal;
use result::Result;
use serde_json;
use signature::PublicKey;
2018-03-26 21:07:11 -07:00
use std::cmp::max;
use std::collections::VecDeque;
2018-03-29 11:54:10 -07:00
use std::io::Write;
2018-03-29 12:09:21 -07:00
use std::net::{SocketAddr, UdpSocket};
2018-03-22 13:05:23 -07:00
use std::sync::atomic::{AtomicBool, Ordering};
2018-05-02 15:35:37 -07:00
use std::sync::mpsc::{channel, Receiver, Sender, SyncSender};
2018-04-18 12:02:54 -07:00
use std::sync::{Arc, Mutex, RwLock};
use std::thread::{spawn, JoinHandle};
use std::time::Duration;
use streamer;
use transaction::Transaction;
2018-02-28 09:07:54 -08:00
2018-04-19 15:43:19 -07:00
use subscribers;
2018-02-28 09:07:54 -08:00
2018-03-26 11:17:19 -07:00
pub struct AccountantSkel<W: Write + Send + 'static> {
2018-04-02 08:30:10 -07:00
acc: Accountant,
last_id: Hash,
2018-03-26 11:17:19 -07:00
writer: W,
2018-05-02 15:35:37 -07:00
historian_input: SyncSender<Signal>,
historian: Historian,
entry_info_subscribers: Vec<SocketAddr>,
2018-02-28 09:07:54 -08:00
}
2018-03-22 13:59:25 -07:00
#[cfg_attr(feature = "cargo-clippy", allow(large_enum_variant))]
2018-04-11 17:30:53 -07:00
#[derive(Serialize, Deserialize, Debug, Clone)]
2018-02-28 09:07:54 -08:00
pub enum Request {
Transaction(Transaction),
GetBalance { key: PublicKey },
2018-04-02 08:30:10 -07:00
GetLastId,
Subscribe { subscriptions: Vec<Subscription> },
}
#[derive(Serialize, Deserialize, Debug, Clone)]
pub enum Subscription {
EntryInfo,
}
#[derive(Serialize, Deserialize, Debug, Clone)]
pub struct EntryInfo {
pub id: Hash,
pub num_hashes: u64,
pub num_events: u64,
2018-02-28 09:07:54 -08:00
}
impl Request {
/// Verify the request is valid.
pub fn verify(&self) -> bool {
match *self {
2018-03-26 21:07:11 -07:00
Request::Transaction(ref tr) => tr.verify_plan(),
_ => true,
}
}
}
2018-02-28 13:16:50 -08:00
#[derive(Serialize, Deserialize, Debug)]
2018-02-28 09:07:54 -08:00
pub enum Response {
Balance { key: PublicKey, val: Option<i64> },
EntryInfo(EntryInfo),
2018-04-02 08:30:10 -07:00
LastId { id: Hash },
2018-02-28 09:07:54 -08:00
}
2018-03-26 11:17:19 -07:00
impl<W: Write + Send + 'static> AccountantSkel<W> {
2018-03-29 11:20:54 -07:00
/// Create a new AccountantSkel that wraps the given Accountant.
2018-05-02 15:35:37 -07:00
pub fn new(
acc: Accountant,
last_id: Hash,
writer: W,
historian_input: SyncSender<Signal>,
historian: Historian,
) -> Self {
AccountantSkel {
acc,
last_id,
2018-04-02 08:30:10 -07:00
writer,
2018-05-02 15:35:37 -07:00
historian_input,
historian,
entry_info_subscribers: vec![],
}
}
fn notify_entry_info_subscribers(&mut self, entry: &Entry) {
// TODO: No need to bind().
let socket = UdpSocket::bind("127.0.0.1:0").expect("bind");
for addr in &self.entry_info_subscribers {
let entry_info = EntryInfo {
id: entry.id,
num_hashes: entry.num_hashes,
num_events: entry.events.len() as u64,
};
let data = serialize(&Response::EntryInfo(entry_info)).expect("serialize EntryInfo");
let _res = socket.send_to(&data, addr);
}
2018-03-20 22:15:44 -07:00
}
2018-03-29 11:20:54 -07:00
/// Process any Entry items that have been published by the Historian.
2018-03-26 11:17:19 -07:00
pub fn sync(&mut self) -> Hash {
2018-05-02 14:54:53 -07:00
while let Ok(entry) = self.historian.output.try_recv() {
2018-03-20 22:15:44 -07:00
self.last_id = entry.id;
self.acc.register_entry_id(&self.last_id);
2018-03-28 13:40:58 -07:00
writeln!(self.writer, "{}", serde_json::to_string(&entry).unwrap()).unwrap();
self.notify_entry_info_subscribers(&entry);
2018-03-20 22:15:44 -07:00
}
self.last_id
2018-02-28 13:16:50 -08:00
}
2018-03-29 11:20:54 -07:00
/// Process Request items sent by clients.
2018-04-11 10:17:00 -07:00
pub fn process_request(
&mut self,
msg: Request,
rsp_addr: SocketAddr,
) -> Option<(Response, SocketAddr)> {
2018-02-28 09:07:54 -08:00
match msg {
Request::GetBalance { key } => {
let val = self.acc.get_balance(&key);
2018-04-11 10:17:00 -07:00
Some((Response::Balance { key, val }, rsp_addr))
2018-02-28 09:07:54 -08:00
}
2018-04-11 10:17:00 -07:00
Request::GetLastId => Some((Response::LastId { id: self.sync() }, rsp_addr)),
Request::Transaction(_) => unreachable!(),
Request::Subscribe { subscriptions } => {
for subscription in subscriptions {
match subscription {
Subscription::EntryInfo => self.entry_info_subscribers.push(rsp_addr),
}
}
None
}
2018-02-28 09:07:54 -08:00
}
}
2018-04-11 08:02:33 -07:00
fn recv_batch(recvr: &streamer::PacketReceiver) -> Result<Vec<SharedPackets>> {
2018-03-26 21:07:11 -07:00
let timer = Duration::new(1, 0);
let msgs = recvr.recv_timeout(timer)?;
trace!("got msgs");
2018-04-11 08:02:33 -07:00
let mut batch = vec![msgs];
2018-03-26 21:07:11 -07:00
while let Ok(more) = recvr.try_recv() {
trace!("got more msgs");
2018-04-11 08:02:33 -07:00
batch.push(more);
2018-03-26 21:07:11 -07:00
}
2018-04-11 08:02:33 -07:00
info!("batch len {}", batch.len());
Ok(batch)
}
fn verify_batch(batch: Vec<SharedPackets>) -> Vec<Vec<(SharedPackets, Vec<u8>)>> {
let chunk_size = max(1, (batch.len() + 3) / 4);
let batches: Vec<_> = batch.chunks(chunk_size).map(|x| x.to_vec()).collect();
batches
2018-03-26 21:07:11 -07:00
.into_par_iter()
2018-04-11 08:02:33 -07:00
.map(|batch| {
let r = ecdsa::ed25519_verify(&batch);
batch.into_iter().zip(r).collect()
})
.collect()
}
fn verifier(
recvr: &streamer::PacketReceiver,
sendr: &Sender<Vec<(SharedPackets, Vec<u8>)>>,
) -> Result<()> {
let batch = Self::recv_batch(recvr)?;
let verified_batches = Self::verify_batch(batch);
for xs in verified_batches {
sendr.send(xs)?;
2018-03-26 21:07:11 -07:00
}
Ok(())
}
pub fn deserialize_packets(p: &packet::Packets) -> Vec<Option<(Request, SocketAddr)>> {
2018-04-06 14:52:58 -07:00
p.packets
.par_iter()
.map(|x| {
deserialize(&x.data[0..x.meta.size])
.map(|req| (req, x.meta.addr()))
.ok()
})
.collect()
2018-03-26 21:07:11 -07:00
}
2018-04-11 10:17:00 -07:00
/// Split Request list into verified transactions and the rest
fn partition_requests(
req_vers: Vec<(Request, SocketAddr, u8)>,
) -> (Vec<Transaction>, Vec<(Request, SocketAddr)>) {
let mut trs = vec![];
let mut reqs = vec![];
for (msg, rsp_addr, verify) in req_vers {
match msg {
Request::Transaction(tr) => {
if verify != 0 {
trs.push(tr);
}
}
_ => reqs.push((msg, rsp_addr)),
}
}
(trs, reqs)
}
fn process_packets(
2018-04-11 10:17:00 -07:00
&mut self,
2018-04-10 20:43:53 -07:00
req_vers: Vec<(Request, SocketAddr, u8)>,
2018-04-11 10:17:00 -07:00
) -> Result<Vec<(Response, SocketAddr)>> {
let (trs, reqs) = Self::partition_requests(req_vers);
// Process the transactions in parallel and then log the successful ones.
for result in self.acc.process_verified_transactions(trs) {
if let Ok(tr) = result {
2018-05-02 15:35:37 -07:00
self.historian_input
2018-04-11 10:17:00 -07:00
.send(Signal::Event(Event::Transaction(tr)))?;
}
}
// Let validators know they should not attempt to process additional
// transactions in parallel.
2018-05-02 15:35:37 -07:00
self.historian_input.send(Signal::Tick)?;
2018-04-11 10:17:00 -07:00
// Process the remaining requests serially.
let rsps = reqs.into_iter()
.filter_map(|(req, rsp_addr)| self.process_request(req, rsp_addr))
.collect();
Ok(rsps)
}
fn serialize_response(
resp: Response,
rsp_addr: SocketAddr,
blob_recycler: &packet::BlobRecycler,
) -> Result<packet::SharedBlob> {
let blob = blob_recycler.allocate();
{
let mut b = blob.write().unwrap();
let v = serialize(&resp)?;
let len = v.len();
b.data[..len].copy_from_slice(&v);
b.meta.size = len;
b.meta.set_addr(&rsp_addr);
}
Ok(blob)
}
fn serialize_responses(
rsps: Vec<(Response, SocketAddr)>,
2018-04-06 15:12:13 -07:00
blob_recycler: &packet::BlobRecycler,
) -> Result<VecDeque<packet::SharedBlob>> {
let mut blobs = VecDeque::new();
for (resp, rsp_addr) in rsps {
blobs.push_back(Self::serialize_response(resp, rsp_addr, blob_recycler)?);
2018-04-06 15:12:13 -07:00
}
Ok(blobs)
2018-04-06 15:12:13 -07:00
}
fn process(
obj: &Arc<Mutex<AccountantSkel<W>>>,
verified_receiver: &Receiver<Vec<(SharedPackets, Vec<u8>)>>,
blob_sender: &streamer::BlobSender,
packet_recycler: &packet::PacketRecycler,
blob_recycler: &packet::BlobRecycler,
) -> Result<()> {
let timer = Duration::new(1, 0);
let mms = verified_receiver.recv_timeout(timer)?;
for (msgs, vers) in mms {
2018-04-06 15:12:13 -07:00
let reqs = Self::deserialize_packets(&msgs.read().unwrap());
2018-04-10 20:43:53 -07:00
let req_vers = reqs.into_iter()
.zip(vers)
.filter_map(|(req, ver)| req.map(|(msg, addr)| (msg, addr, ver)))
.filter(|x| x.0.verify())
.collect();
2018-04-11 10:17:00 -07:00
let rsps = obj.lock().unwrap().process_packets(req_vers)?;
let blobs = Self::serialize_responses(rsps, blob_recycler)?;
if !blobs.is_empty() {
2018-03-26 21:07:11 -07:00
//don't wake up the other side if there is nothing
blob_sender.send(blobs)?;
2018-03-26 21:07:11 -07:00
}
2018-04-06 14:58:11 -07:00
packet_recycler.recycle(msgs);
// Write new entries to the ledger and notify subscribers.
obj.lock().unwrap().sync();
}
Ok(())
}
2018-04-18 12:29:33 -07:00
/// Process verified blobs, already in order
/// Respond with a signed hash of the state
fn replicate_state(
obj: &Arc<Mutex<AccountantSkel<W>>>,
2018-04-19 15:43:19 -07:00
verified_receiver: &streamer::BlobReceiver,
2018-04-18 12:29:33 -07:00
blob_recycler: &packet::BlobRecycler,
) -> Result<()> {
let timer = Duration::new(1, 0);
let blobs = verified_receiver.recv_timeout(timer)?;
2018-04-19 15:43:19 -07:00
for msgs in &blobs {
let blob = msgs.read().unwrap();
let entries: Vec<Entry> = deserialize(&blob.data()[..blob.meta.size]).unwrap();
for entry in entries {
obj.lock().unwrap().acc.register_entry_id(&entry.id);
obj.lock()
.unwrap()
.acc
.process_verified_events(entry.events)?;
2018-04-18 12:29:33 -07:00
}
2018-04-18 20:12:30 -07:00
//TODO respond back to leader with hash of the state
2018-04-18 12:29:33 -07:00
}
2018-04-19 15:43:19 -07:00
for blob in blobs {
blob_recycler.recycle(blob);
}
2018-04-18 12:29:33 -07:00
Ok(())
}
2018-02-28 09:07:54 -08:00
2018-03-29 11:20:54 -07:00
/// Create a UDP microservice that forwards messages the given AccountantSkel.
2018-04-17 19:26:19 -07:00
/// This service is the network leader
2018-03-29 11:20:54 -07:00
/// Set `exit` to shutdown its threads.
pub fn serve(
obj: &Arc<Mutex<AccountantSkel<W>>>,
addr: &str,
2018-03-22 13:05:23 -07:00
exit: Arc<AtomicBool>,
2018-03-23 20:49:28 -07:00
) -> Result<Vec<JoinHandle<()>>> {
let read = UdpSocket::bind(addr)?;
// make sure we are on the same interface
let mut local = read.local_addr()?;
local.set_port(0);
let write = UdpSocket::bind(local)?;
2018-02-28 09:07:54 -08:00
let packet_recycler = packet::PacketRecycler::default();
let blob_recycler = packet::BlobRecycler::default();
let (packet_sender, packet_receiver) = channel();
let t_receiver =
streamer::receiver(read, exit.clone(), packet_recycler.clone(), packet_sender)?;
let (blob_sender, blob_receiver) = channel();
2018-03-25 00:06:48 -07:00
let t_responder =
streamer::responder(write, exit.clone(), blob_recycler.clone(), blob_receiver);
2018-03-26 21:07:11 -07:00
let (verified_sender, verified_receiver) = channel();
let exit_ = exit.clone();
let t_verifier = spawn(move || loop {
let e = Self::verifier(&packet_receiver, &verified_sender);
if e.is_err() && exit_.load(Ordering::Relaxed) {
break;
}
});
let skel = obj.clone();
let t_server = spawn(move || loop {
2018-04-18 12:02:54 -07:00
let e = Self::process(
&skel,
2018-03-26 21:07:11 -07:00
&verified_receiver,
&blob_sender,
&packet_recycler,
&blob_recycler,
);
if e.is_err() {
// Assume this was a timeout, so sync any empty entries.
skel.lock().unwrap().sync();
if exit.load(Ordering::Relaxed) {
break;
}
}
});
2018-03-26 21:07:11 -07:00
Ok(vec![t_receiver, t_responder, t_server, t_verifier])
}
2018-04-17 19:26:19 -07:00
2018-04-18 12:02:54 -07:00
/// This service receives messages from a leader in the network and processes the transactions
/// on the accountant state.
/// # Arguments
2018-04-18 20:12:30 -07:00
/// * `obj` - The accountant state.
2018-04-18 12:02:54 -07:00
/// * `rsubs` - The subscribers.
/// * `exit` - The exit signal.
/// # Remarks
2018-04-18 20:12:30 -07:00
/// The pipeline is constructed as follows:
2018-04-18 12:02:54 -07:00
/// 1. receive blobs from the network, these are out of order
2018-04-18 12:05:12 -07:00
/// 2. verify blobs, PoH, signatures (TODO)
/// 3. reconstruct contiguous window
2018-04-18 12:02:54 -07:00
/// a. order the blobs
/// b. use erasure coding to reconstruct missing blobs
2018-04-18 12:29:33 -07:00
/// c. ask the network for missing blobs, if erasure coding is insufficient
2018-04-18 12:05:12 -07:00
/// d. make sure that the blobs PoH sequences connect (TODO)
2018-04-18 12:02:54 -07:00
/// 4. process the transaction state machine
/// 5. respond with the hash of the state back to the leader
2018-04-17 19:26:19 -07:00
pub fn replicate(
obj: &Arc<Mutex<AccountantSkel<W>>>,
2018-04-19 15:43:19 -07:00
rsubs: subscribers::Subscribers,
2018-04-17 19:26:19 -07:00
exit: Arc<AtomicBool>,
) -> Result<Vec<JoinHandle<()>>> {
let read = UdpSocket::bind(rsubs.me.addr)?;
// make sure we are on the same interface
let mut local = read.local_addr()?;
local.set_port(0);
let write = UdpSocket::bind(local)?;
let blob_recycler = packet::BlobRecycler::default();
let (blob_sender, blob_receiver) = channel();
let t_blob_receiver = streamer::blob_receiver(
exit.clone(),
blob_recycler.clone(),
read,
blob_sender.clone(),
)?;
2018-04-17 19:26:19 -07:00
let (window_sender, window_receiver) = channel();
2018-04-17 20:09:37 -07:00
let (retransmit_sender, retransmit_receiver) = channel();
2018-04-17 19:26:19 -07:00
let subs = Arc::new(RwLock::new(rsubs));
2018-04-19 10:32:02 -07:00
let t_retransmit = streamer::retransmitter(
2018-04-17 20:09:37 -07:00
write,
exit.clone(),
2018-04-19 15:43:19 -07:00
subs.clone(),
2018-04-17 20:09:37 -07:00
blob_recycler.clone(),
retransmit_receiver,
);
2018-04-18 12:02:54 -07:00
//TODO
//the packets coming out of blob_receiver need to be sent to the GPU and verified
2018-04-18 12:02:54 -07:00
//then sent to the window, which does the erasure coding reconstruction
let t_window = streamer::window(
exit.clone(),
2018-04-18 20:12:30 -07:00
subs,
2018-04-18 12:02:54 -07:00
blob_recycler.clone(),
blob_receiver,
window_sender,
retransmit_sender,
);
2018-03-26 21:07:11 -07:00
let skel = obj.clone();
let t_server = spawn(move || loop {
let e = Self::replicate_state(&skel, &window_receiver, &blob_recycler);
if e.is_err() && exit.load(Ordering::Relaxed) {
break;
}
});
2018-04-18 12:02:54 -07:00
Ok(vec![t_blob_receiver, t_retransmit, t_window, t_server])
2018-03-26 21:07:11 -07:00
}
}
2018-04-11 17:30:53 -07:00
#[cfg(test)]
pub fn to_packets(r: &packet::PacketRecycler, reqs: Vec<Request>) -> Vec<SharedPackets> {
let mut out = vec![];
for rrs in reqs.chunks(packet::NUM_PACKETS) {
let p = r.allocate();
p.write()
.unwrap()
.packets
.resize(rrs.len(), Default::default());
for (i, o) in rrs.iter().zip(p.write().unwrap().packets.iter_mut()) {
let v = serialize(&i).expect("serialize request");
let len = v.len();
o.data[..len].copy_from_slice(&v);
o.meta.size = len;
}
out.push(p);
}
return out;
}
2018-03-26 21:07:11 -07:00
#[cfg(test)]
mod tests {
2018-04-11 17:30:53 -07:00
use accountant_skel::{to_packets, Request};
2018-03-26 21:07:11 -07:00
use bincode::serialize;
use ecdsa;
use packet::{BlobRecycler, PacketRecycler, NUM_PACKETS};
2018-03-26 21:07:11 -07:00
use transaction::{memfind, test_tx};
use accountant::Accountant;
2018-04-12 16:04:11 -07:00
use accountant_skel::AccountantSkel;
use accountant_stub::AccountantStub;
use entry::Entry;
use futures::Future;
use historian::Historian;
2018-04-12 16:04:11 -07:00
use mint::Mint;
use plan::Plan;
use recorder::Signal;
2018-04-12 16:04:11 -07:00
use signature::{KeyPair, KeyPairUtil};
use std::io::sink;
use std::net::{SocketAddr, UdpSocket};
use std::sync::atomic::{AtomicBool, Ordering};
2018-05-02 15:35:37 -07:00
use std::sync::mpsc::sync_channel;
2018-04-12 16:04:11 -07:00
use std::sync::{Arc, Mutex};
use std::thread::sleep;
use std::time::Duration;
use transaction::Transaction;
use chrono::prelude::*;
2018-05-02 09:21:20 -07:00
use entry;
use event::Event;
use hash::{hash, Hash};
use std::collections::VecDeque;
use std::sync::mpsc::channel;
use streamer;
use subscribers::{Node, Subscribers};
2018-04-19 15:43:19 -07:00
2018-03-26 21:07:11 -07:00
#[test]
fn test_layout() {
let tr = test_tx();
let tx = serialize(&tr).unwrap();
let packet = serialize(&Request::Transaction(tr)).unwrap();
assert_matches!(memfind(&packet, &tx), Some(ecdsa::TX_OFFSET));
2018-04-11 17:30:53 -07:00
assert_matches!(memfind(&packet, &[0, 1, 2, 3, 4, 5, 6, 7, 8, 9]), None);
}
#[test]
fn test_to_packets() {
let tr = Request::Transaction(test_tx());
let re = PacketRecycler::default();
let rv = to_packets(&re, vec![tr.clone(); 1]);
assert_eq!(rv.len(), 1);
assert_eq!(rv[0].read().unwrap().packets.len(), 1);
let rv = to_packets(&re, vec![tr.clone(); NUM_PACKETS]);
assert_eq!(rv.len(), 1);
assert_eq!(rv[0].read().unwrap().packets.len(), NUM_PACKETS);
let rv = to_packets(&re, vec![tr.clone(); NUM_PACKETS + 1]);
assert_eq!(rv.len(), 2);
assert_eq!(rv[0].read().unwrap().packets.len(), NUM_PACKETS);
assert_eq!(rv[1].read().unwrap().packets.len(), 1);
2018-02-28 09:07:54 -08:00
}
#[test]
fn test_accounting_sequential_consistency() {
// In this attack we'll demonstrate that a verifier can interpret the ledger
// differently if either the server doesn't signal the ledger to add an
// Entry OR if the verifier tries to parallelize across multiple Entries.
let mint = Mint::new(2);
let acc = Accountant::new(&mint);
let rsp_addr: SocketAddr = "0.0.0.0:0".parse().expect("socket address");
2018-05-02 15:35:37 -07:00
let (input, event_receiver) = sync_channel(10);
let historian = Historian::new(event_receiver, &mint.last_id(), None);
let mut skel = AccountantSkel::new(acc, mint.last_id(), sink(), input, historian);
// Process a batch that includes a transaction that receives two tokens.
let alice = KeyPair::new();
let tr = Transaction::new(&mint.keypair(), alice.pubkey(), 2, mint.last_id());
let req_vers = vec![(Request::Transaction(tr), rsp_addr, 1_u8)];
assert!(skel.process_packets(req_vers).is_ok());
// Process a second batch that spends one of those tokens.
let tr = Transaction::new(&alice, mint.pubkey(), 1, mint.last_id());
let req_vers = vec![(Request::Transaction(tr), rsp_addr, 1_u8)];
assert!(skel.process_packets(req_vers).is_ok());
// Collect the ledger and feed it to a new accountant.
2018-05-02 15:35:37 -07:00
skel.historian_input.send(Signal::Tick).unwrap();
drop(skel.historian_input);
2018-05-02 14:54:53 -07:00
let entries: Vec<Entry> = skel.historian.output.iter().collect();
// Assert the user holds one token, not two. If the server only output one
// entry, then the second transaction will be rejected, because it drives
// the account balance below zero before the credit is added.
let acc = Accountant::new(&mint);
for entry in entries {
acc.process_verified_events(entry.events).unwrap();
}
assert_eq!(acc.get_balance(&alice.pubkey()), Some(1));
}
#[test]
fn test_accountant_bad_sig() {
let serve_port = 9002;
let send_port = 9003;
let addr = format!("127.0.0.1:{}", serve_port);
let send_addr = format!("127.0.0.1:{}", send_port);
let alice = Mint::new(10_000);
let acc = Accountant::new(&alice);
let bob_pubkey = KeyPair::new().pubkey();
let exit = Arc::new(AtomicBool::new(false));
2018-05-02 15:35:37 -07:00
let (input, event_receiver) = sync_channel(10);
let historian = Historian::new(event_receiver, &alice.last_id(), Some(30));
let acc = Arc::new(Mutex::new(AccountantSkel::new(
acc,
alice.last_id(),
sink(),
2018-05-02 15:35:37 -07:00
input,
historian,
)));
let _threads = AccountantSkel::serve(&acc, &addr, exit.clone()).unwrap();
sleep(Duration::from_millis(300));
let socket = UdpSocket::bind(send_addr).unwrap();
socket.set_read_timeout(Some(Duration::new(5, 0))).unwrap();
let mut acc = AccountantStub::new(&addr, socket);
let last_id = acc.get_last_id().wait().unwrap();
let tr = Transaction::new(&alice.keypair(), bob_pubkey, 500, last_id);
let _sig = acc.transfer_signed(tr).unwrap();
let last_id = acc.get_last_id().wait().unwrap();
let mut tr2 = Transaction::new(&alice.keypair(), bob_pubkey, 501, last_id);
tr2.data.tokens = 502;
tr2.data.plan = Plan::new_payment(502, bob_pubkey);
let _sig = acc.transfer_signed(tr2).unwrap();
assert_eq!(acc.get_balance(&bob_pubkey).wait().unwrap(), 500);
exit.store(true, Ordering::Relaxed);
}
use std::sync::{Once, ONCE_INIT};
extern crate env_logger;
static INIT: Once = ONCE_INIT;
/// Setup function that is only run once, even if called multiple times.
fn setup() {
INIT.call_once(|| {
env_logger::init().unwrap();
});
}
2018-04-19 15:43:19 -07:00
#[test]
fn test_replicate() {
setup();
let leader_sock = UdpSocket::bind("127.0.0.1:0").expect("bind");
let leader_addr = leader_sock.local_addr().unwrap();
let me_addr = "127.0.0.1:9010".parse().unwrap();
let target_peer_sock = UdpSocket::bind("127.0.0.1:0").expect("bind");
let target_peer_addr = target_peer_sock.local_addr().unwrap();
let source_peer_sock = UdpSocket::bind("127.0.0.1:0").expect("bind");
let exit = Arc::new(AtomicBool::new(false));
let node_me = Node::new([0, 0, 0, 0, 0, 0, 0, 1], 10, me_addr);
let node_subs = vec![Node::new([0, 0, 0, 0, 0, 0, 0, 2], 8, target_peer_addr); 1];
let node_leader = Node::new([0, 0, 0, 0, 0, 0, 0, 3], 20, leader_addr);
let subs = Subscribers::new(node_me, node_leader, &node_subs);
// setup some blob services to send blobs into the socket
// to simulate the source peer and get blobs out of the socket to
// simulate target peer
let recv_recycler = BlobRecycler::default();
let resp_recycler = BlobRecycler::default();
let (s_reader, r_reader) = channel();
let t_receiver = streamer::blob_receiver(
exit.clone(),
recv_recycler.clone(),
target_peer_sock,
s_reader,
).unwrap();
let (s_responder, r_responder) = channel();
let t_responder = streamer::responder(
source_peer_sock,
exit.clone(),
resp_recycler.clone(),
r_responder,
);
let starting_balance = 10_000;
let alice = Mint::new(starting_balance);
2018-04-19 15:43:19 -07:00
let acc = Accountant::new(&alice);
2018-05-02 15:35:37 -07:00
let (input, event_receiver) = sync_channel(10);
let historian = Historian::new(event_receiver, &alice.last_id(), Some(30));
2018-04-19 15:43:19 -07:00
let acc = Arc::new(Mutex::new(AccountantSkel::new(
acc,
alice.last_id(),
sink(),
2018-05-02 15:35:37 -07:00
input,
2018-04-19 15:43:19 -07:00
historian,
)));
2018-04-19 15:43:19 -07:00
let _threads = AccountantSkel::replicate(&acc, subs, exit.clone()).unwrap();
let mut alice_ref_balance = starting_balance;
let mut msgs = VecDeque::new();
let mut cur_hash = Hash::default();
let num_blobs = 10;
let transfer_amount = 501;
let bob_keypair = KeyPair::new();
for i in 0..num_blobs {
let b = resp_recycler.allocate();
let b_ = b.clone();
let mut w = b.write().unwrap();
w.set_index(i).unwrap();
let tr0 = Event::new_timestamp(&bob_keypair, Utc::now());
let entry0 = entry::create_entry(&cur_hash, i, vec![tr0]);
acc.lock().unwrap().acc.register_entry_id(&cur_hash);
cur_hash = hash(&cur_hash);
let tr1 = Transaction::new(
&alice.keypair(),
bob_keypair.pubkey(),
transfer_amount,
cur_hash,
);
acc.lock().unwrap().acc.register_entry_id(&cur_hash);
cur_hash = hash(&cur_hash);
let entry1 =
entry::create_entry(&cur_hash, i + num_blobs, vec![Event::Transaction(tr1)]);
acc.lock().unwrap().acc.register_entry_id(&cur_hash);
cur_hash = hash(&cur_hash);
alice_ref_balance -= transfer_amount;
let serialized_entry = serialize(&vec![entry0, entry1]).unwrap();
w.data_mut()[..serialized_entry.len()].copy_from_slice(&serialized_entry);
w.set_size(serialized_entry.len());
w.meta.set_addr(&me_addr);
drop(w);
msgs.push_back(b_);
}
// send the blobs into the socket
s_responder.send(msgs).expect("send");
// receive retransmitted messages
let timer = Duration::new(1, 0);
let mut msgs: Vec<_> = Vec::new();
while let Ok(msg) = r_reader.recv_timeout(timer) {
trace!("msg: {:?}", msg);
msgs.push(msg);
}
let alice_balance = acc.lock()
.unwrap()
.acc
.get_balance(&alice.keypair().pubkey())
.unwrap();
assert_eq!(alice_balance, alice_ref_balance);
let bob_balance = acc.lock()
.unwrap()
.acc
.get_balance(&bob_keypair.pubkey())
.unwrap();
assert_eq!(bob_balance, starting_balance - alice_ref_balance);
2018-04-19 15:43:19 -07:00
exit.store(true, Ordering::Relaxed);
t_receiver.join().expect("join");
t_responder.join().expect("join");
2018-04-19 15:43:19 -07:00
}
2018-02-28 09:07:54 -08:00
}
2018-04-11 13:05:29 -07:00
#[cfg(all(feature = "unstable", test))]
mod bench {
extern crate test;
use self::test::Bencher;
use accountant::{Accountant, MAX_ENTRY_IDS};
2018-04-11 19:24:14 -07:00
use accountant_skel::*;
use bincode::serialize;
use hash::hash;
2018-04-11 13:05:29 -07:00
use mint::Mint;
2018-04-11 19:24:14 -07:00
use signature::{KeyPair, KeyPairUtil};
2018-04-11 13:05:29 -07:00
use std::collections::HashSet;
use std::io::sink;
2018-05-03 08:38:59 -07:00
use std::sync::mpsc::sync_channel;
2018-04-11 13:05:29 -07:00
use std::time::Instant;
2018-04-11 19:24:14 -07:00
use transaction::Transaction;
2018-04-11 13:05:29 -07:00
#[bench]
fn process_packets_bench(_bencher: &mut Bencher) {
let mint = Mint::new(100_000_000);
let acc = Accountant::new(&mint);
let rsp_addr: SocketAddr = "0.0.0.0:0".parse().expect("socket address");
// Create transactions between unrelated parties.
let txs = 100_000;
let last_ids: Mutex<HashSet<Hash>> = Mutex::new(HashSet::new());
let transactions: Vec<_> = (0..txs)
.into_par_iter()
.map(|i| {
// Seed the 'to' account and a cell for its signature.
let dummy_id = i % (MAX_ENTRY_IDS as i32);
let last_id = hash(&serialize(&dummy_id).unwrap()); // Semi-unique hash
{
let mut last_ids = last_ids.lock().unwrap();
if !last_ids.contains(&last_id) {
last_ids.insert(last_id);
acc.register_entry_id(&last_id);
}
}
// Seed the 'from' account.
let rando0 = KeyPair::new();
let tr = Transaction::new(&mint.keypair(), rando0.pubkey(), 1_000, last_id);
2018-05-02 09:21:20 -07:00
acc.process_verified_transaction(&tr).unwrap();
2018-04-11 13:05:29 -07:00
let rando1 = KeyPair::new();
let tr = Transaction::new(&rando0, rando1.pubkey(), 2, last_id);
2018-05-02 09:21:20 -07:00
acc.process_verified_transaction(&tr).unwrap();
2018-04-11 13:05:29 -07:00
// Finally, return a transaction that's unique
Transaction::new(&rando0, rando1.pubkey(), 1, last_id)
})
.collect();
let req_vers = transactions
.into_iter()
.map(|tr| (Request::Transaction(tr), rsp_addr, 1_u8))
.collect();
2018-05-02 15:35:37 -07:00
let (input, event_receiver) = sync_channel(10);
let historian = Historian::new(event_receiver, &mint.last_id(), None);
let mut skel = AccountantSkel::new(acc, mint.last_id(), sink(), input, historian);
2018-04-11 13:05:29 -07:00
let now = Instant::now();
assert!(skel.process_packets(req_vers).is_ok());
let duration = now.elapsed();
let sec = duration.as_secs() as f64 + duration.subsec_nanos() as f64 / 1_000_000_000.0;
let tps = txs as f64 / sec;
// Ensure that all transactions were successfully logged.
2018-05-03 08:38:59 -07:00
drop(skel.historian_input);
2018-05-02 14:54:53 -07:00
let entries: Vec<Entry> = skel.historian.output.iter().collect();
2018-04-11 13:05:29 -07:00
assert_eq!(entries.len(), 1);
2018-05-02 09:21:20 -07:00
assert_eq!(entries[0].events.len(), txs as usize);
2018-04-11 13:05:29 -07:00
println!("{} tps", tps);
}
}