openethereum/miner/src/pool/queue.rs

446 lines
14 KiB
Rust
Raw Normal View History

New Transaction Queue implementation (#8074) * Implementation of Verifier, Scoring and Ready. * Queue in progress. * TransactionPool. * Prepare for txpool release. * Miner refactor [WiP] * WiP reworking miner. * Make it compile. * Add some docs. * Split blockchain access to a separate file. * Work on miner API. * Fix ethcore tests. * Refactor miner interface for sealing/work packages. * Implement next nonce. * RPC compiles. * Implement couple of missing methdods for RPC. * Add transaction queue listeners. * Compiles! * Clean-up and parallelize. * Get rid of RefCell in header. * Revert "Get rid of RefCell in header." This reverts commit 0f2424c9b7319a786e1565ea2a8a6d801a21b4fb. * Override Sync requirement. * Fix status display. * Unify logging. * Extract some cheap checks. * Measurements and optimizations. * Fix scoring bug, heap size of bug and add cache * Disable tx queueing and parallel verification. * Make ethcore and ethcore-miner compile again. * Make RPC compile again. * Bunch of txpool tests. * Migrate transaction queue tests. * Nonce Cap * Nonce cap cache and tests. * Remove stale future transactions from the queue. * Optimize scoring and write some tests. * Simple penalization. * Clean up and support for different scoring algorithms. * Add CLI parameters for the new queue. * Remove banning queue. * Disable debug build. * Change per_sender limit to be 1% instead of 5% * Avoid cloning when propagating transactions. * Remove old todo. * Post-review fixes. * Fix miner options default. * Implement back ready transactions for light client. * Get rid of from_pending_block * Pass rejection reason. * Add more details to drop. * Rollback heap size of. * Avoid cloning hashes when propagating and include more details on rejection. * Fix tests. * Introduce nonces cache. * Remove uneccessary hashes allocation. * Lower the mem limit. * Re-enable parallel verification. * Add miner log. Don't check the type if not below min_gas_price. * Add more traces, fix disabling miner. * Fix creating pending blocks twice on AuRa authorities. * Fix tests. * re-use pending blocks in AuRa * Use reseal_min_period to prevent too frequent update_sealing. * Fix log to contain hash not sender. * Optimize local transactions. * Fix aura tests. * Update locks comments. * Get rid of unsafe Sync impl. * Review fixes. * Remove excessive matches. * Fix compilation errors. * Use new pool in private transactions. * Fix private-tx test. * Fix secret store tests. * Actually use gas_floor_target * Fix config tests. * Fix pool tests. * Address grumbles.
2018-04-13 17:34:27 +02:00
// Copyright 2015-2017 Parity Technologies (UK) Ltd.
// This file is part of Parity.
// Parity is free software: you can redistribute it and/or modify
// it under the terms of the GNU General Public License as published by
// the Free Software Foundation, either version 3 of the License, or
// (at your option) any later version.
// Parity is distributed in the hope that it will be useful,
// but WITHOUT ANY WARRANTY; without even the implied warranty of
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
// GNU General Public License for more details.
// You should have received a copy of the GNU General Public License
// along with Parity. If not, see <http://www.gnu.org/licenses/>.
//! Ethereum Transaction Queue
use std::{cmp, fmt};
use std::sync::Arc;
use std::sync::atomic::{self, AtomicUsize};
use std::collections::BTreeMap;
use ethereum_types::{H256, U256, Address};
use parking_lot::RwLock;
use rayon::prelude::*;
use transaction;
use txpool::{self, Verifier};
use pool::{self, scoring, verifier, client, ready, listener, PrioritizationStrategy};
use pool::local_transactions::LocalTransactionsList;
type Listener = (LocalTransactionsList, (listener::Notifier, listener::Logger));
type Pool = txpool::Pool<pool::VerifiedTransaction, scoring::NonceAndGasPrice, Listener>;
/// Max cache time in milliseconds for pending transactions.
///
/// Pending transactions are cached and will only be computed again
/// if last cache has been created earler than `TIMESTAMP_CACHE` ms ago.
/// This timeout applies only if there are local pending transactions
/// since it only affects transaction Condition.
const TIMESTAMP_CACHE: u64 = 1000;
/// Transaction queue status.
#[derive(Debug, Clone, PartialEq)]
pub struct Status {
/// Verifier options.
pub options: verifier::Options,
/// Current status of the transaction pool.
pub status: txpool::LightStatus,
/// Current limits of the transaction pool.
pub limits: txpool::Options,
}
impl fmt::Display for Status {
fn fmt(&self, fmt: &mut fmt::Formatter) -> fmt::Result {
writeln!(
fmt,
"Pool: {current}/{max} ({senders} senders; {mem}/{mem_max} kB) [minGasPrice: {gp} Mwei, maxGas: {max_gas}]",
current = self.status.transaction_count,
max = self.limits.max_count,
senders = self.status.senders,
mem = self.status.mem_usage / 1024,
mem_max = self.limits.max_mem_usage / 1024,
gp = self.options.minimal_gas_price / 1_000_000.into(),
max_gas = cmp::min(self.options.block_gas_limit, self.options.tx_gas_limit),
)
}
}
#[derive(Debug)]
struct CachedPending {
block_number: u64,
current_timestamp: u64,
nonce_cap: Option<U256>,
has_local_pending: bool,
pending: Option<Vec<Arc<pool::VerifiedTransaction>>>,
}
impl CachedPending {
/// Creates new `CachedPending` without cached set.
pub fn none() -> Self {
CachedPending {
block_number: 0,
current_timestamp: 0,
has_local_pending: false,
pending: None,
nonce_cap: None,
}
}
/// Remove cached pending set.
pub fn clear(&mut self) {
self.pending = None;
}
/// Returns cached pending set (if any) if it's valid.
pub fn pending(
&self,
block_number: u64,
current_timestamp: u64,
nonce_cap: Option<&U256>,
) -> Option<Vec<Arc<pool::VerifiedTransaction>>> {
// First check if we have anything in cache.
let pending = self.pending.as_ref()?;
if block_number != self.block_number {
return None;
}
// In case we don't have any local pending transactions
// there is no need to invalidate the cache because of timestamp.
// Timestamp only affects local `PendingTransactions` with `Condition::Timestamp`.
if self.has_local_pending && current_timestamp > self.current_timestamp + TIMESTAMP_CACHE {
return None;
}
// It's fine to return limited set even if `nonce_cap` is `None`.
// The worst thing that may happen is that some transactions won't get propagated in current round,
// but they are not really valid in current block anyway. We will propagate them in the next round.
// Also there is no way to have both `Some` with different numbers since it depends on the block number
// and a constant parameter in schedule (`nonce_cap_increment`)
if self.nonce_cap.is_none() && nonce_cap.is_some() {
return None;
}
Some(pending.clone())
}
}
/// Ethereum Transaction Queue
///
/// Responsible for:
/// - verifying incoming transactions
/// - maintaining a pool of verified transactions.
/// - returning an iterator for transactions that are ready to be included in block (pending)
#[derive(Debug)]
pub struct TransactionQueue {
insertion_id: Arc<AtomicUsize>,
pool: RwLock<Pool>,
options: RwLock<verifier::Options>,
cached_pending: RwLock<CachedPending>,
}
impl TransactionQueue {
/// Create new queue with given pool limits and initial verification options.
pub fn new(
limits: txpool::Options,
verification_options: verifier::Options,
strategy: PrioritizationStrategy,
) -> Self {
TransactionQueue {
insertion_id: Default::default(),
pool: RwLock::new(txpool::Pool::new(Default::default(), scoring::NonceAndGasPrice(strategy), limits)),
options: RwLock::new(verification_options),
cached_pending: RwLock::new(CachedPending::none()),
}
}
/// Update verification options
///
/// Some parameters of verification may vary in time (like block gas limit or minimal gas price).
pub fn set_verifier_options(&self, options: verifier::Options) {
*self.options.write() = options;
}
/// Import a set of transactions to the pool.
///
/// Given blockchain and state access (Client)
/// verifies and imports transactions to the pool.
pub fn import<C: client::Client>(
&self,
client: C,
transactions: Vec<verifier::Transaction>,
) -> Vec<Result<(), transaction::Error>> {
// Run verification
let _timer = ::trace_time::PerfTimer::new("queue::verifyAndImport");
let options = self.options.read().clone();
let verifier = verifier::Verifier::new(client, options, self.insertion_id.clone());
let results = transactions
.into_par_iter()
.map(|transaction| verifier.verify_transaction(transaction))
.map(|result| result.and_then(|verified| {
self.pool.write().import(verified)
.map(|_imported| ())
.map_err(convert_error)
}))
.collect::<Vec<_>>();
// Notify about imported transactions.
(self.pool.write().listener_mut().1).0.notify();
if results.iter().any(|r| r.is_ok()) {
self.cached_pending.write().clear();
}
results
}
/// Returns all transactions in the queue ordered by priority.
pub fn all_transactions(&self) -> Vec<Arc<pool::VerifiedTransaction>> {
let ready = |_tx: &pool::VerifiedTransaction| txpool::Readiness::Ready;
self.pool.read().pending(ready).collect()
}
/// Returns current pneding transactions.
///
/// NOTE: This may return a cached version of pending transaction set.
/// Re-computing the pending set is possible with `#collect_pending` method,
/// but be aware that it's a pretty expensive operation.
pub fn pending<C>(
&self,
client: C,
block_number: u64,
current_timestamp: u64,
nonce_cap: Option<U256>,
) -> Vec<Arc<pool::VerifiedTransaction>> where
C: client::NonceClient,
{
if let Some(pending) = self.cached_pending.read().pending(block_number, current_timestamp, nonce_cap.as_ref()) {
return pending;
}
// Double check after acquiring write lock
let mut cached_pending = self.cached_pending.write();
if let Some(pending) = cached_pending.pending(block_number, current_timestamp, nonce_cap.as_ref()) {
return pending;
}
let pending: Vec<_> = self.collect_pending(client, block_number, current_timestamp, nonce_cap, |i| i.collect());
*cached_pending = CachedPending {
block_number,
current_timestamp,
nonce_cap,
has_local_pending: self.has_local_pending_transactions(),
pending: Some(pending.clone()),
};
pending
}
/// Collect pending transactions.
///
/// NOTE This is re-computing the pending set and it might be expensive to do so.
/// Prefer using cached pending set using `#pending` method.
pub fn collect_pending<C, F, T>(
&self,
client: C,
block_number: u64,
current_timestamp: u64,
nonce_cap: Option<U256>,
collect: F,
) -> T where
C: client::NonceClient,
F: FnOnce(txpool::PendingIterator<
pool::VerifiedTransaction,
(ready::Condition, ready::State<C>),
scoring::NonceAndGasPrice,
Listener,
>) -> T,
{
let pending_readiness = ready::Condition::new(block_number, current_timestamp);
// don't mark any transactions as stale at this point.
let stale_id = None;
let state_readiness = ready::State::new(client, stale_id, nonce_cap);
let ready = (pending_readiness, state_readiness);
collect(self.pool.read().pending(ready))
}
/// Culls all stalled transactions from the pool.
pub fn cull<C: client::NonceClient>(
&self,
client: C,
) {
// We don't care about future transactions, so nonce_cap is not important.
let nonce_cap = None;
// We want to clear stale transactions from the queue as well.
// (Transactions that are occuping the queue for a long time without being included)
let stale_id = {
let current_id = self.insertion_id.load(atomic::Ordering::Relaxed) as u64;
// wait at least for half of the queue to be replaced
let gap = self.pool.read().options().max_count / 2;
// but never less than 100 transactions
let gap = cmp::max(100, gap) as u64;
current_id.checked_sub(gap)
};
let state_readiness = ready::State::new(client, stale_id, nonce_cap);
let removed = self.pool.write().cull(None, state_readiness);
debug!(target: "txqueue", "Removed {} stalled transactions. {}", removed, self.status());
}
/// Returns next valid nonce for given sender
/// or `None` if there are no pending transactions from that sender.
pub fn next_nonce<C: client::NonceClient>(
&self,
client: C,
address: &Address,
) -> Option<U256> {
// Do not take nonce_cap into account when determining next nonce.
let nonce_cap = None;
// Also we ignore stale transactions in the queue.
let stale_id = None;
let state_readiness = ready::State::new(client, stale_id, nonce_cap);
self.pool.read().pending_from_sender(state_readiness, address)
.last()
.map(|tx| tx.signed().nonce + 1.into())
}
/// Retrieve a transaction from the pool.
///
/// Given transaction hash looks up that transaction in the pool
/// and returns a shared pointer to it or `None` if it's not present.
pub fn find(
&self,
hash: &H256,
) -> Option<Arc<pool::VerifiedTransaction>> {
self.pool.read().find(hash)
}
/// Remove a set of transactions from the pool.
///
/// Given an iterator of transaction hashes
/// removes them from the pool.
/// That method should be used if invalid transactions are detected
/// or you want to cancel a transaction.
pub fn remove<'a, T: IntoIterator<Item = &'a H256>>(
&self,
hashes: T,
is_invalid: bool,
) -> Vec<Option<Arc<pool::VerifiedTransaction>>> {
let results = {
let mut pool = self.pool.write();
hashes
.into_iter()
.map(|hash| pool.remove(hash, is_invalid))
.collect::<Vec<_>>()
};
if results.iter().any(Option::is_some) {
self.cached_pending.write().clear();
}
results
}
/// Clear the entire pool.
pub fn clear(&self) {
self.pool.write().clear();
}
/// Penalize given senders.
pub fn penalize<'a, T: IntoIterator<Item = &'a Address>>(&self, senders: T) {
let mut pool = self.pool.write();
for sender in senders {
pool.update_scores(sender, ());
}
}
/// Returns gas price of currently the worst transaction in the pool.
pub fn current_worst_gas_price(&self) -> U256 {
match self.pool.read().worst_transaction() {
Some(tx) => tx.signed().gas_price,
None => self.options.read().minimal_gas_price,
}
}
/// Returns a status of the queue.
pub fn status(&self) -> Status {
let pool = self.pool.read();
let status = pool.light_status();
let limits = pool.options();
let options = self.options.read().clone();
Status {
options,
status,
limits,
}
}
/// Check if there are any local transactions in the pool.
///
/// Returns `true` if there are any transactions in the pool
/// that has been marked as local.
///
/// Local transactions are the ones from accounts managed by this node
/// and transactions submitted via local RPC (`eth_sendRawTransaction`)
pub fn has_local_pending_transactions(&self) -> bool {
self.pool.read().listener().0.has_pending()
}
/// Returns status of recently seen local transactions.
pub fn local_transactions(&self) -> BTreeMap<H256, pool::local_transactions::Status> {
self.pool.read().listener().0.all_transactions().iter().map(|(a, b)| (*a, b.clone())).collect()
}
/// Add a callback to be notified about all transactions entering the pool.
pub fn add_listener(&self, f: Box<Fn(&[H256]) + Send + Sync>) {
let mut pool = self.pool.write();
(pool.listener_mut().1).0.add(f);
}
}
fn convert_error(err: txpool::Error) -> transaction::Error {
use self::txpool::ErrorKind;
match *err.kind() {
ErrorKind::AlreadyImported(..) => transaction::Error::AlreadyImported,
ErrorKind::TooCheapToEnter(..) => transaction::Error::LimitReached,
ErrorKind::TooCheapToReplace(..) => transaction::Error::TooCheapToReplace,
ref e => {
warn!(target: "txqueue", "Unknown import error: {:?}", e);
transaction::Error::NotAllowed
},
}
}
#[cfg(test)]
mod tests {
use super::*;
use pool::tests::client::TestClient;
#[test]
fn should_get_pending_transactions() {
let queue = TransactionQueue::new(txpool::Options::default(), verifier::Options::default(), PrioritizationStrategy::GasPriceOnly);
let pending: Vec<_> = queue.pending(TestClient::default(), 0, 0, None);
for tx in pending {
assert!(tx.signed().nonce > 0.into());
}
}
}