Minimal Transaction Pool (#1067)

* verify a tx like we verify a block (experimental)

* first minimal_pool test up and running but not testing what we need to

* rework tx_pool validation to use txhashset extension

* minimal tx pool wired up but rough

* works locally (rough statew though)
delete "legacy" pool and graph code

* rework the new pool into TransactionPool and Pool impls

* rework pool to store pool entries
with associated timer and source etc.

* all_transactions

* extra_txs so we can validate stempool against existing txpool

* rework reconcile_block

* txhashset apply_raw_tx can now rewind to a checkpoint (prev raw tx)

* wip - txhashset tx tests

* more flexible rewind on MMRs

* add tests to cover apply_raw_txs on txhashset extension

* add_to_stempool and add_to_txpool

* deaggregate multi kernel tx when adding to txpoool

* handle freshness in stempool
handle propagation of stempool txs via dandelion monitor

* patience timer and fluff if we cannot propagate
to next relay

* aggregate and fluff stempool is we have no relay

* refactor coinbase maturity

* rewrote basic tx pool tests to use a real txhashset via chain adapter

* rework dandelion monitor to reflect recent discussion
works locally but needs a cleanup

* refactor dandelion_monitor - split out phases

* more pool test coverage

* remove old test code from pool (still wip)

* block_building and block_reconciliation tests

* tracked down chain test failure...

* fix test_coinbase_maturity

* dandelion_monitor now runs...

* refactor dandelion config, shared across p2p and pool components

* fix pool tests with new config

* fix p2p tests

* rework tx pool to deal with duplicate commitments (testnet2 limitation)

* cleanup and address some PR feedback

* add big comment about pre_tx...
This commit is contained in:
Antioch Peverell 2018-05-30 16:57:13 -04:00 committed by GitHub
parent b6c31ebc78
commit 4fda7a6899
No known key found for this signature in database
GPG key ID: 4AEE18F83AFDEB23
61 changed files with 2265 additions and 3569 deletions

1
Cargo.lock generated
View file

@ -714,6 +714,7 @@ name = "grin_pool"
version = "0.2.0"
dependencies = [
"blake2-rfc 0.2.18 (registry+https://github.com/rust-lang/crates.io-index)",
"grin_chain 0.2.0",
"grin_core 0.2.0",
"grin_keychain 0.2.0",
"grin_util 0.2.0",

View file

@ -614,10 +614,9 @@ where
fn handle(&self, _req: &mut Request) -> IronResult<Response> {
let pool_arc = w(&self.tx_pool);
let pool = pool_arc.read().unwrap();
json_response(&PoolInfo {
pool_size: pool.pool_size(),
orphans_size: pool.orphans_size(),
total_size: pool.total_size(),
pool_size: pool.total_size(),
})
}
}
@ -628,10 +627,8 @@ struct TxWrapper {
tx_hex: String,
}
// Push new transactions to our stem transaction pool, that should broadcast it
// to the network if valid.
// Push new transaction to our local transaction pool.
struct PoolPushHandler<T> {
peers: Weak<p2p::Peers>,
tx_pool: Weak<RwLock<pool::TransactionPool<T>>>,
}
@ -667,21 +664,12 @@ where
}
}
// Will not do a stem transaction if our dandelion peer relay is empty
if !fluff && w(&self.peers).get_dandelion_relay().is_empty() {
debug!(
LOGGER,
"Missing Dandelion relay: will push stem transaction normally"
);
fluff = true;
}
// Push into the pool or stempool
let pool_arc = w(&self.tx_pool);
let res = pool_arc
.write()
.unwrap()
.add_to_memory_pool(source, tx, !fluff);
// Push to tx pool.
let res = {
let pool_arc = w(&self.tx_pool);
let mut tx_pool = pool_arc.write().unwrap();
tx_pool.add_to_pool(source, tx, !fluff)
};
match res {
Ok(()) => Ok(Response::with(status::Ok)),
@ -764,7 +752,6 @@ pub fn start_rest_apis<T>(
tx_pool: tx_pool.clone(),
};
let pool_push_handler = PoolPushHandler {
peers: peers.clone(),
tx_pool: tx_pool.clone(),
};
let peers_all_handler = PeersAllHandler {

View file

@ -14,23 +14,23 @@
use std::sync::Arc;
use core::{core, ser};
use chain;
use core::core::hash::Hashed;
use core::core::pmmr::MerkleProof;
use chain;
use core::{core, ser};
use p2p;
use serde;
use serde::de::MapAccess;
use serde::ser::SerializeStruct;
use std::fmt;
use util;
use util::secp::pedersen;
use serde;
use serde::ser::SerializeStruct;
use serde::de::MapAccess;
use std::fmt;
macro_rules! no_dup {
($field: ident) => {
($field:ident) => {
if $field.is_some() {
return Err(serde::de::Error::duplicate_field("$field"));
}
}
};
}
@ -616,10 +616,6 @@ pub struct OutputListing {
pub struct PoolInfo {
/// Size of the pool
pub pool_size: usize,
/// Size of orphans
pub orphans_size: usize,
/// Total size of pool + orphans
pub total_size: usize,
}
#[cfg(test)]

View file

@ -20,10 +20,10 @@ use std::fs::File;
use std::sync::{Arc, Mutex, RwLock};
use std::time::{Duration, Instant};
use core::core::{Block, BlockHeader, Input, Output, OutputFeatures, OutputIdentifier, TxKernel};
use core::core::hash::{Hash, Hashed};
use core::core::pmmr::MerkleProof;
use core::core::target::Difficulty;
use core::core::{Block, BlockHeader, Output, OutputIdentifier, Transaction, TxKernel};
use core::global;
use grin_store::Error::NotFoundErr;
use pipe;
@ -150,9 +150,9 @@ impl Chain {
}
}
/// Initializes the blockchain and returns a new Chain instance. Does a check
/// on the current chain head to make sure it exists and creates one based
/// on the genesis block if necessary.
/// Initializes the blockchain and returns a new Chain instance. Does a
/// check on the current chain head to make sure it exists and creates one
/// based on the genesis block if necessary.
pub fn init(
db_root: String,
adapter: Arc<ChainAdapter>,
@ -450,15 +450,57 @@ impl Chain {
}
}
/// For the given commitment find the unspent output and return the associated
/// Return an error if the output does not exist or has been spent.
/// This querying is done in a way that is consistent with the current chain state,
/// specifically the current winning (valid, most work) fork.
/// For the given commitment find the unspent output and return the
/// associated Return an error if the output does not exist or has been
/// spent. This querying is done in a way that is consistent with the
/// current chain state, specifically the current winning (valid, most
/// work) fork.
pub fn is_unspent(&self, output_ref: &OutputIdentifier) -> Result<Hash, Error> {
let mut txhashset = self.txhashset.write().unwrap();
txhashset.is_unspent(output_ref)
}
/// Validate a vector of "raw" transactions against the current chain state.
pub fn validate_raw_txs(
&self,
txs: Vec<Transaction>,
pre_tx: Option<Transaction>,
) -> Result<Vec<Transaction>, Error> {
let bh = self.head_header()?;
let mut txhashset = self.txhashset.write().unwrap();
txhashset::extending_readonly(&mut txhashset, |extension| {
let valid_txs = extension.validate_raw_txs(txs, pre_tx, bh.height)?;
Ok(valid_txs)
})
}
fn next_block_height(&self) -> Result<u64, Error> {
let bh = self.head_header()?;
Ok(bh.height + 1)
}
/// Verify we are not attempting to spend a coinbase output
/// that has not yet sufficiently matured.
pub fn verify_coinbase_maturity(&self, tx: &Transaction) -> Result<(), Error> {
let height = self.next_block_height()?;
let mut txhashset = self.txhashset.write().unwrap();
txhashset::extending_readonly(&mut txhashset, |extension| {
extension.verify_coinbase_maturity(&tx.inputs, height)?;
Ok(())
})
}
/// Verify that the tx has a lock_height that is less than or equal to
/// the height of the next block.
pub fn verify_tx_lock_height(&self, tx: &Transaction) -> Result<(), Error> {
let height = self.next_block_height()?;
if tx.lock_height() <= height {
Ok(())
} else {
Err(Error::TxLockHeight)
}
}
/// Validate the current chain state.
pub fn validate(&self, skip_rproofs: bool) -> Result<(), Error> {
let header = self.store.head_header()?;
@ -481,28 +523,13 @@ impl Chain {
})
}
/// Check if the input has matured sufficiently for the given block height.
/// This only applies to inputs spending coinbase outputs.
/// An input spending a non-coinbase output will always pass this check.
pub fn is_matured(&self, input: &Input, height: u64) -> Result<(), Error> {
if input.features.contains(OutputFeatures::COINBASE_OUTPUT) {
let mut txhashset = self.txhashset.write().unwrap();
let output = OutputIdentifier::from_input(&input);
let hash = txhashset.is_unspent(&output)?;
let header = self.get_block_header(&input.block_hash())?;
input.verify_maturity(hash, &header, height)?;
}
Ok(())
}
/// Sets the txhashset roots on a brand new block by applying the block on the
/// current txhashset state.
/// Sets the txhashset roots on a brand new block by applying the block on
/// the current txhashset state.
pub fn set_txhashset_roots(&self, b: &mut Block, is_fork: bool) -> Result<(), Error> {
let mut txhashset = self.txhashset.write().unwrap();
let store = self.store.clone();
let roots = txhashset::extending_readonly(&mut txhashset, |extension| {
// apply the block on the txhashset and check the resulting root
if is_fork {
pipe::rewind_and_apply_fork(b, store, extension)?;
}
@ -810,7 +837,8 @@ impl Chain {
}
/// Verifies the given block header is actually on the current chain.
/// Checks the header_by_height index to verify the header is where we say it is
/// Checks the header_by_height index to verify the header is where we say
/// it is
pub fn is_on_current_chain(&self, header: &BlockHeader) -> Result<(), Error> {
self.store
.is_on_current_chain(header)

View file

@ -45,4 +45,4 @@ pub mod types;
// Re-export the base interface
pub use chain::{Chain, MAX_ORPHAN_SIZE};
pub use types::{ChainAdapter, ChainStore, Error, Options, Tip};
pub use types::{BlockSums, ChainAdapter, ChainStore, Error, Options, Tip};

View file

@ -20,13 +20,13 @@ use time;
use core::consensus;
use core::core::hash::{Hash, Hashed};
use core::core::{Block, BlockHeader};
use core::core::target::Difficulty;
use core::core::{Block, BlockHeader};
use core::global;
use grin_store;
use types::*;
use store;
use txhashset;
use core::global;
use types::*;
use util::LOGGER;
/// Contextual information required to process a new block and either reject or
@ -100,7 +100,14 @@ pub fn process_block(b: &Block, mut ctx: BlockContext) -> Result<Option<Tip>, Er
// start a chain extension unit of work dependent on the success of the
// internal validation and saving operations
let result = txhashset::extending(&mut txhashset, |mut extension| {
validate_block_via_txhashset(b, &mut ctx, &mut extension)?;
// First we rewind the txhashset extension if necessary
// to put it into a consistent state for validating the block.
// We can skip this step if the previous header is the latest header we saw.
if b.header.previous != ctx.head.last_block_h {
rewind_and_apply_fork(b, ctx.store.clone(), extension)?;
}
validate_block_via_txhashset(b, &mut extension)?;
trace!(
LOGGER,
"pipe: process_block: {} at {} is valid, save and append.",
@ -143,8 +150,9 @@ pub fn sync_block_header(
}
/// Process block header as part of "header first" block propagation.
/// We validate the header but we do not store it or update header head based on this.
/// We will update these once we get the block back after requesting it.
/// We validate the header but we do not store it or update header head based
/// on this. We will update these once we get the block back after requesting
/// it.
pub fn process_block_header(bh: &BlockHeader, mut ctx: BlockContext) -> Result<(), Error> {
debug!(
LOGGER,
@ -326,14 +334,10 @@ fn validate_block(b: &Block, ctx: &mut BlockContext) -> Result<(), Error> {
/// and checking the roots.
/// Rewind and reapply forked blocks if necessary to put the txhashset extension
/// in the correct state to accept the block.
fn validate_block_via_txhashset(
b: &Block,
ctx: &mut BlockContext,
ext: &mut txhashset::Extension,
) -> Result<(), Error> {
if b.header.previous != ctx.head.last_block_h {
rewind_and_apply_fork(b, ctx.store.clone(), ext)?;
}
fn validate_block_via_txhashset(b: &Block, ext: &mut txhashset::Extension) -> Result<(), Error> {
// First check we are not attempting to spend any coinbase outputs
// before they have matured sufficiently.
ext.verify_coinbase_maturity(&b.inputs, b.header.height)?;
// apply the new block to the MMR trees and check the new root hashes
ext.apply_block(&b)?;
@ -515,7 +519,7 @@ pub fn rewind_and_apply_fork(
let forked_block = store.get_block_header(&current)?;
debug!(
trace!(
LOGGER,
"rewind_and_apply_fork @ {} [{}], was @ {} [{}]",
forked_block.height,
@ -527,9 +531,10 @@ pub fn rewind_and_apply_fork(
// rewind the sum trees up to the forking block
ext.rewind(&forked_block)?;
debug!(
trace!(
LOGGER,
"rewind_and_apply_fork: blocks on fork: {:?}", hashes
"rewind_and_apply_fork: blocks on fork: {:?}",
hashes,
);
// apply all forked blocks, including this new one

View file

@ -28,7 +28,7 @@ use core::consensus::REWARD;
use core::core::hash::{Hash, Hashed};
use core::core::pmmr::{self, MerkleProof, PMMR};
use core::core::{Block, BlockHeader, Committed, Input, Output, OutputFeatures, OutputIdentifier,
TxKernel};
Transaction, TxKernel};
use core::global;
use core::ser::{PMMRIndexHashable, PMMRable};
@ -257,14 +257,16 @@ where
trace!(LOGGER, "Starting new txhashset (readonly) extension.");
let mut extension = Extension::new(trees, commit_index);
extension.force_rollback();
res = inner(&mut extension);
sizes = extension.sizes();
}
debug!(
trace!(
LOGGER,
"Rollbacking txhashset (readonly) extension. sizes {:?}", sizes
"Rollbacking txhashset (readonly) extension. sizes {:?}",
sizes
);
trees.output_pmmr_h.backend.discard();
@ -315,12 +317,12 @@ where
}
Ok(r) => {
if rollback {
debug!(LOGGER, "Rollbacking txhashset extension. sizes {:?}", sizes);
trace!(LOGGER, "Rollbacking txhashset extension. sizes {:?}", sizes);
trees.output_pmmr_h.backend.discard();
trees.rproof_pmmr_h.backend.discard();
trees.kernel_pmmr_h.backend.discard();
} else {
debug!(LOGGER, "Committing txhashset extension. sizes {:?}", sizes);
trace!(LOGGER, "Committing txhashset extension. sizes {:?}", sizes);
trees.output_pmmr_h.backend.sync()?;
trees.rproof_pmmr_h.backend.sync()?;
trees.kernel_pmmr_h.backend.sync()?;
@ -402,6 +404,106 @@ impl<'a> Extension<'a> {
}
}
/// Apply a "raw" transaction to the txhashset.
/// We will never commit a txhashset extension that includes raw txs.
/// But we can use this when validating txs in the tx pool.
/// If we can add a tx to the tx pool and then successfully add the
/// aggregated tx from the tx pool to the current chain state (via a
/// txhashset extension) then we know the tx pool is valid (including the
/// new tx).
pub fn apply_raw_tx(&mut self, tx: &Transaction, height: u64) -> Result<(), Error> {
// This should *never* be called on a writeable extension...
if !self.rollback {
panic!("attempted to apply a raw tx to a writeable txhashset extension");
}
// Checkpoint the MMR positions before we apply the new tx,
// anything goes wrong we will rewind to these positions.
let output_pos = self.output_pmmr.unpruned_size();
let kernel_pos = self.kernel_pmmr.unpruned_size();
// When applying blocks we can apply the coinbase output first
// but we cannot do this here, so we need to apply outputs first.
for ref output in &tx.outputs {
if let Err(e) = self.apply_output(output) {
self.rewind_to_pos(output_pos, kernel_pos, height)?;
return Err(e);
}
}
for ref input in &tx.inputs {
if let Err(e) = self.apply_input(input, height) {
self.rewind_to_pos(output_pos, kernel_pos, height)?;
return Err(e);
}
}
for ref kernel in &tx.kernels {
if let Err(e) = self.apply_kernel(kernel) {
self.rewind_to_pos(output_pos, kernel_pos, height)?;
return Err(e);
}
}
Ok(())
}
/// Validate a vector of "raw" transactions against the current chain state.
/// We support rewind on a "dirty" txhashset - so we can apply each tx in
/// turn, rewinding if any particular tx is not valid and continuing
/// through the vec of txs provided. This allows us to efficiently apply
/// all the txs, filtering out those that are not valid and returning the
/// final vec of txs that were successfully validated against the txhashset.
///
/// Note: We also pass in a "pre_tx". This tx is applied to and validated
/// before we start applying the vec of txs. We use this when validating
/// txs in the stempool as we need to account for txs in the txpool as
/// well (new_tx + stempool + txpool + txhashset). So we aggregate the
/// contents of the txpool into a single aggregated tx and pass it in here
/// as the "pre_tx" so we apply it to the txhashset before we start
/// validating the stempool txs.
/// This is optional and we pass in None when validating the txpool txs
/// themselves.
///
pub fn validate_raw_txs(
&mut self,
txs: Vec<Transaction>,
pre_tx: Option<Transaction>,
height: u64,
) -> Result<Vec<Transaction>, Error> {
let mut height = height;
let mut valid_txs = vec![];
if let Some(tx) = pre_tx {
height += 1;
self.apply_raw_tx(&tx, height)?;
}
for tx in txs {
height += 1;
if self.apply_raw_tx(&tx, height).is_ok() {
valid_txs.push(tx);
}
}
Ok(valid_txs)
}
/// Verify we are not attempting to spend any coinbase outputs
/// that have not sufficiently matured.
pub fn verify_coinbase_maturity(
&mut self,
inputs: &Vec<Input>,
height: u64,
) -> Result<(), Error> {
for x in inputs {
if x.features.contains(OutputFeatures::COINBASE_OUTPUT) {
let header = self.commit_index.get_block_header(&x.block_hash())?;
let pos = self.get_output_pos(&x.commitment())?;
let out_hash = self.output_pmmr.get_hash(pos).ok_or(Error::OutputNotFound)?;
x.verify_maturity(out_hash, &header, height)?;
}
}
Ok(())
}
/// Apply a new set of blocks on top the existing sum trees. Blocks are
/// applied in order of the provided Vec. If pruning is enabled, inputs also
/// prune MMR data.
@ -443,8 +545,9 @@ impl<'a> Extension<'a> {
Ok(())
}
// Store all new output pos in the index.
// Also store any new block_markers.
fn save_indexes(&self) -> Result<(), Error> {
// store all new output pos in the index
for (commit, pos) in &self.new_output_commits {
self.commit_index.save_output_pos(commit, *pos)?;
}
@ -463,23 +566,13 @@ impl<'a> Extension<'a> {
// check hash from pmmr matches hash from input (or corresponding output)
// if not then the input is not being honest about
// what it is attempting to spend...
let read_elem = self.output_pmmr.get_data(pos);
if output_id_hash != read_hash
|| output_id_hash
!= read_elem
.expect("no output at position")
.hash_with_index(pos - 1)
{
let read_elem_hash = read_elem
.expect("no output at pos")
.hash_with_index(pos - 1);
if output_id_hash != read_hash || output_id_hash != read_elem_hash {
return Err(Error::TxHashSetErr(format!("output pmmr hash mismatch")));
}
// check coinbase maturity with the Merkle Proof on the input
if input.features.contains(OutputFeatures::COINBASE_OUTPUT) {
let header = self.commit_index.get_block_header(&input.block_hash())?;
input.verify_maturity(read_hash, &header, height)?;
}
}
// Now prune the output_pmmr, rproof_pmmr and their storage.
@ -502,7 +595,6 @@ impl<'a> Extension<'a> {
fn apply_output(&mut self, out: &Output) -> Result<(), Error> {
let commit = out.commitment();
if let Ok(pos) = self.get_output_pos(&commit) {
// we need to check whether the commitment is in the current MMR view
// as well as the index doesn't support rewind and is non-authoritative
@ -581,28 +673,40 @@ impl<'a> Extension<'a> {
pub fn rewind(&mut self, block_header: &BlockHeader) -> Result<(), Error> {
let hash = block_header.hash();
let height = block_header.height;
debug!(LOGGER, "Rewind to header {} @ {}", height, hash);
trace!(LOGGER, "Rewind to header {} @ {}", height, hash);
// rewind our MMRs to the appropriate pos
// based on block height and block marker
let marker = self.commit_index.get_block_marker(&hash)?;
self.rewind_to_marker(height, &marker)?;
self.rewind_to_pos(marker.output_pos, marker.kernel_pos, height)?;
Ok(())
}
/// Rewinds the MMRs to the provided positions, given the output and
/// kernel we want to rewind to.
fn rewind_to_marker(&mut self, height: u64, marker: &BlockMarker) -> Result<(), Error> {
debug!(LOGGER, "Rewind txhashset to {}, {:?}", height, marker);
fn rewind_to_pos(
&mut self,
output_pos: u64,
kernel_pos: u64,
height: u64,
) -> Result<(), Error> {
trace!(
LOGGER,
"Rewind txhashset to output {}, kernel {}, height {}",
output_pos,
kernel_pos,
height
);
self.output_pmmr
.rewind(marker.output_pos, height as u32)
.rewind(output_pos, height as u32)
.map_err(&Error::TxHashSetErr)?;
self.rproof_pmmr
.rewind(marker.output_pos, height as u32)
.rewind(output_pos, height as u32)
.map_err(&Error::TxHashSetErr)?;
self.kernel_pmmr
.rewind(marker.kernel_pos, height as u32)
.rewind(kernel_pos, height as u32)
.map_err(&Error::TxHashSetErr)?;
Ok(())

View file

@ -99,6 +99,8 @@ pub enum Error {
SerErr(ser::Error),
/// Error with the txhashset
TxHashSetErr(String),
/// Tx not valid based on lock_height.
TxLockHeight,
/// No chain exists and genesis block is required
GenesisBlockRequired,
/// Error from underlying tx handling

View file

@ -24,8 +24,8 @@ extern crate time;
use std::fs;
use std::sync::Arc;
use chain::Chain;
use chain::types::*;
use chain::Chain;
use core::core::target::Difficulty;
use core::core::{Block, BlockHeader, Transaction};
use core::global;
@ -160,7 +160,12 @@ fn _prepare_block_nosum(
let fees = txs.iter().map(|tx| tx.fee()).sum();
let reward = libtx::reward::output(&kc, &key_id, fees, prev.height).unwrap();
let mut b = match core::core::Block::new(prev, txs, Difficulty::from_num(diff), reward) {
let mut b = match core::core::Block::new(
prev,
txs.into_iter().cloned().collect(),
Difficulty::from_num(diff),
reward,
) {
Err(e) => panic!("{:?}", e),
Ok(b) => b,
};

View file

@ -24,8 +24,8 @@ extern crate time;
use std::fs;
use std::sync::Arc;
use chain::Chain;
use chain::types::*;
use chain::Chain;
use core::consensus;
use core::core::hash::Hashed;
use core::core::target::Difficulty;
@ -248,8 +248,6 @@ fn spend_in_fork_and_compact() {
let merkle_proof = chain.get_merkle_proof(&out_id, &b.header).unwrap();
println!("First block");
// now mine three further blocks
for n in 3..6 {
let b = prepare_block(&kc, &fork_head, &chain, n);
@ -257,10 +255,8 @@ fn spend_in_fork_and_compact() {
chain.process_block(b, chain::Options::SKIP_POW).unwrap();
}
let lock_height = 1 + global::coinbase_maturity();
assert_eq!(lock_height, 4);
println!("3 Further Blocks: should have 4 blocks or 264 bytes in file ");
// Check the height of the "fork block".
assert_eq!(fork_head.height, 4);
let tx1 = build::transaction(
vec![
@ -276,8 +272,6 @@ fn spend_in_fork_and_compact() {
&kc,
).unwrap();
println!("Built coinbase input and output");
let next = prepare_block_tx(&kc, &fork_head, &chain, 7, vec![&tx1]);
let prev_main = next.header.clone();
chain
@ -285,8 +279,6 @@ fn spend_in_fork_and_compact() {
.unwrap();
chain.validate(false).unwrap();
println!("tx 1 processed, should have 6 outputs or 396 bytes in file, first skipped");
let tx2 = build::transaction(
vec![
build::input(consensus::REWARD - 20000, kc.derive_key_id(30).unwrap()),
@ -299,10 +291,9 @@ fn spend_in_fork_and_compact() {
let next = prepare_block_tx(&kc, &prev_main, &chain, 9, vec![&tx2]);
let prev_main = next.header.clone();
chain.process_block(next, chain::Options::SKIP_POW).unwrap();
chain.validate(false).unwrap();
println!("tx 2 processed");
/* panic!("Stop"); */
// Full chain validation for completeness.
chain.validate(false).unwrap();
// mine 2 forked blocks from the first
let fork = prepare_fork_block_tx(&kc, &fork_head, &chain, 6, vec![&tx1]);
@ -314,6 +305,7 @@ fn spend_in_fork_and_compact() {
chain
.process_block(fork_next, chain::Options::SKIP_POW)
.unwrap();
chain.validate(false).unwrap();
// check state
@ -414,7 +406,12 @@ fn prepare_block_nosum(
let fees = txs.iter().map(|tx| tx.fee()).sum();
let reward = libtx::reward::output(&kc, &key_id, fees, prev.height).unwrap();
let mut b = match core::core::Block::new(prev, txs, Difficulty::from_num(diff), reward) {
let mut b = match core::core::Block::new(
prev,
txs.into_iter().cloned().collect(),
Difficulty::from_num(diff),
reward,
) {
Err(e) => panic!("{:?}", e),
Ok(b) => b,
};

View file

@ -22,10 +22,10 @@ extern crate rand;
use std::fs;
use chain::{ChainStore, Tip};
use core::core::Block;
use core::core::BlockHeader;
use core::core::hash::Hashed;
use core::core::target::Difficulty;
use core::core::Block;
use core::core::BlockHeader;
use core::global;
use core::global::ChainTypes;
use core::pow;

View file

@ -25,9 +25,9 @@ use std::sync::Arc;
use chain::types::*;
use core::consensus;
use core::core::OutputIdentifier;
use core::core::target::Difficulty;
use core::core::transaction;
use core::core::OutputIdentifier;
use core::global;
use core::global::ChainTypes;
use wallet::libtx::build;
@ -116,7 +116,7 @@ fn test_coinbase_maturity() {
&keychain,
).unwrap();
let txs = vec![&coinbase_txn];
let txs = vec![coinbase_txn.clone()];
let fees = txs.iter().map(|tx| tx.fee()).sum();
let reward = libtx::reward::output(&keychain, &key_id3, fees, prev.height).unwrap();
let mut block = core::core::Block::new(&prev, txs, Difficulty::one(), reward).unwrap();
@ -124,9 +124,13 @@ fn test_coinbase_maturity() {
let difficulty = consensus::next_difficulty(chain.difficulty_iter()).unwrap();
match chain.set_txhashset_roots(&mut block, false) {
Err(Error::Transaction(transaction::Error::ImmatureCoinbase)) => (),
_ => panic!("expected ImmatureCoinbase error here"),
chain.set_txhashset_roots(&mut block, false).unwrap();
// Confirm the tx attempting to spend the coinbase output
// is not valid at the current block height given the current chain state.
match chain.verify_coinbase_maturity(&coinbase_txn) {
Err(Error::Transaction(transaction::Error::ImmatureCoinbase)) => {}
_ => panic!("Expected transaction error with immature coinbase."),
}
pow::pow_size(
@ -164,20 +168,14 @@ fn test_coinbase_maturity() {
let prev = chain.head_header().unwrap();
let coinbase_txn = build::transaction(
vec![
build::coinbase_input(amount, block_hash, merkle_proof.clone(), key_id1.clone()),
build::output(amount - 2, key_id2.clone()),
build::with_fee(2),
],
&keychain,
).unwrap();
// Confirm the tx spending the coinbase output is now valid.
// The coinbase output has matured sufficiently based on current chain state.
chain.verify_coinbase_maturity(&coinbase_txn).unwrap();
let txs = vec![&coinbase_txn];
let txs = vec![coinbase_txn];
let fees = txs.iter().map(|tx| tx.fee()).sum();
let reward = libtx::reward::output(&keychain, &key_id4, fees, prev.height).unwrap();
let mut block =
core::core::Block::new(&prev, vec![&coinbase_txn], Difficulty::one(), reward).unwrap();
let mut block = core::core::Block::new(&prev, txs, Difficulty::one(), reward).unwrap();
block.header.timestamp = prev.timestamp + time::Duration::seconds(60);

View file

@ -0,0 +1,134 @@
// Copyright 2018 The Grin Developers
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
extern crate grin_chain as chain;
extern crate grin_core as core;
extern crate grin_keychain as keychain;
extern crate grin_wallet as wallet;
use std::fs;
use std::sync::Arc;
use chain::store::ChainKVStore;
use chain::txhashset;
use chain::txhashset::TxHashSet;
use chain::types::Tip;
use chain::ChainStore;
use core::core::pmmr::MerkleProof;
use core::core::target::Difficulty;
use core::core::{Block, BlockHeader};
use keychain::Keychain;
use wallet::libtx::{build, reward};
fn clean_output_dir(dir_name: &str) {
let _ = fs::remove_dir_all(dir_name);
}
#[test]
fn test_some_raw_txs() {
let db_root = format!(".grin_txhashset_raw_txs");
clean_output_dir(&db_root);
let store = Arc::new(ChainKVStore::new(db_root.clone()).unwrap());
let mut txhashset = TxHashSet::open(db_root.clone(), store.clone()).unwrap();
let keychain = Keychain::from_random_seed().unwrap();
let key_id1 = keychain.derive_key_id(1).unwrap();
let key_id2 = keychain.derive_key_id(2).unwrap();
let key_id3 = keychain.derive_key_id(3).unwrap();
let key_id4 = keychain.derive_key_id(4).unwrap();
let key_id5 = keychain.derive_key_id(5).unwrap();
let key_id6 = keychain.derive_key_id(6).unwrap();
// Create a simple block with a single coinbase output
// so we have something to spend.
let prev_header = BlockHeader::default();
let reward_out = reward::output(&keychain, &key_id1, 0, prev_header.height).unwrap();
let block = Block::new(&prev_header, vec![], Difficulty::one(), reward_out).unwrap();
// Now apply this initial block to the (currently empty) MMRs.
// Note: this results in an output MMR with a single leaf node.
// We need to be careful with pruning while processing the txs below
// as we cannot prune a tree with a single node in it (no sibling or parent).
txhashset::extending(&mut txhashset, |extension| extension.apply_block(&block)).unwrap();
// Make sure we setup the head in the store based on block we just accepted.
let head = Tip::from_block(&block.header);
store.save_head(&head).unwrap();
let coinbase_reward = 60_000_000_000;
// tx1 spends the original coinbase output from the block
let tx1 = build::transaction_with_offset(
vec![
build::coinbase_input(
coinbase_reward,
block.hash(),
MerkleProof::default(),
key_id1.clone(),
),
build::output(100, key_id2.clone()),
build::output(150, key_id3.clone()),
],
&keychain,
).unwrap();
// tx2 attempts to "double spend" the coinbase output from the block (conflicts
// with tx1)
let tx2 = build::transaction_with_offset(
vec![
build::coinbase_input(
coinbase_reward,
block.hash(),
MerkleProof::default(),
key_id1.clone(),
),
build::output(100, key_id4.clone()),
],
&keychain,
).unwrap();
// tx3 spends one output from tx1
let tx3 = build::transaction_with_offset(
vec![
build::input(100, key_id2.clone()),
build::output(90, key_id5.clone()),
],
&keychain,
).unwrap();
// tx4 spends the other output from tx1 and the output from tx3
let tx4 = build::transaction_with_offset(
vec![
build::input(150, key_id3.clone()),
build::input(90, key_id5.clone()),
build::output(220, key_id6.clone()),
],
&keychain,
).unwrap();
// Now validate the txs against the txhashset (via a readonly extension).
// Note: we use a single txhashset extension and we can continue to
// apply txs successfully after a failure.
let _ = txhashset::extending_readonly(&mut txhashset, |extension| {
// Note: we pass in an increasing "height" here so we can rollback
// each tx individually as necessary, while maintaining a long lived
// txhashset extension.
assert!(extension.apply_raw_tx(&tx1, 3).is_ok());
assert!(extension.apply_raw_tx(&tx2, 4).is_err());
assert!(extension.apply_raw_tx(&tx3, 5).is_ok());
assert!(extension.apply_raw_tx(&tx4, 6).is_ok());
Ok(())
});
}

View file

@ -14,22 +14,22 @@
//! Blocks and blockheaders
use time;
use rand::{thread_rng, Rng};
use std::collections::HashSet;
use time;
use core::{Commitment, Committed, Input, KernelFeatures, Output, OutputFeatures, Proof, ShortId,
Transaction, TxKernel};
use consensus;
use consensus::{exceeds_weight, reward, VerifySortOrder, REWARD};
use core::hash::{Hash, HashWriter, Hashed, ZERO_HASH};
use core::id::ShortIdentifiable;
use core::target::Difficulty;
use core::transaction;
use ser::{self, read_and_verify_sorted, Readable, Reader, Writeable, WriteableSorted, Writer};
use core::{Commitment, Committed, Input, KernelFeatures, Output, OutputFeatures, Proof, ShortId,
Transaction, TxKernel};
use global;
use keychain;
use keychain::BlindingFactor;
use ser::{self, read_and_verify_sorted, Readable, Reader, Writeable, WriteableSorted, Writer};
use util::LOGGER;
use util::{secp, static_secp_instance};
@ -222,9 +222,9 @@ impl BlockHeader {
/// Compact representation of a full block.
/// Each input/output/kernel is represented as a short_id.
/// A node is reasonably likely to have already seen all tx data (tx broadcast before block)
/// and can go request missing tx data from peers if necessary to hydrate a compact block
/// into a full block.
/// A node is reasonably likely to have already seen all tx data (tx broadcast
/// before block) and can go request missing tx data from peers if necessary to
/// hydrate a compact block into a full block.
#[derive(Debug, Clone)]
pub struct CompactBlock {
/// The header with metadata and commitments to the rest of the data
@ -240,9 +240,9 @@ pub struct CompactBlock {
pub kern_ids: Vec<ShortId>,
}
/// Implementation of Writeable for a compact block, defines how to write the block to a
/// binary writer. Differentiates between writing the block for the purpose of
/// full serialization and the one of just extracting a hash.
/// Implementation of Writeable for a compact block, defines how to write the
/// block to a binary writer. Differentiates between writing the block for the
/// purpose of full serialization and the one of just extracting a hash.
impl Writeable for CompactBlock {
fn write<W: Writer>(&self, writer: &mut W) -> Result<(), ser::Error> {
try!(self.header.write(writer));
@ -271,8 +271,8 @@ impl Writeable for CompactBlock {
}
}
/// Implementation of Readable for a compact block, defines how to read a compact block
/// from a binary stream.
/// Implementation of Readable for a compact block, defines how to read a
/// compact block from a binary stream.
impl Readable for CompactBlock {
fn read(reader: &mut Reader) -> Result<CompactBlock, ser::Error> {
let header = try!(BlockHeader::read(reader));
@ -400,7 +400,7 @@ impl Block {
///
pub fn new(
prev: &BlockHeader,
txs: Vec<&Transaction>,
txs: Vec<Transaction>,
difficulty: Difficulty,
reward_output: (Output, TxKernel),
) -> Result<Block, Error> {
@ -409,7 +409,8 @@ impl Block {
}
/// Hydrate a block from a compact block.
/// Note: caller must validate the block themselves, we do not validate it here.
/// Note: caller must validate the block themselves, we do not validate it
/// here.
pub fn hydrate_from(cb: CompactBlock, txs: Vec<Transaction>) -> Block {
trace!(
LOGGER,
@ -495,7 +496,7 @@ impl Block {
/// that all transactions are valid and calculates the Merkle tree.
pub fn with_reward(
prev: &BlockHeader,
txs: Vec<&Transaction>,
txs: Vec<Transaction>,
reward_out: Output,
reward_kern: TxKernel,
difficulty: Difficulty,
@ -670,10 +671,11 @@ impl Block {
}
/// We can verify the Merkle proof (for coinbase inputs) here in isolation.
/// But we cannot check the following as we need data from the index and the PMMR.
/// So we must be sure to check these at the appropriate point during block validation.
/// * node is in the correct pos in the PMMR
/// * block is the correct one (based on output_root from block_header via the index)
/// But we cannot check the following as we need data from the index and
/// the PMMR. So we must be sure to check these at the appropriate point
/// during block validation. * node is in the correct pos in the PMMR
/// * block is the correct one (based on output_root from block_header
/// via the index)
fn verify_inputs(&self) -> Result<(), Error> {
let coinbase_inputs = self.inputs
.iter()
@ -733,7 +735,8 @@ impl Block {
Ok((io_sum, kernel_sum))
}
/// Validate the coinbase outputs generated by miners. Entails 2 main checks:
/// Validate the coinbase outputs generated by miners. Entails 2 main
/// checks:
///
/// * That the sum of all coinbase-marked outputs equal the supply.
/// * That the sum of blinding factors for all coinbase-marked outputs match

View file

@ -35,12 +35,12 @@
//! The underlying Hashes are stored in a Backend implementation that can
//! either be a simple Vec or a database.
use std::clone::Clone;
use std::marker;
use core::hash::Hash;
use ser;
use ser::{Readable, Reader, Writeable, Writer};
use ser::{PMMRIndexHashable, PMMRable};
use ser::{Readable, Reader, Writeable, Writer};
use std::clone::Clone;
use std::marker;
use util;
use util::LOGGER;
@ -52,9 +52,10 @@ pub trait Backend<T>
where
T: PMMRable,
{
/// Append the provided Hashes to the backend storage, and optionally an associated
/// data element to flatfile storage (for leaf nodes only). The position of the
/// first element of the Vec in the MMR is provided to help the implementation.
/// Append the provided Hashes to the backend storage, and optionally an
/// associated data element to flatfile storage (for leaf nodes only). The
/// position of the first element of the Vec in the MMR is provided to
/// help the implementation.
fn append(&mut self, position: u64, data: Vec<(Hash, Option<T>)>) -> Result<(), String>;
/// Rewind the backend state to a previous position, as if all append
@ -102,8 +103,8 @@ pub const MAX_PATH: u64 = 100;
/// Proves inclusion of an output (node) in the output MMR.
/// We can use this to prove an output was unspent at the time of a given block
/// as the root will match the output_root of the block header.
/// The path and left_right can be used to reconstruct the peak hash for a given tree
/// in the MMR.
/// The path and left_right can be used to reconstruct the peak hash for a
/// given tree in the MMR.
/// The root is the result of hashing all the peaks together.
#[derive(Clone, Debug, Eq, Ord, PartialEq, PartialOrd, Serialize, Deserialize)]
pub struct MerkleProof {
@ -207,8 +208,9 @@ impl MerkleProof {
/// Verify the Merkle proof.
/// We do this by verifying the folloiwing -
/// * inclusion of the node beneath a peak (via the Merkle path/branch of siblings)
/// * inclusion of the peak in the "bag of peaks" beneath the root
/// * inclusion of the node beneath a peak (via the Merkle path/branch of
/// siblings) * inclusion of the peak in the "bag of peaks" beneath the
/// root
pub fn verify(&self) -> bool {
// if we have no further elements in the path
// then this proof verifies successfully if our node is
@ -455,7 +457,6 @@ where
break;
}
}
self.backend.remove(to_prune, index)?;
Ok(true)
}
@ -937,8 +938,9 @@ pub fn bintree_postorder_height(num: u64) -> u64 {
}
/// Is this position a leaf in the MMR?
/// We know the positions of all leaves based on the postorder height of an MMR of any size
/// (somewhat unintuitively but this is how the PMMR is "append only").
/// We know the positions of all leaves based on the postorder height of an MMR
/// of any size (somewhat unintuitively but this is how the PMMR is "append
/// only").
pub fn is_leaf(pos: u64) -> bool {
bintree_postorder_height(pos) == 0
}
@ -1037,9 +1039,9 @@ fn most_significant_pos(num: u64) -> u64 {
#[cfg(test)]
mod test {
use super::*;
use ser::{Error, Readable, Writeable};
use core::{Reader, Writer};
use core::hash::Hash;
use core::{Reader, Writer};
use ser::{Error, Readable, Writeable};
use ser::{PMMRIndexHashable, PMMRable};
/// Simple MMR backend implementation based on a Vector. Pruning does not

View file

@ -13,11 +13,13 @@
// limitations under the License.
//! Transactions
use std::cmp::Ordering;
use std::cmp::max;
use std::collections::HashSet;
use std::io::Cursor;
use std::{error, fmt};
use util::secp::pedersen::{Commitment, ProofMessage, RangeProof};
use util::secp::{self, Message, Signature};
use util::{kernel_sig_msg, static_secp_instance};
@ -34,7 +36,6 @@ use keychain::BlindingFactor;
use ser::{self, read_and_verify_sorted, ser_vec, PMMRable, Readable, Reader, Writeable,
WriteableSorted, Writer};
use util;
use util::LOGGER;
bitflags! {
/// Options for a kernel's structure or use
@ -74,6 +75,9 @@ pub enum Error {
/// Returns if the value hidden within the a RangeProof message isn't
/// repeated 3 times, indicating it's incorrect
InvalidProofMessage,
/// Error when sums do not verify correctly during tx aggregation.
/// Likely a "double spend" across two unconfirmed txs.
AggregationError,
}
impl error::Error for Error {
@ -391,24 +395,30 @@ impl Transaction {
.fold(0, |acc, ref x| max(acc, x.lock_height))
}
/// To verify transaction kernels we check that -
/// * all kernels have an even fee
/// * sum of input/output commitments matches sum of kernel commitments
/// after applying offset * each kernel sig is valid (i.e. tx commitments
/// sum to zero, given above is true)
fn verify_kernels(&self) -> Result<(), Error> {
// Verify all the output rangeproofs.
// Note: this is expensive.
for x in &self.outputs {
x.verify_proof()?;
}
fn verify_kernel_signatures(&self) -> Result<(), Error> {
// Verify the kernel signatures.
// Note: this is expensive.
for x in &self.kernels {
x.verify()?;
}
Ok(())
}
fn verify_rangeproofs(&self) -> Result<(), Error> {
// Verify all the output rangeproofs.
// Note: this is expensive.
for x in &self.outputs {
x.verify_proof()?;
}
Ok(())
}
/// To verify transaction kernels we check that -
/// * all kernels have an even fee
/// * sum of input/output commitments matches sum of kernel commitments
/// after applying offset * each kernel sig is valid (i.e. tx commitments
/// sum to zero, given above is true)
fn verify_kernel_sums(&self) -> Result<(), Error> {
// Sum all input|output|overage commitments.
let overage = self.fee() as i64;
let io_sum = self.sum_commitments(overage, None)?;
@ -433,8 +443,9 @@ impl Transaction {
return Err(Error::TooManyInputs);
}
self.verify_sorted()?;
self.verify_inputs()?;
self.verify_kernels()?;
self.verify_kernel_sums()?;
self.verify_rangeproofs()?;
self.verify_kernel_signatures()?;
Ok(())
}
@ -472,32 +483,11 @@ impl Transaction {
self.kernels.verify_sort_order()?;
Ok(())
}
/// We can verify the Merkle proof (for coinbase inputs) here in isolation.
/// But we cannot check the following as we need data from the index and
/// the PMMR. So we must be sure to check these at the appropriate point
/// during block validation. * node is in the correct pos in the PMMR
/// * block is the correct one (based on output_root from block_header
/// via the index)
fn verify_inputs(&self) -> Result<(), Error> {
let coinbase_inputs = self.inputs
.iter()
.filter(|x| x.features.contains(OutputFeatures::COINBASE_OUTPUT));
for input in coinbase_inputs {
let merkle_proof = input.merkle_proof();
if !merkle_proof.verify() {
return Err(Error::MerkleProof);
}
}
Ok(())
}
}
/// Aggregate a vec of transactions into a multi-kernel transaction with
/// cut_through
pub fn aggregate_with_cut_through(transactions: Vec<Transaction>) -> Result<Transaction, Error> {
pub fn aggregate(transactions: Vec<Transaction>) -> Result<Transaction, Error> {
let mut inputs: Vec<Input> = vec![];
let mut outputs: Vec<Output> = vec![];
let mut kernels: Vec<TxKernel> = vec![];
@ -565,58 +555,14 @@ pub fn aggregate_with_cut_through(transactions: Vec<Transaction>) -> Result<Tran
new_outputs.sort();
kernels.sort();
let tx = Transaction::new(new_inputs, new_outputs, kernels);
let tx = Transaction::new(new_inputs, new_outputs, kernels).with_offset(total_kernel_offset);
Ok(tx.with_offset(total_kernel_offset))
}
// We need to check sums here as aggregation/cut-through may have created an
// invalid tx.
tx.verify_kernel_sums()
.map_err(|_| Error::AggregationError)?;
/// Aggregate a vec of transactions into a multi-kernel transaction
pub fn aggregate(transactions: Vec<Transaction>) -> Result<Transaction, Error> {
let mut inputs: Vec<Input> = vec![];
let mut outputs: Vec<Output> = vec![];
let mut kernels: Vec<TxKernel> = vec![];
// we will sum these together at the end to give us the overall offset for the
// transaction
let mut kernel_offsets = vec![];
for mut transaction in transactions {
// we will summ these later to give a single aggregate offset
kernel_offsets.push(transaction.offset);
inputs.append(&mut transaction.inputs);
outputs.append(&mut transaction.outputs);
kernels.append(&mut transaction.kernels);
}
// now sum the kernel_offsets up to give us an aggregate offset for the
// transaction
let total_kernel_offset = {
let secp = static_secp_instance();
let secp = secp.lock().unwrap();
let mut keys = kernel_offsets
.iter()
.cloned()
.filter(|x| *x != BlindingFactor::zero())
.filter_map(|x| x.secret_key(&secp).ok())
.collect::<Vec<_>>();
if keys.is_empty() {
BlindingFactor::zero()
} else {
let sum = secp.blind_sum(keys, vec![])?;
BlindingFactor::from_secret_key(sum)
}
};
// sort them lexicographically
inputs.sort();
outputs.sort();
kernels.sort();
let tx = Transaction::new(inputs, outputs, kernels);
Ok(tx.with_offset(total_kernel_offset))
Ok(tx)
}
/// Attempt to deaggregate a multi-kernel transaction based on multiple
@ -855,11 +801,6 @@ impl Input {
if lock_height > height {
return Err(Error::ImmatureCoinbase);
}
debug!(
LOGGER,
"input: verify_maturity: success via Merkle proof: {} vs {}", lock_height, height,
);
}
Ok(())
}
@ -1206,11 +1147,9 @@ mod test {
let key_id = keychain.derive_key_id(1).unwrap();
let commit = keychain.commit(1003, &key_id).unwrap();
println!("commit: {:?}", commit);
let key_id = keychain.derive_key_id(1).unwrap();
let commit_2 = keychain.commit(1003, &key_id).unwrap();
println!("commit2 : {:?}", commit_2);
assert!(commit == commit_2);
}

View file

@ -54,12 +54,12 @@ fn too_large_block() {
let now = Instant::now();
parts.append(&mut vec![input(500000, pks.pop().unwrap()), with_fee(2)]);
let mut tx = build::transaction(parts, &keychain).unwrap();
let tx = build::transaction(parts, &keychain).unwrap();
println!("Build tx: {}", now.elapsed().as_secs());
let prev = BlockHeader::default();
let key_id = keychain.derive_key_id(1).unwrap();
let b = new_block(vec![&mut tx], &keychain, &prev, &key_id);
let b = new_block(vec![&tx], &keychain, &prev, &key_id);
assert!(b.validate(&zero_commit, &zero_commit).is_err());
}

View file

@ -19,9 +19,9 @@ extern crate grin_keychain as keychain;
extern crate grin_util as util;
extern crate grin_wallet as wallet;
use grin_core::core::Transaction;
use grin_core::core::block::{Block, BlockHeader};
use grin_core::core::target::Difficulty;
use grin_core::core::Transaction;
use keychain::{Identifier, Keychain};
use wallet::libtx::build::{self, input, output, with_fee};
use wallet::libtx::reward;
@ -86,7 +86,12 @@ pub fn new_block(
) -> Block {
let fees = txs.iter().map(|tx| tx.fee()).sum();
let reward_output = reward::output(keychain, &key_id, fees, previous_header.height).unwrap();
Block::new(&previous_header, txs, Difficulty::one(), reward_output).unwrap()
Block::new(
&previous_header,
txs.into_iter().cloned().collect(),
Difficulty::one(),
reward_output,
).unwrap()
}
// utility producing a transaction that spends an output with the provided

View file

@ -24,8 +24,7 @@ use common::{new_block, tx1i1o, tx1i2o, tx2i1o};
use grin_core::core::block::BlockHeader;
use grin_core::core::block::Error::KernelLockHeight;
use grin_core::core::hash::{Hashed, ZERO_HASH};
use grin_core::core::{aggregate, aggregate_with_cut_through, deaggregate, KernelFeatures, Output,
Transaction};
use grin_core::core::{aggregate, deaggregate, KernelFeatures, Output, Transaction};
use grin_core::ser;
use keychain::Keychain;
use util::{secp_static, static_secp_instance};
@ -128,7 +127,7 @@ fn transaction_cut_through() {
assert!(tx2.validate().is_ok());
// now build a "cut_through" tx from tx1 and tx2
let tx3 = aggregate_with_cut_through(vec![tx1, tx2]).unwrap();
let tx3 = aggregate(vec![tx1, tx2]).unwrap();
assert!(tx3.validate().is_ok());
}

View file

@ -65,7 +65,7 @@ run_tui = true
#Whether to run the wallet listener with the server by default
run_wallet_listener = true
#Whether to run a test miner. This is only for developer testing (chaintype
#Whether to run a test miner. This is only for developer testing (chaintype
#usertesting) at cuckoo 16, and will only mine into the default wallet port.
#real mining should use the standalone grin-miner
run_test_miner = false
@ -73,10 +73,23 @@ run_test_miner = false
#test miner wallet URL (burns if this doesn't exist)
#test_miner_wallet_url = "http://127.0.0.1:13415"
[server.dandelion_config]
#dandelion relay time (choose new relay peer every n secs)
#relay_secs = 600
#fluff and broadcast after embargo expires if tx not seen on network
#embargo_secs = 300
#run dandelion stem/fluff processing every n secs (stem tx aggregation in this window)
#patience_secs = 30
#dandelion stem probability (stem 90% of the time, fluff 10% of the time)
#stem_probability = 90
#The P2P server details (i.e. the server that communicates with other
#grin server nodes
[server.p2p_config]
host = "0.0.0.0"
port = 13414
@ -86,9 +99,6 @@ port = 13414
#will *never* connect to peers in deny list
#peers_deny = ["192.168.0.3:13414", "192.168.0.4:13414"]
#dandelion relay time
#dandelion_relay_time = 600
#how long a banned peer should stay banned
#ban_window = 10800

View file

@ -53,5 +53,6 @@ pub use peer::Peer;
pub use peers::Peers;
pub use serv::{DummyAdapter, Server};
pub use store::{PeerData, State};
pub use types::{Capabilities, ChainAdapter, Direction, Error, P2PConfig, PeerInfo, ReasonForBan,
TxHashSetRead, MAX_BLOCK_HEADERS, MAX_LOCATORS, MAX_PEER_ADDRS};
pub use types::{Capabilities, ChainAdapter, DandelionConfig, Direction, Error, P2PConfig,
PeerInfo, ReasonForBan, TxHashSetRead, MAX_BLOCK_HEADERS, MAX_LOCATORS,
MAX_PEER_ADDRS};

View file

@ -21,9 +21,9 @@ use std::thread;
use std::time;
use core::consensus::{MAX_MSG_LEN, MAX_TX_INPUTS, MAX_TX_KERNELS, MAX_TX_OUTPUTS};
use core::core::BlockHeader;
use core::core::hash::Hash;
use core::core::target::Difficulty;
use core::core::BlockHeader;
use core::ser::{self, Readable, Reader, Writeable, Writer};
use types::*;

View file

@ -254,25 +254,16 @@ impl Peer {
}
}
/// Sends the provided stem transaction to the remote peer. The request may
/// be dropped if the remote peer is known to already have the stem
/// transaction.
/// Sends the provided stem transaction to the remote peer.
/// Note: tracking adapter is ignored for stem transactions (while under
/// embargo).
pub fn send_stem_transaction(&self, tx: &core::Transaction) -> Result<(), Error> {
if !self.tracking_adapter.has(tx.hash()) {
debug!(LOGGER, "Send tx {} to {}", tx.hash(), self.info.addr);
self.connection
.as_ref()
.unwrap()
.send(tx, msg::Type::StemTransaction)
} else {
debug!(
LOGGER,
"Not sending tx {} to {} (already seen)",
tx.hash(),
self.info.addr
);
Ok(())
}
debug!(LOGGER, "Send (stem) tx {} to {}", tx.hash(), self.info.addr);
self.connection
.as_ref()
.unwrap()
.send(tx, msg::Type::StemTransaction)?;
Ok(())
}
/// Sends a request for block headers from the provided block locator
@ -397,7 +388,12 @@ impl ChainAdapter for TrackingAdapter {
}
fn transaction_received(&self, tx: core::Transaction, stem: bool) {
self.push(tx.hash());
// Do not track the tx hash for stem txs.
// Otherwise we fail to handle the subsequent fluff or embargo expiration
// correctly.
if !stem {
self.push(tx.hash());
}
self.adapter.transaction_received(tx, stem)
}

View file

@ -96,10 +96,10 @@ impl Peers {
None => debug!(LOGGER, "Could not update dandelion relay"),
};
}
// Get the dandelion relay
pub fn get_dandelion_relay(&self) -> HashMap<i64, Arc<RwLock<Peer>>> {
let res = self.dandelion_relay.read().unwrap().clone();
res
self.dandelion_relay.read().unwrap().clone()
}
pub fn is_known(&self, addr: &SocketAddr) -> bool {
@ -378,15 +378,16 @@ impl Peers {
}
/// Broadcasts the provided stem transaction to our peer relay.
pub fn broadcast_stem_transaction(&self, tx: &core::Transaction) {
pub fn broadcast_stem_transaction(&self, tx: &core::Transaction) -> Result<(), Error> {
let dandelion_relay = self.get_dandelion_relay();
if dandelion_relay.is_empty() {
debug!(LOGGER, "No dandelion relay updating");
debug!(LOGGER, "No dandelion relay, updating.");
self.update_dandelion_relay();
}
// If still empty broadcast then broadcast transaction normally
// If still return an error, let the caller handle this as they see fit.
// The caller will "fluff" at this point as the stem phase is finished.
if dandelion_relay.is_empty() {
self.broadcast_transaction(tx);
return Err(Error::NoDandelionRelay);
}
for relay in dandelion_relay.values() {
let relay = relay.read().unwrap();
@ -399,6 +400,7 @@ impl Peers {
}
}
}
Ok(())
}
/// Broadcasts the provided transaction to PEER_PREFERRED_COUNT of our

View file

@ -15,8 +15,8 @@
use std::fs::File;
use std::io;
use std::net::{Shutdown, SocketAddr, TcpListener, TcpStream};
use std::sync::{Arc, RwLock};
use std::sync::atomic::{AtomicBool, Ordering};
use std::sync::{Arc, RwLock};
use std::thread;
use std::time::Duration;
@ -34,6 +34,7 @@ use util::LOGGER;
/// peers, receiving connections from other peers and keep track of all of them.
pub struct Server {
pub config: P2PConfig,
pub dandelion_config: DandelionConfig,
capabilities: Capabilities,
handshake: Arc<Handshake>,
pub peers: Arc<Peers>,
@ -50,6 +51,7 @@ impl Server {
db_root: String,
mut capab: Capabilities,
config: P2PConfig,
dandelion_config: DandelionConfig,
adapter: Arc<ChainAdapter>,
genesis: Hash,
stop: Arc<AtomicBool>,
@ -79,6 +81,7 @@ impl Server {
}
Ok(Server {
config: config.clone(),
dandelion_config: dandelion_config.clone(),
capabilities: capab,
handshake: Arc::new(Handshake::new(genesis, config.clone())),
peers: Arc::new(Peers::new(PeerStore::new(db_root)?, adapter, config)),

View file

@ -37,9 +37,6 @@ pub const MAX_PEER_ADDRS: u32 = 256;
/// Maximum number of block header hashes to send as part of a locator
pub const MAX_LOCATORS: u32 = 20;
/// Dandelion relay time
const DANDELION_RELAY_TIME: i64 = 600;
/// How long a banned peer should be banned for
const BAN_WINDOW: i64 = 10800;
@ -60,6 +57,7 @@ pub enum Error {
Timeout,
Store(grin_store::Error),
PeerWithSelf,
NoDandelionRelay,
ProtocolMismatch {
us: u32,
peer: u32,
@ -91,11 +89,23 @@ impl<T> From<mpsc::TrySendError<T>> for Error {
Error::Send(e.to_string())
}
}
// impl From<TimerError> for Error {
// fn from(_: TimerError) -> Error {
// Error::Timeout
// }
// }
/// Configuration for "Dandelion".
/// Note: shared between p2p and pool.
/// Look in top-level server config for info on configuring these parameters.
#[derive(Debug, Clone, Serialize, Deserialize)]
pub struct DandelionConfig {
/// Choose new Dandelion relay peer every n secs.
pub relay_secs: u64,
/// Dandelion embargo, fluff and broadcast tx if not seen on network before
/// embargo expires.
pub embargo_secs: u64,
/// Dandelion patience timer, fluff/stem processing runs every n secs.
/// Tx aggregation happens on stem txs received within this window.
pub patience_secs: u64,
/// Dandelion stem probability (stem 90% of the time, fluff 10% etc.)
pub stem_probability: usize,
}
/// Configuration for the peer-to-peer server.
#[derive(Debug, Clone, Serialize, Deserialize)]
@ -107,8 +117,6 @@ pub struct P2PConfig {
pub peers_deny: Option<Vec<String>>,
pub dandelion_relay_time: Option<i64>,
pub ban_window: Option<i64>,
pub peer_max_count: Option<u32>,
@ -125,7 +133,6 @@ impl Default for P2PConfig {
port: 13414,
peers_allow: None,
peers_deny: None,
dandelion_relay_time: Some(DANDELION_RELAY_TIME),
ban_window: Some(BAN_WINDOW),
peer_max_count: Some(PEER_MAX_COUNT),
peer_min_preferred_count: Some(PEER_MIN_PREFERRED_COUNT),
@ -136,14 +143,6 @@ impl Default for P2PConfig {
/// Note certain fields are options just so they don't have to be
/// included in grin.toml, but we don't want them to ever return none
impl P2PConfig {
/// return dandelion_relay_time
pub fn dandelion_relay_time(&self) -> i64 {
match self.dandelion_relay_time {
Some(n) => n,
None => DANDELION_RELAY_TIME,
}
}
/// return ban window
pub fn ban_window(&self) -> i64 {
match self.ban_window {

View file

@ -22,8 +22,8 @@ use std::sync::atomic::AtomicBool;
use std::thread;
use std::time;
use core::core::target::Difficulty;
use core::core::hash::Hash;
use core::core::target::Difficulty;
use p2p::Peer;
fn open_port() -> u16 {
@ -40,19 +40,26 @@ fn open_port() -> u16 {
fn peer_handshake() {
util::init_test_logger();
let p2p_conf = p2p::P2PConfig {
let p2p_config = p2p::P2PConfig {
host: "0.0.0.0".parse().unwrap(),
port: open_port(),
peers_allow: None,
peers_deny: None,
..p2p::P2PConfig::default()
};
let dandelion_config = p2p::DandelionConfig {
relay_secs: 600,
embargo_secs: 30,
patience_secs: 10,
stem_probability: 90,
};
let net_adapter = Arc::new(p2p::DummyAdapter {});
let server = Arc::new(
p2p::Server::new(
".grin".to_owned(),
p2p::Capabilities::UNKNOWN,
p2p_conf.clone(),
p2p_config.clone(),
dandelion_config.clone(),
net_adapter.clone(),
Hash::from_vec(vec![]),
Arc::new(AtomicBool::new(false)),
@ -66,7 +73,7 @@ fn peer_handshake() {
thread::sleep(time::Duration::from_secs(1));
let addr = SocketAddr::new(p2p_conf.host, p2p_conf.port);
let addr = SocketAddr::new(p2p_config.host, p2p_config.port);
let mut socket = TcpStream::connect_timeout(&addr, time::Duration::from_secs(10)).unwrap();
let my_addr = "127.0.0.1:5000".parse().unwrap();
@ -75,7 +82,7 @@ fn peer_handshake() {
p2p::Capabilities::UNKNOWN,
Difficulty::one(),
my_addr,
&p2p::handshake::Handshake::new(Hash::from_vec(vec![]), p2p_conf.clone()),
&p2p::handshake::Handshake::new(Hash::from_vec(vec![]), p2p_config.clone()),
net_adapter,
).unwrap();

View file

@ -19,3 +19,4 @@ grin_util = { path = "../util" }
[dev-dependencies]
grin_wallet = { path = "../wallet" }
grin_chain = { path = "../chain" }

View file

@ -1,184 +0,0 @@
// Copyright 2018 The Grin Developers
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
//! This file is (hopefully) temporary.
//!
//! It contains a trait based on (but not exactly equal to) the trait defined
//! for the blockchain Output set, discussed at
//! https://github.com/ignopeverell/grin/issues/29, and a dummy implementation
//! of said trait.
//! Notably, OutputDiff has been left off, and the question of how to handle
//! abstract return types has been deferred.
use std::collections::HashMap;
use std::clone::Clone;
use std::sync::RwLock;
use core::core::{block, hash, transaction};
use core::core::{Input, OutputFeatures, OutputIdentifier};
use core::global;
use core::core::hash::Hashed;
use types::{BlockChain, PoolError};
use util::secp::pedersen::Commitment;
/// A DummyOutputSet for mocking up the chain
pub struct DummyOutputSet {
outputs: HashMap<Commitment, transaction::Output>,
}
#[allow(dead_code)]
impl DummyOutputSet {
/// Empty output set
pub fn empty() -> DummyOutputSet {
DummyOutputSet {
outputs: HashMap::new(),
}
}
/// roots
pub fn root(&self) -> hash::Hash {
hash::ZERO_HASH
}
/// apply a block
pub fn apply(&self, b: &block::Block) -> DummyOutputSet {
let mut new_outputs = self.outputs.clone();
for input in &b.inputs {
new_outputs.remove(&input.commitment());
}
for output in &b.outputs {
new_outputs.insert(output.commitment(), output.clone());
}
DummyOutputSet {
outputs: new_outputs,
}
}
/// create with block
pub fn with_block(&mut self, b: &block::Block) {
for input in &b.inputs {
self.outputs.remove(&input.commitment());
}
for output in &b.outputs {
self.outputs.insert(output.commitment(), output.clone());
}
}
/// rewind
pub fn rewind(&self, _: &block::Block) -> DummyOutputSet {
DummyOutputSet {
outputs: HashMap::new(),
}
}
/// get an output
pub fn get_output(&self, output_ref: &Commitment) -> Option<&transaction::Output> {
self.outputs.get(output_ref)
}
fn clone(&self) -> DummyOutputSet {
DummyOutputSet {
outputs: self.outputs.clone(),
}
}
/// only for testing: add an output to the map
pub fn with_output(&self, output: transaction::Output) -> DummyOutputSet {
let mut new_outputs = self.outputs.clone();
new_outputs.insert(output.commitment(), output);
DummyOutputSet {
outputs: new_outputs,
}
}
}
/// A DummyChain is the mocked chain for playing with what methods we would
/// need
#[allow(dead_code)]
pub struct DummyChainImpl {
output: RwLock<DummyOutputSet>,
block_headers: RwLock<Vec<block::BlockHeader>>,
}
#[allow(dead_code)]
impl DummyChainImpl {
/// new dummy chain
pub fn new() -> DummyChainImpl {
DummyChainImpl {
output: RwLock::new(DummyOutputSet {
outputs: HashMap::new(),
}),
block_headers: RwLock::new(vec![]),
}
}
}
impl BlockChain for DummyChainImpl {
fn is_unspent(&self, output_ref: &OutputIdentifier) -> Result<hash::Hash, PoolError> {
match self.output.read().unwrap().get_output(&output_ref.commit) {
Some(_) => Ok(hash::Hash::default()),
None => Err(PoolError::GenericPoolError),
}
}
fn is_matured(&self, input: &Input, height: u64) -> Result<(), PoolError> {
if !input.features.contains(OutputFeatures::COINBASE_OUTPUT) {
return Ok(());
}
let block_hash = input.block_hash.expect("requires a block hash");
let headers = self.block_headers.read().unwrap();
if let Some(h) = headers.iter().find(|x| x.hash() == block_hash) {
if h.height + global::coinbase_maturity() < height {
return Ok(());
}
}
Err(PoolError::InvalidTx(transaction::Error::ImmatureCoinbase))
}
fn head_header(&self) -> Result<block::BlockHeader, PoolError> {
let headers = self.block_headers.read().unwrap();
if headers.len() > 0 {
Ok(headers[0].clone())
} else {
Err(PoolError::GenericPoolError)
}
}
}
impl DummyChain for DummyChainImpl {
fn update_output_set(&mut self, new_output: DummyOutputSet) {
self.output = RwLock::new(new_output);
}
fn apply_block(&self, b: &block::Block) {
self.output.write().unwrap().with_block(b);
self.store_head_header(&b.header)
}
fn store_head_header(&self, block_header: &block::BlockHeader) {
let mut headers = self.block_headers.write().unwrap();
headers.insert(0, block_header.clone());
}
}
/// Dummy chain trait
pub trait DummyChain: BlockChain {
/// update output set
fn update_output_set(&mut self, new_output: DummyOutputSet);
/// apply a block
fn apply_block(&self, b: &block::Block);
/// store header
fn store_head_header(&self, block_header: &block::BlockHeader);
}

View file

@ -1,293 +0,0 @@
// Copyright 2018 The Grin Developers
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
//! Base types for the transaction pool's Directed Acyclic Graphs
use std::vec::Vec;
use std::collections::{HashMap, HashSet};
use util::secp::pedersen::Commitment;
use time;
use std::fmt;
use core::core;
use core::core::hash::Hashed;
use core::core::OutputIdentifier;
/// An entry in the transaction pool.
/// These are the vertices of both of the graph structures
#[derive(Debug, PartialEq, Clone)]
pub struct PoolEntry {
// Core data
/// Unique identifier of this pool entry and the corresponding transaction
pub transaction_hash: core::hash::Hash,
// Metadata
/// Size estimate
pub size_estimate: u64,
/// Receive timestamp
pub receive_ts: time::Tm,
}
impl PoolEntry {
/// Create new transaction pool entry
pub fn new(tx: &core::transaction::Transaction) -> PoolEntry {
PoolEntry {
transaction_hash: transaction_identifier(tx),
size_estimate: estimate_transaction_size(tx),
receive_ts: time::now_utc(),
}
}
}
/// TODO guessing this needs implementing
fn estimate_transaction_size(_tx: &core::transaction::Transaction) -> u64 {
0
}
/// An edge connecting graph vertices.
/// For various use cases, one of either the source or destination may be
/// unpopulated
#[derive(Clone)]
pub struct Edge {
// Source and Destination are the vertex id's, the transaction (kernel)
// hash.
source: Option<core::hash::Hash>,
destination: Option<core::hash::Hash>,
// Output is the output hash which this input/output pairing corresponds
// to.
output: OutputIdentifier,
}
impl Edge {
/// Create new edge
pub fn new(
source: Option<core::hash::Hash>,
destination: Option<core::hash::Hash>,
output: OutputIdentifier,
) -> Edge {
Edge {
source: source,
destination: destination,
output: output,
}
}
/// Create new edge with a source
pub fn with_source(&self, src: Option<core::hash::Hash>) -> Edge {
Edge {
source: src,
destination: self.destination,
output: self.output.clone(),
}
}
/// Create new edge with destination
pub fn with_destination(&self, dst: Option<core::hash::Hash>) -> Edge {
Edge {
source: self.source,
destination: dst,
output: self.output.clone(),
}
}
/// The output_identifier of the edge.
pub fn output(&self) -> OutputIdentifier {
self.output.clone()
}
/// The output commitment of the edge
pub fn output_commitment(&self) -> Commitment {
self.output.commit
}
/// The destination hash of the edge
pub fn destination_hash(&self) -> Option<core::hash::Hash> {
self.destination
}
/// The source hash of the edge
pub fn source_hash(&self) -> Option<core::hash::Hash> {
self.source
}
}
impl fmt::Debug for Edge {
fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result {
write!(
f,
"Edge {{source: {:?}, destination: {:?}, commitment: {:?}}}",
self.source, self.destination, self.output
)
}
}
/// The generic graph container. Both graphs, the pool and orphans, embed this
/// structure and add additional capability on top of it.
pub struct DirectedGraph {
/// Edges
pub edges: HashMap<Commitment, Edge>,
/// Vertices
pub vertices: Vec<PoolEntry>,
/// A small optimization: keeping roots (vertices with in-degree 0) in a
/// separate list makes topological sort a bit faster. (This is true for
/// Kahn's, not sure about other implementations)
pub roots: Vec<PoolEntry>,
}
impl DirectedGraph {
/// Create an empty directed graph
pub fn empty() -> DirectedGraph {
DirectedGraph {
edges: HashMap::new(),
vertices: Vec::new(),
roots: Vec::new(),
}
}
/// Get an edge by its commitment
pub fn get_edge_by_commitment(&self, output_commitment: &Commitment) -> Option<&Edge> {
self.edges.get(output_commitment)
}
/// Remove an edge by its commitment
pub fn remove_edge_by_commitment(&mut self, output_commitment: &Commitment) -> Option<Edge> {
self.edges.remove(output_commitment)
}
/// Remove a vertex by its hash
pub fn remove_vertex(&mut self, tx_hash: core::hash::Hash) -> Option<PoolEntry> {
match self.roots
.iter()
.position(|x| x.transaction_hash == tx_hash)
{
Some(i) => Some(self.roots.swap_remove(i)),
None => match self.vertices
.iter()
.position(|x| x.transaction_hash == tx_hash)
{
Some(i) => Some(self.vertices.swap_remove(i)),
None => None,
},
}
}
/// Promote any non-root vertices to roots based on current edges.
/// For a given tx, if there are no edges with that tx as destination then
/// it is a root.
pub fn update_roots(&mut self) {
let mut new_vertices: Vec<PoolEntry> = vec![];
// first find the set of all destinations from the edges in the graph
// a root is a vertex that is not a destination of any edge
let destinations = self.edges
.values()
.filter_map(|edge| edge.destination)
.collect::<HashSet<_>>();
// now iterate over the current non-root vertices
// and check if it is now a root based on the set of edge destinations
for x in &self.vertices {
if destinations.contains(&x.transaction_hash) {
new_vertices.push(x.clone());
} else {
self.roots.push(x.clone());
}
}
// now update our vertices to reflect the updated list
self.vertices = new_vertices;
}
/// Adds a vertex and a set of incoming edges to the graph.
///
/// The PoolEntry at vertex is added to the graph; depending on the
/// number of incoming edges, the vertex is either added to the vertices
/// or to the roots.
///
/// Outgoing edges must not be included in edges; this method is designed
/// for adding vertices one at a time and only accepts incoming edges as
/// internal edges.
pub fn add_entry(&mut self, vertex: PoolEntry, mut edges: Vec<Edge>) {
if edges.len() == 0 {
self.roots.push(vertex);
} else {
self.vertices.push(vertex);
for edge in edges.drain(..) {
self.edges.insert(edge.output_commitment(), edge);
}
}
}
/// add_vertex_only adds a vertex, meant to be complemented by add_edge_only
/// in cases where delivering a vector of edges is not feasible or efficient
pub fn add_vertex_only(&mut self, vertex: PoolEntry, is_root: bool) {
if is_root {
self.roots.push(vertex);
} else {
self.vertices.push(vertex);
}
}
/// add_edge_only adds an edge
pub fn add_edge_only(&mut self, edge: Edge) {
self.edges.insert(edge.output_commitment(), edge);
}
/// Number of vertices (root + internal)
pub fn len_vertices(&self) -> usize {
self.vertices.len() + self.roots.len()
}
/// Number of root vertices only
pub fn len_roots(&self) -> usize {
self.roots.len()
}
/// Number of edges
pub fn len_edges(&self) -> usize {
self.edges.len()
}
/// Get the current list of roots
pub fn get_roots(&self) -> Vec<core::hash::Hash> {
self.roots.iter().map(|x| x.transaction_hash).collect()
}
/// Get list of all vertices in this graph including the roots
pub fn get_vertices(&self) -> Vec<core::hash::Hash> {
let mut hashes = self.roots
.iter()
.map(|x| x.transaction_hash)
.collect::<Vec<_>>();
let non_root_hashes = self.vertices
.iter()
.map(|x| x.transaction_hash)
.collect::<Vec<_>>();
hashes.extend(&non_root_hashes);
return hashes;
}
}
/// Using transaction merkle_inputs_outputs to calculate a deterministic hash;
/// this hashing mechanism has some ambiguity issues especially around range
/// proofs and any extra data the kernel may cover, but it is used initially
/// for testing purposes.
pub fn transaction_identifier(tx: &core::transaction::Transaction) -> core::hash::Hash {
// core::transaction::merkle_inputs_outputs(&tx.inputs, &tx.outputs)
tx.hash()
}

View file

@ -12,19 +12,13 @@
// See the License for the specific language governing permissions and
// limitations under the License.
//! The transaction pool, keeping a view of currently-valid transactions that
//! The transaction pool, keeping a view of currently valid transactions that
//! may be confirmed soon.
#![deny(non_upper_case_globals)]
#![deny(non_camel_case_types)]
#![deny(non_snake_case)]
#![deny(unused_mut)]
#![warn(missing_docs)]
pub mod graph;
pub mod types;
pub mod blockchain;
pub mod pool;
extern crate blake2_rfc as blake2;
extern crate grin_core as core;
@ -39,5 +33,10 @@ extern crate serde_derive;
extern crate slog;
extern crate time;
pub use pool::TransactionPool;
pub use types::{BlockChain, PoolAdapter, PoolConfig, PoolError, TxSource};
mod pool;
pub mod transaction_pool;
pub mod types;
pub use transaction_pool::TransactionPool;
pub use types::{BlockChain, DandelionConfig, PoolAdapter, PoolConfig, PoolEntryState, PoolError,
TxSource};

File diff suppressed because it is too large Load diff

View file

@ -0,0 +1,178 @@
// Copyright 2018 The Grin Developers
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
//! Transaction pool implementation leveraging txhashset for chain state
//! validation. It is a valid operation to add a tx to the tx pool if the
//! resulting tx pool can be added to the current chain state to produce a
//! valid chain state.
use std::sync::Arc;
use time;
use core::core::transaction;
use core::core::{Block, CompactBlock, Transaction};
use pool::Pool;
use types::*;
/// Transaction pool implementation.
pub struct TransactionPool<T> {
/// Pool Config
pub config: PoolConfig,
/// Our transaction pool.
pub txpool: Pool<T>,
/// Our Dandelion "stempool".
pub stempool: Pool<T>,
/// The blockchain
pub blockchain: Arc<T>,
/// The pool adapter
pub adapter: Arc<PoolAdapter>,
}
impl<T> TransactionPool<T>
where
T: BlockChain,
{
/// Create a new transaction pool
pub fn new(config: PoolConfig, chain: Arc<T>, adapter: Arc<PoolAdapter>) -> TransactionPool<T> {
TransactionPool {
config: config,
txpool: Pool::new(chain.clone(), format!("txpool")),
stempool: Pool::new(chain.clone(), format!("stempool")),
blockchain: chain,
adapter: adapter,
}
}
fn add_to_stempool(&mut self, entry: PoolEntry) -> Result<(), PoolError> {
// Add tx to stempool (passing in all txs from txpool to validate against).
self.stempool
.add_to_pool(entry.clone(), self.txpool.all_transactions())?;
// Note: we do not notify the adapter here,
// we let the dandelion monitor handle this.
Ok(())
}
fn add_to_txpool(&mut self, mut entry: PoolEntry) -> Result<(), PoolError> {
// First deaggregate the tx based on current txpool txs.
if entry.tx.kernels.len() > 1 {
let txs = self.txpool
.find_matching_transactions(entry.tx.kernels.clone());
if !txs.is_empty() {
entry.tx = transaction::deaggregate(entry.tx, txs)?;
entry.src.debug_name = "deagg".to_string();
}
}
self.txpool.add_to_pool(entry.clone(), vec![])?;
// We now need to reconcile the stempool based on the new state of the txpool.
// Some stempool txs may no longer be valid and we need to evict them.
let txpool_tx = self.txpool.aggregate_transaction()?;
self.stempool.reconcile(txpool_tx)?;
self.adapter.tx_accepted(&entry.tx);
Ok(())
}
/// Add the given tx to the pool, directing it to either the stempool or
/// txpool based on stem flag provided.
pub fn add_to_pool(
&mut self,
src: TxSource,
tx: Transaction,
stem: bool,
) -> Result<(), PoolError> {
// Do we have the capacity to accept this transaction?
self.is_acceptable(&tx)?;
// Make sure the transaction is valid before anything else.
tx.validate().map_err(|e| PoolError::InvalidTx(e))?;
// Check the tx lock_time is valid based on current chain state.
self.blockchain.verify_tx_lock_height(&tx)?;
// Check coinbase maturity before we go any further.
self.blockchain.verify_coinbase_maturity(&tx)?;
let entry = PoolEntry {
state: PoolEntryState::Fresh,
src,
tx_at: time::now_utc().to_timespec(),
tx: tx.clone(),
};
if stem {
self.add_to_stempool(entry)?;
} else {
self.add_to_txpool(entry)?;
}
Ok(())
}
/// Reconcile the transaction pool (both txpool and stempool) against the
/// provided block.
pub fn reconcile_block(&mut self, block: &Block) -> Result<(), PoolError> {
// First reconcile the txpool.
self.txpool.reconcile_block(block)?;
self.txpool.reconcile(None)?;
// Then reconcile the stempool, accounting for the txpool txs.
let txpool_tx = self.txpool.aggregate_transaction()?;
self.stempool.reconcile_block(block)?;
self.stempool.reconcile(txpool_tx)?;
Ok(())
}
/// Retrieve all transactions matching the provided "compact block"
/// based on the kernel set.
/// Note: we only look in the txpool for this (stempool is under embargo).
pub fn retrieve_transactions(&self, cb: &CompactBlock) -> Vec<Transaction> {
self.txpool.retrieve_transactions(cb)
}
/// Whether the transaction is acceptable to the pool, given both how
/// full the pool is and the transaction weight.
fn is_acceptable(&self, tx: &Transaction) -> Result<(), PoolError> {
if self.total_size() > self.config.max_pool_size {
// TODO evict old/large transactions instead
return Err(PoolError::OverCapacity);
}
// for a basic transaction (1 input, 2 outputs) -
// (-1 * 1) + (4 * 2) + 1 = 8
// 8 * 10 = 80
if self.config.accept_fee_base > 0 {
let threshold = (tx.tx_weight() as u64) * self.config.accept_fee_base;
if tx.fee() < threshold {
return Err(PoolError::LowFeeTransaction(threshold));
}
}
Ok(())
}
/// Get the total size of the pool.
/// Note: we only consider the txpool here as stempool is under embargo.
pub fn total_size(&self) -> usize {
self.txpool.size()
}
/// Returns a vector of transactions from the txpool so we can build a
/// block from them.
pub fn prepare_mineable_transactions(&self, num_to_fetch: u32) -> Vec<Transaction> {
self.txpool.prepare_mineable_transactions(num_to_fetch)
}
}

View file

@ -15,18 +15,29 @@
//! The primary module containing the implementations of the transaction pool
//! and its top-level members.
use std::collections::{HashMap, HashSet};
use std::iter::Iterator;
use std::vec::Vec;
use std::{error, fmt};
use util::secp::pedersen::Commitment;
pub use graph;
use time::Timespec;
use core::consensus;
use core::core::transaction::{Input, OutputIdentifier};
use core::core::{block, hash, transaction};
use core::core::transaction;
use core::core::transaction::Transaction;
/// Configuration for "Dandelion".
/// Note: shared between p2p and pool.
/// Look in top-level server config for info on configuring these parameters.
#[derive(Debug, Clone, Serialize, Deserialize)]
pub struct DandelionConfig {
/// Choose new Dandelion relay peer every n secs.
pub relay_secs: u64,
/// Dandelion embargo, fluff and broadcast tx if not seen on network before
/// embargo expires.
pub embargo_secs: u64,
/// Dandelion patience timer, fluff/stem processing runs every n secs.
/// Tx aggregation happens on stem txs received within this window.
pub patience_secs: u64,
/// Dandelion stem probability (stem 90% of the time, fluff 10% etc.)
pub stem_probability: usize,
}
/// Transaction pool configuration
#[derive(Clone, Debug, Serialize, Deserialize)]
@ -40,14 +51,6 @@ pub struct PoolConfig {
/// Maximum capacity of the pool in number of transactions
#[serde = "default_max_pool_size"]
pub max_pool_size: usize,
/// Maximum capacity of the pool in number of transactions
#[serde = "default_dandelion_probability"]
pub dandelion_probability: usize,
/// Default embargo for Dandelion transaction
#[serde = "default_dandelion_embargo"]
pub dandelion_embargo: i64,
}
impl Default for PoolConfig {
@ -55,8 +58,6 @@ impl Default for PoolConfig {
PoolConfig {
accept_fee_base: default_accept_fee_base(),
max_pool_size: default_max_pool_size(),
dandelion_probability: default_dandelion_probability(),
dandelion_embargo: default_dandelion_embargo(),
}
}
}
@ -67,11 +68,34 @@ fn default_accept_fee_base() -> u64 {
fn default_max_pool_size() -> usize {
50_000
}
fn default_dandelion_probability() -> usize {
90
/// Represents a single entry in the pool.
/// A single (possibly aggregated) transaction.
#[derive(Clone, Debug)]
pub struct PoolEntry {
/// The state of the pool entry.
pub state: PoolEntryState,
/// Info on where this tx originated from.
pub src: TxSource,
/// Timestamp of when this tx was originally added to the pool.
pub tx_at: Timespec,
/// The transaction itself.
pub tx: Transaction,
}
fn default_dandelion_embargo() -> i64 {
30
/// The possible states a pool entry can be in.
#[derive(Clone, Debug, PartialEq)]
pub enum PoolEntryState {
/// A new entry, not yet processed.
Fresh,
/// Tx to be included in the next "stem" run.
ToStem,
/// Tx previously "stemmed" and propagated.
Stemmed,
/// Tx to be included in the next "fluff" run.
ToFluff,
/// Tx previously "fluffed" and broadcast.
Fluffed,
}
/// Placeholder: the data representing where we heard about a tx from.
@ -82,6 +106,7 @@ fn default_dandelion_embargo() -> i64 {
///
/// Most likely this will evolve to contain some sort of network identifier,
/// once we get a better sense of what transaction building might look like.
#[derive(Clone, Debug)]
pub struct TxSource {
/// Human-readable name used for logging and errors.
pub debug_name: String,
@ -89,93 +114,32 @@ pub struct TxSource {
pub identifier: String,
}
/// This enum describes the parent for a given input of a transaction.
#[derive(Clone)]
pub enum Parent {
/// Unknown
Unknown,
/// Block Transaction
BlockTransaction,
/// Pool Transaction
PoolTransaction {
/// Transaction reference
tx_ref: hash::Hash,
},
/// StemPool Transaction
StemPoolTransaction {
/// Transaction reference
tx_ref: hash::Hash,
},
/// AlreadySpent
AlreadySpent {
/// Other transaction reference
other_tx: hash::Hash,
},
}
impl fmt::Debug for Parent {
fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result {
match self {
&Parent::Unknown => write!(f, "Parent: Unknown"),
&Parent::BlockTransaction => write!(f, "Parent: Block Transaction"),
&Parent::PoolTransaction { tx_ref: x } => {
write!(f, "Parent: Pool Transaction ({:?})", x)
}
&Parent::StemPoolTransaction { tx_ref: x } => {
write!(f, "Parent: Stempool Transaction ({:?})", x)
}
&Parent::AlreadySpent { other_tx: x } => write!(f, "Parent: Already Spent By {:?}", x),
}
}
}
// TODO document this enum more accurately
/// Enum of errors
/// Possible errors when interacting with the transaction pool.
#[derive(Debug)]
pub enum PoolError {
/// An invalid pool entry caused by underlying tx validation error
InvalidTx(transaction::Error),
/// An entry already in the pool
AlreadyInPool,
/// An entry already in the stempool
AlreadyInStempool,
/// A duplicate output
DuplicateOutput {
/// The other transaction
other_tx: Option<hash::Hash>,
/// Is in chain?
in_chain: bool,
/// The output
output: Commitment,
},
/// A double spend
DoubleSpend {
/// The other transaction
other_tx: hash::Hash,
/// The spent output
spent_output: Commitment,
},
/// A failed deaggregation error
FailedDeaggregation,
/// Attempt to add a transaction to the pool with lock_height
/// greater than height of current block
ImmatureTransaction {
/// The lock height of the invalid transaction
lock_height: u64,
},
/// An orphan successfully added to the orphans set
OrphanTransaction,
/// TODO - wip, just getting imports working, remove this and use more
/// specific errors
GenericPoolError,
/// TODO - is this the right level of abstraction for pool errors?
OutputNotFound,
/// TODO - is this the right level of abstraction for pool errors?
OutputSpent,
ImmatureTransaction,
/// Attempt to spend a coinbase output before it has sufficiently matured.
ImmatureCoinbase,
/// Problem propagating a stem tx to the next Dandelion relay node.
DandelionError,
/// Transaction pool is over capacity, can't accept more transactions
OverCapacity,
/// Transaction fee is too low given its weight
LowFeeTransaction(u64),
/// Attempt to add a duplicate output to the pool.
DuplicateCommitment,
/// Other kinds of error (not yet pulled out into meaningful errors).
Other(String),
}
impl From<transaction::Error> for PoolError {
fn from(e: transaction::Error) -> PoolError {
PoolError::InvalidTx(e)
}
}
impl error::Error for PoolError {
@ -196,21 +160,21 @@ impl fmt::Display for PoolError {
/// Interface that the pool requires from a blockchain implementation.
pub trait BlockChain {
/// Get an unspent output by its commitment. Will return an error if the
/// output is spent or if it doesn't exist. The blockchain is expected to
/// produce a result with its current view of the most worked chain,
/// ignoring orphans, etc.
/// We do not maintain outputs themselves. The only information we have is
/// the hash from the output MMR.
fn is_unspent(&self, output_ref: &OutputIdentifier) -> Result<hash::Hash, PoolError>;
/// Validate a vec of txs against the current chain state after applying the
/// pre_tx to the chain state.
fn validate_raw_txs(
&self,
txs: Vec<transaction::Transaction>,
pre_tx: Option<transaction::Transaction>,
) -> Result<Vec<transaction::Transaction>, PoolError>;
/// Check if an output being spent by the input has sufficiently matured.
/// This is only applicable for coinbase outputs (1,000 blocks).
/// Non-coinbase outputs will always pass this check.
fn is_matured(&self, input: &Input, height: u64) -> Result<(), PoolError>;
/// Verify any coinbase outputs being spent
/// have matured sufficiently.
fn verify_coinbase_maturity(&self, tx: &transaction::Transaction) -> Result<(), PoolError>;
/// Get the block header at the head
fn head_header(&self) -> Result<block::BlockHeader, PoolError>;
/// Verify any coinbase outputs being spent
/// have matured sufficiently.
fn verify_tx_lock_height(&self, tx: &transaction::Transaction) -> Result<(), PoolError>;
}
/// Bridge between the transaction pool and the rest of the system. Handles
@ -221,391 +185,19 @@ pub trait PoolAdapter: Send + Sync {
/// it to its internal cache.
fn tx_accepted(&self, tx: &transaction::Transaction);
/// The stem transaction pool has accepted this transactions as valid and
/// added it to its internal cache.
fn stem_tx_accepted(&self, tx: &transaction::Transaction);
/// added it to its internal cache, we have waited for the "patience" timer
/// to fire and we now want to propagate the tx to the next Dandelion relay.
fn stem_tx_accepted(&self, tx: &transaction::Transaction) -> Result<(), PoolError>;
}
/// Dummy adapter used as a placeholder for real implementations
// TODO: do we need this dummy, if it's never used?
#[allow(dead_code)]
pub struct NoopAdapter {}
impl PoolAdapter for NoopAdapter {
fn tx_accepted(&self, _: &transaction::Transaction) {}
fn stem_tx_accepted(&self, _: &transaction::Transaction) {}
}
/// Pool contains the elements of the graph that are connected, in full, to
/// the blockchain.
/// Reservations of outputs by orphan transactions (not fully connected) are
/// not respected.
/// Spending references (input -> output) exist in two structures: internal
/// graph references are contained in the pool edge sets, while references
/// sourced from the blockchain's Output set are contained in the
/// blockchain_connections set.
/// Spent by references (output-> input) exist in two structures: pool-pool
/// connections are in the pool edge set, while unspent (dangling) references
/// exist in the available_outputs set.
pub struct Pool {
graph: graph::DirectedGraph,
// available_outputs are unspent outputs of the current pool set,
// maintained as edges with empty destinations, keyed by the
// output's hash.
available_outputs: HashMap<Commitment, graph::Edge>,
// Consumed blockchain output's are kept in a separate map.
consumed_blockchain_outputs: HashMap<Commitment, graph::Edge>,
}
impl Pool {
/// Return an empty pool
pub fn empty() -> Pool {
Pool {
graph: graph::DirectedGraph::empty(),
available_outputs: HashMap::new(),
consumed_blockchain_outputs: HashMap::new(),
}
}
/// Given an output, check if a spending reference (input -> output)
/// already exists in the pool.
/// Returns the transaction (kernel) hash corresponding to the conflicting
/// transaction
pub fn check_double_spend(&self, o: &transaction::Output) -> Option<hash::Hash> {
self.graph
.get_edge_by_commitment(&o.commitment())
.or(self.consumed_blockchain_outputs.get(&o.commitment()))
.map(|x| x.destination_hash().unwrap())
}
/// Length of roots
pub fn len_roots(&self) -> usize {
self.graph.len_roots()
}
/// Length of vertices
pub fn len_vertices(&self) -> usize {
self.graph.len_vertices()
}
/// Consumed outputs
pub fn get_blockchain_spent(&self, c: &Commitment) -> Option<&graph::Edge> {
self.consumed_blockchain_outputs.get(c)
}
/// Add transaction
pub fn add_pool_transaction(
&mut self,
pool_entry: graph::PoolEntry,
mut blockchain_refs: Vec<graph::Edge>,
pool_refs: Vec<graph::Edge>,
mut new_unspents: Vec<graph::Edge>,
) {
// Removing consumed available_outputs
for new_edge in &pool_refs {
// All of these should correspond to an existing unspent
assert!(
self.available_outputs
.remove(&new_edge.output_commitment())
.is_some()
);
}
// Accounting for consumed blockchain outputs
for new_blockchain_edge in blockchain_refs.drain(..) {
self.consumed_blockchain_outputs
.insert(new_blockchain_edge.output_commitment(), new_blockchain_edge);
}
// Adding the transaction to the vertices list along with internal
// pool edges
self.graph.add_entry(pool_entry, pool_refs);
// Adding the new unspents to the unspent map
for unspent_output in new_unspents.drain(..) {
self.available_outputs
.insert(unspent_output.output_commitment(), unspent_output);
}
}
/// More relax way for stempool transaction in order to accept scenario such as:
/// Parent is in mempool, child is allowed in stempool
///
pub fn add_stempool_transaction(
&mut self,
pool_entry: graph::PoolEntry,
mut blockchain_refs: Vec<graph::Edge>,
pool_refs: Vec<graph::Edge>,
mut new_unspents: Vec<graph::Edge>,
) {
// Removing consumed available_outputs
for new_edge in &pool_refs {
// All of these *can* correspond to an existing unspent
self.available_outputs.remove(&new_edge.output_commitment());
}
// Accounting for consumed blockchain outputs
for new_blockchain_edge in blockchain_refs.drain(..) {
self.consumed_blockchain_outputs
.insert(new_blockchain_edge.output_commitment(), new_blockchain_edge);
}
// Adding the transaction to the vertices list along with internal
// pool edges
self.graph.add_entry(pool_entry, pool_refs);
// Adding the new unspents to the unspent map
for unspent_output in new_unspents.drain(..) {
self.available_outputs
.insert(unspent_output.output_commitment(), unspent_output);
}
}
/// Update roots
pub fn update_roots(&mut self) {
self.graph.update_roots()
}
/// Remove transaction
pub fn remove_pool_transaction(
&mut self,
tx: &transaction::Transaction,
marked_txs: &HashSet<hash::Hash>,
) {
self.graph.remove_vertex(graph::transaction_identifier(tx));
for input in tx.inputs.iter().map(|x| x.commitment()) {
match self.graph.remove_edge_by_commitment(&input) {
Some(x) => if !marked_txs.contains(&x.source_hash().unwrap()) {
self.available_outputs
.insert(x.output_commitment(), x.with_destination(None));
},
None => {
self.consumed_blockchain_outputs.remove(&input);
}
};
}
for output in tx.outputs.iter().map(|x| x.commitment()) {
match self.graph.remove_edge_by_commitment(&output) {
Some(x) => if !marked_txs.contains(&x.destination_hash().unwrap()) {
self.consumed_blockchain_outputs
.insert(x.output_commitment(), x.with_source(None));
},
None => {
self.available_outputs.remove(&output);
}
};
}
}
/// Currently a single rule for miner preference -
/// return all txs if less than num_to_fetch txs in the entire pool
/// otherwise return num_to_fetch of just the roots
pub fn get_mineable_transactions(&self, num_to_fetch: u32) -> Vec<hash::Hash> {
if self.graph.len_vertices() <= num_to_fetch as usize {
self.graph.get_vertices()
} else {
let mut roots = self.graph.get_roots();
roots.truncate(num_to_fetch as usize);
roots
}
}
}
impl TransactionGraphContainer for Pool {
fn get_graph(&self) -> &graph::DirectedGraph {
&self.graph
}
fn get_available_output(&self, output: &Commitment) -> Option<&graph::Edge> {
self.available_outputs.get(output)
}
fn get_external_spent_output(&self, output: &Commitment) -> Option<&graph::Edge> {
self.consumed_blockchain_outputs.get(output)
}
fn get_internal_spent_output(&self, output: &Commitment) -> Option<&graph::Edge> {
self.graph.get_edge_by_commitment(output)
}
}
/// Orphans contains the elements of the transaction graph that have not been
/// connected in full to the blockchain.
pub struct Orphans {
graph: graph::DirectedGraph,
// available_outputs are unspent outputs of the current orphan set,
// maintained as edges with empty destinations.
available_outputs: HashMap<Commitment, graph::Edge>,
// missing_outputs are spending references (inputs) with missing
// corresponding outputs, maintained as edges with empty sources.
missing_outputs: HashMap<Commitment, graph::Edge>,
// pool_connections are bidirectional edges which connect to the pool
// graph. They should map one-to-one to pool graph available_outputs.
// pool_connections should not be viewed authoritatively, they are
// merely informational until the transaction is officially connected to
// the pool.
pool_connections: HashMap<Commitment, graph::Edge>,
}
impl Orphans {
/// empty set
pub fn empty() -> Orphans {
Orphans {
graph: graph::DirectedGraph::empty(),
available_outputs: HashMap::new(),
missing_outputs: HashMap::new(),
pool_connections: HashMap::new(),
}
}
/// Checks for a double spent output, given the hash of the output,
/// ONLY in the data maintained by the orphans set. This includes links
/// to the pool as well as links internal to orphan transactions.
/// Returns the transaction hash corresponding to the conflicting
/// transaction.
pub fn check_double_spend(&self, o: transaction::Output) -> Option<hash::Hash> {
self.graph
.get_edge_by_commitment(&o.commitment())
.or(self.pool_connections.get(&o.commitment()))
.map(|x| x.destination_hash().unwrap())
}
/// unknown output
pub fn get_unknown_output(&self, output: &Commitment) -> Option<&graph::Edge> {
self.missing_outputs.get(output)
}
/// Add an orphan transaction to the orphans set.
///
/// This method adds a given transaction (represented by the PoolEntry at
/// orphan_entry) to the orphans set.
///
/// This method has no failure modes. All checks should be passed before
/// entry.
///
/// Expects a HashMap at is_missing describing the indices of orphan_refs
/// which correspond to missing (vs orphan-to-orphan) links.
pub fn add_orphan_transaction(
&mut self,
orphan_entry: graph::PoolEntry,
mut pool_refs: Vec<graph::Edge>,
mut orphan_refs: Vec<graph::Edge>,
is_missing: HashMap<usize, ()>,
mut new_unspents: Vec<graph::Edge>,
) {
// Removing consumed available_outputs
for (i, new_edge) in orphan_refs.drain(..).enumerate() {
if is_missing.contains_key(&i) {
self.missing_outputs
.insert(new_edge.output_commitment(), new_edge);
} else {
assert!(
self.available_outputs
.remove(&new_edge.output_commitment())
.is_some()
);
self.graph.add_edge_only(new_edge);
}
}
// Accounting for consumed blockchain and pool outputs
for external_edge in pool_refs.drain(..) {
self.pool_connections
.insert(external_edge.output_commitment(), external_edge);
}
// if missing_refs is the same length as orphan_refs, we have
// no orphan-orphan links for this transaction and it is a
// root transaction of the orphans set
self.graph
.add_vertex_only(orphan_entry, is_missing.len() == orphan_refs.len());
// Adding the new unspents to the unspent map
for unspent_output in new_unspents.drain(..) {
self.available_outputs
.insert(unspent_output.output_commitment(), unspent_output);
}
}
}
impl TransactionGraphContainer for Orphans {
fn get_graph(&self) -> &graph::DirectedGraph {
&self.graph
}
fn get_available_output(&self, output: &Commitment) -> Option<&graph::Edge> {
self.available_outputs.get(output)
}
fn get_external_spent_output(&self, output: &Commitment) -> Option<&graph::Edge> {
self.pool_connections.get(output)
}
fn get_internal_spent_output(&self, output: &Commitment) -> Option<&graph::Edge> {
self.graph.get_edge_by_commitment(output)
}
}
/// Trait for types that embed a graph and connect to external state.
///
/// The types implementing this trait consist of a graph with nodes and edges
/// representing transactions and outputs, respectively. Outputs fall into one
/// of three categories:
/// 1) External spent: An output sourced externally consumed by a transaction
/// in this graph,
/// 2) Internal spent: An output produced by a transaction in this graph and
/// consumed by another transaction in this graph,
/// 3) [External] Unspent: An output produced by a transaction in this graph
/// that is not yet spent.
///
/// There is no concept of an external "spent by" reference (output produced by
/// a transaction in the graph spent by a transaction in another source), as
/// these references are expected to be maintained by descendent graph. Outputs
/// follow a heirarchy (Blockchain -> Pool -> Orphans) where each descendent
/// exists at a lower priority than their parent. An output consumed by a
/// child graph is marked as unspent in the parent graph and an external spent
/// in the child. This ensures that no descendent set must modify state in a
/// set of higher priority.
pub trait TransactionGraphContainer {
/// Accessor for graph object
fn get_graph(&self) -> &graph::DirectedGraph;
/// Accessor for internal spents
fn get_internal_spent_output(&self, output: &Commitment) -> Option<&graph::Edge>;
/// Accessor for external unspents
fn get_available_output(&self, output: &Commitment) -> Option<&graph::Edge>;
/// Accessor for external spents
fn get_external_spent_output(&self, output: &Commitment) -> Option<&graph::Edge>;
/// Checks if the available_output set has the output at the given
/// commitment
fn has_available_output(&self, c: &Commitment) -> bool {
self.get_available_output(c).is_some()
}
/// Checks if the pool has anything by this output already, between
/// available outputs and internal ones.
fn find_output(&self, c: &Commitment) -> Option<hash::Hash> {
self.get_available_output(c)
.or(self.get_internal_spent_output(c))
.map(|x| x.source_hash().unwrap())
}
/// Search for a spent reference internal to the graph
fn get_internal_spent(&self, c: &Commitment) -> Option<&graph::Edge> {
self.get_internal_spent_output(c)
}
/// number of root transactions
fn num_root_transactions(&self) -> usize {
self.get_graph().len_roots()
}
/// number of transactions
fn num_transactions(&self) -> usize {
self.get_graph().len_vertices()
}
/// number of output edges
fn num_output_edges(&self) -> usize {
self.get_graph().len_edges()
fn stem_tx_accepted(&self, _: &transaction::Transaction) -> Result<(), PoolError> {
Ok(())
}
}

View file

@ -0,0 +1,147 @@
// Copyright 2018 The Grin Developers
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
extern crate blake2_rfc as blake2;
extern crate grin_chain as chain;
extern crate grin_core as core;
extern crate grin_keychain as keychain;
extern crate grin_pool as pool;
extern crate grin_util as util;
extern crate grin_wallet as wallet;
extern crate rand;
extern crate time;
pub mod common;
use std::sync::{Arc, RwLock};
use core::core::{Block, BlockHeader};
use chain::txhashset;
use chain::types::Tip;
use chain::ChainStore;
use core::core::target::Difficulty;
use keychain::Keychain;
use wallet::libtx;
use common::*;
#[test]
fn test_transaction_pool_block_building() {
let keychain = Keychain::from_random_seed().unwrap();
let db_root = ".grin_block_building".to_string();
clean_output_dir(db_root.clone());
let chain = ChainAdapter::init(db_root.clone()).unwrap();
// Initialize the chain/txhashset with an initial block
// so we have a non-empty UTXO set.
let header = {
let height = 1;
let key_id = keychain.derive_key_id(height as u32).unwrap();
let reward = libtx::reward::output(&keychain, &key_id, 0, height).unwrap();
let block = Block::new(&BlockHeader::default(), vec![], Difficulty::one(), reward).unwrap();
let mut txhashset = chain.txhashset.write().unwrap();
txhashset::extending(&mut txhashset, |extension| extension.apply_block(&block)).unwrap();
let tip = Tip::from_block(&block.header);
chain.store.save_block_header(&block.header).unwrap();
chain.store.save_head(&tip).unwrap();
block.header
};
// Initialize a new pool with our chain adapter.
let pool = RwLock::new(test_setup(&Arc::new(chain.clone())));
// Now create tx to spend that first coinbase (now matured).
// Provides us with some useful outputs to test with.
let initial_tx = test_transaction_spending_coinbase(&keychain, &header, vec![10, 20, 30, 40]);
// Add this tx to the pool (stem=false, direct to txpool).
{
let mut write_pool = pool.write().unwrap();
write_pool
.add_to_pool(test_source(), initial_tx, false)
.unwrap();
assert_eq!(write_pool.total_size(), 1);
}
let root_tx_1 = test_transaction(&keychain, vec![10, 20], vec![24]);
let root_tx_2 = test_transaction(&keychain, vec![30], vec![28]);
let root_tx_3 = test_transaction(&keychain, vec![40], vec![38]);
let child_tx_1 = test_transaction(&keychain, vec![24], vec![22]);
let child_tx_2 = test_transaction(&keychain, vec![38], vec![32]);
{
let mut write_pool = pool.write().unwrap();
// Add the three root txs to the pool.
write_pool
.add_to_pool(test_source(), root_tx_1, false)
.unwrap();
write_pool
.add_to_pool(test_source(), root_tx_2, false)
.unwrap();
write_pool
.add_to_pool(test_source(), root_tx_3, false)
.unwrap();
// Now add the two child txs to the pool.
write_pool
.add_to_pool(test_source(), child_tx_1.clone(), false)
.unwrap();
write_pool
.add_to_pool(test_source(), child_tx_2.clone(), false)
.unwrap();
assert_eq!(write_pool.total_size(), 6);
}
let txs = {
let read_pool = pool.read().unwrap();
read_pool.prepare_mineable_transactions(4)
};
assert_eq!(txs.len(), 4);
let block = {
let key_id = keychain.derive_key_id(2).unwrap();
let fees = txs.iter().map(|tx| tx.fee()).sum();
let reward = libtx::reward::output(&keychain, &key_id, fees, 0).unwrap();
Block::new(&header, txs, Difficulty::one(), reward)
}.unwrap();
{
let mut txhashset = chain.txhashset.write().unwrap();
txhashset::extending(&mut txhashset, |extension| {
extension.apply_block(&block)?;
Ok(())
}).unwrap();
}
// Now reconcile the transaction pool with the new block
// and check the resulting contents of the pool are what we expect.
{
let mut write_pool = pool.write().unwrap();
write_pool.reconcile_block(&block).unwrap();
assert_eq!(write_pool.total_size(), 2);
assert_eq!(write_pool.txpool.entries[0].tx, child_tx_1);
assert_eq!(write_pool.txpool.entries[1].tx, child_tx_2);
}
}

View file

@ -0,0 +1,212 @@
// Copyright 2018 The Grin Developers
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
extern crate blake2_rfc as blake2;
extern crate grin_chain as chain;
extern crate grin_core as core;
extern crate grin_keychain as keychain;
extern crate grin_pool as pool;
extern crate grin_util as util;
extern crate grin_wallet as wallet;
extern crate rand;
extern crate time;
pub mod common;
use std::sync::{Arc, RwLock};
use core::core::{Block, BlockHeader};
use chain::txhashset;
use chain::types::Tip;
use chain::ChainStore;
use core::core::target::Difficulty;
use keychain::Keychain;
use wallet::libtx;
use common::*;
#[test]
fn test_transaction_pool_block_reconciliation() {
let keychain = Keychain::from_random_seed().unwrap();
let db_root = ".grin_block_reconcilliation".to_string();
clean_output_dir(db_root.clone());
let chain = ChainAdapter::init(db_root.clone()).unwrap();
// Initialize the chain/txhashset with an initial block
// so we have a non-empty UTXO set.
let header = {
let height = 1;
let key_id = keychain.derive_key_id(height as u32).unwrap();
let reward = libtx::reward::output(&keychain, &key_id, 0, height).unwrap();
let block = Block::new(&BlockHeader::default(), vec![], Difficulty::one(), reward).unwrap();
let mut txhashset = chain.txhashset.write().unwrap();
txhashset::extending(&mut txhashset, |extension| extension.apply_block(&block)).unwrap();
let tip = Tip::from_block(&block.header);
chain.store.save_block_header(&block.header).unwrap();
chain.store.save_head(&tip).unwrap();
block.header
};
// Initialize a new pool with our chain adapter.
let pool = RwLock::new(test_setup(&Arc::new(chain.clone())));
// Now create tx to spend that first coinbase (now matured).
// Provides us with some useful outputs to test with.
let initial_tx = test_transaction_spending_coinbase(&keychain, &header, vec![10, 20, 30, 40]);
let header = {
let key_id = keychain.derive_key_id(2).unwrap();
let fees = initial_tx.fee();
let reward = libtx::reward::output(&keychain, &key_id, fees, 0).unwrap();
let block = Block::new(&header, vec![initial_tx], Difficulty::one(), reward).unwrap();
{
let mut txhashset = chain.txhashset.write().unwrap();
txhashset::extending(&mut txhashset, |extension| {
extension.apply_block(&block)?;
Ok(())
}).unwrap();
}
let tip = Tip::from_block(&block.header);
chain.store.save_block_header(&block.header).unwrap();
chain.store.save_head(&tip).unwrap();
block.header
};
// Preparation: We will introduce three root pool transactions.
// 1. A transaction that should be invalidated because it is exactly
// contained in the block.
// 2. A transaction that should be invalidated because the input is
// consumed in the block, although it is not exactly consumed.
// 3. A transaction that should remain after block reconciliation.
let block_transaction = test_transaction(&keychain, vec![10], vec![8]);
let conflict_transaction = test_transaction(&keychain, vec![20], vec![12, 6]);
let valid_transaction = test_transaction(&keychain, vec![30], vec![13, 15]);
// We will also introduce a few children:
// 4. A transaction that descends from transaction 1, that is in
// turn exactly contained in the block.
let block_child = test_transaction(&keychain, vec![8], vec![5, 1]);
// 5. A transaction that descends from transaction 4, that is not
// contained in the block at all and should be valid after
// reconciliation.
let pool_child = test_transaction(&keychain, vec![5], vec![3]);
// 6. A transaction that descends from transaction 2 that does not
// conflict with anything in the block in any way, but should be
// invalidated (orphaned).
let conflict_child = test_transaction(&keychain, vec![12], vec![2]);
// 7. A transaction that descends from transaction 2 that should be
// valid due to its inputs being satisfied by the block.
let conflict_valid_child = test_transaction(&keychain, vec![6], vec![4]);
// 8. A transaction that descends from transaction 3 that should be
// invalidated due to an output conflict.
let valid_child_conflict = test_transaction(&keychain, vec![13], vec![9]);
// 9. A transaction that descends from transaction 3 that should remain
// valid after reconciliation.
let valid_child_valid = test_transaction(&keychain, vec![15], vec![11]);
// 10. A transaction that descends from both transaction 6 and
// transaction 9
let mixed_child = test_transaction(&keychain, vec![2, 11], vec![7]);
let txs_to_add = vec![
block_transaction,
conflict_transaction,
valid_transaction.clone(),
block_child,
pool_child.clone(),
conflict_child,
conflict_valid_child.clone(),
valid_child_conflict.clone(),
valid_child_valid.clone(),
mixed_child,
];
// First we add the above transactions to the pool.
// All should be accepted.
{
let mut write_pool = pool.write().unwrap();
assert_eq!(write_pool.total_size(), 0);
for tx in &txs_to_add {
write_pool
.add_to_pool(test_source(), tx.clone(), false)
.unwrap();
}
assert_eq!(write_pool.total_size(), txs_to_add.len());
}
// Now we prepare the block that will cause the above conditions to be met.
// First, the transactions we want in the block:
// - Copy of 1
let block_tx_1 = test_transaction(&keychain, vec![10], vec![8]);
// - Conflict w/ 2, satisfies 7
let block_tx_2 = test_transaction(&keychain, vec![20], vec![6]);
// - Copy of 4
let block_tx_3 = test_transaction(&keychain, vec![8], vec![5, 1]);
// - Output conflict w/ 8
let block_tx_4 = test_transaction(&keychain, vec![40], vec![9, 31]);
let block_txs = vec![block_tx_1, block_tx_2, block_tx_3, block_tx_4];
// Now apply this block.
let block = {
let key_id = keychain.derive_key_id(3).unwrap();
let fees = block_txs.iter().map(|tx| tx.fee()).sum();
let reward = libtx::reward::output(&keychain, &key_id, fees, 0).unwrap();
let block = Block::new(&header, block_txs, Difficulty::one(), reward).unwrap();
{
let mut txhashset = chain.txhashset.write().unwrap();
txhashset::extending(&mut txhashset, |extension| {
extension.apply_block(&block)?;
Ok(())
}).unwrap();
}
block
};
// And reconcile the pool with this latest block.
{
let mut write_pool = pool.write().unwrap();
write_pool.reconcile_block(&block).unwrap();
assert_eq!(write_pool.total_size(), 4);
assert_eq!(write_pool.txpool.entries[0].tx, valid_transaction);
// TODO - this is the "correct" behavior (see below)
// assert_eq!(write_pool.txpool.entries[1].tx, pool_child);
// assert_eq!(write_pool.txpool.entries[2].tx, conflict_valid_child);
// assert_eq!(write_pool.txpool.entries[3].tx, valid_child_valid);
//
// TODO - once the hash() vs hash_with_index(pos - 1) change is made in
// txhashset.apply_output() TODO - and we no longer incorrectly allow
// duplicate outputs in the MMR TODO - then this test will fail
//
// TODO - wtf is with these name permutations...
//
assert_eq!(write_pool.txpool.entries[1].tx, conflict_valid_child);
assert_eq!(write_pool.txpool.entries[2].tx, valid_child_conflict);
assert_eq!(write_pool.txpool.entries[3].tx, valid_child_valid);
}
}

View file

@ -0,0 +1,100 @@
// Copyright 2018 The Grin Developers
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
extern crate blake2_rfc as blake2;
extern crate grin_chain as chain;
extern crate grin_core as core;
extern crate grin_keychain as keychain;
extern crate grin_pool as pool;
extern crate grin_util as util;
extern crate grin_wallet as wallet;
extern crate rand;
extern crate time;
pub mod common;
use std::sync::{Arc, RwLock};
use core::core::Transaction;
use keychain::Keychain;
use pool::TransactionPool;
use pool::types::*;
use common::*;
pub fn test_setup(
chain: &Arc<CoinbaseMaturityErrorChainAdapter>,
) -> TransactionPool<CoinbaseMaturityErrorChainAdapter> {
TransactionPool::new(
PoolConfig {
accept_fee_base: 0,
max_pool_size: 50,
},
chain.clone(),
Arc::new(NoopAdapter {}),
)
}
#[derive(Clone)]
pub struct CoinbaseMaturityErrorChainAdapter {}
impl CoinbaseMaturityErrorChainAdapter {
pub fn new() -> CoinbaseMaturityErrorChainAdapter {
CoinbaseMaturityErrorChainAdapter {}
}
}
impl BlockChain for CoinbaseMaturityErrorChainAdapter {
fn validate_raw_txs(
&self,
_txs: Vec<Transaction>,
_pre_tx: Option<Transaction>,
) -> Result<Vec<Transaction>, PoolError> {
Err(PoolError::Other(
"not implemented, not a real chain adapter...".to_string(),
))
}
// Returns an ImmatureCoinbase for every tx we pass in.
fn verify_coinbase_maturity(&self, _tx: &Transaction) -> Result<(), PoolError> {
Err(PoolError::ImmatureCoinbase)
}
// Mocking this out for these tests.
fn verify_tx_lock_height(&self, _tx: &Transaction) -> Result<(), PoolError> {
Ok(())
}
}
/// Test we correctly verify coinbase maturity when adding txs to the pool.
#[test]
fn test_coinbase_maturity() {
let keychain = Keychain::from_random_seed().unwrap();
// Mocking this up with an adapter that will raise an error for coinbase
// maturity.
let chain = CoinbaseMaturityErrorChainAdapter::new();
let pool = RwLock::new(test_setup(&Arc::new(chain.clone())));
{
let mut write_pool = pool.write().unwrap();
let tx = test_transaction(&keychain, vec![50], vec![49]);
match write_pool.add_to_pool(test_source(), tx.clone(), true) {
Err(PoolError::ImmatureCoinbase) => {}
_ => panic!("Expected an immature coinbase error here."),
}
}
}

181
pool/tests/common/mod.rs Normal file
View file

@ -0,0 +1,181 @@
// Copyright 2018 The Grin Developers
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
//! Common test functions
extern crate blake2_rfc as blake2;
extern crate grin_chain as chain;
extern crate grin_core as core;
extern crate grin_keychain as keychain;
extern crate grin_pool as pool;
extern crate grin_util as util;
extern crate grin_wallet as wallet;
extern crate rand;
extern crate time;
use std::fs;
use std::sync::{Arc, RwLock};
use core::core::{BlockHeader, Transaction};
use chain::store::ChainKVStore;
use chain::txhashset;
use chain::txhashset::TxHashSet;
use chain::ChainStore;
use core::core::hash::Hashed;
use core::core::pmmr::MerkleProof;
use pool::*;
use keychain::Keychain;
use wallet::libtx;
use pool::types::*;
use pool::TransactionPool;
#[derive(Clone)]
pub struct ChainAdapter {
pub txhashset: Arc<RwLock<TxHashSet>>,
pub store: Arc<ChainKVStore>,
}
impl ChainAdapter {
pub fn init(db_root: String) -> Result<ChainAdapter, String> {
let target_dir = format!("target/{}", db_root);
let chain_store = ChainKVStore::new(target_dir.clone())
.map_err(|e| format!("failed to init chain_store, {}", e))?;
let store = Arc::new(chain_store);
let txhashset = TxHashSet::open(target_dir.clone(), store.clone())
.map_err(|e| format!("failed to init txhashset, {}", e))?;
Ok(ChainAdapter {
txhashset: Arc::new(RwLock::new(txhashset)),
store: store.clone(),
})
}
}
impl BlockChain for ChainAdapter {
fn validate_raw_txs(
&self,
txs: Vec<Transaction>,
pre_tx: Option<Transaction>,
) -> Result<Vec<Transaction>, PoolError> {
let header = self.store.head_header().unwrap();
let mut txhashset = self.txhashset.write().unwrap();
let res = txhashset::extending_readonly(&mut txhashset, |extension| {
let valid_txs = extension.validate_raw_txs(txs, pre_tx, header.height)?;
Ok(valid_txs)
}).map_err(|e| PoolError::Other(format!("Error: test chain adapter: {:?}", e)))?;
Ok(res)
}
// Mocking this check out for these tests.
// We will test the Merkle proof verification logic elsewhere.
fn verify_coinbase_maturity(&self, _tx: &Transaction) -> Result<(), PoolError> {
Ok(())
}
// Mocking this out for these tests.
fn verify_tx_lock_height(&self, _tx: &Transaction) -> Result<(), PoolError> {
Ok(())
}
}
pub fn test_setup(chain: &Arc<ChainAdapter>) -> TransactionPool<ChainAdapter> {
TransactionPool::new(
PoolConfig {
accept_fee_base: 0,
max_pool_size: 50,
},
chain.clone(),
Arc::new(NoopAdapter {}),
)
}
pub fn test_transaction_spending_coinbase(
keychain: &Keychain,
header: &BlockHeader,
output_values: Vec<u64>,
) -> Transaction {
let output_sum = output_values.iter().sum::<u64>() as i64;
let coinbase_reward: u64 = 60_000_000_000;
let fees: i64 = coinbase_reward as i64 - output_sum;
assert!(fees >= 0);
let mut tx_elements = Vec::new();
// single input spending a single coinbase (deterministic key_id aka height)
{
let key_id = keychain.derive_key_id(header.height as u32).unwrap();
tx_elements.push(libtx::build::coinbase_input(
coinbase_reward,
header.hash(),
MerkleProof::default(),
key_id,
));
}
for output_value in output_values {
let key_id = keychain.derive_key_id(output_value as u32).unwrap();
tx_elements.push(libtx::build::output(output_value, key_id));
}
tx_elements.push(libtx::build::with_fee(fees as u64));
libtx::build::transaction(tx_elements, &keychain).unwrap()
}
pub fn test_transaction(
keychain: &Keychain,
input_values: Vec<u64>,
output_values: Vec<u64>,
) -> Transaction {
let input_sum = input_values.iter().sum::<u64>() as i64;
let output_sum = output_values.iter().sum::<u64>() as i64;
let fees: i64 = input_sum - output_sum;
assert!(fees >= 0);
let mut tx_elements = Vec::new();
for input_value in input_values {
let key_id = keychain.derive_key_id(input_value as u32).unwrap();
tx_elements.push(libtx::build::input(input_value, key_id));
}
for output_value in output_values {
let key_id = keychain.derive_key_id(output_value as u32).unwrap();
tx_elements.push(libtx::build::output(output_value, key_id));
}
tx_elements.push(libtx::build::with_fee(fees as u64));
libtx::build::transaction(tx_elements, &keychain).unwrap()
}
pub fn test_source() -> TxSource {
TxSource {
debug_name: format!("test"),
identifier: format!("127.0.0.1"),
}
}
pub fn clean_output_dir(db_root: String) {
if let Err(e) = fs::remove_dir_all(format!("target/{}", db_root)) {
println!("cleaning output dir failed - {:?}", e)
}
}

View file

@ -1,96 +0,0 @@
// Copyright 2018 The Grin Developers
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
//! Top-level Graph tests
extern crate grin_core as core;
extern crate grin_keychain as keychain;
extern crate grin_pool as pool;
extern crate grin_wallet as wallet;
extern crate rand;
use core::core::OutputFeatures;
use core::core::transaction::ProofMessageElements;
use keychain::Keychain;
use wallet::libtx::proof;
#[test]
fn test_add_entry() {
let keychain = Keychain::from_random_seed().unwrap();
let key_id1 = keychain.derive_key_id(1).unwrap();
let key_id2 = keychain.derive_key_id(2).unwrap();
let key_id3 = keychain.derive_key_id(3).unwrap();
let output_commit = keychain.commit(70, &key_id1).unwrap();
let inputs = vec![
core::core::transaction::Input::new(
OutputFeatures::DEFAULT_OUTPUT,
keychain.commit(50, &key_id2).unwrap(),
None,
None,
),
core::core::transaction::Input::new(
OutputFeatures::DEFAULT_OUTPUT,
keychain.commit(25, &key_id3).unwrap(),
None,
None,
),
];
let msg = ProofMessageElements::new(100, &key_id1);
let output = core::core::transaction::Output {
features: OutputFeatures::DEFAULT_OUTPUT,
commit: output_commit,
proof: proof::create(
&keychain,
100,
&key_id1,
output_commit,
None,
msg.to_proof_message(),
).unwrap(),
};
let kernel = core::core::transaction::TxKernel::empty()
.with_fee(5)
.with_lock_height(0);
let test_transaction =
core::core::transaction::Transaction::new(inputs, vec![output], vec![kernel]);
let test_pool_entry = pool::graph::PoolEntry::new(&test_transaction);
let incoming_edge_1 = pool::graph::Edge::new(
Some(random_hash()),
Some(core::core::hash::ZERO_HASH),
core::core::OutputIdentifier::from_output(&output),
);
let mut test_graph = pool::graph::DirectedGraph::empty();
test_graph.add_entry(test_pool_entry, vec![incoming_edge_1]);
assert_eq!(test_graph.vertices.len(), 1);
assert_eq!(test_graph.roots.len(), 0);
assert_eq!(test_graph.edges.len(), 1);
}
/// For testing/debugging: a random tx hash
fn random_hash() -> core::core::hash::Hash {
let hash_bytes: [u8; 32] = rand::random();
core::core::hash::Hash(hash_bytes)
}

File diff suppressed because it is too large Load diff

View file

@ -0,0 +1,223 @@
// Copyright 2018 The Grin Developers
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
extern crate blake2_rfc as blake2;
extern crate grin_chain as chain;
extern crate grin_core as core;
extern crate grin_keychain as keychain;
extern crate grin_pool as pool;
extern crate grin_util as util;
extern crate grin_wallet as wallet;
extern crate rand;
extern crate time;
pub mod common;
use std::sync::{Arc, RwLock};
use core::core::{Block, BlockHeader};
use chain::txhashset;
use chain::types::Tip;
use chain::ChainStore;
use core::core::target::Difficulty;
use core::core::transaction;
use keychain::Keychain;
use wallet::libtx;
use common::*;
/// Test we can add some txs to the pool (both stempool and txpool).
#[test]
fn test_the_transaction_pool() {
let keychain = Keychain::from_random_seed().unwrap();
let db_root = ".grin_transaction_pool".to_string();
clean_output_dir(db_root.clone());
let chain = ChainAdapter::init(db_root.clone()).unwrap();
// Initialize the chain/txhashset with a few blocks,
// so we have a non-empty UTXO set.
let header = {
let height = 1;
let key_id = keychain.derive_key_id(height as u32).unwrap();
let reward = libtx::reward::output(&keychain, &key_id, 0, height).unwrap();
let block = Block::new(&BlockHeader::default(), vec![], Difficulty::one(), reward).unwrap();
let mut txhashset = chain.txhashset.write().unwrap();
txhashset::extending(&mut txhashset, |extension| extension.apply_block(&block)).unwrap();
let tip = Tip::from_block(&block.header);
chain.store.save_block_header(&block.header).unwrap();
chain.store.save_head(&tip).unwrap();
block.header
};
// Initialize a new pool with our chain adapter.
let pool = RwLock::new(test_setup(&Arc::new(chain.clone())));
// Now create tx to spend a coinbase, giving us some useful outputs for testing
// with.
let initial_tx = {
test_transaction_spending_coinbase(
&keychain,
&header,
vec![500, 600, 700, 800, 900, 1000, 1100, 1200, 1300, 1400],
)
};
// Add this tx to the pool (stem=false, direct to txpool).
{
let mut write_pool = pool.write().unwrap();
write_pool
.add_to_pool(test_source(), initial_tx, false)
.unwrap();
assert_eq!(write_pool.total_size(), 1);
}
// tx1 spends some outputs from the initial test tx.
let tx1 = test_transaction(&keychain, vec![500, 600], vec![499, 599]);
// tx2 spends some outputs from both tx1 and the initial test tx.
let tx2 = test_transaction(&keychain, vec![499, 700], vec![498]);
// Take a write lock and add a couple of tx entries to the pool.
{
let mut write_pool = pool.write().unwrap();
// Check we have a single initial tx in the pool.
assert_eq!(write_pool.total_size(), 1);
// First, add a simple tx to the pool in "stem" mode.
write_pool
.add_to_pool(test_source(), tx1.clone(), true)
.unwrap();
assert_eq!(write_pool.total_size(), 1);
assert_eq!(write_pool.stempool.size(), 1);
// Add another tx spending outputs from the previous tx.
write_pool
.add_to_pool(test_source(), tx2.clone(), true)
.unwrap();
assert_eq!(write_pool.total_size(), 1);
assert_eq!(write_pool.stempool.size(), 2);
}
// Test adding the exact same tx multiple times (same kernel signature).
// This will fail during tx aggregation due to duplicate outputs and duplicate
// kernels.
{
let mut write_pool = pool.write().unwrap();
assert!(
write_pool
.add_to_pool(test_source(), tx1.clone(), true)
.is_err()
);
}
// Test adding a duplicate tx with the same input and outputs (not the *same*
// tx).
{
let tx1a = test_transaction(&keychain, vec![500, 600], vec![499, 599]);
let mut write_pool = pool.write().unwrap();
assert!(write_pool.add_to_pool(test_source(), tx1a, true).is_err());
}
// Test adding a tx attempting to spend a non-existent output.
{
let bad_tx = test_transaction(&keychain, vec![10_001], vec![10_000]);
let mut write_pool = pool.write().unwrap();
assert!(write_pool.add_to_pool(test_source(), bad_tx, true).is_err());
}
// Test adding a tx that would result in a duplicate output (conflicts with
// output from tx2). For reasons of security all outputs in the UTXO set must
// be unique. Otherwise spending one will almost certainly cause the other
// to be immediately stolen via a "replay" tx.
{
let tx = test_transaction(&keychain, vec![900], vec![498]);
let mut write_pool = pool.write().unwrap();
assert!(write_pool.add_to_pool(test_source(), tx, true).is_err());
}
// Confirm the tx pool correctly identifies an invalid tx (already spent).
{
let mut write_pool = pool.write().unwrap();
let tx3 = test_transaction(&keychain, vec![500], vec![497]);
assert!(write_pool.add_to_pool(test_source(), tx3, true).is_err());
assert_eq!(write_pool.total_size(), 1);
assert_eq!(write_pool.stempool.size(), 2);
}
// Check we can take some entries from the stempool and "fluff" them into the
// txpool. This also exercises multi-kernel txs.
{
let mut write_pool = pool.write().unwrap();
let agg_tx = write_pool
.stempool
.aggregate_transaction()
.unwrap()
.unwrap();
assert_eq!(agg_tx.kernels.len(), 2);
write_pool
.add_to_pool(test_source(), agg_tx, false)
.unwrap();
assert_eq!(write_pool.total_size(), 2);
}
// Now check we can correctly deaggregate a multi-kernel tx based on current
// contents of the txpool.
// We will do this be adding a new tx to the pool
// that is a superset of a tx already in the pool.
{
let mut write_pool = pool.write().unwrap();
let tx4 = test_transaction(&keychain, vec![800], vec![799]);
// tx1 and tx2 are already in the txpool (in aggregated form)
// tx4 is the "new" part of this aggregated tx that we care about
let agg_tx = transaction::aggregate(vec![tx1.clone(), tx2.clone(), tx4]).unwrap();
write_pool
.add_to_pool(test_source(), agg_tx, false)
.unwrap();
assert_eq!(write_pool.total_size(), 3);
let entry = write_pool.txpool.entries.last().unwrap();
assert_eq!(entry.tx.kernels.len(), 1);
assert_eq!(entry.src.debug_name, "deagg");
}
// Check we cannot "double spend" an output spent in a previous block.
// We use the initial coinbase output here for convenience.
{
let mut write_pool = pool.write().unwrap();
let double_spend_tx =
{ test_transaction_spending_coinbase(&keychain, &header, vec![1000]) };
// check we cannot add a double spend to the stempool
assert!(
write_pool
.add_to_pool(test_source(), double_spend_tx.clone(), true)
.is_err()
);
// check we cannot add a double spend to the txpool
assert!(
write_pool
.add_to_pool(test_source(), double_spend_tx.clone(), false)
.is_err()
);
}
}

View file

@ -15,28 +15,28 @@
//! Adapters connecting new block, new transaction, and accepted transaction
//! events to consumers of those events.
use rand;
use rand::Rng;
use std::fs::File;
use std::net::SocketAddr;
use std::ops::Deref;
use std::sync::{Arc, RwLock, Weak};
use std::sync::atomic::{AtomicBool, Ordering};
use std::sync::{Arc, RwLock, Weak};
use std::thread;
use std::time::Instant;
use rand;
use rand::Rng;
use chain::{self, ChainAdapter, Options, Tip};
use common::types::{ChainValidationMode, ServerConfig};
use core::core;
use core::core::block::BlockHeader;
use core::core::hash::{Hash, Hashed};
use core::core::target::Difficulty;
use core::core::transaction::{Input, OutputIdentifier};
use core::core::transaction::Transaction;
use p2p;
use pool;
use util::OneTime;
use store;
use common::types::{ChainValidationMode, ServerConfig};
use util::LOGGER;
use util::OneTime;
// All adapters use `Weak` references instead of `Arc` to avoid cycles that
// can never be destroyed. These 2 functions are simple helpers to reduce the
@ -75,35 +75,23 @@ impl p2p::ChainAdapter for NetToChainAdapter {
debug_name: "p2p".to_string(),
identifier: "?.?.?.?".to_string(),
};
debug!(
LOGGER,
"Received tx {} from {}, going to process.",
"Received tx {} from {:?}, going to process.",
tx.hash(),
source.identifier,
source,
);
let h = tx.hash();
if !stem && tx.kernels.len() != 1 {
debug!(
LOGGER,
"Received regular multi-kernel transaction will attempt to deaggregate"
);
if let Err(e) = self.tx_pool
.write()
.unwrap()
.deaggregate_and_add_to_memory_pool(source, tx, stem)
{
debug!(LOGGER, "Transaction {} rejected: {:?}", h, e);
}
} else {
if let Err(e) = self.tx_pool
.write()
.unwrap()
.add_to_memory_pool(source, tx, stem)
{
debug!(LOGGER, "Transaction {} rejected: {:?}", h, e);
}
let res = {
let mut tx_pool = self.tx_pool.write().unwrap();
tx_pool.add_to_pool(source, tx, stem)
};
if let Err(e) = res {
debug!(LOGGER, "Transaction {} rejected: {:?}", h, e);
}
}
@ -635,8 +623,8 @@ impl ChainToPoolAndNetAdapter {
}
}
/// Initialize a ChainToPoolAndNetAdapter instance with hanlde to a Peers object.
/// Should only be called once.
/// Initialize a ChainToPoolAndNetAdapter instance with hanlde to a Peers
/// object. Should only be called once.
pub fn init(&self, peers: Weak<p2p::Peers>) {
self.peers.init(peers);
}
@ -649,8 +637,11 @@ pub struct PoolToNetAdapter {
}
impl pool::PoolAdapter for PoolToNetAdapter {
fn stem_tx_accepted(&self, tx: &core::Transaction) {
wo(&self.peers).broadcast_stem_transaction(tx);
fn stem_tx_accepted(&self, tx: &core::Transaction) -> Result<(), pool::PoolError> {
wo(&self.peers)
.broadcast_stem_transaction(tx)
.map_err(|_| pool::PoolError::DandelionError)?;
Ok(())
}
fn tx_accepted(&self, tx: &core::Transaction) {
wo(&self.peers).broadcast_transaction(tx);
@ -694,26 +685,25 @@ impl PoolToChainAdapter {
}
impl pool::BlockChain for PoolToChainAdapter {
fn is_unspent(&self, output_ref: &OutputIdentifier) -> Result<Hash, pool::PoolError> {
wo(&self.chain).is_unspent(output_ref).map_err(|e| match e {
chain::types::Error::OutputNotFound => pool::PoolError::OutputNotFound,
chain::types::Error::OutputSpent => pool::PoolError::OutputSpent,
_ => pool::PoolError::GenericPoolError,
fn validate_raw_txs(
&self,
txs: Vec<Transaction>,
pre_tx: Option<Transaction>,
) -> Result<(Vec<Transaction>), pool::PoolError> {
wo(&self.chain).validate_raw_txs(txs, pre_tx).map_err(|_| {
pool::PoolError::Other("Chain adapter failed to validate_raw_txs.".to_string())
})
}
fn is_matured(&self, input: &Input, height: u64) -> Result<(), pool::PoolError> {
fn verify_coinbase_maturity(&self, tx: &Transaction) -> Result<(), pool::PoolError> {
wo(&self.chain)
.is_matured(input, height)
.map_err(|e| match e {
chain::types::Error::OutputNotFound => pool::PoolError::OutputNotFound,
_ => pool::PoolError::GenericPoolError,
})
.verify_coinbase_maturity(tx)
.map_err(|_| pool::PoolError::ImmatureCoinbase)
}
fn head_header(&self) -> Result<BlockHeader, pool::PoolError> {
fn verify_tx_lock_height(&self, tx: &Transaction) -> Result<(), pool::PoolError> {
wo(&self.chain)
.head_header()
.map_err(|_| pool::PoolError::GenericPoolError)
.verify_tx_lock_height(tx)
.map_err(|_| pool::PoolError::ImmatureTransaction)
}
}

View file

@ -19,12 +19,24 @@ use std::convert::From;
use api;
use chain;
use core::core;
use core::global::ChainTypes;
use core::pow;
use p2p;
use pool;
use store;
use wallet;
use core::global::ChainTypes;
use core::pow;
/// Dandelion relay timer
const DANDELION_RELAY_SECS: u64 = 600;
/// Dandelion emabargo timer
const DANDELION_EMBARGO_SECS: u64 = 180;
/// Dandelion patience timer
const DANDELION_PATIENCE_SECS: u64 = 10;
/// Dandelion stem probability (stem 90% of the time, fluff 10%).
const DANDELION_STEM_PROBABILITY: usize = 90;
/// Error type wrapping underlying module errors.
#[derive(Debug)]
@ -123,6 +135,54 @@ impl Default for Seeding {
}
}
fn default_dandelion_stem_probability() -> usize {
DANDELION_STEM_PROBABILITY
}
fn default_dandelion_relay_secs() -> u64 {
DANDELION_RELAY_SECS
}
fn default_dandelion_embargo_secs() -> u64 {
DANDELION_EMBARGO_SECS
}
fn default_dandelion_patience_secs() -> u64 {
DANDELION_PATIENCE_SECS
}
/// Dandelion config.
/// Note: Used by both p2p and pool components.
#[derive(Debug, Clone, Serialize, Deserialize)]
pub struct DandelionConfig {
/// Choose new Dandelion relay peer every n secs.
#[serde = "default_dandelion_relay_secs"]
pub relay_secs: u64,
/// Dandelion embargo, fluff and broadcast tx if not seen on network before
/// embargo expires.
#[serde = "default_dandelion_embargo_secs"]
pub embargo_secs: u64,
/// Dandelion patience timer, fluff/stem processing runs every n secs.
/// Tx aggregation happens on stem txs received within this window.
#[serde = "default_dandelion_patience_secs"]
pub patience_secs: u64,
/// Dandelion stem probability.
#[serde = "default_dandelion_stem_probability"]
pub stem_probability: usize,
}
/// Default address for peer-to-peer connections.
impl Default for DandelionConfig {
fn default() -> DandelionConfig {
DandelionConfig {
relay_secs: default_dandelion_relay_secs(),
embargo_secs: default_dandelion_embargo_secs(),
patience_secs: default_dandelion_patience_secs(),
stem_probability: default_dandelion_stem_probability(),
}
}
}
/// Full server configuration, aggregating configurations required for the
/// different components.
#[derive(Debug, Clone, Serialize, Deserialize)]
@ -167,6 +227,10 @@ pub struct ServerConfig {
#[serde(default)]
pub pool_config: pool::PoolConfig,
/// Dandelion configuration
#[serde(default)]
pub dandelion_config: DandelionConfig,
/// Whether to skip the sync timeout on startup
/// (To assist testing on solo chains)
pub skip_sync_wait: Option<bool>,
@ -185,6 +249,28 @@ pub struct ServerConfig {
pub test_miner_wallet_url: Option<String>,
}
impl ServerConfig {
/// Adapter for configuring Dandelion on the pool component.
pub fn pool_dandelion_config(&self) -> pool::DandelionConfig {
pool::DandelionConfig {
relay_secs: self.dandelion_config.relay_secs,
embargo_secs: self.dandelion_config.embargo_secs,
patience_secs: self.dandelion_config.patience_secs,
stem_probability: self.dandelion_config.stem_probability,
}
}
/// Adapter for configuring Dandelion on the p2p component.
pub fn p2p_dandelion_config(&self) -> p2p::DandelionConfig {
p2p::DandelionConfig {
relay_secs: self.dandelion_config.relay_secs,
embargo_secs: self.dandelion_config.embargo_secs,
patience_secs: self.dandelion_config.patience_secs,
stem_probability: self.dandelion_config.stem_probability,
}
}
}
impl Default for ServerConfig {
fn default() -> ServerConfig {
ServerConfig {
@ -194,6 +280,7 @@ impl Default for ServerConfig {
seeding_type: Seeding::default(),
seeds: None,
p2p_config: p2p::P2PConfig::default(),
dandelion_config: DandelionConfig::default(),
stratum_mining_config: Some(StratumServerConfig::default()),
chain_type: ChainTypes::default(),
archive_mode: None,

View file

@ -12,17 +12,19 @@
// See the License for the specific language governing permissions and
// limitations under the License.
use rand;
use rand::Rng;
use std::sync::atomic::{AtomicBool, Ordering};
use std::sync::{Arc, RwLock};
use std::thread;
use std::time::Duration;
use time::now_utc;
use util::LOGGER;
use pool::BlockChain;
use pool::PoolConfig;
use pool::TransactionPool;
use pool::TxSource;
use core::core::hash::Hashed;
use core::core::transaction;
use p2p::DandelionConfig;
use pool::{BlockChain, PoolEntryState, PoolError, TransactionPool, TxSource};
use util::LOGGER;
/// A process to monitor transactions in the stempool.
/// With Dandelion, transaction can be broadcasted in stem or fluff phase.
@ -33,53 +35,218 @@ use pool::TxSource;
/// the transaction will be sent in fluff phase (to multiple peers) instead of
/// sending only to the peer relay.
pub fn monitor_transactions<T>(
config: PoolConfig,
dandelion_config: DandelionConfig,
tx_pool: Arc<RwLock<TransactionPool<T>>>,
stop: Arc<AtomicBool>,
) where
T: BlockChain + Send + Sync + 'static,
{
debug!(LOGGER, "Started Dandelion transaction monitor");
debug!(LOGGER, "Started Dandelion transaction monitor.");
let _ = thread::Builder::new()
.name("dandelion".to_string())
.spawn(move || {
loop {
let tx_pool = tx_pool.clone();
let stem_transactions = tx_pool.read().unwrap().stem_transactions.clone();
let time_stem_transactions = tx_pool.read().unwrap().time_stem_transactions.clone();
for tx_hash in stem_transactions.keys() {
let time_transaction = time_stem_transactions.get(tx_hash).unwrap();
let interval = now_utc().to_timespec().sec - time_transaction;
if interval >= config.dandelion_embargo {
let source = TxSource {
debug_name: "dandelion-monitor".to_string(),
identifier: "?.?.?.?".to_string(),
};
let stem_transaction = stem_transactions.get(tx_hash).unwrap();
let res = tx_pool.write().unwrap().add_to_memory_pool(
source,
stem_transaction.clone(),
false,
);
match res {
Ok(()) => {
info!(LOGGER, "Fluffing transaction after embargo timer expired.")
}
Err(e) => debug!(LOGGER, "error - {:?}", e),
};
// Remove from tx pool
tx_pool.write().unwrap().remove_from_stempool(tx_hash);
}
}
thread::sleep(Duration::from_secs(1));
if stop.load(Ordering::Relaxed) {
break;
}
// This is the patience timer, we loop every n secs.
let patience_secs = dandelion_config.patience_secs;
thread::sleep(Duration::from_secs(patience_secs));
let tx_pool = tx_pool.clone();
// Step 1: find all "ToStem" entries in stempool from last run.
// Aggregate them up to give a single (valid) aggregated tx and propagate it
// to the next Dandelion relay along the stem.
if process_stem_phase(tx_pool.clone()).is_err() {
error!(LOGGER, "dand_mon: Problem with stem phase.");
}
// Step 2: find all "ToFluff" entries in stempool from last run.
// Aggregate them up to give a single (valid) aggregated tx and (re)add it
// to our pool with stem=false (which will then broadcast it).
if process_fluff_phase(tx_pool.clone()).is_err() {
error!(LOGGER, "dand_mon: Problem with fluff phase.");
}
// Step 3: now find all "Fresh" entries in stempool since last run.
// Coin flip for each (90/10) and label them as either "ToStem" or "ToFluff".
// We will process these in the next run (waiting patience secs).
if process_fresh_entries(dandelion_config.clone(), tx_pool.clone()).is_err() {
error!(LOGGER, "dand_mon: Problem processing fresh pool entries.");
}
// Step 4: now find all expired entries based on embargo timer.
if process_expired_entries(dandelion_config.clone(), tx_pool.clone()).is_err() {
error!(LOGGER, "dand_mon: Problem processing fresh pool entries.");
}
}
});
}
fn process_stem_phase<T>(tx_pool: Arc<RwLock<TransactionPool<T>>>) -> Result<(), PoolError>
where
T: BlockChain + Send + Sync + 'static,
{
let mut tx_pool = tx_pool.write().unwrap();
let txpool_tx = tx_pool.txpool.aggregate_transaction()?;
let stem_txs = tx_pool.stempool.select_valid_transactions(
PoolEntryState::ToStem,
PoolEntryState::Stemmed,
txpool_tx,
)?;
if stem_txs.len() > 0 {
debug!(
LOGGER,
"dand_mon: Found {} txs for stemming.",
stem_txs.len()
);
let agg_tx = transaction::aggregate(stem_txs)?;
let res = tx_pool.adapter.stem_tx_accepted(&agg_tx);
if res.is_err() {
debug!(
LOGGER,
"dand_mon: Unable to propagate stem tx. No relay, fluffing instead."
);
let src = TxSource {
debug_name: "no_relay".to_string(),
identifier: "?.?.?.?".to_string(),
};
tx_pool.add_to_pool(src, agg_tx, false)?;
}
}
Ok(())
}
fn process_fluff_phase<T>(tx_pool: Arc<RwLock<TransactionPool<T>>>) -> Result<(), PoolError>
where
T: BlockChain + Send + Sync + 'static,
{
let mut tx_pool = tx_pool.write().unwrap();
let txpool_tx = tx_pool.txpool.aggregate_transaction()?;
let stem_txs = tx_pool.stempool.select_valid_transactions(
PoolEntryState::ToFluff,
PoolEntryState::Fluffed,
txpool_tx,
)?;
if stem_txs.len() > 0 {
debug!(
LOGGER,
"dand_mon: Found {} txs for fluffing.",
stem_txs.len()
);
let agg_tx = transaction::aggregate(stem_txs)?;
let src = TxSource {
debug_name: "fluff".to_string(),
identifier: "?.?.?.?".to_string(),
};
tx_pool.add_to_pool(src, agg_tx, false)?;
}
Ok(())
}
fn process_fresh_entries<T>(
dandelion_config: DandelionConfig,
tx_pool: Arc<RwLock<TransactionPool<T>>>,
) -> Result<(), PoolError>
where
T: BlockChain + Send + Sync + 'static,
{
let mut tx_pool = tx_pool.write().unwrap();
let mut rng = rand::thread_rng();
let fresh_entries = &mut tx_pool
.stempool
.entries
.iter_mut()
.filter(|x| x.state == PoolEntryState::Fresh)
.collect::<Vec<_>>();
if fresh_entries.len() > 0 {
debug!(
LOGGER,
"dand_mon: Found {} fresh entries in stempool.",
fresh_entries.len()
);
for x in &mut fresh_entries.iter_mut() {
let random = rng.gen_range(0, 101);
if random <= dandelion_config.stem_probability {
x.state = PoolEntryState::ToStem;
} else {
x.state = PoolEntryState::ToFluff;
}
}
}
Ok(())
}
fn process_expired_entries<T>(
dandelion_config: DandelionConfig,
tx_pool: Arc<RwLock<TransactionPool<T>>>,
) -> Result<(), PoolError>
where
T: BlockChain + Send + Sync + 'static,
{
let now = now_utc().to_timespec().sec;
let embargo_sec = dandelion_config.embargo_secs + rand::thread_rng().gen_range(0, 31);
let cutoff = now - embargo_sec as i64;
let mut expired_entries = vec![];
{
let tx_pool = tx_pool.read().unwrap();
for entry in tx_pool
.stempool
.entries
.iter()
.filter(|x| x.tx_at.sec < cutoff)
{
debug!(
LOGGER,
"dand_mon: Embargo timer expired for {:?}",
entry.tx.hash()
);
expired_entries.push(entry.clone());
}
}
if expired_entries.len() > 0 {
debug!(
LOGGER,
"dand_mon: Found {} expired txs.",
expired_entries.len()
);
{
let mut tx_pool = tx_pool.write().unwrap();
for entry in expired_entries {
let src = TxSource {
debug_name: "embargo_expired".to_string(),
identifier: "?.?.?.?".to_string(),
};
match tx_pool.add_to_pool(src, entry.tx, false) {
Ok(_) => debug!(
LOGGER,
"dand_mon: embargo expired, fluffed tx successfully."
),
Err(e) => debug!(LOGGER, "dand_mon: Failed to fluff expired tx - {:?}", e),
};
}
}
}
Ok(())
}

View file

@ -20,10 +20,10 @@ use std::io::Read;
use std::net::SocketAddr;
use std::net::ToSocketAddrs;
use std::str;
use std::sync::{mpsc, Arc};
use std::sync::atomic::{AtomicBool, Ordering};
use std::time::Duration;
use std::sync::{mpsc, Arc};
use std::thread;
use std::time::Duration;
use time::{self, now_utc};
use hyper;
@ -73,7 +73,7 @@ pub fn connect_and_monitor(
tx.clone(),
);
update_dandelion_relay(peers.clone(), p2p_server.config.clone());
update_dandelion_relay(peers.clone(), p2p_server.dandelion_config.clone());
prev = current_time;
}
@ -159,7 +159,7 @@ fn monitor_peers(
}
}
fn update_dandelion_relay(peers: Arc<p2p::Peers>, config: p2p::P2PConfig) {
fn update_dandelion_relay(peers: Arc<p2p::Peers>, dandelion_config: p2p::DandelionConfig) {
// Dandelion Relay Updater
let dandelion_relay = peers.get_dandelion_relay();
if dandelion_relay.is_empty() {
@ -168,7 +168,7 @@ fn update_dandelion_relay(peers: Arc<p2p::Peers>, config: p2p::P2PConfig) {
} else {
for last_added in dandelion_relay.keys() {
let dandelion_interval = now_utc().to_timespec().sec - last_added;
if dandelion_interval >= config.dandelion_relay_time() {
if dandelion_interval >= dandelion_config.relay_secs as i64 {
debug!(LOGGER, "monitor_peers: updating expired dandelion relay");
peers.update_dandelion_relay();
}

View file

@ -17,27 +17,27 @@
//! as a facade.
use std::net::SocketAddr;
use std::sync::{Arc, RwLock};
use std::sync::atomic::{AtomicBool, Ordering};
use std::sync::{Arc, RwLock};
use std::thread;
use std::time;
use common::adapters::*;
use api;
use chain;
use core::{consensus, genesis, global, pow};
use core::core::target::Difficulty;
use common::adapters::*;
use common::stats::*;
use common::types::*;
use core::core::hash::Hashed;
use core::core::target::Difficulty;
use core::{consensus, genesis, global, pow};
use grin::dandelion_monitor;
use mining::stratumserver;
use p2p;
use pool;
use grin::seed;
use grin::sync;
use common::types::*;
use common::stats::*;
use util::LOGGER;
use mining::stratumserver;
use mining::test_miner::Miner;
use p2p;
use pool;
use util::LOGGER;
/// Grin server holding internal structures.
pub struct Server {
@ -157,11 +157,11 @@ impl Server {
Err(_) => None,
};
let p2p_config = config.p2p_config.clone();
let p2p_server = Arc::new(p2p::Server::new(
config.db_root.clone(),
config.capabilities,
p2p_config,
config.p2p_config.clone(),
config.p2p_dandelion_config(),
net_adapter.clone(),
genesis.hash(),
stop.clone(),
@ -232,7 +232,7 @@ impl Server {
"Starting dandelion monitor: {}", &config.api_http_addr
);
dandelion_monitor::monitor_transactions(
config.pool_config.clone(),
config.p2p_dandelion_config(),
tx_pool.clone(),
stop.clone(),
);
@ -288,9 +288,9 @@ impl Server {
});
}
/// Start mining for blocks internally on a separate thread. Relies on internal miner,
/// and should only be used for automated testing. Burns reward if wallet_listener_url
/// is 'None'
/// Start mining for blocks internally on a separate thread. Relies on
/// internal miner, and should only be used for automated testing. Burns
/// reward if wallet_listener_url is 'None'
pub fn start_test_miner(&self, wallet_listener_url: Option<String>) {
let currently_syncing = self.currently_syncing.clone();
let config_wallet_url = match wallet_listener_url.clone() {

View file

@ -28,7 +28,6 @@ use common::adapters::PoolToChainAdapter;
use common::types::Error;
use core::consensus;
use core::core;
use core::core::Transaction;
use core::core::hash::Hashed;
use core::ser;
use core::ser::AsFixedBytes;
@ -163,8 +162,6 @@ fn build_block(
.unwrap()
.prepare_mineable_transactions(max_tx);
let txs: Vec<&Transaction> = txs.iter().collect();
// build the coinbase and the block itself
let fees = txs.iter().map(|tx| tx.fee()).sum();
let height = head.height + 1;

View file

@ -17,8 +17,8 @@ use std::fs;
use std::io;
use std::marker;
use core::core::pmmr::{self, family, Backend};
use core::core::hash::Hash;
use core::core::pmmr::{self, family, Backend};
use core::ser;
use core::ser::PMMRable;
use types::*;
@ -272,15 +272,6 @@ where
self.get_data_file_path()
}
/// Return last written buffer positions for the hash file and the data file
// pub fn last_file_positions(&self) -> PMMRFileMetadata {
// PMMRFileMetadata {
// block_height: 0,
// last_hash_file_pos: self.hash_file.last_buffer_pos() as u64,
// last_data_file_pos: self.data_file.last_buffer_pos() as u64,
// }
// }
/// Checks the length of the remove log to see if it should get compacted.
/// If so, the remove log is flushed into the pruned list, which itself gets
/// saved, and the hash and data files are rewritten, cutting the removed

View file

@ -16,15 +16,15 @@ use memmap;
use std::cmp;
use std::fs::{self, File, OpenOptions};
use std::io::Read;
use std::io::{self, BufRead, BufReader, BufWriter, ErrorKind, Write};
use std::os::unix::io::AsRawFd;
use std::io::Read;
use std::path::Path;
#[cfg(any(target_os = "linux"))]
use libc::{ftruncate64, off64_t};
#[cfg(not(any(target_os = "linux", target_os = "android")))]
use libc::{ftruncate as ftruncate64, off_t as off64_t};
#[cfg(any(target_os = "linux"))]
use libc::{ftruncate64, off64_t};
use core::ser;
@ -82,12 +82,25 @@ impl AppendOnlyFile {
/// Rewinds the data file back to a lower position. The new position needs
/// to be the one of the first byte the next time data is appended.
pub fn rewind(&mut self, pos: u64) {
if self.buffer_start_bak > 0 || self.buffer.len() > 0 {
panic!("Can't rewind on a dirty state.");
/// Supports two scenarios currently -
/// * rewind from a clean state (rewinding to handle a forked block)
/// * rewind within the buffer itself (raw_tx fails to validate)
/// Note: we do not currently support a rewind() that
/// crosses the buffer boundary.
pub fn rewind(&mut self, file_pos: u64) {
if self.buffer.is_empty() {
// rewinding from clean state, no buffer, not already rewound anything
self.buffer_start_bak = self.buffer_start;
self.buffer_start = file_pos as usize;
} else {
// rewinding (within) the buffer
if self.buffer_start as u64 > file_pos {
panic!("cannot rewind buffer beyond buffer_start");
} else {
let buffer_len = file_pos - self.buffer_start as u64;
self.buffer.truncate(buffer_len as usize);
}
}
self.buffer_start_bak = self.buffer_start;
self.buffer_start = pos as usize;
}
/// Syncs all writes (fsync), reallocating the memory map to make the newly
@ -263,16 +276,18 @@ impl RemoveLog {
/// In practice the index is a block height, so we rewind back to that block
/// keeping everything in the rm_log up to and including that block.
pub fn rewind(&mut self, idx: u32) -> io::Result<()> {
// simplifying assumption: we always remove older than what's in tmp
self.removed_tmp = vec![];
// backing it up before truncating
self.removed_bak = self.removed.clone();
// backing it up before truncating (unless we already have a backup)
if self.removed_bak.is_empty() {
self.removed_bak = self.removed.clone();
}
if idx == 0 {
self.removed = vec![];
self.removed_tmp = vec![];
} else {
// retain rm_log entries up to and including those at the provided index
self.removed.retain(|&(_, x)| x <= idx);
self.removed_tmp.retain(|&(_, x)| x <= idx);
}
Ok(())
}

View file

@ -16,15 +16,15 @@
//! the wallet storage and update them.
use failure::ResultExt;
use std::collections::HashMap;
use std::collections::hash_map::Entry;
use std::collections::HashMap;
use api;
use keychain::Identifier;
use libwallet::types::*;
use util;
use util::LOGGER;
use util::secp::pedersen;
use util::LOGGER;
pub fn refresh_outputs<T>(wallet: &mut T) -> Result<(), Error>
where

View file

@ -18,13 +18,13 @@ use std::cmp::min;
use std::collections::HashMap;
use std::fs::{self, File};
use std::io::{Read, Write};
use std::path::MAIN_SEPARATOR;
use std::path::Path;
use std::path::MAIN_SEPARATOR;
use serde_json;
use tokio_core::reactor;
use tokio_retry::Retry;
use tokio_retry::strategy::FibonacciBackoff;
use tokio_retry::Retry;
use failure::{Fail, ResultExt};

View file

@ -14,9 +14,9 @@
use std::sync::{Arc, RwLock};
use bodyparser;
use iron::Handler;
use iron::prelude::*;
use iron::status;
use iron::Handler;
use serde_json;
use core::ser;

View file

@ -13,9 +13,9 @@
// limitations under the License.
//! Aggsig helper functions used in transaction creation.. should be only
//! interface into the underlying secp library
use keychain::Keychain;
use keychain::blind::BlindingFactor;
use keychain::extkey::Identifier;
use keychain::Keychain;
use libtx::error::Error;
use util::kernel_sig_msg;
use util::secp::key::{PublicKey, SecretKey};

View file

@ -15,8 +15,8 @@
//! Rangeproof library functions
use blake2;
use keychain::Keychain;
use keychain::extkey::Identifier;
use keychain::Keychain;
use libtx::error::Error;
use util::logger::LOGGER;
use util::secp::key::SecretKey;

View file

@ -22,8 +22,8 @@ use keychain::{BlindSum, BlindingFactor, Keychain};
use libtx::error::Error;
use libtx::{aggsig, build, tx_fee};
use util::secp::Signature;
use util::secp::key::{PublicKey, SecretKey};
use util::secp::Signature;
use util::{secp, LOGGER};
/// Public data for each participant in the slate

View file

@ -16,7 +16,7 @@
use failure::ResultExt;
use keychain::Identifier;
use libtx::{build, tx_fee, slate::Slate};
use libtx::{build, slate::Slate, tx_fee};
use libwallet::types::*;
use libwallet::{keys, sigcontext};

View file

@ -357,8 +357,11 @@ pub enum ErrorKind {
#[fail(display = "Fee dispute: sender fee {}, recipient fee {}", sender_fee, recipient_fee)]
FeeDispute { sender_fee: u64, recipient_fee: u64 },
#[fail(display = "Fee exceeds amount: sender amount {}, recipient fee {}", sender_amount,
recipient_fee)]
#[fail(
display = "Fee exceeds amount: sender amount {}, recipient fee {}",
sender_amount,
recipient_fee
)]
FeeExceedsAmount {
sender_amount: u64,
recipient_fee: u64,

View file

@ -19,9 +19,9 @@
use std::sync::{Arc, RwLock};
use bodyparser;
use iron::Handler;
use iron::prelude::*;
use iron::status;
use iron::Handler;
use serde_json;
use api;

View file

@ -20,8 +20,8 @@ use keychain::Identifier;
use libtx::proof;
use libwallet::types::*;
use util;
use util::LOGGER;
use util::secp::pedersen;
use util::LOGGER;
pub fn get_chain_height(node_addr: &str) -> Result<u64, Error> {
let url = format!("{}/v1/chain", node_addr);

View file

@ -13,8 +13,8 @@
// limitations under the License.
//! Common functions to facilitate wallet, walletlib and transaction testing
use std::collections::HashMap;
use std::collections::hash_map::Entry;
use std::collections::HashMap;
extern crate grin_api as api;
extern crate grin_chain as chain;
@ -126,7 +126,12 @@ fn get_output_local(
pub fn add_block_with_reward(chain: &Chain, txs: Vec<&Transaction>, reward: (Output, TxKernel)) {
let prev = chain.head_header().unwrap();
let difficulty = consensus::next_difficulty(chain.difficulty_iter()).unwrap();
let mut b = core::core::Block::new(&prev, txs, difficulty.clone(), reward).unwrap();
let mut b = core::core::Block::new(
&prev,
txs.into_iter().cloned().collect(),
difficulty.clone(),
reward,
).unwrap();
b.header.timestamp = prev.timestamp + time::Duration::seconds(60);
chain.set_txhashset_roots(&mut b, false).unwrap();
pow::pow_size(

View file

@ -29,8 +29,8 @@ mod common;
use std::fs;
use std::sync::Arc;
use chain::Chain;
use chain::types::*;
use chain::Chain;
use core::global::ChainTypes;
use core::{global, pow};
use util::LOGGER;