finish incoming tx handler

This commit is contained in:
Boog900 2024-10-13 01:22:20 +01:00
parent d2c7e49e80
commit b6d94cf780
No known key found for this signature in database
GPG key ID: 42AB1287CB0041C2
14 changed files with 328 additions and 239 deletions

View file

@ -20,7 +20,7 @@ cuprate-levin = { path = "../../net/levin" }
cuprate-wire = { path = "../../net/wire" }
cuprate-p2p = { path = "../../p2p/p2p" }
cuprate-p2p-core = { path = "../../p2p/p2p-core" }
cuprate-dandelion-tower = { path = "../../p2p/dandelion-tower" }
cuprate-dandelion-tower = { path = "../../p2p/dandelion-tower", features = ["txpool"] }
cuprate-async-buffer = { path = "../../p2p/async-buffer" }
cuprate-address-book = { path = "../../p2p/address-book" }
cuprate-blockchain = { path = "../../storage/blockchain", features = ["service"] }

View file

@ -4,5 +4,4 @@
mod dandelion;
mod incoming_tx;
mod manager;
mod txs_being_handled;

View file

@ -1,6 +1,8 @@
use std::time::Duration;
use bytes::Bytes;
use cuprate_dandelion_tower::pool::DandelionPoolService;
use cuprate_dandelion_tower::{DandelionConfig, DandelionRouter};
use cuprate_dandelion_tower::{DandelionConfig, DandelionRouter, Graph};
use cuprate_p2p::NetworkInterface;
use cuprate_p2p_core::ClearNet;
use cuprate_txpool::service::{TxpoolReadHandle, TxpoolWriteHandle};
@ -11,10 +13,17 @@ mod stem_service;
mod tx_store;
#[derive(Clone)]
struct DandelionTx(Bytes);
pub struct DandelionTx(Bytes);
type TxId = [u8; 32];
const DANDELION_CONFIG: DandelionConfig = DandelionConfig {
time_between_hop: Duration::from_millis(175),
epoch_duration: Duration::from_secs(10 * 60),
fluff_probability: 0.12,
graph: Graph::FourRegular,
};
type ConcreteDandelionRouter = DandelionRouter<
stem_service::OutboundPeerStream,
diffuse_service::DiffuseService,
@ -35,12 +44,7 @@ pub fn start_dandelion_pool_manager(
txpool_read_handle,
txpool_write_handle,
},
DandelionConfig {
time_between_hop: Default::default(),
epoch_duration: Default::default(),
fluff_probability: 0.0,
graph: Default::default(),
},
DANDELION_CONFIG,
)
}
@ -49,14 +53,7 @@ pub fn dandelion_router(clear_net: NetworkInterface<ClearNet>) -> ConcreteDandel
diffuse_service::DiffuseService {
clear_net_broadcast_service: clear_net.broadcast_svc(),
},
stem_service::OutboundPeerStream {
clear_net: clear_net.clone(),
},
DandelionConfig {
time_between_hop: Default::default(),
epoch_duration: Default::default(),
fluff_probability: 0.0,
graph: Default::default(),
},
stem_service::OutboundPeerStream { clear_net },
DANDELION_CONFIG,
)
}

View file

