solana/src/bank.rs

721 lines
26 KiB
Rust
Raw Normal View History

2018-06-06 08:49:22 -07:00
//! The `bank` module tracks client balances and the progress of smart
//! contracts. It offers a high-level API that signs transactions
//! on behalf of the caller, and a low-level API for when they have
2018-03-29 11:20:54 -07:00
//! already been signed and verified.
2018-02-23 13:08:19 -08:00
2018-03-26 21:07:11 -07:00
extern crate libc;
use chrono::prelude::*;
use entry::Entry;
use hash::Hash;
use mint::Mint;
2018-05-29 12:55:21 -07:00
use payment_plan::{Payment, PaymentPlan, Witness};
use signature::{KeyPair, PublicKey, Signature};
use std::collections::hash_map::Entry::Occupied;
use std::collections::{HashMap, HashSet, VecDeque};
use std::result;
2018-06-22 10:45:22 -07:00
use std::sync::atomic::{AtomicUsize, Ordering};
use std::sync::RwLock;
2018-06-07 19:35:38 -07:00
use std::time::Instant;
2018-06-22 10:44:31 -07:00
use timing::duration_as_us;
2018-06-22 10:45:22 -07:00
use transaction::{Instruction, Plan, Transaction};
2018-06-06 08:49:22 -07:00
/// The number of most recent `last_id` values that the bank will track the signatures
/// of. Once the bank discards a `last_id`, it will reject any transactions that use
/// that `last_id` in a transaction. Lowering this value reduces memory consumption,
/// but requires clients to update its `last_id` more frequently. Raising the value
/// lengthens the time a client must wait to be certain a missing transaction will
/// not be processed by the network.
2018-06-22 10:45:14 -07:00
pub const MAX_ENTRY_IDS: usize = 1024 * 16;
2018-06-06 08:49:22 -07:00
/// Reasons a transaction might be rejected.
#[derive(Debug, PartialEq, Eq)]
2018-05-14 14:33:11 -07:00
pub enum BankError {
2018-06-06 08:49:22 -07:00
/// Attempt to debit from `PublicKey`, but no found no record of a prior credit.
2018-05-23 11:26:32 -07:00
AccountNotFound(PublicKey),
2018-06-06 08:49:22 -07:00
/// The requested debit from `PublicKey` has the potential to draw the balance
/// below zero. This can occur when a debit and credit are processed in parallel.
/// The bank may reject the debit or push it to a future entry.
2018-05-23 11:26:32 -07:00
InsufficientFunds(PublicKey),
2018-06-06 08:49:22 -07:00
/// The bank has seen `Signature` before. This can occur under normal operation
/// when a UDP packet is duplicated, as a user error from a client not updating
/// its `last_id`, or as a double-spend attack.
2018-06-28 15:59:32 -07:00
DuplicateSignature(Signature),
2018-06-06 08:49:22 -07:00
/// The bank has not seen the given `last_id` or the transaction is too old and
/// the `last_id` has been discarded.
2018-05-29 20:02:32 -07:00
LastIdNotFound(Hash),
2018-06-06 08:49:22 -07:00
/// The transaction is invalid and has requested a debit or credit of negative
/// tokens.
2018-05-29 20:02:32 -07:00
NegativeTokens,
}
2018-05-14 14:33:11 -07:00
pub type Result<T> = result::Result<T, BankError>;
2018-02-23 13:08:19 -08:00
2018-06-06 08:49:22 -07:00
/// The state of all accounts and contracts after processing its entries.
2018-05-14 14:33:11 -07:00
pub struct Bank {
2018-06-06 08:49:22 -07:00
/// A map of account public keys to the balance in that account.
2018-06-07 19:35:38 -07:00
balances: RwLock<HashMap<PublicKey, i64>>,
2018-06-06 08:49:22 -07:00
/// A map of smart contract transaction signatures to what remains of its payment
/// plan. Each transaction that targets the plan should cause it to be reduced.
/// Once it cannot be reduced, final payments are made and it is discarded.
pending: RwLock<HashMap<Signature, Plan>>,
2018-06-06 08:49:22 -07:00
/// A FIFO queue of `last_id` items, where each item is a set of signatures
2018-06-22 10:45:22 -07:00
/// that have been processed using that `last_id`. Rejected `last_id`
/// values are so old that the `last_id` has been pulled out of the queue.
2018-06-22 10:44:31 -07:00
last_ids: RwLock<VecDeque<Hash>>,
2018-06-22 10:45:22 -07:00
// Mapping of hashes to signature sets. The bank uses this data to
/// reject transactions with signatures its seen before
last_ids_sigs: RwLock<HashMap<Hash, HashSet<Signature>>>,
2018-06-06 08:49:22 -07:00
/// The set of trusted timekeepers. A Timestamp transaction from a `PublicKey`
/// outside this set will be discarded. Note that if validators do not have the
/// same set as leaders, they may interpret the ledger differently.
time_sources: RwLock<HashSet<PublicKey>>,
2018-06-06 08:49:22 -07:00
/// The most recent timestamp from a trusted timekeeper. This timestamp is applied
/// to every smart contract when it enters the system. If it is waiting on a
/// timestamp witness before that timestamp, the bank will execute it immediately.
last_time: RwLock<DateTime<Utc>>,
2018-06-06 08:49:22 -07:00
/// The number of transactions the bank has processed without error since the
/// start of the ledger.
2018-05-14 05:49:48 -07:00
transaction_count: AtomicUsize,
2018-02-23 13:08:19 -08:00
}
2018-05-14 14:33:11 -07:00
impl Bank {
/// Create an Bank using a deposit.
pub fn new_from_deposit(deposit: &Payment) -> Self {
let bank = Bank {
balances: RwLock::new(HashMap::new()),
pending: RwLock::new(HashMap::new()),
last_ids: RwLock::new(VecDeque::new()),
2018-06-22 10:44:31 -07:00
last_ids_sigs: RwLock::new(HashMap::new()),
time_sources: RwLock::new(HashSet::new()),
last_time: RwLock::new(Utc.timestamp(0, 0)),
2018-05-14 05:49:48 -07:00
transaction_count: AtomicUsize::new(0),
};
2018-06-07 19:35:38 -07:00
bank.apply_payment(deposit, &mut bank.balances.write().unwrap());
bank
2018-04-02 12:51:44 -07:00
}
2018-05-14 14:33:11 -07:00
/// Create an Bank with only a Mint. Typically used by unit tests.
pub fn new(mint: &Mint) -> Self {
2018-04-02 12:51:44 -07:00
let deposit = Payment {
to: mint.pubkey(),
tokens: mint.tokens,
};
2018-05-14 14:33:11 -07:00
let bank = Self::new_from_deposit(&deposit);
bank.register_entry_id(&mint.last_id());
bank
2018-04-02 12:51:44 -07:00
}
2018-06-22 12:11:27 -07:00
/// Commit funds to the `payment.to` party.
2018-06-07 19:35:38 -07:00
fn apply_payment(&self, payment: &Payment, balances: &mut HashMap<PublicKey, i64>) {
if balances.contains_key(&payment.to) {
*balances.get_mut(&payment.to).unwrap() += payment.tokens;
} else {
balances.insert(payment.to, payment.tokens);
}
}
2018-06-06 08:49:22 -07:00
/// Return the last entry ID registered.
2018-05-03 12:24:35 -07:00
pub fn last_id(&self) -> Hash {
let last_ids = self.last_ids.read().expect("'last_ids' read lock");
2018-05-09 17:19:12 -07:00
let last_item = last_ids.iter().last().expect("empty 'last_ids' list");
2018-06-22 10:44:31 -07:00
*last_item
2018-05-03 12:24:35 -07:00
}
2018-06-06 08:49:22 -07:00
/// Store the given signature. The bank will reject any transaction with the same signature.
2018-06-22 10:45:22 -07:00
fn reserve_signature(signatures: &mut HashSet<Signature>, sig: &Signature) -> Result<()> {
if let Some(sig) = signatures.get(sig) {
2018-06-28 15:59:32 -07:00
return Err(BankError::DuplicateSignature(*sig));
2018-06-22 10:44:31 -07:00
}
2018-06-22 10:45:22 -07:00
signatures.insert(*sig);
Ok(())
}
2018-06-06 08:49:22 -07:00
/// Forget the given `signature` because its transaction was rejected.
2018-06-22 10:45:22 -07:00
fn forget_signature(signatures: &mut HashSet<Signature>, signature: &Signature) {
signatures.remove(signature);
2018-04-26 04:22:11 -07:00
}
2018-06-06 08:49:22 -07:00
/// Forget the given `signature` with `last_id` because the transaction was rejected.
fn forget_signature_with_last_id(&self, signature: &Signature, last_id: &Hash) {
2018-06-22 10:44:31 -07:00
if let Some(entry) = self.last_ids_sigs
2018-06-22 10:45:22 -07:00
.write()
.expect("'last_ids' read lock in forget_signature_with_last_id")
2018-06-22 10:45:22 -07:00
.get_mut(last_id)
2018-04-26 04:22:11 -07:00
{
2018-06-22 10:44:31 -07:00
Self::forget_signature(entry, signature);
2018-04-26 04:22:11 -07:00
}
}
2018-06-06 08:49:22 -07:00
fn reserve_signature_with_last_id(&self, signature: &Signature, last_id: &Hash) -> Result<()> {
2018-06-22 10:44:31 -07:00
if let Some(entry) = self.last_ids_sigs
2018-06-22 10:45:22 -07:00
.write()
.expect("'last_ids' read lock in reserve_signature_with_last_id")
2018-06-22 10:45:22 -07:00
.get_mut(last_id)
{
2018-06-22 10:45:22 -07:00
return Self::reserve_signature(entry, signature);
}
2018-05-29 20:02:32 -07:00
Err(BankError::LastIdNotFound(*last_id))
}
2018-05-14 14:33:11 -07:00
/// Tell the bank which Entry IDs exist on the ledger. This function
/// assumes subsequent calls correspond to later entries, and will boot
/// the oldest ones once its internal cache is full. Once boot, the
2018-05-14 14:33:11 -07:00
/// bank will reject transactions using that `last_id`.
pub fn register_entry_id(&self, last_id: &Hash) {
2018-05-30 12:07:28 -07:00
let mut last_ids = self.last_ids
2018-05-11 11:38:52 -07:00
.write()
.expect("'last_ids' write lock in register_entry_id");
2018-06-22 10:44:31 -07:00
let mut last_ids_sigs = self.last_ids_sigs
.write()
.expect("last_ids_sigs write lock");
if last_ids.len() >= MAX_ENTRY_IDS {
2018-06-22 10:44:31 -07:00
let id = last_ids.pop_front().unwrap();
last_ids_sigs.remove(&id);
}
2018-06-22 10:45:22 -07:00
last_ids_sigs.insert(*last_id, HashSet::new());
2018-06-22 10:44:31 -07:00
last_ids.push_back(*last_id);
}
2018-06-22 12:11:27 -07:00
/// Deduct tokens from the 'from' address the account has sufficient
/// funds and isn't a duplicate.
fn apply_debits(&self, tx: &Transaction, bals: &mut HashMap<PublicKey, i64>) -> Result<()> {
let option = bals.get_mut(&tx.from);
2018-06-07 13:59:22 -07:00
if option.is_none() {
2018-06-22 12:11:27 -07:00
return Err(BankError::AccountNotFound(tx.from));
2018-06-07 13:59:22 -07:00
}
2018-06-22 10:45:22 -07:00
let bal = option.unwrap();
2018-06-07 13:59:22 -07:00
2018-06-22 12:11:27 -07:00
self.reserve_signature_with_last_id(&tx.sig, &tx.last_id)?;
2018-06-07 13:59:22 -07:00
2018-06-22 12:11:27 -07:00
if let Instruction::NewContract(contract) = &tx.instruction {
2018-06-07 19:35:38 -07:00
if contract.tokens < 0 {
return Err(BankError::NegativeTokens);
}
2018-06-07 13:59:22 -07:00
if *bal < contract.tokens {
2018-06-22 12:11:27 -07:00
self.forget_signature_with_last_id(&tx.sig, &tx.last_id);
return Err(BankError::InsufficientFunds(tx.from));
2018-06-07 13:59:22 -07:00
}
*bal -= contract.tokens;
};
Ok(())
}
2018-06-22 12:11:27 -07:00
/// Apply only a transaction's credits. Credits from multiple transactions
/// may safely be applied in parallel.
2018-06-07 19:35:38 -07:00
fn apply_credits(&self, tx: &Transaction, balances: &mut HashMap<PublicKey, i64>) {
2018-05-25 15:05:37 -07:00
match &tx.instruction {
2018-05-22 20:42:04 -07:00
Instruction::NewContract(contract) => {
let mut plan = contract.plan.clone();
2018-05-30 12:07:28 -07:00
plan.apply_witness(&Witness::Timestamp(*self.last_time
.read()
.expect("timestamp creation in apply_credits")));
2018-05-22 20:42:04 -07:00
if let Some(payment) = plan.final_payment() {
2018-06-07 19:35:38 -07:00
self.apply_payment(&payment, balances);
2018-05-22 20:42:04 -07:00
} else {
2018-05-30 12:07:28 -07:00
let mut pending = self.pending
2018-05-22 20:42:04 -07:00
.write()
2018-05-29 09:01:40 -07:00
.expect("'pending' write lock in apply_credits");
2018-05-25 15:05:37 -07:00
pending.insert(tx.sig, plan);
2018-05-22 20:42:04 -07:00
}
}
Instruction::ApplyTimestamp(dt) => {
2018-05-29 09:01:40 -07:00
let _ = self.apply_timestamp(tx.from, *dt);
2018-05-22 20:42:04 -07:00
}
Instruction::ApplySignature(tx_sig) => {
2018-05-29 09:01:40 -07:00
let _ = self.apply_signature(tx.from, *tx_sig);
2018-05-22 20:42:04 -07:00
}
}
}
2018-06-06 08:49:22 -07:00
/// Process a Transaction. If it contains a payment plan that requires a witness
/// to progress, the payment plan will be stored in the bank.
2018-05-29 08:58:47 -07:00
fn process_transaction(&self, tx: &Transaction) -> Result<()> {
2018-06-07 19:35:38 -07:00
let bals = &mut self.balances.write().unwrap();
self.apply_debits(tx, bals)?;
self.apply_credits(tx, bals);
self.transaction_count.fetch_add(1, Ordering::Relaxed);
2018-05-02 07:44:41 -07:00
Ok(())
2018-02-23 13:08:19 -08:00
}
2018-06-22 10:45:22 -07:00
/// Process a batch of transactions.
#[must_use]
2018-05-29 09:12:27 -07:00
pub fn process_transactions(&self, txs: Vec<Transaction>) -> Vec<Result<Transaction>> {
2018-06-07 19:35:38 -07:00
let bals = &mut self.balances.write().unwrap();
2018-06-22 10:44:31 -07:00
debug!("processing Transactions {}", txs.len());
let txs_len = txs.len();
2018-06-07 19:35:38 -07:00
let now = Instant::now();
let results: Vec<_> = txs.into_iter()
.map(|tx| self.apply_debits(&tx, bals).map(|_| tx))
.collect(); // Calling collect() here forces all debits to complete before moving on.
2018-06-22 10:44:31 -07:00
let debits = now.elapsed();
let now = Instant::now();
2018-06-07 19:35:38 -07:00
let res: Vec<_> = results
.into_iter()
2018-04-11 10:17:00 -07:00
.map(|result| {
2018-05-25 15:05:37 -07:00
result.map(|tx| {
2018-06-07 19:35:38 -07:00
self.apply_credits(&tx, bals);
2018-05-25 15:05:37 -07:00
tx
2018-04-11 10:17:00 -07:00
})
})
2018-06-07 19:35:38 -07:00
.collect();
2018-06-22 10:45:22 -07:00
debug!(
"debits: {} us credits: {:?} us tx: {}",
duration_as_us(&debits),
duration_as_us(&now.elapsed()),
txs_len
);
2018-06-22 12:11:27 -07:00
let mut tx_count = 0;
2018-06-07 19:35:38 -07:00
for r in &res {
if r.is_ok() {
2018-06-22 12:11:27 -07:00
tx_count += 1;
2018-06-22 10:44:31 -07:00
} else {
info!("tx error: {:?}", r);
2018-06-07 19:35:38 -07:00
}
}
2018-06-22 10:45:22 -07:00
self.transaction_count
2018-06-22 12:11:27 -07:00
.fetch_add(tx_count, Ordering::Relaxed);
2018-06-07 19:35:38 -07:00
res
}
2018-06-06 08:49:22 -07:00
/// Process an ordered list of entries.
2018-06-27 12:35:58 -07:00
pub fn process_entries<I>(&self, entries: I) -> Result<u64>
where
I: IntoIterator<Item = Entry>,
{
2018-06-27 12:35:58 -07:00
let mut entry_count = 0;
for entry in entries {
2018-06-27 12:35:58 -07:00
entry_count += 1;
2018-06-14 16:32:39 -07:00
if !entry.transactions.is_empty() {
for result in self.process_transactions(entry.transactions) {
result?;
}
}
// TODO: verify this is ok in cases like:
// 1. an untrusted genesis or tx-<DATE>.log
// 2. a crazy leader..
if !entry.has_more {
self.register_entry_id(&entry.id);
}
}
2018-06-27 12:35:58 -07:00
Ok(entry_count)
}
2018-06-06 08:49:22 -07:00
/// Process a Witness Signature. Any payment plans waiting on this signature
/// will progress one step.
2018-05-29 09:01:40 -07:00
fn apply_signature(&self, from: PublicKey, tx_sig: Signature) -> Result<()> {
2018-05-30 12:07:28 -07:00
if let Occupied(mut e) = self.pending
2018-05-11 11:38:52 -07:00
.write()
2018-05-29 09:01:40 -07:00
.expect("write() in apply_signature")
2018-05-11 11:38:52 -07:00
.entry(tx_sig)
{
2018-03-22 13:38:06 -07:00
e.get_mut().apply_witness(&Witness::Signature(from));
2018-05-15 04:35:41 -07:00
if let Some(payment) = e.get().final_payment() {
2018-06-07 19:35:38 -07:00
self.apply_payment(&payment, &mut self.balances.write().unwrap());
e.remove_entry();
}
2018-03-10 23:11:08 -08:00
};
Ok(())
}
2018-06-06 08:49:22 -07:00
/// Process a Witness Timestamp. Any payment plans waiting on this timestamp
/// will progress one step.
2018-05-29 09:01:40 -07:00
fn apply_timestamp(&self, from: PublicKey, dt: DateTime<Utc>) -> Result<()> {
2018-03-08 09:05:00 -08:00
// If this is the first timestamp we've seen, it probably came from the genesis block,
// so we'll trust it.
2018-05-30 12:07:28 -07:00
if *self.last_time
2018-05-11 11:38:52 -07:00
.read()
.expect("'last_time' read lock on first timestamp check")
== Utc.timestamp(0, 0)
{
self.time_sources
.write()
.expect("'time_sources' write lock on first timestamp")
.insert(from);
2018-03-08 09:05:00 -08:00
}
2018-05-30 12:07:28 -07:00
if self.time_sources
2018-05-11 11:38:52 -07:00
.read()
.expect("'time_sources' read lock")
.contains(&from)
{
if dt > *self.last_time.read().expect("'last_time' read lock") {
*self.last_time.write().expect("'last_time' write lock") = dt;
2018-03-08 09:05:00 -08:00
}
} else {
return Ok(());
2018-03-08 09:05:00 -08:00
}
// Check to see if any timelocked transactions can be completed.
let mut completed = vec![];
// Hold 'pending' write lock until the end of this function. Otherwise another thread can
// double-spend if it enters before the modified plan is removed from 'pending'.
2018-05-30 12:07:28 -07:00
let mut pending = self.pending
2018-05-11 11:38:52 -07:00
.write()
2018-05-29 09:01:40 -07:00
.expect("'pending' write lock in apply_timestamp");
for (key, plan) in pending.iter_mut() {
2018-05-30 12:07:28 -07:00
plan.apply_witness(&Witness::Timestamp(*self.last_time
.read()
.expect("'last_time' read lock when creating timestamp")));
if let Some(payment) = plan.final_payment() {
2018-06-07 19:35:38 -07:00
self.apply_payment(&payment, &mut self.balances.write().unwrap());
2018-03-10 23:30:01 -08:00
completed.push(key.clone());
}
}
for key in completed {
pending.remove(&key);
}
Ok(())
}
2018-03-29 11:20:54 -07:00
/// Create, sign, and process a Transaction from `keypair` to `to` of
/// `n` tokens where `last_id` is the last Entry ID observed by the client.
2018-02-23 13:08:19 -08:00
pub fn transfer(
&self,
n: i64,
keypair: &KeyPair,
2018-02-28 09:07:54 -08:00
to: PublicKey,
2018-03-20 22:15:44 -07:00
last_id: Hash,
) -> Result<Signature> {
2018-05-25 15:05:37 -07:00
let tx = Transaction::new(keypair, to, n, last_id);
let sig = tx.sig;
self.process_transaction(&tx).map(|_| sig)
2018-02-23 13:08:19 -08:00
}
2018-03-29 11:20:54 -07:00
/// Create, sign, and process a postdated Transaction from `keypair`
/// to `to` of `n` tokens on `dt` where `last_id` is the last Entry ID
/// observed by the client.
pub fn transfer_on_date(
&self,
n: i64,
keypair: &KeyPair,
to: PublicKey,
dt: DateTime<Utc>,
2018-03-20 22:15:44 -07:00
last_id: Hash,
) -> Result<Signature> {
2018-05-25 15:05:37 -07:00
let tx = Transaction::new_on_date(keypair, to, dt, n, last_id);
let sig = tx.sig;
self.process_transaction(&tx).map(|_| sig)
}
pub fn get_balance(&self, pubkey: &PublicKey) -> Option<i64> {
2018-05-30 12:07:28 -07:00
let bals = self.balances
2018-05-11 11:38:52 -07:00
.read()
.expect("'balances' read lock in get_balance");
2018-06-07 19:35:38 -07:00
bals.get(pubkey).map(|x| *x)
2018-02-23 13:08:19 -08:00
}
2018-05-14 05:49:48 -07:00
pub fn transaction_count(&self) -> usize {
self.transaction_count.load(Ordering::Relaxed)
}
2018-02-23 13:08:19 -08:00
}
#[cfg(test)]
mod tests {
use super::*;
use bincode::serialize;
2018-06-14 16:32:39 -07:00
use entry::next_entry;
use hash::hash;
use signature::KeyPairUtil;
2018-02-23 13:08:19 -08:00
#[test]
2018-06-06 08:49:22 -07:00
fn test_two_payments_to_one_party() {
2018-05-14 14:39:34 -07:00
let mint = Mint::new(10_000);
let pubkey = KeyPair::new().pubkey();
let bank = Bank::new(&mint);
assert_eq!(bank.last_id(), mint.last_id());
2018-05-03 12:24:35 -07:00
2018-05-14 14:39:34 -07:00
bank.transfer(1_000, &mint.keypair(), pubkey, mint.last_id())
2018-03-20 22:15:44 -07:00
.unwrap();
2018-05-14 14:39:34 -07:00
assert_eq!(bank.get_balance(&pubkey).unwrap(), 1_000);
2018-02-23 13:08:19 -08:00
2018-05-14 14:39:34 -07:00
bank.transfer(500, &mint.keypair(), pubkey, mint.last_id())
2018-03-20 22:15:44 -07:00
.unwrap();
2018-05-14 14:39:34 -07:00
assert_eq!(bank.get_balance(&pubkey).unwrap(), 1_500);
2018-05-14 14:33:11 -07:00
assert_eq!(bank.transaction_count(), 2);
2018-02-23 13:08:19 -08:00
}
2018-02-27 10:28:10 -08:00
2018-05-29 19:35:14 -07:00
#[test]
2018-06-06 08:49:22 -07:00
fn test_negative_tokens() {
2018-05-29 19:35:14 -07:00
let mint = Mint::new(1);
let pubkey = KeyPair::new().pubkey();
let bank = Bank::new(&mint);
assert_eq!(
bank.transfer(-1, &mint.keypair(), pubkey, mint.last_id()),
2018-05-29 20:02:32 -07:00
Err(BankError::NegativeTokens)
2018-05-29 19:35:14 -07:00
);
assert_eq!(bank.transaction_count(), 0);
}
#[test]
fn test_account_not_found() {
let mint = Mint::new(1);
2018-05-14 14:33:11 -07:00
let bank = Bank::new(&mint);
2018-05-23 11:26:32 -07:00
let keypair = KeyPair::new();
assert_eq!(
2018-05-23 11:26:32 -07:00
bank.transfer(1, &keypair, mint.pubkey(), mint.last_id()),
Err(BankError::AccountNotFound(keypair.pubkey()))
);
2018-05-14 14:33:11 -07:00
assert_eq!(bank.transaction_count(), 0);
}
2018-02-27 10:28:10 -08:00
#[test]
2018-06-06 08:49:22 -07:00
fn test_insufficient_funds() {
2018-05-14 14:39:34 -07:00
let mint = Mint::new(11_000);
let bank = Bank::new(&mint);
let pubkey = KeyPair::new().pubkey();
bank.transfer(1_000, &mint.keypair(), pubkey, mint.last_id())
2018-03-20 22:15:44 -07:00
.unwrap();
2018-05-14 14:33:11 -07:00
assert_eq!(bank.transaction_count(), 1);
assert_eq!(
2018-05-14 14:39:34 -07:00
bank.transfer(10_001, &mint.keypair(), pubkey, mint.last_id()),
2018-05-23 11:26:32 -07:00
Err(BankError::InsufficientFunds(mint.pubkey()))
);
2018-05-14 14:33:11 -07:00
assert_eq!(bank.transaction_count(), 1);
2018-05-14 14:39:34 -07:00
let mint_pubkey = mint.keypair().pubkey();
assert_eq!(bank.get_balance(&mint_pubkey).unwrap(), 10_000);
assert_eq!(bank.get_balance(&pubkey).unwrap(), 1_000);
2018-02-27 10:28:10 -08:00
}
#[test]
fn test_transfer_to_newb() {
2018-05-14 14:39:34 -07:00
let mint = Mint::new(10_000);
let bank = Bank::new(&mint);
let pubkey = KeyPair::new().pubkey();
bank.transfer(500, &mint.keypair(), pubkey, mint.last_id())
2018-03-20 22:15:44 -07:00
.unwrap();
2018-05-14 14:39:34 -07:00
assert_eq!(bank.get_balance(&pubkey).unwrap(), 500);
2018-02-27 10:28:10 -08:00
}
#[test]
fn test_transfer_on_date() {
2018-05-14 14:39:34 -07:00
let mint = Mint::new(1);
let bank = Bank::new(&mint);
let pubkey = KeyPair::new().pubkey();
let dt = Utc::now();
2018-05-14 14:39:34 -07:00
bank.transfer_on_date(1, &mint.keypair(), pubkey, dt, mint.last_id())
.unwrap();
2018-05-14 14:39:34 -07:00
// Mint's balance will be zero because all funds are locked up.
assert_eq!(bank.get_balance(&mint.pubkey()), Some(0));
2018-05-14 05:49:48 -07:00
// tx count is 1, because debits were applied.
2018-05-14 14:33:11 -07:00
assert_eq!(bank.transaction_count(), 1);
2018-05-14 05:49:48 -07:00
2018-05-14 14:39:34 -07:00
// pubkey's balance will be None because the funds have not been
// sent.
2018-05-14 14:39:34 -07:00
assert_eq!(bank.get_balance(&pubkey), None);
// Now, acknowledge the time in the condition occurred and
2018-05-14 14:39:34 -07:00
// that pubkey's funds are now available.
2018-05-29 09:01:40 -07:00
bank.apply_timestamp(mint.pubkey(), dt).unwrap();
2018-05-14 14:39:34 -07:00
assert_eq!(bank.get_balance(&pubkey), Some(1));
2018-05-25 14:51:41 -07:00
// tx count is still 1, because we chose not to count timestamp transactions
2018-05-14 05:49:48 -07:00
// tx count.
2018-05-14 14:33:11 -07:00
assert_eq!(bank.transaction_count(), 1);
2018-05-14 05:49:48 -07:00
2018-05-29 09:01:40 -07:00
bank.apply_timestamp(mint.pubkey(), dt).unwrap(); // <-- Attack! Attempt to process completed transaction.
2018-05-14 14:39:34 -07:00
assert_ne!(bank.get_balance(&pubkey), Some(2));
}
2018-03-08 14:39:03 -08:00
#[test]
fn test_transfer_after_date() {
2018-05-14 14:39:34 -07:00
let mint = Mint::new(1);
let bank = Bank::new(&mint);
let pubkey = KeyPair::new().pubkey();
2018-03-08 14:39:03 -08:00
let dt = Utc::now();
2018-05-29 09:01:40 -07:00
bank.apply_timestamp(mint.pubkey(), dt).unwrap();
2018-03-08 14:39:03 -08:00
// It's now past now, so this transfer should be processed immediately.
2018-05-14 14:39:34 -07:00
bank.transfer_on_date(1, &mint.keypair(), pubkey, dt, mint.last_id())
2018-03-08 14:39:03 -08:00
.unwrap();
2018-05-14 14:39:34 -07:00
assert_eq!(bank.get_balance(&mint.pubkey()), Some(0));
assert_eq!(bank.get_balance(&pubkey), Some(1));
2018-03-08 14:39:03 -08:00
}
#[test]
fn test_cancel_transfer() {
2018-05-14 14:39:34 -07:00
let mint = Mint::new(1);
let bank = Bank::new(&mint);
let pubkey = KeyPair::new().pubkey();
let dt = Utc::now();
2018-05-30 12:07:28 -07:00
let sig = bank.transfer_on_date(1, &mint.keypair(), pubkey, dt, mint.last_id())
.unwrap();
2018-05-14 05:49:48 -07:00
// Assert the debit counts as a transaction.
2018-05-14 14:33:11 -07:00
assert_eq!(bank.transaction_count(), 1);
2018-05-14 05:49:48 -07:00
2018-05-14 14:39:34 -07:00
// Mint's balance will be zero because all funds are locked up.
assert_eq!(bank.get_balance(&mint.pubkey()), Some(0));
2018-05-14 14:39:34 -07:00
// pubkey's balance will be None because the funds have not been
// sent.
2018-05-14 14:39:34 -07:00
assert_eq!(bank.get_balance(&pubkey), None);
2018-05-14 14:39:34 -07:00
// Now, cancel the trancaction. Mint gets her funds back, pubkey never sees them.
2018-05-29 09:01:40 -07:00
bank.apply_signature(mint.pubkey(), sig).unwrap();
2018-05-14 14:39:34 -07:00
assert_eq!(bank.get_balance(&mint.pubkey()), Some(1));
assert_eq!(bank.get_balance(&pubkey), None);
2018-05-14 05:49:48 -07:00
// Assert cancel doesn't cause count to go backward.
2018-05-14 14:33:11 -07:00
assert_eq!(bank.transaction_count(), 1);
2018-05-14 05:49:48 -07:00
2018-05-29 09:01:40 -07:00
bank.apply_signature(mint.pubkey(), sig).unwrap(); // <-- Attack! Attempt to cancel completed transaction.
2018-05-14 14:39:34 -07:00
assert_ne!(bank.get_balance(&mint.pubkey()), Some(2));
}
#[test]
2018-05-25 14:51:41 -07:00
fn test_duplicate_transaction_signature() {
2018-05-14 14:39:34 -07:00
let mint = Mint::new(1);
let bank = Bank::new(&mint);
let sig = Signature::default();
assert!(
bank.reserve_signature_with_last_id(&sig, &mint.last_id())
.is_ok()
);
assert_eq!(
bank.reserve_signature_with_last_id(&sig, &mint.last_id()),
2018-06-28 15:59:32 -07:00
Err(BankError::DuplicateSignature(sig))
);
}
2018-04-26 04:22:11 -07:00
#[test]
fn test_forget_signature() {
2018-05-14 14:39:34 -07:00
let mint = Mint::new(1);
let bank = Bank::new(&mint);
2018-04-26 04:22:11 -07:00
let sig = Signature::default();
bank.reserve_signature_with_last_id(&sig, &mint.last_id())
.unwrap();
bank.forget_signature_with_last_id(&sig, &mint.last_id());
assert!(
bank.reserve_signature_with_last_id(&sig, &mint.last_id())
.is_ok()
);
2018-04-26 04:22:11 -07:00
}
#[test]
2018-06-06 08:49:22 -07:00
fn test_reject_old_last_id() {
2018-05-14 14:39:34 -07:00
let mint = Mint::new(1);
let bank = Bank::new(&mint);
let sig = Signature::default();
for i in 0..MAX_ENTRY_IDS {
let last_id = hash(&serialize(&i).unwrap()); // Unique hash
2018-05-14 14:33:11 -07:00
bank.register_entry_id(&last_id);
}
// Assert we're no longer able to use the oldest entry ID.
assert_eq!(
bank.reserve_signature_with_last_id(&sig, &mint.last_id()),
2018-05-29 20:02:32 -07:00
Err(BankError::LastIdNotFound(mint.last_id()))
);
}
#[test]
fn test_debits_before_credits() {
let mint = Mint::new(2);
2018-05-14 14:33:11 -07:00
let bank = Bank::new(&mint);
2018-05-14 14:39:34 -07:00
let keypair = KeyPair::new();
2018-05-29 09:12:27 -07:00
let tx0 = Transaction::new(&mint.keypair(), keypair.pubkey(), 2, mint.last_id());
let tx1 = Transaction::new(&keypair, mint.pubkey(), 1, mint.last_id());
let txs = vec![tx0, tx1];
let results = bank.process_transactions(txs);
2018-05-14 05:49:48 -07:00
assert!(results[1].is_err());
// Assert bad transactions aren't counted.
2018-05-14 14:33:11 -07:00
assert_eq!(bank.transaction_count(), 1);
}
2018-06-14 16:32:39 -07:00
#[test]
fn test_process_empty_entry_is_registered() {
let mint = Mint::new(1);
let bank = Bank::new(&mint);
let keypair = KeyPair::new();
let entry = next_entry(&mint.last_id(), 1, vec![]);
let tx = Transaction::new(&mint.keypair(), keypair.pubkey(), 1, entry.id);
// First, ensure the TX is rejected because of the unregistered last ID
assert_eq!(
bank.process_transaction(&tx),
Err(BankError::LastIdNotFound(entry.id))
);
// Now ensure the TX is accepted despite pointing to the ID of an empty entry.
bank.process_entries(vec![entry]).unwrap();
assert!(bank.process_transaction(&tx).is_ok());
}
2018-02-23 13:08:19 -08:00
}
#[cfg(all(feature = "unstable", test))]
mod bench {
extern crate test;
use self::test::Bencher;
2018-05-14 14:33:11 -07:00
use bank::*;
use bincode::serialize;
use hash::hash;
2018-06-22 10:45:22 -07:00
use rayon::prelude::*;
use signature::KeyPairUtil;
#[bench]
fn bench_process_transaction(bencher: &mut Bencher) {
2018-04-04 15:01:43 -07:00
let mint = Mint::new(100_000_000);
2018-05-14 14:33:11 -07:00
let bank = Bank::new(&mint);
2018-04-04 15:01:43 -07:00
// Create transactions between unrelated parties.
let transactions: Vec<_> = (0..4096)
.into_par_iter()
.map(|i| {
// Seed the 'from' account.
2018-04-04 15:01:43 -07:00
let rando0 = KeyPair::new();
2018-05-25 15:05:37 -07:00
let tx = Transaction::new(&mint.keypair(), rando0.pubkey(), 1_000, mint.last_id());
bank.process_transaction(&tx).unwrap();
// Seed the 'to' account and a cell for its signature.
let last_id = hash(&serialize(&i).unwrap()); // Unique hash
2018-05-14 14:33:11 -07:00
bank.register_entry_id(&last_id);
2018-04-04 15:01:43 -07:00
let rando1 = KeyPair::new();
2018-05-25 15:05:37 -07:00
let tx = Transaction::new(&rando0, rando1.pubkey(), 1, last_id);
bank.process_transaction(&tx).unwrap();
// Finally, return a transaction that's unique
Transaction::new(&rando0, rando1.pubkey(), 1, last_id)
})
.collect();
bencher.iter(|| {
// Since benchmarker runs this multiple times, we need to clear the signatures.
2018-06-22 10:44:31 -07:00
for (_, sigs) in bank.last_ids_sigs.write().unwrap().iter_mut() {
sigs.clear();
}
assert!(
bank.process_transactions(transactions.clone())
.iter()
.all(|x| x.is_ok())
);
});
}
}