@ -1,19 +1,26 @@
use std::task::{Context, Poll};
use std::{
future::{ready, Ready},
task::{Context, Poll},
};
use futures::FutureExt;
use tower::Service;
use crate::txpool::dandelion::DandelionTx;
use cuprate_dandelion_tower::traits::DiffuseRequest;
use cuprate_p2p::{BroadcastRequest, BroadcastSvc, NetworkInterface};
use cuprate_p2p_core::ClearNet;
use super::DandelionTx;
/// The dandelion diffusion service.
pub struct DiffuseService {
pub clear_net_broadcast_service: BroadcastSvc<ClearNet>,
}
impl Service<DiffuseRequest<DandelionTx>> for DiffuseService {
type Response = BroadcastSvc::Response;
type Response = ();
type Error = tower::BoxError;
type Future = BroadcastSvc::Future;
type Future = Ready<Result<Self::Response, Self::Error>>;
fn poll_ready(&mut self, cx: &mut Context<'_>) -> Poll<Result<(), Self::Error>> {
self.clear_net_broadcast_service
@ -22,11 +29,17 @@ impl Service<DiffuseRequest<DandelionTx>> for DiffuseService {
}
fn call(&mut self, req: DiffuseRequest<DandelionTx>) -> Self::Future {
// TODO: Call `into_inner` when 1.82.0 stabilizes
self.clear_net_broadcast_service
.call(BroadcastRequest::Transaction {
tx_bytes: req.0 .0,
direction: None,
received_from: None,
})
.now_or_never()
.unwrap()
.expect("Broadcast service is Infallible");
ready(Ok(()))
}
}

View file

@ -1,17 +1,23 @@
use super::DandelionTx;
use bytes::Bytes;
use cuprate_dandelion_tower::traits::StemRequest;
use cuprate_dandelion_tower::OutboundPeer;
use std::{
pin::Pin,
task::{Context, Poll},
};
use cuprate_dandelion_tower::{traits::StemRequest, OutboundPeer};
use cuprate_p2p::NetworkInterface;
use cuprate_p2p_core::client::Client;
use cuprate_p2p_core::{ClearNet, NetworkZone, PeerRequest, ProtocolRequest};
use cuprate_wire::protocol::NewTransactions;
use cuprate_wire::NetworkAddress;
use cuprate_p2p_core::{
client::{Client, InternalPeerID},
ClearNet, NetworkZone, PeerRequest, ProtocolRequest,
};
use cuprate_wire::{protocol::NewTransactions, NetworkAddress};
use bytes::Bytes;
use futures::Stream;
use std::pin::Pin;
use std::task::{Context, Poll};
use tower::Service;
use super::DandelionTx;
/// The dandelion outbound peer stream.
pub struct OutboundPeerStream {
pub clear_net: NetworkInterface<ClearNet>,
}
@ -20,22 +26,29 @@ impl Stream for OutboundPeerStream {
type Item = Result<OutboundPeer<NetworkAddress, StemPeerService<ClearNet>>, tower::BoxError>;
fn poll_next(self: Pin<&mut Self>, _: &mut Context<'_>) -> Poll<Option<Self::Item>> {
// TODO: make the outbound peer choice random.
Poll::Ready(Some(Ok(self
.clear_net
.client_pool()
.outbound_client()
.map_or(OutboundPeer::Exhausted, |client| {
OutboundPeer::Peer(client.info.id.into(), StemPeerService(client))
let addr = match client.info.id {
InternalPeerID::KnownAddr(addr) => addr,
InternalPeerID::Unknown(_) => panic!("Outbound peer had an unknown address"),
};
OutboundPeer::Peer(addr.into(), StemPeerService(client))
}))))
}
}
pub struct StemPeerService<N>(Client<N>);
/// The stem service, used to send stem txs.
pub struct StemPeerService<N: NetworkZone>(Client<N>);
impl<N: NetworkZone> Service<StemRequest<DandelionTx>> for StemPeerService<N> {
type Response = ();
type Response = <Client<N> as Service<PeerRequest>>::Response;
type Error = tower::BoxError;
type Future = Client::Future;
type Future = <Client<N> as Service<PeerRequest>>::Future;
fn poll_ready(&mut self, cx: &mut Context<'_>) -> Poll<Result<(), Self::Error>> {
self.0.poll_ready(cx)

View file

@ -1,15 +1,27 @@
use crate::txpool::dandelion::{DandelionTx, TxId};
use bytes::Bytes;
use cuprate_dandelion_tower::traits::{TxStoreRequest, TxStoreResponse};
use cuprate_database::RuntimeError;
use cuprate_txpool::service::interface::{TxpoolReadRequest, TxpoolReadResponse};
use cuprate_txpool::service::{TxpoolReadHandle, TxpoolWriteHandle};
use futures::future::BoxFuture;
use futures::{FutureExt, StreamExt, TryFutureExt};
use std::task::{Context, Poll};
use tower::util::Oneshot;
use tower::{Service, ServiceExt};
use std::{
f32::consts::E,
task::{Context, Poll},
};
use bytes::Bytes;
use futures::{future::BoxFuture, FutureExt, StreamExt, TryFutureExt};
use tower::{util::Oneshot, Service, ServiceExt};
use cuprate_dandelion_tower::{
traits::{TxStoreRequest, TxStoreResponse},
State,
};
use cuprate_database::RuntimeError;
use cuprate_txpool::service::{
interface::{TxpoolReadRequest, TxpoolReadResponse, TxpoolWriteRequest, TxpoolWriteResponse},
TxpoolReadHandle, TxpoolWriteHandle,
};
use super::{DandelionTx, TxId};
/// The dandelion tx-store service.
///
/// This is just mapping the interface [`cuprate_dandelion_tower`] wants to what [`cuprate_txpool`] provides.
pub struct TxStoreService {
pub txpool_read_handle: TxpoolReadHandle,
pub txpool_write_handle: TxpoolWriteHandle,
@ -31,17 +43,34 @@ impl Service<TxStoreRequest<TxId>> for TxStoreService {
.clone()
.oneshot(TxpoolReadRequest::TxBlob(tx_id))
.map(|res| match res {
Ok(TxpoolReadResponse::TxBlob(blob)) => Ok(TxStoreResponse::Transaction(Some(
(DandelionTx(Bytes::from(blob)), todo!()),
))),
Ok(TxpoolReadResponse::TxBlob {
tx_blob,
state_stem,
}) => {
let state = if state_stem {
State::Stem
} else {
State::Fluff
};
Ok(TxStoreResponse::Transaction(Some((
DandelionTx(Bytes::from(tx_blob)),
state,
))))
}
Err(RuntimeError::KeyNotFound) => Ok(TxStoreResponse::Transaction(None)),
Err(e) => Err(e.into()),
Ok(_) => unreachable!(),
})
.boxed(),
TxStoreRequest::Promote(tx_id) => {
todo!()
}
TxStoreRequest::Promote(tx_id) => self
.txpool_write_handle
.oneshot(TxpoolWriteRequest::Promote(tx_id))
.map(|res| match res {
Ok(_) | Err(RuntimeError::KeyNotFound) => TxStoreResponse::Ok,
Err(e) => Err(e.into()),
})
.boxed(),
}
}
}

View file

@ -1,58 +1,76 @@
use std::collections::HashSet;
use std::future::ready;
use std::sync::Arc;
use std::task::{Context, Poll};
use std::{
collections::HashSet,
future::ready,
sync::Arc,
task::{Context, Poll},
};
use crate::blockchain::ConcreteTxVerifierService;
use crate::txpool::txs_being_handled::{tx_blob_hash, TxBeingHandledLocally, TxsBeingHandled};
use bytes::Bytes;
use cuprate_consensus::transactions::new_tx_verification_data;
use cuprate_consensus::{
BlockChainContextRequest, BlockChainContextResponse, BlockChainContextService,
ExtendedConsensusError, TxVerifierService, VerifyTxRequest, VerifyTxResponse,
};
use cuprate_dandelion_tower::pool::{DandelionPoolService, IncomingTx, IncomingTxBuilder};
use cuprate_dandelion_tower::TxState;
use cuprate_helper::asynch::rayon_spawn_async;
use cuprate_txpool::service::interface::{
TxpoolReadRequest, TxpoolWriteRequest, TxpoolWriteResponse,
};
use cuprate_txpool::service::{TxpoolReadHandle, TxpoolWriteHandle};
use cuprate_wire::NetworkAddress;
use dashmap::DashSet;
use futures::future::BoxFuture;
use futures::FutureExt;
use futures::{future::BoxFuture, FutureExt};
use monero_serai::transaction::Transaction;
use sha3::{Digest, Sha3_256};
use tower::{Service, ServiceExt};
use cuprate_consensus::{
transactions::new_tx_verification_data, BlockChainContextRequest, BlockChainContextResponse,
BlockChainContextService, ExtendedConsensusError, TxVerifierService, VerifyTxRequest,
VerifyTxResponse,
};
use cuprate_dandelion_tower::{
pool::{DandelionPoolService, IncomingTx, IncomingTxBuilder},
TxState,
};
use cuprate_helper::asynch::rayon_spawn_async;
use cuprate_txpool::service::{
interface::{TxpoolReadRequest, TxpoolReadResponse, TxpoolWriteRequest, TxpoolWriteResponse},
TxpoolReadHandle, TxpoolWriteHandle,
};
use cuprate_types::TransactionVerificationData;
use cuprate_wire::NetworkAddress;
use crate::{
blockchain::ConcreteTxVerifierService,
constants::PANIC_CRITICAL_SERVICE_ERROR,
signals::REORG_LOCK,
txpool::txs_being_handled::{tx_blob_hash, TxBeingHandledLocally, TxsBeingHandled},
};
/// An error that can happen handling an incoming tx.
pub enum IncomingTxError {
Parse(std::io::Error),
Consensus(ExtendedConsensusError),
DuplicateTransaction,
}
pub enum IncomingTxs {
Bytes {
txs: Vec<Bytes>,
state: TxState<NetworkAddress>,
},
/// Incoming transactions.
pub struct IncomingTxs {
pub txs: Vec<Bytes>,
pub state: TxState<NetworkAddress>,
}
/// The transaction type used for dandelion++.
#[derive(Clone)]
struct DandelionTx(Bytes);
/// A transaction ID/hash.
type TxId = [u8; 32];
/// The service than handles incoming transaction pool transactions.
///
/// This service handles everything including verifying the tx, adding it to the pool and routing it to other nodes.
pub struct IncomingTxHandler {
txs_being_added: Arc<TxsBeingHandled>,
/// A store of txs currently being handled in incoming tx requests.
txs_being_handled: TxsBeingHandled,
/// The blockchain context cache.
blockchain_context_cache: BlockChainContextService,
/// The dandelion txpool manager.
dandelion_pool_manager: DandelionPoolService<DandelionTx, TxId, NetworkAddress>,
/// The transaction verifier service.
tx_verifier_service: ConcreteTxVerifierService,
/// The txpool write handle.
txpool_write_handle: TxpoolWriteHandle,
/// The txpool read handle.
txpool_read_handle: TxpoolReadHandle,
}
@ -66,70 +84,18 @@ impl Service<IncomingTxs> for IncomingTxHandler {
}
fn call(&mut self, req: IncomingTxs) -> Self::Future {
let IncomingTxs::Bytes { mut txs, state } = req;
let IncomingTxs::Bytes { txs, state } = req;
let mut local_tracker = self.txs_being_added.local_tracker();
txs.retain(|bytes| local_tracker.try_add_tx(bytes.as_ref()));
if txs.is_empty() {
return ready(Ok(())).boxed();
}
let mut blockchain_context_cache = self.blockchain_context_cache.clone();
let mut tx_verifier_service = self.tx_verifier_service.clone();
let mut txpool_write_handle = self.txpool_write_handle.clone();
async move {
let txs = rayon_spawn_async(move || {
txs.into_iter()
.map(|bytes| {
let tx = Transaction::read(&mut bytes.as_ref())
.map_err(IncomingTxError::Parse)?;
let tx = new_tx_verification_data(tx)
.map_err(|e| IncomingTxError::Consensus(e.into()))?;
Ok(Arc::new(tx))
})
.collect::<Result<Vec<_>, IncomingTxError>>()
})
.await?;
let BlockChainContextResponse::Context(context) = blockchain_context_cache
.ready()
.await?
.call(BlockChainContextRequest::GetContext)
.await?
else {
unreachable!()
};
let context = context.unchecked_blockchain_context();
tx_verifier_service
.ready()
.await?
.call(VerifyTxRequest::Prepped {
txs: txs.clone(),
current_chain_height: context.chain_height,
top_hash: context.top_hash,
time_for_time_lock: context.current_adjusted_timestamp_for_time_lock(),
hf: context.current_hf,
})
.await?;
txpool_write_handle
.ready()
.await?
.call(TxpoolWriteRequest::AddTransaction {
tx,
state_stem: state.state_stem(),
})
.await;
todo!()
}
handle_incoming_txs(
txs,
state,
self.txs_being_handled.clone(),
self.blockchain_context_cache.clone(),
self.tx_verifier_service.clone(),
self.txpool_write_handle.clone(),
self.txpool_read_handle.clone(),
self.dandelion_pool_manager.clone(),
)
.boxed()
}
}
@ -137,38 +103,106 @@ impl Service<IncomingTxs> for IncomingTxHandler {
async fn handle_incoming_txs(
txs: Vec<Bytes>,
state: TxState<NetworkAddress>,
tx_being_handled_locally: TxBeingHandledLocally,
txs_being_handled: TxsBeingHandled,
mut blockchain_context_cache: BlockChainContextService,
mut tx_verifier_service: ConcreteTxVerifierService,
mut txpool_write_handle: TxpoolWriteHandle,
mut txpool_read_handle: TxpoolReadHandle,
mut dandelion_pool_manager: DandelionPoolService<DandelionTx, TxId, NetworkAddress>,
) -> Result<(), IncomingTxError> {
let mut tx_blob_hashes = HashSet::new();
let reorg_guard = REORG_LOCK.read().await;
let txs = txs
.into_iter()
.map(|tx_blob| {
let tx_blob_hash = tx_blob_hash(tx_blob.as_ref());
if !tx_blob_hashes.insert(tx_blob_hash) {
return Err(IncomingTxError::DuplicateTransaction);
}
let (txs, txs_being_handled_guard) =
prepare_incoming_txs(txs, txs_being_handled, &mut txpool_read_handle).await?;
Ok((tx_blob_hash, tx_blob))
})
.collect::<Result<Vec<_>, _>>()?;
let TxpoolReadRequest::FilterKnownTxBlobHashes(tx_blob_hashes) = txpool_read_handle
let BlockChainContextResponse::Context(context) = blockchain_context_cache
.ready()
.await?
.call(TxpoolReadRequest::FilterKnownTxBlobHashes(tx_blob_hashes))
.await?
.await
.expect(PANIC_CRITICAL_SERVICE_ERROR)
.call(BlockChainContextRequest::GetContext)
.await
.expect(PANIC_CRITICAL_SERVICE_ERROR)
else {
unreachable!()
};
let txs = rayon_spawn_async(move || {
txs.into_iter()
let context = context.unchecked_blockchain_context();
tx_verifier_service
.ready()
.await
.expect(PANIC_CRITICAL_SERVICE_ERROR)
.call(VerifyTxRequest::Prepped {
txs: txs.clone(),
current_chain_height: context.chain_height,
top_hash: context.top_hash,
time_for_time_lock: context.current_adjusted_timestamp_for_time_lock(),
hf: context.current_hf,
})
.await
.map_err(IncomingTxError::Consensus)?;
for tx in txs {
handle_valid_tx(
tx,
state.clone(),
&mut txpool_write_handle,
&mut dandelion_pool_manager,
)
.await
}
Ok(())
}
/// Prepares the incoming transactions for verification.
///
/// This will filter out all transactions already in the pool or txs already being handled in another request.
async fn prepare_incoming_txs(
tx_blobs: Vec<Bytes>,
txs_being_handled: TxsBeingHandled,
txpool_read_handle: &mut TxpoolReadHandle,
) -> Result<(Vec<Arc<TransactionVerificationData>>, TxBeingHandledLocally), IncomingTxError> {
let mut tx_blob_hashes = HashSet::new();
let mut txs_being_handled_loacally = txs_being_handled.local_tracker();
// Compute the blob hash for each tx and filter out the txs currently being handled by another incoming tx batch.
let txs = tx_blobs
.into_iter()
.filter_map(|tx_blob| {
let tx_blob_hash = tx_blob_hash(tx_blob.as_ref());
// If a duplicate is in here the incoming tx batch contained the same tx twice.
if !tx_blob_hashes.insert(tx_blob_hash) {
return Some(Err(IncomingTxError::DuplicateTransaction));
}
// If a duplicate is here it is being handled in another batch.
if !txs_being_handled_loacally.try_add_tx(tx_blob_hash) {
return None;
}
Some(Ok((tx_blob_hash, tx_blob)))
})
.collect::<Result<Vec<_>, _>>()?;
// Filter the txs already in the txpool out.
// This will leave the txs already in the pool in [`TxBeingHandledLocally`] but that shouldn't be an issue.
let TxpoolReadResponse::FilterKnownTxBlobHashes(tx_blob_hashes) = txpool_read_handle
.ready()
.await
.expect(PANIC_CRITICAL_SERVICE_ERROR)
.call(TxpoolReadRequest::FilterKnownTxBlobHashes(tx_blob_hashes))
.await
.expect(PANIC_CRITICAL_SERVICE_ERROR)
else {
unreachable!()
};
// Now prepare the txs for verification.
rayon_spawn_async(move || {
let txs = txs
.into_iter()
.filter_map(|(tx_blob_hash, tx_blob)| {
if tx_blob_hashes.contains(&tx_blob_hash) {
Some(tx_blob)
@ -184,66 +218,55 @@ async fn handle_incoming_txs(
Ok(Arc::new(tx))
})
.collect::<Result<Vec<_>, IncomingTxError>>()
})
.await?;
.collect::<Result<Vec<_>, IncomingTxError>>()?;
let BlockChainContextResponse::Context(context) = blockchain_context_cache
Ok((txs, txs_being_handled_loacally))
})
.await
}
async fn handle_valid_tx(
tx: Arc<TransactionVerificationData>,
state: TxState<NetworkAddress>,
txpool_write_handle: &mut TxpoolWriteHandle,
dandelion_pool_manager: &mut DandelionPoolService<DandelionTx, TxId, NetworkAddress>,
) {
let incoming_tx =
IncomingTxBuilder::new(DandelionTx(Bytes::copy_from_slice(&tx.tx_blob)), tx.tx_hash);
let TxpoolWriteResponse::AddTransaction(double_spend) = txpool_write_handle
.ready()
.await?
.call(BlockChainContextRequest::GetContext)
.await?
.await
.expect(PANIC_CRITICAL_SERVICE_ERROR)
.call(TxpoolWriteRequest::AddTransaction {
tx,
state_stem: state.state_stem(),
})
.await
.expect("TODO")
else {
unreachable!()
};
let context = context.unchecked_blockchain_context();
// TODO: track double spends to quickly ignore them from their blob hash.
if let Some(tx_hash) = double_spend {
return;
};
tx_verifier_service
// TODO: check blockchain for double spends to prevent a race condition.
// TODO: fill this in properly.
let incoming_tx = incoming_tx
.with_routing_state(state)
.with_state_in_db(None)
.build()
.unwrap();
dandelion_pool_manager
.ready()
.await?
.call(VerifyTxRequest::Prepped {
txs: txs.clone(),
current_chain_height: context.chain_height,
top_hash: context.top_hash,
time_for_time_lock: context.current_adjusted_timestamp_for_time_lock(),
hf: context.current_hf,
})
.await?;
for tx in txs {
let incoming_tx = IncomingTxBuilder::new(Bytes::copy_from_slice(&tx.tx_blob), tx.tx_hash);
let TxpoolWriteResponse::AddTransaction(double_spend) = txpool_write_handle
.ready()
.await?
.call(TxpoolWriteRequest::AddTransaction {
tx,
state_stem: state.state_stem(),
})
.await?
else {
unreachable!()
};
// TODO: track double spends to quickly ignore them from their blob hash.
if let Some(tx_hash) = double_spend {
continue;
};
// TODO: check blockchain for double spends to prevent a race condition.
// TODO: fill this in properly.
let incoming_tx = incoming_tx
.with_routing_state(state.clone())
.with_state_in_db(None)
.build()
.unwrap();
dandelion_pool_manager
.ready()
.await?
.call(incoming_tx)
.await?;
}
.await
.expect(PANIC_CRITICAL_SERVICE_ERROR)
.call(incoming_tx)
.await
.expect(PANIC_CRITICAL_SERVICE_ERROR);
}

View file

@ -1 +0,0 @@

View file

@ -7,6 +7,7 @@ pub fn tx_blob_hash(tx_bytes: &[u8]) -> [u8; 32] {
hasher.update(tx_bytes);
hasher.finalize().into()
}
#[derive(Clone)]
pub struct TxsBeingHandled(Arc<DashSet<[u8; 32]>>);
@ -25,11 +26,7 @@ pub struct TxBeingHandledLocally {
}
impl TxBeingHandledLocally {
pub fn try_add_tx(&mut self, tx_bytes: &[u8]) -> bool {
let mut hasher = Sha3_256::new();
hasher.update(tx_bytes);
let tx_blob_hash = hasher.finalize().into();
pub fn try_add_tx(&mut self, tx_blob_hash: [u8; 32]) -> bool {
if !self.txs_being_handled.0.insert(tx_blob_hash) {
return false;
}

View file

@ -74,7 +74,7 @@ pub enum TxState<Id> {
}
impl<Id> TxState<Id> {
pub const fn state_stem(&self) -> bool {
pub fn state_stem(&self) -> bool {
matches!(self, Self::Local | Self::Stem { .. })
}
}

View file

@ -21,7 +21,7 @@ const WRITER_THREAD_NAME: &str = concat!(module_path!(), "::DatabaseWriter");
/// Calling [`tower::Service::call`] with a [`DatabaseWriteHandle`]
/// will return an `async`hronous channel that can be `.await`ed upon
/// to receive the corresponding response.
#[derive(Debug, Clone)]
#[derive(Debug)]
pub struct DatabaseWriteHandle<Req, Res> {
/// Sender channel to the database write thread-pool.
///
@ -30,6 +30,14 @@ pub struct DatabaseWriteHandle<Req, Res> {
crossbeam::channel::Sender<(Req, oneshot::Sender<Result<Res, RuntimeError>>)>,
}
impl<Req, Res> Clone for DatabaseWriteHandle<Req, Res> {
fn clone(&self) -> Self {
Self {
sender: self.sender.clone(),
}
}
}
impl<Req, Res> DatabaseWriteHandle<Req, Res>
where
Req: Send + 'static,

View file

@ -1,13 +1,11 @@
//! Tx-pool [`service`](super) interface.
//!
//! This module contains `cuprate_txpool`'s [`tower::Service`] request and response enums.
use std::collections::HashSet;
use std::sync::Arc;
use std::{collections::HashSet, sync::Arc};
use cuprate_types::TransactionVerificationData;
use crate::types::TransactionHash;
use crate::types::{TransactionBlobHash, TransactionHash};
//---------------------------------------------------------------------------------------------------- TxpoolReadRequest
/// The transaction pool [`tower::Service`] read request type.
@ -17,8 +15,10 @@ pub enum TxpoolReadRequest {
TxBlob(TransactionHash),
/// A request for the [`TransactionVerificationData`] of a transaction in the tx pool.
TxVerificationData(TransactionHash),
FilterKnownTxBlobHashes(HashSet<TransactionHash>),
/// A request to filter (remove) all **known** transactions from the set.
///
/// The hash is **not** the transaction hash, it is the hash of the serialized tx-blob.
FilterKnownTxBlobHashes(HashSet<TransactionBlobHash>),
}
//---------------------------------------------------------------------------------------------------- TxpoolReadResponse
@ -26,12 +26,14 @@ pub enum TxpoolReadRequest {
#[expect(clippy::large_enum_variant)]
pub enum TxpoolReadResponse {
/// A response containing the raw bytes of a transaction.
// TODO: use bytes::Bytes.
TxBlob(Vec<u8>),
TxBlob {
tx_blob: Vec<u8>,
state_stem: bool,
},
/// A response of [`TransactionVerificationData`].
TxVerificationData(TransactionVerificationData),
FilterKnownTxBlobHashes(HashSet<TransactionHash>),
/// The response for [`TxpoolReadRequest::FilterKnownTxBlobHashes`].
FilterKnownTxBlobHashes(HashSet<TransactionBlobHash>),
}
//---------------------------------------------------------------------------------------------------- TxpoolWriteRequest
@ -53,6 +55,11 @@ pub enum TxpoolWriteRequest {
///
/// Returns [`TxpoolWriteResponse::Ok`].
RemoveTransaction(TransactionHash),
/// Promote a transaction from the stem pool to the fluff pool.
/// If the tx is already in the fluff pool this does nothing.
///
/// Returns [`TxpoolWriteResponse::Ok`].
Promote(TransactionHash),
}
//---------------------------------------------------------------------------------------------------- TxpoolWriteResponse

View file

@ -58,6 +58,7 @@ fn map_request(
match request {
TxpoolReadRequest::TxBlob(tx_hash) => tx_blob(env, &tx_hash),
TxpoolReadRequest::TxVerificationData(tx_hash) => tx_verification_data(env, &tx_hash),
_ => todo!(),
}
}

View file

@ -17,6 +17,9 @@ pub type KeyImage = [u8; 32];
/// A transaction hash.
pub type TransactionHash = [u8; 32];
/// A transaction blob hash.
pub type TransactionBlobHash = [u8; 32];
bitflags::bitflags! {
/// Flags representing the state of the transaction in the pool.
#[derive(Copy, Clone, Debug, PartialEq, PartialOrd, Eq, Ord, Hash, Pod, Zeroable)]