From 241088e2736e8b5f29d2703a6f1e846f6f437867 Mon Sep 17 00:00:00 2001 From: Boog900 Date: Sun, 17 Nov 2024 20:32:41 +0000 Subject: [PATCH 01/10] Wire: fix IPv4 Endianness (#342) * fix IPv4 Endianness * fix import order --- net/wire/src/network_address.rs | 6 ++++-- net/wire/src/network_address/epee_builder.rs | 12 ++++++++---- 2 files changed, 12 insertions(+), 6 deletions(-) diff --git a/net/wire/src/network_address.rs b/net/wire/src/network_address.rs index ad599b70..3e15c467 100644 --- a/net/wire/src/network_address.rs +++ b/net/wire/src/network_address.rs @@ -17,10 +17,12 @@ //! Monero network. Core Monero has 4 main addresses: IPv4, IPv6, Tor, //! I2p. Currently this module only has IPv(4/6). //! -use bytes::BufMut; -use cuprate_epee_encoding::EpeeObject; use std::{hash::Hash, net, net::SocketAddr}; +use bytes::BufMut; + +use cuprate_epee_encoding::EpeeObject; + mod epee_builder; use epee_builder::*; diff --git a/net/wire/src/network_address/epee_builder.rs b/net/wire/src/network_address/epee_builder.rs index c1d17423..bd481a5e 100644 --- a/net/wire/src/network_address/epee_builder.rs +++ b/net/wire/src/network_address/epee_builder.rs @@ -1,9 +1,10 @@ -use bytes::Buf; use std::net::{Ipv4Addr, Ipv6Addr, SocketAddr, SocketAddrV4, SocketAddrV6}; -use cuprate_epee_encoding::{epee_object, EpeeObjectBuilder}; +use bytes::Buf; use thiserror::Error; +use cuprate_epee_encoding::{epee_object, EpeeObjectBuilder}; + use crate::NetworkAddress; #[derive(Default)] @@ -77,7 +78,7 @@ impl From for TaggedNetworkAddress { SocketAddr::V4(addr) => Self { ty: Some(1), addr: Some(AllFieldsNetworkAddress { - m_ip: Some(u32::from_be_bytes(addr.ip().octets())), + m_ip: Some(u32::from_le_bytes(addr.ip().octets())), m_port: Some(addr.port()), addr: None, }), @@ -112,7 +113,10 @@ epee_object!( impl AllFieldsNetworkAddress { fn try_into_network_address(self, ty: u8) -> Option { Some(match ty { - 1 => NetworkAddress::from(SocketAddrV4::new(Ipv4Addr::from(self.m_ip?), self.m_port?)), + 1 => NetworkAddress::from(SocketAddrV4::new( + Ipv4Addr::from(self.m_ip?.to_le_bytes()), + self.m_port?, + )), 2 => NetworkAddress::from(SocketAddrV6::new( Ipv6Addr::from(self.addr?), self.m_port?, From e8598a082d5df0660c5f52a2375c68777408b9f4 Mon Sep 17 00:00:00 2001 From: hinto-janai Date: Mon, 18 Nov 2024 10:21:52 -0500 Subject: [PATCH 02/10] books/architecture: add `Monero oddities` (#343) * add `oddities/` * swap `Expected`, `Why` --- books/architecture/src/SUMMARY.md | 5 +++ books/architecture/src/oddities/intro.md | 37 ++++++++++++++++++++++ books/architecture/src/oddities/le-ipv4.md | 24 ++++++++++++++ 3 files changed, 66 insertions(+) create mode 100644 books/architecture/src/oddities/intro.md create mode 100644 books/architecture/src/oddities/le-ipv4.md diff --git a/books/architecture/src/SUMMARY.md b/books/architecture/src/SUMMARY.md index bf668609..0961d8fc 100644 --- a/books/architecture/src/SUMMARY.md +++ b/books/architecture/src/SUMMARY.md @@ -157,6 +157,11 @@ --- +- [🟢 Monero oddities](oddities/intro.md) + - [🟡 Little-endian IPv4 addresses](oddities/le-ipv4.md) + +--- + - [⚪️ Appendix](appendix/intro.md) - [🟢 Crates](appendix/crates.md) - [🔴 Contributing](appendix/contributing.md) diff --git a/books/architecture/src/oddities/intro.md b/books/architecture/src/oddities/intro.md new file mode 100644 index 00000000..c0275b35 --- /dev/null +++ b/books/architecture/src/oddities/intro.md @@ -0,0 +1,37 @@ +# Monero oddities +This section is a list of any peculiar, interesting, +or non-standard behavior that Monero has that is not +planned on being changed or deprecated. + +This section exists to hold all the small yet noteworthy knowledge in one place, +instead of in any single contributor's mind. + +These are usually behaviors stemming from implementation rather than protocol/cryptography. + +## Formatting +This is the markdown formatting for each entry in this section. + +If applicable, consider using this formatting when adding to this section. + +```md +# + +## What +A detailed description of the behavior. + +## Expected +The norm or standard behavior that is usually expected. + +## Why +The reasoning behind why this behavior exists and/or +any links to more detailed discussion on the behavior. + +## Affects +A (potentially non-exhaustive) list of places that this behavior can/does affect. + +## Example +An example link or section of code where the behavior occurs. + +## Source +A link to original `monerod` code that defines the behavior. +``` \ No newline at end of file diff --git a/books/architecture/src/oddities/le-ipv4.md b/books/architecture/src/oddities/le-ipv4.md new file mode 100644 index 00000000..f64c1d78 --- /dev/null +++ b/books/architecture/src/oddities/le-ipv4.md @@ -0,0 +1,24 @@ +# Little-endian IPv4 addresses + +## What +Monero encodes IPv4 addresses in [little-endian](https://en.wikipedia.org/wiki/Endianness) byte order. + +## Expected +In general, [networking-related protocols/code use _networking order_ (big-endian)](https://en.wikipedia.org/wiki/Endianness#Networking). + +## Why +TODO + +- +- + +## Affects +Any representation and (de)serialization of IPv4 addresses must keep little +endian in-mind, e.g. the P2P wire format or `int` encoded IPv4 addresses in RPC. + +For example, [the `ip` field in `set_bans`](https://www.getmonero.org/resources/developer-guides/daemon-rpc.html#set_bans). + +For Cuprate, this means Rust's [`Ipv4Addr::from_bits/from`](https://doc.rust-lang.org/1.82.0/src/core/net/ip_addr.rs.html#1182) cannot be used in these cases as [it assumes big-endian encoding](https://doc.rust-lang.org/1.82.0/src/core/net/ip_addr.rs.html#540). + +## Source +- From c54bb0c8b20b209c8e841ab1187c0bf772ecc33c Mon Sep 17 00:00:00 2001 From: Boog900 Date: Wed, 20 Nov 2024 01:37:52 +0000 Subject: [PATCH 03/10] P2P: Change `ClientPool` to `PeerSet` (#337) * add WeakClient * todo * client pool -> peer set * more peer set changes * fix cuprated builds * add docs * more docs + better disconnect handling * more docs * fix imports * review fixes --- Cargo.lock | 1 - binaries/cuprated/src/blockchain/syncer.rs | 27 ++- binaries/cuprated/src/txpool/dandelion.rs | 2 +- .../src/txpool/dandelion/stem_service.rs | 71 ++++-- p2p/p2p-core/src/client.rs | 13 ++ p2p/p2p-core/src/client/weak.rs | 114 +++++++++ p2p/p2p/Cargo.toml | 4 +- p2p/p2p/src/block_downloader.rs | 68 +++--- .../src/block_downloader/download_batch.rs | 8 +- p2p/p2p/src/block_downloader/request_chain.rs | 24 +- p2p/p2p/src/block_downloader/tests.rs | 15 +- p2p/p2p/src/client_pool.rs | 188 --------------- p2p/p2p/src/client_pool/disconnect_monitor.rs | 83 ------- p2p/p2p/src/client_pool/drop_guard_client.rs | 41 ---- p2p/p2p/src/connection_maintainer.rs | 11 +- p2p/p2p/src/inbound_server.rs | 9 +- p2p/p2p/src/lib.rs | 38 +-- p2p/p2p/src/peer_set.rs | 217 ++++++++++++++++++ p2p/p2p/src/peer_set/client_wrappers.rs | 86 +++++++ 19 files changed, 602 insertions(+), 418 deletions(-) create mode 100644 p2p/p2p-core/src/client/weak.rs delete mode 100644 p2p/p2p/src/client_pool.rs delete mode 100644 p2p/p2p/src/client_pool/disconnect_monitor.rs delete mode 100644 p2p/p2p/src/client_pool/drop_guard_client.rs create mode 100644 p2p/p2p/src/peer_set.rs create mode 100644 p2p/p2p/src/peer_set/client_wrappers.rs diff --git a/Cargo.lock b/Cargo.lock index b446bf6d..a947a159 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -837,7 +837,6 @@ dependencies = [ "cuprate-test-utils", "cuprate-types", "cuprate-wire", - "dashmap", "futures", "indexmap", "monero-serai", diff --git a/binaries/cuprated/src/blockchain/syncer.rs b/binaries/cuprated/src/blockchain/syncer.rs index 913c9834..69ad3303 100644 --- a/binaries/cuprated/src/blockchain/syncer.rs +++ b/binaries/cuprated/src/blockchain/syncer.rs @@ -12,7 +12,7 @@ use tracing::instrument; use cuprate_consensus::{BlockChainContext, BlockChainContextRequest, BlockChainContextResponse}; use cuprate_p2p::{ block_downloader::{BlockBatch, BlockDownloaderConfig, ChainSvcRequest, ChainSvcResponse}, - NetworkInterface, + NetworkInterface, PeerSetRequest, PeerSetResponse, }; use cuprate_p2p_core::ClearNet; @@ -28,15 +28,11 @@ pub enum SyncerError { } /// The syncer tasks that makes sure we are fully synchronised with our connected peers. -#[expect( - clippy::significant_drop_tightening, - reason = "Client pool which will be removed" -)] #[instrument(level = "debug", skip_all)] pub async fn syncer( mut context_svc: C, our_chain: CN, - clearnet_interface: NetworkInterface, + mut clearnet_interface: NetworkInterface, incoming_block_batch_tx: mpsc::Sender, stop_current_block_downloader: Arc, block_downloader_config: BlockDownloaderConfig, @@ -67,8 +63,6 @@ where unreachable!(); }; - let client_pool = clearnet_interface.client_pool(); - tracing::debug!("Waiting for new sync info in top sync channel"); loop { @@ -79,9 +73,20 @@ where check_update_blockchain_context(&mut context_svc, &mut blockchain_ctx).await?; let raw_blockchain_context = blockchain_ctx.unchecked_blockchain_context(); - if !client_pool.contains_client_with_more_cumulative_difficulty( - raw_blockchain_context.cumulative_difficulty, - ) { + let PeerSetResponse::MostPoWSeen { + cumulative_difficulty, + .. + } = clearnet_interface + .peer_set() + .ready() + .await? + .call(PeerSetRequest::MostPoWSeen) + .await? + else { + unreachable!(); + }; + + if cumulative_difficulty <= raw_blockchain_context.cumulative_difficulty { continue; } diff --git a/binaries/cuprated/src/txpool/dandelion.rs b/binaries/cuprated/src/txpool/dandelion.rs index d791b62d..00d9f5a5 100644 --- a/binaries/cuprated/src/txpool/dandelion.rs +++ b/binaries/cuprated/src/txpool/dandelion.rs @@ -59,7 +59,7 @@ pub fn dandelion_router(clear_net: NetworkInterface) -> ConcreteDandel diffuse_service::DiffuseService { clear_net_broadcast_service: clear_net.broadcast_svc(), }, - stem_service::OutboundPeerStream { clear_net }, + stem_service::OutboundPeerStream::new(clear_net), DANDELION_CONFIG, ) } diff --git a/binaries/cuprated/src/txpool/dandelion/stem_service.rs b/binaries/cuprated/src/txpool/dandelion/stem_service.rs index 5c0ba65e..2debfd42 100644 --- a/binaries/cuprated/src/txpool/dandelion/stem_service.rs +++ b/binaries/cuprated/src/txpool/dandelion/stem_service.rs @@ -1,14 +1,15 @@ use std::{ + future::Future, pin::Pin, - task::{Context, Poll}, + task::{ready, Context, Poll}, }; use bytes::Bytes; -use futures::Stream; +use futures::{future::BoxFuture, FutureExt, Stream}; use tower::Service; use cuprate_dandelion_tower::{traits::StemRequest, OutboundPeer}; -use cuprate_p2p::{ClientPoolDropGuard, NetworkInterface}; +use cuprate_p2p::{ClientDropGuard, NetworkInterface, PeerSetRequest, PeerSetResponse}; use cuprate_p2p_core::{ client::{Client, InternalPeerID}, ClearNet, NetworkZone, PeerRequest, ProtocolRequest, @@ -19,7 +20,17 @@ use crate::{p2p::CrossNetworkInternalPeerId, txpool::dandelion::DandelionTx}; /// The dandelion outbound peer stream. pub struct OutboundPeerStream { - pub clear_net: NetworkInterface, + clear_net: NetworkInterface, + state: OutboundPeerStreamState, +} + +impl OutboundPeerStream { + pub const fn new(clear_net: NetworkInterface) -> Self { + Self { + clear_net, + state: OutboundPeerStreamState::Standby, + } + } } impl Stream for OutboundPeerStream { @@ -28,23 +39,49 @@ impl Stream for OutboundPeerStream { tower::BoxError, >; - fn poll_next(self: Pin<&mut Self>, _: &mut Context<'_>) -> Poll> { - // 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( - CrossNetworkInternalPeerId::ClearNet(client.info.id), - StemPeerService(client), - ) - })))) + fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + loop { + match &mut self.state { + OutboundPeerStreamState::Standby => { + let peer_set = self.clear_net.peer_set(); + let res = ready!(peer_set.poll_ready(cx)); + + self.state = OutboundPeerStreamState::AwaitingPeer( + peer_set.call(PeerSetRequest::StemPeer).boxed(), + ); + } + OutboundPeerStreamState::AwaitingPeer(fut) => { + let res = ready!(fut.poll_unpin(cx)); + + return Poll::Ready(Some(res.map(|res| { + let PeerSetResponse::StemPeer(stem_peer) = res else { + unreachable!() + }; + + match stem_peer { + Some(peer) => OutboundPeer::Peer( + CrossNetworkInternalPeerId::ClearNet(peer.info.id), + StemPeerService(peer), + ), + None => OutboundPeer::Exhausted, + } + }))); + } + } + } } } +/// The state of the [`OutboundPeerStream`]. +enum OutboundPeerStreamState { + /// Standby state. + Standby, + /// Awaiting a response from the peer-set. + AwaitingPeer(BoxFuture<'static, Result, tower::BoxError>>), +} + /// The stem service, used to send stem txs. -pub struct StemPeerService(ClientPoolDropGuard); +pub struct StemPeerService(ClientDropGuard); impl Service> for StemPeerService { type Response = as Service>::Response; diff --git a/p2p/p2p-core/src/client.rs b/p2p/p2p-core/src/client.rs index 73b33ba6..f2fde670 100644 --- a/p2p/p2p-core/src/client.rs +++ b/p2p/p2p-core/src/client.rs @@ -27,9 +27,11 @@ mod connector; pub mod handshaker; mod request_handler; mod timeout_monitor; +mod weak; pub use connector::{ConnectRequest, Connector}; pub use handshaker::{DoHandshakeRequest, HandshakeError, HandshakerBuilder}; +pub use weak::WeakClient; /// An internal identifier for a given peer, will be their address if known /// or a random u128 if not. @@ -128,6 +130,17 @@ impl Client { } .into() } + + /// Create a [`WeakClient`] for this [`Client`]. + pub fn downgrade(&self) -> WeakClient { + WeakClient { + info: self.info.clone(), + connection_tx: self.connection_tx.downgrade(), + semaphore: self.semaphore.clone(), + permit: None, + error: self.error.clone(), + } + } } impl Service for Client { diff --git a/p2p/p2p-core/src/client/weak.rs b/p2p/p2p-core/src/client/weak.rs new file mode 100644 index 00000000..90f25dd6 --- /dev/null +++ b/p2p/p2p-core/src/client/weak.rs @@ -0,0 +1,114 @@ +use std::task::{ready, Context, Poll}; + +use futures::channel::oneshot; +use tokio::sync::{mpsc, OwnedSemaphorePermit}; +use tokio_util::sync::PollSemaphore; +use tower::Service; + +use cuprate_helper::asynch::InfallibleOneshotReceiver; + +use crate::{ + client::{connection, PeerInformation}, + NetworkZone, PeerError, PeerRequest, PeerResponse, SharedError, +}; + +/// A weak handle to a [`Client`](super::Client). +/// +/// When this is dropped the peer will not be disconnected. +pub struct WeakClient { + /// Information on the connected peer. + pub info: PeerInformation, + + /// The channel to the [`Connection`](connection::Connection) task. + pub(super) connection_tx: mpsc::WeakSender, + + /// The semaphore that limits the requests sent to the peer. + pub(super) semaphore: PollSemaphore, + /// A permit for the semaphore, will be [`Some`] after `poll_ready` returns ready. + pub(super) permit: Option, + + /// The error slot shared between the [`Client`] and [`Connection`](connection::Connection). + pub(super) error: SharedError, +} + +impl WeakClient { + /// Internal function to set an error on the [`SharedError`]. + fn set_err(&self, err: PeerError) -> tower::BoxError { + let err_str = err.to_string(); + match self.error.try_insert_err(err) { + Ok(()) => err_str, + Err(e) => e.to_string(), + } + .into() + } +} + +impl Service for WeakClient { + type Response = PeerResponse; + type Error = tower::BoxError; + type Future = InfallibleOneshotReceiver>; + + fn poll_ready(&mut self, cx: &mut Context<'_>) -> Poll> { + if let Some(err) = self.error.try_get_err() { + return Poll::Ready(Err(err.to_string().into())); + } + + if self.connection_tx.strong_count() == 0 { + let err = self.set_err(PeerError::ClientChannelClosed); + return Poll::Ready(Err(err)); + } + + if self.permit.is_some() { + return Poll::Ready(Ok(())); + } + + let permit = ready!(self.semaphore.poll_acquire(cx)) + .expect("Client semaphore should not be closed!"); + + self.permit = Some(permit); + + Poll::Ready(Ok(())) + } + + #[expect(clippy::significant_drop_tightening)] + fn call(&mut self, request: PeerRequest) -> Self::Future { + let permit = self + .permit + .take() + .expect("poll_ready did not return ready before call to call"); + + let (tx, rx) = oneshot::channel(); + let req = connection::ConnectionTaskRequest { + response_channel: tx, + request, + permit: Some(permit), + }; + + match self.connection_tx.upgrade() { + None => { + self.set_err(PeerError::ClientChannelClosed); + + let resp = Err(PeerError::ClientChannelClosed.into()); + drop(req.response_channel.send(resp)); + } + Some(sender) => { + if let Err(e) = sender.try_send(req) { + // The connection task could have closed between a call to `poll_ready` and the call to + // `call`, which means if we don't handle the error here the receiver would panic. + use mpsc::error::TrySendError; + + match e { + TrySendError::Closed(req) | TrySendError::Full(req) => { + self.set_err(PeerError::ClientChannelClosed); + + let resp = Err(PeerError::ClientChannelClosed.into()); + drop(req.response_channel.send(resp)); + } + } + } + } + } + + rx.into() + } +} diff --git a/p2p/p2p/Cargo.toml b/p2p/p2p/Cargo.toml index 866fb918..e6ebccb8 100644 --- a/p2p/p2p/Cargo.toml +++ b/p2p/p2p/Cargo.toml @@ -20,12 +20,12 @@ monero-serai = { workspace = true, features = ["std"] } tower = { workspace = true, features = ["buffer"] } tokio = { workspace = true, features = ["rt", "rt-multi-thread"] } -rayon = { workspace = true } tokio-util = { workspace = true } +rayon = { workspace = true } tokio-stream = { workspace = true, features = ["sync", "time"] } futures = { workspace = true, features = ["std"] } pin-project = { workspace = true } -dashmap = { workspace = true } +indexmap = { workspace = true, features = ["std"] } thiserror = { workspace = true } bytes = { workspace = true, features = ["std"] } diff --git a/p2p/p2p/src/block_downloader.rs b/p2p/p2p/src/block_downloader.rs index fcc9eb65..faac4d5d 100644 --- a/p2p/p2p/src/block_downloader.rs +++ b/p2p/p2p/src/block_downloader.rs @@ -8,7 +8,6 @@ use std::{ cmp::{max, min, Reverse}, collections::{BTreeMap, BinaryHeap}, - sync::Arc, time::Duration, }; @@ -18,7 +17,7 @@ use tokio::{ task::JoinSet, time::{interval, timeout, MissedTickBehavior}, }; -use tower::{Service, ServiceExt}; +use tower::{util::BoxCloneService, Service, ServiceExt}; use tracing::{instrument, Instrument, Span}; use cuprate_async_buffer::{BufferAppender, BufferStream}; @@ -27,11 +26,11 @@ use cuprate_p2p_core::{handles::ConnectionHandle, NetworkZone}; use cuprate_pruning::PruningSeed; use crate::{ - client_pool::{ClientPool, ClientPoolDropGuard}, constants::{ BLOCK_DOWNLOADER_REQUEST_TIMEOUT, EMPTY_CHAIN_ENTRIES_BEFORE_TOP_ASSUMED, LONG_BAN, MAX_BLOCK_BATCH_LEN, MAX_DOWNLOAD_FAILURES, }, + peer_set::ClientDropGuard, }; mod block_queue; @@ -41,6 +40,7 @@ mod request_chain; #[cfg(test)] mod tests; +use crate::peer_set::{PeerSetRequest, PeerSetResponse}; use block_queue::{BlockQueue, ReadyQueueBatch}; use chain_tracker::{BlocksToRetrieve, ChainEntry, ChainTracker}; use download_batch::download_batch_task; @@ -135,7 +135,7 @@ pub enum ChainSvcResponse { /// call this function again, so it can start the search again. #[instrument(level = "error", skip_all, name = "block_downloader")] pub fn download_blocks( - client_pool: Arc>, + peer_set: BoxCloneService, tower::BoxError>, our_chain_svc: C, config: BlockDownloaderConfig, ) -> BufferStream @@ -147,8 +147,7 @@ where { let (buffer_appender, buffer_stream) = cuprate_async_buffer::new_buffer(config.buffer_size); - let block_downloader = - BlockDownloader::new(client_pool, our_chain_svc, buffer_appender, config); + let block_downloader = BlockDownloader::new(peer_set, our_chain_svc, buffer_appender, config); tokio::spawn( block_downloader @@ -186,8 +185,8 @@ where /// - download an already requested batch of blocks (this might happen due to an error in the previous request /// or because the queue of ready blocks is too large, so we need the oldest block to clear it). struct BlockDownloader { - /// The client pool. - client_pool: Arc>, + /// The peer set. + peer_set: BoxCloneService, tower::BoxError>, /// The service that holds our current chain state. our_chain_svc: C, @@ -208,7 +207,7 @@ struct BlockDownloader { /// /// Returns a result of the chain entry or an error. #[expect(clippy::type_complexity)] - chain_entry_task: JoinSet, ChainEntry), BlockDownloadError>>, + chain_entry_task: JoinSet, ChainEntry), BlockDownloadError>>, /// The current inflight requests. /// @@ -235,13 +234,13 @@ where { /// Creates a new [`BlockDownloader`] fn new( - client_pool: Arc>, + peer_set: BoxCloneService, tower::BoxError>, our_chain_svc: C, buffer_appender: BufferAppender, config: BlockDownloaderConfig, ) -> Self { Self { - client_pool, + peer_set, our_chain_svc, amount_of_blocks_to_request: config.initial_batch_size, amount_of_blocks_to_request_updated_at: 0, @@ -259,7 +258,7 @@ where fn check_pending_peers( &mut self, chain_tracker: &mut ChainTracker, - pending_peers: &mut BTreeMap>>, + pending_peers: &mut BTreeMap>>, ) { tracing::debug!("Checking if we can give any work to pending peers."); @@ -286,11 +285,11 @@ where /// This function will find the batch(es) that we are waiting on to clear our ready queue and sends another request /// for them. /// - /// Returns the [`ClientPoolDropGuard`] back if it doesn't have the batch according to its pruning seed. + /// Returns the [`ClientDropGuard`] back if it doesn't have the batch according to its pruning seed. fn request_inflight_batch_again( &mut self, - client: ClientPoolDropGuard, - ) -> Option> { + client: ClientDropGuard, + ) -> Option> { tracing::debug!( "Requesting an inflight batch, current ready queue size: {}", self.block_queue.size() @@ -336,13 +335,13 @@ where /// /// The batch requested will depend on our current state, failed batches will be prioritised. /// - /// Returns the [`ClientPoolDropGuard`] back if it doesn't have the data we currently need according + /// Returns the [`ClientDropGuard`] back if it doesn't have the data we currently need according /// to its pruning seed. fn request_block_batch( &mut self, chain_tracker: &mut ChainTracker, - client: ClientPoolDropGuard, - ) -> Option> { + client: ClientDropGuard, + ) -> Option> { tracing::trace!("Using peer to request a batch of blocks."); // First look to see if we have any failed requests. while let Some(failed_request) = self.failed_batches.peek() { @@ -416,13 +415,13 @@ where /// This function will use our current state to decide if we should send a request for a chain entry /// or if we should request a batch of blocks. /// - /// Returns the [`ClientPoolDropGuard`] back if it doesn't have the data we currently need according + /// Returns the [`ClientDropGuard`] back if it doesn't have the data we currently need according /// to its pruning seed. fn try_handle_free_client( &mut self, chain_tracker: &mut ChainTracker, - client: ClientPoolDropGuard, - ) -> Option> { + client: ClientDropGuard, + ) -> Option> { // We send 2 requests, so if one of them is slow or doesn't have the next chain, we still have a backup. if self.chain_entry_task.len() < 2 // If we have had too many failures then assume the tip has been found so no more chain entries. @@ -463,7 +462,7 @@ where async fn check_for_free_clients( &mut self, chain_tracker: &mut ChainTracker, - pending_peers: &mut BTreeMap>>, + pending_peers: &mut BTreeMap>>, ) -> Result<(), BlockDownloadError> { tracing::debug!("Checking for free peers"); @@ -478,10 +477,19 @@ where panic!("Chain service returned wrong response."); }; - for client in self - .client_pool - .clients_with_more_cumulative_difficulty(current_cumulative_difficulty) - { + let PeerSetResponse::PeersWithMorePoW(clients) = self + .peer_set + .ready() + .await? + .call(PeerSetRequest::PeersWithMorePoW( + current_cumulative_difficulty, + )) + .await? + else { + unreachable!(); + }; + + for client in clients { pending_peers .entry(client.info.pruning_seed) .or_default() @@ -497,9 +505,9 @@ where async fn handle_download_batch_res( &mut self, start_height: usize, - res: Result<(ClientPoolDropGuard, BlockBatch), BlockDownloadError>, + res: Result<(ClientDropGuard, BlockBatch), BlockDownloadError>, chain_tracker: &mut ChainTracker, - pending_peers: &mut BTreeMap>>, + pending_peers: &mut BTreeMap>>, ) -> Result<(), BlockDownloadError> { tracing::debug!("Handling block download response"); @@ -593,7 +601,7 @@ where /// Starts the main loop of the block downloader. async fn run(mut self) -> Result<(), BlockDownloadError> { let mut chain_tracker = - initial_chain_search(&self.client_pool, &mut self.our_chain_svc).await?; + initial_chain_search(&mut self.peer_set, &mut self.our_chain_svc).await?; let mut pending_peers = BTreeMap::new(); @@ -662,7 +670,7 @@ struct BlockDownloadTaskResponse { /// The start height of the batch. start_height: usize, /// A result containing the batch or an error. - result: Result<(ClientPoolDropGuard, BlockBatch), BlockDownloadError>, + result: Result<(ClientDropGuard, BlockBatch), BlockDownloadError>, } /// Returns if a peer has all the blocks in a range, according to its [`PruningSeed`]. diff --git a/p2p/p2p/src/block_downloader/download_batch.rs b/p2p/p2p/src/block_downloader/download_batch.rs index bbb14b3b..ef621ce8 100644 --- a/p2p/p2p/src/block_downloader/download_batch.rs +++ b/p2p/p2p/src/block_downloader/download_batch.rs @@ -16,8 +16,8 @@ use cuprate_wire::protocol::{GetObjectsRequest, GetObjectsResponse}; use crate::{ block_downloader::{BlockBatch, BlockDownloadError, BlockDownloadTaskResponse}, - client_pool::ClientPoolDropGuard, constants::{BLOCK_DOWNLOADER_REQUEST_TIMEOUT, MAX_TRANSACTION_BLOB_SIZE, MEDIUM_BAN}, + peer_set::ClientDropGuard, }; /// Attempts to request a batch of blocks from a peer, returning [`BlockDownloadTaskResponse`]. @@ -32,7 +32,7 @@ use crate::{ )] #[expect(clippy::used_underscore_binding)] pub async fn download_batch_task( - client: ClientPoolDropGuard, + client: ClientDropGuard, ids: ByteArrayVec<32>, previous_id: [u8; 32], expected_start_height: usize, @@ -49,11 +49,11 @@ pub async fn download_batch_task( /// This function will validate the blocks that were downloaded were the ones asked for and that they match /// the expected height. async fn request_batch_from_peer( - mut client: ClientPoolDropGuard, + mut client: ClientDropGuard, ids: ByteArrayVec<32>, previous_id: [u8; 32], expected_start_height: usize, -) -> Result<(ClientPoolDropGuard, BlockBatch), BlockDownloadError> { +) -> Result<(ClientDropGuard, BlockBatch), BlockDownloadError> { let request = PeerRequest::Protocol(ProtocolRequest::GetObjects(GetObjectsRequest { blocks: ids.clone(), pruned: false, diff --git a/p2p/p2p/src/block_downloader/request_chain.rs b/p2p/p2p/src/block_downloader/request_chain.rs index d6a2a0af..4e0f855b 100644 --- a/p2p/p2p/src/block_downloader/request_chain.rs +++ b/p2p/p2p/src/block_downloader/request_chain.rs @@ -1,7 +1,7 @@ -use std::{mem, sync::Arc}; +use std::mem; use tokio::{task::JoinSet, time::timeout}; -use tower::{Service, ServiceExt}; +use tower::{util::BoxCloneService, Service, ServiceExt}; use tracing::{instrument, Instrument, Span}; use cuprate_p2p_core::{ @@ -15,11 +15,11 @@ use crate::{ chain_tracker::{ChainEntry, ChainTracker}, BlockDownloadError, ChainSvcRequest, ChainSvcResponse, }, - client_pool::{ClientPool, ClientPoolDropGuard}, constants::{ BLOCK_DOWNLOADER_REQUEST_TIMEOUT, INITIAL_CHAIN_REQUESTS_TO_SEND, MAX_BLOCKS_IDS_IN_CHAIN_ENTRY, MEDIUM_BAN, }, + peer_set::{ClientDropGuard, PeerSetRequest, PeerSetResponse}, }; /// Request a chain entry from a peer. @@ -27,9 +27,9 @@ use crate::{ /// Because the block downloader only follows and downloads one chain we only have to send the block hash of /// top block we have found and the genesis block, this is then called `short_history`. pub(crate) async fn request_chain_entry_from_peer( - mut client: ClientPoolDropGuard, + mut client: ClientDropGuard, short_history: [[u8; 32]; 2], -) -> Result<(ClientPoolDropGuard, ChainEntry), BlockDownloadError> { +) -> Result<(ClientDropGuard, ChainEntry), BlockDownloadError> { let PeerResponse::Protocol(ProtocolResponse::GetChain(chain_res)) = client .ready() .await? @@ -80,7 +80,7 @@ pub(crate) async fn request_chain_entry_from_peer( /// We then wait for their response and choose the peer who claims the highest cumulative difficulty. #[instrument(level = "error", skip_all)] pub async fn initial_chain_search( - client_pool: &Arc>, + peer_set: &mut BoxCloneService, tower::BoxError>, mut our_chain_svc: C, ) -> Result, BlockDownloadError> where @@ -102,9 +102,15 @@ where let our_genesis = *block_ids.last().expect("Blockchain had no genesis block."); - let mut peers = client_pool - .clients_with_more_cumulative_difficulty(cumulative_difficulty) - .into_iter(); + let PeerSetResponse::PeersWithMorePoW(clients) = peer_set + .ready() + .await? + .call(PeerSetRequest::PeersWithMorePoW(cumulative_difficulty)) + .await? + else { + unreachable!(); + }; + let mut peers = clients.into_iter(); let mut futs = JoinSet::new(); diff --git a/p2p/p2p/src/block_downloader/tests.rs b/p2p/p2p/src/block_downloader/tests.rs index 83dd417c..6799482d 100644 --- a/p2p/p2p/src/block_downloader/tests.rs +++ b/p2p/p2p/src/block_downloader/tests.rs @@ -14,8 +14,8 @@ use monero_serai::{ transaction::{Input, Timelock, Transaction, TransactionPrefix}, }; use proptest::{collection::vec, prelude::*}; -use tokio::time::timeout; -use tower::{service_fn, Service}; +use tokio::{sync::mpsc, time::timeout}; +use tower::{buffer::Buffer, service_fn, Service, ServiceExt}; use cuprate_fixed_bytes::ByteArrayVec; use cuprate_p2p_core::{ @@ -31,7 +31,7 @@ use cuprate_wire::{ use crate::{ block_downloader::{download_blocks, BlockDownloaderConfig, ChainSvcRequest, ChainSvcResponse}, - client_pool::ClientPool, + peer_set::PeerSet, }; proptest! { @@ -48,19 +48,20 @@ proptest! { let tokio_pool = tokio::runtime::Builder::new_multi_thread().enable_all().build().unwrap(); - #[expect(clippy::significant_drop_tightening)] tokio_pool.block_on(async move { timeout(Duration::from_secs(600), async move { - let client_pool = ClientPool::new(); + let (new_connection_tx, new_connection_rx) = mpsc::channel(peers); + + let peer_set = PeerSet::new(new_connection_rx); for _ in 0..peers { let client = mock_block_downloader_client(Arc::clone(&blockchain)); - client_pool.add_new_client(client); + new_connection_tx.try_send(client).unwrap(); } let stream = download_blocks( - client_pool, + Buffer::new(peer_set, 10).boxed_clone(), OurChainSvc { genesis: *blockchain.blocks.first().unwrap().0 }, diff --git a/p2p/p2p/src/client_pool.rs b/p2p/p2p/src/client_pool.rs deleted file mode 100644 index 67c8f112..00000000 --- a/p2p/p2p/src/client_pool.rs +++ /dev/null @@ -1,188 +0,0 @@ -//! # Client Pool. -//! -//! The [`ClientPool`], is a pool of currently connected peers that can be pulled from. -//! It does _not_ necessarily contain every connected peer as another place could have -//! taken a peer from the pool. -//! -//! When taking peers from the pool they are wrapped in [`ClientPoolDropGuard`], which -//! returns the peer to the pool when it is dropped. -//! -//! Internally the pool is a [`DashMap`] which means care should be taken in `async` code -//! as internally this uses blocking `RwLock`s. -use std::sync::Arc; - -use dashmap::DashMap; -use tokio::sync::mpsc; -use tracing::{Instrument, Span}; - -use cuprate_p2p_core::{ - client::{Client, InternalPeerID}, - handles::ConnectionHandle, - ConnectionDirection, NetworkZone, -}; - -pub(crate) mod disconnect_monitor; -mod drop_guard_client; - -pub use drop_guard_client::ClientPoolDropGuard; - -/// The client pool, which holds currently connected free peers. -/// -/// See the [module docs](self) for more. -pub struct ClientPool { - /// The connected [`Client`]s. - clients: DashMap, Client>, - /// A channel to send new peer ids down to monitor for disconnect. - new_connection_tx: mpsc::UnboundedSender<(ConnectionHandle, InternalPeerID)>, -} - -impl ClientPool { - /// Returns a new [`ClientPool`] wrapped in an [`Arc`]. - pub fn new() -> Arc { - let (tx, rx) = mpsc::unbounded_channel(); - - let pool = Arc::new(Self { - clients: DashMap::new(), - new_connection_tx: tx, - }); - - tokio::spawn( - disconnect_monitor::disconnect_monitor(rx, Arc::clone(&pool)) - .instrument(Span::current()), - ); - - pool - } - - /// Adds a [`Client`] to the pool, the client must have previously been taken from the - /// pool. - /// - /// See [`ClientPool::add_new_client`] to add a [`Client`] which was not taken from the pool before. - /// - /// # Panics - /// This function panics if `client` already exists in the pool. - fn add_client(&self, client: Client) { - let handle = client.info.handle.clone(); - let id = client.info.id; - - // Fast path: if the client is disconnected don't add it to the peer set. - if handle.is_closed() { - return; - } - - assert!(self.clients.insert(id, client).is_none()); - - // We have to check this again otherwise we could have a race condition where a - // peer is disconnected after the first check, the disconnect monitor tries to remove it, - // and then it is added to the pool. - if handle.is_closed() { - self.remove_client(&id); - } - } - - /// Adds a _new_ [`Client`] to the pool, this client should be a new connection, and not already - /// from the pool. - /// - /// # Panics - /// This function panics if `client` already exists in the pool. - pub fn add_new_client(&self, client: Client) { - self.new_connection_tx - .send((client.info.handle.clone(), client.info.id)) - .unwrap(); - - self.add_client(client); - } - - /// Remove a [`Client`] from the pool. - /// - /// [`None`] is returned if the client did not exist in the pool. - fn remove_client(&self, peer: &InternalPeerID) -> Option> { - self.clients.remove(peer).map(|(_, client)| client) - } - - /// Borrows a [`Client`] from the pool. - /// - /// The [`Client`] is wrapped in [`ClientPoolDropGuard`] which - /// will return the client to the pool when it's dropped. - /// - /// See [`Self::borrow_clients`] for borrowing multiple clients. - pub fn borrow_client( - self: &Arc, - peer: &InternalPeerID, - ) -> Option> { - self.remove_client(peer).map(|client| ClientPoolDropGuard { - pool: Arc::clone(self), - client: Some(client), - }) - } - - /// Borrows multiple [`Client`]s from the pool. - /// - /// Note that the returned iterator is not guaranteed to contain every peer asked for. - /// - /// See [`Self::borrow_client`] for borrowing a single client. - pub fn borrow_clients<'a, 'b>( - self: &'a Arc, - peers: &'b [InternalPeerID], - ) -> impl Iterator> + sealed::Captures<(&'a (), &'b ())> { - peers.iter().filter_map(|peer| self.borrow_client(peer)) - } - - /// Borrows all [`Client`]s from the pool that have claimed a higher cumulative difficulty than - /// the amount passed in. - /// - /// The [`Client`]s are wrapped in [`ClientPoolDropGuard`] which - /// will return the clients to the pool when they are dropped. - pub fn clients_with_more_cumulative_difficulty( - self: &Arc, - cumulative_difficulty: u128, - ) -> Vec> { - let peers = self - .clients - .iter() - .filter_map(|element| { - let peer_sync_info = element.value().info.core_sync_data.lock().unwrap(); - - if peer_sync_info.cumulative_difficulty() > cumulative_difficulty { - Some(*element.key()) - } else { - None - } - }) - .collect::>(); - - self.borrow_clients(&peers).collect() - } - - /// Checks all clients in the pool checking if any claim a higher cumulative difficulty than the - /// amount specified. - pub fn contains_client_with_more_cumulative_difficulty( - &self, - cumulative_difficulty: u128, - ) -> bool { - self.clients.iter().any(|element| { - let sync_data = element.value().info.core_sync_data.lock().unwrap(); - sync_data.cumulative_difficulty() > cumulative_difficulty - }) - } - - /// Returns the first outbound peer when iterating over the peers. - pub fn outbound_client(self: &Arc) -> Option> { - let client = self - .clients - .iter() - .find(|element| element.value().info.direction == ConnectionDirection::Outbound)?; - let id = *client.key(); - - Some(self.borrow_client(&id).unwrap()) - } -} - -mod sealed { - /// TODO: Remove me when 2024 Rust - /// - /// - pub trait Captures {} - - impl Captures for T {} -} diff --git a/p2p/p2p/src/client_pool/disconnect_monitor.rs b/p2p/p2p/src/client_pool/disconnect_monitor.rs deleted file mode 100644 index f54b5606..00000000 --- a/p2p/p2p/src/client_pool/disconnect_monitor.rs +++ /dev/null @@ -1,83 +0,0 @@ -//! # Disconnect Monitor -//! -//! This module contains the [`disconnect_monitor`] task, which monitors connected peers for disconnection -//! and then removes them from the [`ClientPool`] if they do. -use std::{ - future::Future, - pin::Pin, - sync::Arc, - task::{Context, Poll}, -}; - -use futures::{stream::FuturesUnordered, StreamExt}; -use tokio::sync::mpsc; -use tokio_util::sync::WaitForCancellationFutureOwned; -use tracing::instrument; - -use cuprate_p2p_core::{client::InternalPeerID, handles::ConnectionHandle, NetworkZone}; - -use super::ClientPool; - -/// The disconnect monitor task. -#[instrument(level = "info", skip_all)] -pub async fn disconnect_monitor( - mut new_connection_rx: mpsc::UnboundedReceiver<(ConnectionHandle, InternalPeerID)>, - client_pool: Arc>, -) { - // We need to hold a weak reference otherwise the client pool and this would hold a reference to - // each other causing the pool to be leaked. - let weak_client_pool = Arc::downgrade(&client_pool); - drop(client_pool); - - tracing::info!("Starting peer disconnect monitor."); - - let mut futs: FuturesUnordered> = FuturesUnordered::new(); - - loop { - tokio::select! { - Some((con_handle, peer_id)) = new_connection_rx.recv() => { - tracing::debug!("Monitoring {peer_id} for disconnect"); - futs.push(PeerDisconnectFut { - closed_fut: con_handle.closed(), - peer_id: Some(peer_id), - }); - } - Some(peer_id) = futs.next() => { - tracing::debug!("{peer_id} has disconnected, removing from client pool."); - let Some(pool) = weak_client_pool.upgrade() else { - tracing::info!("Peer disconnect monitor shutting down."); - return; - }; - - pool.remove_client(&peer_id); - drop(pool); - } - else => { - tracing::info!("Peer disconnect monitor shutting down."); - return; - } - } - } -} - -/// A [`Future`] that resolves when a peer disconnects. -#[pin_project::pin_project] -pub(crate) struct PeerDisconnectFut { - /// The inner [`Future`] that resolves when a peer disconnects. - #[pin] - pub(crate) closed_fut: WaitForCancellationFutureOwned, - /// The peers ID. - pub(crate) peer_id: Option>, -} - -impl Future for PeerDisconnectFut { - type Output = InternalPeerID; - - fn poll(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { - let this = self.project(); - - this.closed_fut - .poll(cx) - .map(|()| this.peer_id.take().unwrap()) - } -} diff --git a/p2p/p2p/src/client_pool/drop_guard_client.rs b/p2p/p2p/src/client_pool/drop_guard_client.rs deleted file mode 100644 index b10c4e9c..00000000 --- a/p2p/p2p/src/client_pool/drop_guard_client.rs +++ /dev/null @@ -1,41 +0,0 @@ -use std::{ - ops::{Deref, DerefMut}, - sync::Arc, -}; - -use cuprate_p2p_core::{client::Client, NetworkZone}; - -use crate::client_pool::ClientPool; - -/// A wrapper around [`Client`] which returns the client to the [`ClientPool`] when dropped. -pub struct ClientPoolDropGuard { - /// The [`ClientPool`] to return the peer to. - pub(super) pool: Arc>, - /// The [`Client`]. - /// - /// This is set to [`Some`] when this guard is created, then - /// [`take`](Option::take)n and returned to the pool when dropped. - pub(super) client: Option>, -} - -impl Deref for ClientPoolDropGuard { - type Target = Client; - - fn deref(&self) -> &Self::Target { - self.client.as_ref().unwrap() - } -} - -impl DerefMut for ClientPoolDropGuard { - fn deref_mut(&mut self) -> &mut Self::Target { - self.client.as_mut().unwrap() - } -} - -impl Drop for ClientPoolDropGuard { - fn drop(&mut self) { - let client = self.client.take().unwrap(); - - self.pool.add_client(client); - } -} diff --git a/p2p/p2p/src/connection_maintainer.rs b/p2p/p2p/src/connection_maintainer.rs index cd9d9311..245fbf12 100644 --- a/p2p/p2p/src/connection_maintainer.rs +++ b/p2p/p2p/src/connection_maintainer.rs @@ -21,7 +21,6 @@ use cuprate_p2p_core::{ }; use crate::{ - client_pool::ClientPool, config::P2PConfig, constants::{HANDSHAKE_TIMEOUT, MAX_SEED_CONNECTIONS, OUTBOUND_CONNECTION_ATTEMPT_TIMEOUT}, }; @@ -46,7 +45,7 @@ pub struct MakeConnectionRequest { /// This handles maintaining a minimum number of connections and making extra connections when needed, upto a maximum. pub struct OutboundConnectionKeeper { /// The pool of currently connected peers. - pub client_pool: Arc>, + pub new_peers_tx: mpsc::Sender>, /// The channel that tells us to make new _extra_ outbound connections. pub make_connection_rx: mpsc::Receiver, /// The address book service @@ -77,7 +76,7 @@ where { pub fn new( config: P2PConfig, - client_pool: Arc>, + new_peers_tx: mpsc::Sender>, make_connection_rx: mpsc::Receiver, address_book_svc: A, connector_svc: C, @@ -86,7 +85,7 @@ where .expect("Gray peer percent is incorrect should be 0..=1"); Self { - client_pool, + new_peers_tx, make_connection_rx, address_book_svc, connector_svc, @@ -149,7 +148,7 @@ where /// Connects to a given outbound peer. #[instrument(level = "info", skip_all)] async fn connect_to_outbound_peer(&mut self, permit: OwnedSemaphorePermit, addr: N::Addr) { - let client_pool = Arc::clone(&self.client_pool); + let new_peers_tx = self.new_peers_tx.clone(); let connection_fut = self .connector_svc .ready() @@ -164,7 +163,7 @@ where async move { #[expect(clippy::significant_drop_in_scrutinee)] if let Ok(Ok(peer)) = timeout(HANDSHAKE_TIMEOUT, connection_fut).await { - client_pool.add_new_client(peer); + drop(new_peers_tx.send(peer).await); } } .instrument(Span::current()), diff --git a/p2p/p2p/src/inbound_server.rs b/p2p/p2p/src/inbound_server.rs index 6e793bd1..0479560b 100644 --- a/p2p/p2p/src/inbound_server.rs +++ b/p2p/p2p/src/inbound_server.rs @@ -6,7 +6,7 @@ use std::{pin::pin, sync::Arc}; use futures::{SinkExt, StreamExt}; use tokio::{ - sync::Semaphore, + sync::{mpsc, Semaphore}, task::JoinSet, time::{sleep, timeout}, }; @@ -24,7 +24,6 @@ use cuprate_wire::{ }; use crate::{ - client_pool::ClientPool, constants::{ HANDSHAKE_TIMEOUT, INBOUND_CONNECTION_COOL_DOWN, PING_REQUEST_CONCURRENCY, PING_REQUEST_TIMEOUT, @@ -36,7 +35,7 @@ use crate::{ /// and initiate handshake if needed, after verifying the address isn't banned. #[instrument(level = "warn", skip_all)] pub async fn inbound_server( - client_pool: Arc>, + new_connection_tx: mpsc::Sender>, mut handshaker: HS, mut address_book: A, config: P2PConfig, @@ -111,13 +110,13 @@ where permit: Some(permit), }); - let cloned_pool = Arc::clone(&client_pool); + let new_connection_tx = new_connection_tx.clone(); tokio::spawn( async move { let client = timeout(HANDSHAKE_TIMEOUT, fut).await; if let Ok(Ok(peer)) = client { - cloned_pool.add_new_client(peer); + drop(new_connection_tx.send(peer).await); } } .instrument(Span::current()), diff --git a/p2p/p2p/src/lib.rs b/p2p/p2p/src/lib.rs index 541784c9..fb506582 100644 --- a/p2p/p2p/src/lib.rs +++ b/p2p/p2p/src/lib.rs @@ -18,17 +18,18 @@ use cuprate_p2p_core::{ pub mod block_downloader; mod broadcast; -pub mod client_pool; pub mod config; pub mod connection_maintainer; pub mod constants; mod inbound_server; +mod peer_set; use block_downloader::{BlockBatch, BlockDownloaderConfig, ChainSvcRequest, ChainSvcResponse}; pub use broadcast::{BroadcastRequest, BroadcastSvc}; -pub use client_pool::{ClientPool, ClientPoolDropGuard}; pub use config::{AddressBookConfig, P2PConfig}; use connection_maintainer::MakeConnectionRequest; +use peer_set::PeerSet; +pub use peer_set::{ClientDropGuard, PeerSetRequest, PeerSetResponse}; /// Initializes the P2P [`NetworkInterface`] for a specific [`NetworkZone`]. /// @@ -54,7 +55,10 @@ where cuprate_address_book::init_address_book(config.address_book_config.clone()).await?; let address_book = Buffer::new( address_book, - config.max_inbound_connections + config.outbound_connections, + config + .max_inbound_connections + .checked_add(config.outbound_connections) + .unwrap(), ); // Use the default config. Changing the defaults affects tx fluff times, which could affect D++ so for now don't allow changing @@ -83,19 +87,25 @@ where let outbound_handshaker = outbound_handshaker_builder.build(); - let client_pool = ClientPool::new(); - + let (new_connection_tx, new_connection_rx) = mpsc::channel( + config + .outbound_connections + .checked_add(config.max_inbound_connections) + .unwrap(), + ); let (make_connection_tx, make_connection_rx) = mpsc::channel(3); let outbound_connector = Connector::new(outbound_handshaker); let outbound_connection_maintainer = connection_maintainer::OutboundConnectionKeeper::new( config.clone(), - Arc::clone(&client_pool), + new_connection_tx.clone(), make_connection_rx, address_book.clone(), outbound_connector, ); + let peer_set = PeerSet::new(new_connection_rx); + let mut background_tasks = JoinSet::new(); background_tasks.spawn( @@ -105,7 +115,7 @@ where ); background_tasks.spawn( inbound_server::inbound_server( - Arc::clone(&client_pool), + new_connection_tx, inbound_handshaker, address_book.clone(), config, @@ -121,7 +131,7 @@ where ); Ok(NetworkInterface { - pool: client_pool, + peer_set: Buffer::new(peer_set, 10).boxed_clone(), broadcast_svc, make_connection_tx, address_book: address_book.boxed_clone(), @@ -133,7 +143,7 @@ where #[derive(Clone)] pub struct NetworkInterface { /// A pool of free connected peers. - pool: Arc>, + peer_set: BoxCloneService, tower::BoxError>, /// A [`Service`] that allows broadcasting to all connected peers. broadcast_svc: BroadcastSvc, /// A channel to request extra connections. @@ -163,7 +173,7 @@ impl NetworkInterface { + 'static, C::Future: Send + 'static, { - block_downloader::download_blocks(Arc::clone(&self.pool), our_chain_service, config) + block_downloader::download_blocks(self.peer_set.clone(), our_chain_service, config) } /// Returns the address book service. @@ -173,8 +183,10 @@ impl NetworkInterface { self.address_book.clone() } - /// Borrows the `ClientPool`, for access to connected peers. - pub const fn client_pool(&self) -> &Arc> { - &self.pool + /// Borrows the `PeerSet`, for access to connected peers. + pub fn peer_set( + &mut self, + ) -> &mut BoxCloneService, tower::BoxError> { + &mut self.peer_set } } diff --git a/p2p/p2p/src/peer_set.rs b/p2p/p2p/src/peer_set.rs new file mode 100644 index 00000000..498eaafc --- /dev/null +++ b/p2p/p2p/src/peer_set.rs @@ -0,0 +1,217 @@ +use std::{ + future::{ready, Future, Ready}, + pin::{pin, Pin}, + task::{Context, Poll}, +}; + +use futures::{stream::FuturesUnordered, StreamExt}; +use indexmap::{IndexMap, IndexSet}; +use rand::{seq::index::sample, thread_rng}; +use tokio::sync::mpsc::Receiver; +use tokio_util::sync::WaitForCancellationFutureOwned; +use tower::Service; + +use cuprate_helper::cast::u64_to_usize; +use cuprate_p2p_core::{ + client::{Client, InternalPeerID}, + ConnectionDirection, NetworkZone, +}; + +mod client_wrappers; + +pub use client_wrappers::ClientDropGuard; +use client_wrappers::StoredClient; + +/// A request to the peer-set. +pub enum PeerSetRequest { + /// The most claimed proof-of-work from a peer in the peer-set. + MostPoWSeen, + /// Peers with more cumulative difficulty than the given cumulative difficulty. + /// + /// Returned peers will be remembered and won't be returned from subsequent calls until the guard is dropped. + PeersWithMorePoW(u128), + /// A random outbound peer. + /// + /// The returned peer will be remembered and won't be returned from subsequent calls until the guard is dropped. + StemPeer, +} + +/// A response from the peer-set. +pub enum PeerSetResponse { + /// [`PeerSetRequest::MostPoWSeen`] + MostPoWSeen { + /// The cumulative difficulty claimed. + cumulative_difficulty: u128, + /// The height claimed. + height: usize, + /// The claimed hash of the top block. + top_hash: [u8; 32], + }, + /// [`PeerSetRequest::PeersWithMorePoW`] + /// + /// Returned peers will be remembered and won't be returned from subsequent calls until the guard is dropped. + PeersWithMorePoW(Vec>), + /// [`PeerSetRequest::StemPeer`] + /// + /// The returned peer will be remembered and won't be returned from subsequent calls until the guard is dropped. + StemPeer(Option>), +} + +/// A [`Future`] that completes when a peer disconnects. +#[pin_project::pin_project] +struct ClosedConnectionFuture { + #[pin] + fut: WaitForCancellationFutureOwned, + id: Option>, +} + +impl Future for ClosedConnectionFuture { + type Output = InternalPeerID; + fn poll(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { + let this = self.project(); + + this.fut.poll(cx).map(|()| this.id.take().unwrap()) + } +} + +/// A collection of all connected peers on a [`NetworkZone`]. +pub(crate) struct PeerSet { + /// The connected peers. + peers: IndexMap, StoredClient>, + /// A [`FuturesUnordered`] that resolves when a peer disconnects. + closed_connections: FuturesUnordered>, + /// The [`InternalPeerID`]s of all outbound peers. + outbound_peers: IndexSet>, + /// A channel of new peers from the inbound server or outbound connector. + new_peers: Receiver>, +} + +impl PeerSet { + pub(crate) fn new(new_peers: Receiver>) -> Self { + Self { + peers: IndexMap::new(), + closed_connections: FuturesUnordered::new(), + outbound_peers: IndexSet::new(), + new_peers, + } + } + + /// Polls the new peers channel for newly connected peers. + fn poll_new_peers(&mut self, cx: &mut Context<'_>) { + while let Poll::Ready(Some(new_peer)) = self.new_peers.poll_recv(cx) { + if new_peer.info.direction == ConnectionDirection::Outbound { + self.outbound_peers.insert(new_peer.info.id); + } + + self.closed_connections.push(ClosedConnectionFuture { + fut: new_peer.info.handle.closed(), + id: Some(new_peer.info.id), + }); + + self.peers + .insert(new_peer.info.id, StoredClient::new(new_peer)); + } + } + + /// Remove disconnected peers from the peer set. + fn remove_dead_peers(&mut self, cx: &mut Context<'_>) { + while let Poll::Ready(Some(dead_peer)) = self.closed_connections.poll_next_unpin(cx) { + let Some(peer) = self.peers.swap_remove(&dead_peer) else { + continue; + }; + + if peer.client.info.direction == ConnectionDirection::Outbound { + self.outbound_peers.swap_remove(&peer.client.info.id); + } + + self.peers.swap_remove(&dead_peer); + } + } + + /// [`PeerSetRequest::MostPoWSeen`] + fn most_pow_seen(&self) -> PeerSetResponse { + let most_pow_chain = self + .peers + .values() + .map(|peer| { + let core_sync_data = peer.client.info.core_sync_data.lock().unwrap(); + + ( + core_sync_data.cumulative_difficulty(), + u64_to_usize(core_sync_data.current_height), + core_sync_data.top_id, + ) + }) + .max_by_key(|(cumulative_difficulty, ..)| *cumulative_difficulty) + .unwrap_or_default(); + + PeerSetResponse::MostPoWSeen { + cumulative_difficulty: most_pow_chain.0, + height: most_pow_chain.1, + top_hash: most_pow_chain.2, + } + } + + /// [`PeerSetRequest::PeersWithMorePoW`] + fn peers_with_more_pow(&self, cumulative_difficulty: u128) -> PeerSetResponse { + PeerSetResponse::PeersWithMorePoW( + self.peers + .values() + .filter(|&client| { + !client.is_downloading_blocks() + && client + .client + .info + .core_sync_data + .lock() + .unwrap() + .cumulative_difficulty() + > cumulative_difficulty + }) + .map(StoredClient::downloading_blocks_guard) + .collect(), + ) + } + + /// [`PeerSetRequest::StemPeer`] + fn random_peer_for_stem(&self) -> PeerSetResponse { + PeerSetResponse::StemPeer( + sample( + &mut thread_rng(), + self.outbound_peers.len(), + self.outbound_peers.len(), + ) + .into_iter() + .find_map(|i| { + let peer = self.outbound_peers.get_index(i).unwrap(); + let client = self.peers.get(peer).unwrap(); + (!client.is_a_stem_peer()).then(|| client.stem_peer_guard()) + }), + ) + } +} + +impl Service for PeerSet { + type Response = PeerSetResponse; + type Error = tower::BoxError; + type Future = Ready>; + + fn poll_ready(&mut self, cx: &mut Context<'_>) -> Poll> { + self.poll_new_peers(cx); + self.remove_dead_peers(cx); + + // TODO: should we return `Pending` if we don't have any peers? + + Poll::Ready(Ok(())) + } + + fn call(&mut self, req: PeerSetRequest) -> Self::Future { + ready(match req { + PeerSetRequest::MostPoWSeen => Ok(self.most_pow_seen()), + PeerSetRequest::PeersWithMorePoW(cumulative_difficulty) => { + Ok(self.peers_with_more_pow(cumulative_difficulty)) + } + PeerSetRequest::StemPeer => Ok(self.random_peer_for_stem()), + }) + } +} diff --git a/p2p/p2p/src/peer_set/client_wrappers.rs b/p2p/p2p/src/peer_set/client_wrappers.rs new file mode 100644 index 00000000..97d74931 --- /dev/null +++ b/p2p/p2p/src/peer_set/client_wrappers.rs @@ -0,0 +1,86 @@ +use std::{ + ops::{Deref, DerefMut}, + sync::{ + atomic::{AtomicBool, Ordering}, + Arc, + }, +}; + +use cuprate_p2p_core::{ + client::{Client, WeakClient}, + NetworkZone, +}; + +/// A client stored in the peer-set. +pub(super) struct StoredClient { + pub client: Client, + /// An [`AtomicBool`] for if the peer is currently downloading blocks. + downloading_blocks: Arc, + /// An [`AtomicBool`] for if the peer is currently being used to stem txs. + stem_peer: Arc, +} + +impl StoredClient { + pub(super) fn new(client: Client) -> Self { + Self { + client, + downloading_blocks: Arc::new(AtomicBool::new(false)), + stem_peer: Arc::new(AtomicBool::new(false)), + } + } + + /// Returns [`true`] if the [`StoredClient`] is currently downloading blocks. + pub(super) fn is_downloading_blocks(&self) -> bool { + self.downloading_blocks.load(Ordering::Relaxed) + } + + /// Returns [`true`] if the [`StoredClient`] is currently being used to stem txs. + pub(super) fn is_a_stem_peer(&self) -> bool { + self.stem_peer.load(Ordering::Relaxed) + } + + /// Returns a [`ClientDropGuard`] that while it is alive keeps the [`StoredClient`] in the downloading blocks state. + pub(super) fn downloading_blocks_guard(&self) -> ClientDropGuard { + self.downloading_blocks.store(true, Ordering::Relaxed); + + ClientDropGuard { + client: self.client.downgrade(), + bool: Arc::clone(&self.downloading_blocks), + } + } + + /// Returns a [`ClientDropGuard`] that while it is alive keeps the [`StoredClient`] in the stemming peers state. + pub(super) fn stem_peer_guard(&self) -> ClientDropGuard { + self.stem_peer.store(true, Ordering::Relaxed); + + ClientDropGuard { + client: self.client.downgrade(), + bool: Arc::clone(&self.stem_peer), + } + } +} + +/// A [`Drop`] guard for a client returned from the peer-set. +pub struct ClientDropGuard { + client: WeakClient, + bool: Arc, +} + +impl Deref for ClientDropGuard { + type Target = WeakClient; + fn deref(&self) -> &Self::Target { + &self.client + } +} + +impl DerefMut for ClientDropGuard { + fn deref_mut(&mut self) -> &mut Self::Target { + &mut self.client + } +} + +impl Drop for ClientDropGuard { + fn drop(&mut self) { + self.bool.store(false, Ordering::Relaxed); + } +} From 4b925b8c78bfce9089a74f12f07ace94cb8b57f7 Mon Sep 17 00:00:00 2001 From: Dmitry Holodov Date: Wed, 20 Nov 2024 19:08:24 -0600 Subject: [PATCH 04/10] ZMQ PUB/SUB JSON Types (#330) --- Cargo.lock | 21 + Cargo.toml | 2 + books/architecture/src/appendix/crates.md | 5 + zmq/types/Cargo.toml | 20 + zmq/types/src/json_message_types.rs | 646 ++++++++++++++++++++++ zmq/types/src/lib.rs | 1 + 6 files changed, 695 insertions(+) create mode 100644 zmq/types/Cargo.toml create mode 100644 zmq/types/src/json_message_types.rs create mode 100644 zmq/types/src/lib.rs diff --git a/Cargo.lock b/Cargo.lock index a947a159..08c017c4 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -68,6 +68,16 @@ version = "0.7.6" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "7c02d123df017efcdfbd739ef81735b36c5ba83ec3c59c80a9d7ecc718f92e50" +[[package]] +name = "assert-json-diff" +version = "2.0.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "47e4f2b81832e72834d7518d8487a0396a28cc408186a2e8854c0f98011faf12" +dependencies = [ + "serde", + "serde_json", +] + [[package]] name = "async-stream" version = "0.3.6" @@ -1012,6 +1022,17 @@ dependencies = [ "thiserror", ] +[[package]] +name = "cuprate-zmq-types" +version = "0.1.0" +dependencies = [ + "assert-json-diff", + "cuprate-types", + "hex", + "serde", + "serde_json", +] + [[package]] name = "cuprated" version = "0.0.1" diff --git a/Cargo.toml b/Cargo.toml index 0f460e8f..1bfd680a 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -30,6 +30,7 @@ members = [ "rpc/json-rpc", "rpc/types", "rpc/interface", + "zmq/types", ] [profile.release] @@ -79,6 +80,7 @@ cuprate-types = { path = "types", default-features = cuprate-json-rpc = { path = "rpc/json-rpc", default-features = false } cuprate-rpc-types = { path = "rpc/types", default-features = false } cuprate-rpc-interface = { path = "rpc/interface", default-features = false } +cuprate-zmq-types = { path = "zmq/types", default-features = false } # External dependencies anyhow = { version = "1", default-features = false } diff --git a/books/architecture/src/appendix/crates.md b/books/architecture/src/appendix/crates.md index ac2780e1..a0dff481 100644 --- a/books/architecture/src/appendix/crates.md +++ b/books/architecture/src/appendix/crates.md @@ -54,6 +54,11 @@ cargo doc --open --package cuprate-blockchain | [`cuprate-rpc-interface`](https://doc.cuprate.org/cuprate_rpc_interface) | [`rpc/interface/`](https://github.com/Cuprate/cuprate/tree/main/rpc/interface) | RPC interface & routing | [`cuprate-rpc-handler`](https://doc.cuprate.org/cuprate_rpc_handler) | [`rpc/handler/`](https://github.com/Cuprate/cuprate/tree/main/rpc/handler) | RPC inner handlers +## ZMQ +| Crate | In-tree path | Purpose | +|-------|--------------|---------| +| [`cuprate-zmq-types`](https://doc.cuprate.org/cuprate_zmq_types) | [`zmq/types/`](https://github.com/Cuprate/cuprate/tree/main/zmq/types) | Message types for ZMQ Pub/Sub interface + ## 1-off crates | Crate | In-tree path | Purpose | |-------|--------------|---------| diff --git a/zmq/types/Cargo.toml b/zmq/types/Cargo.toml new file mode 100644 index 00000000..78e7d00a --- /dev/null +++ b/zmq/types/Cargo.toml @@ -0,0 +1,20 @@ +[package] +name = "cuprate-zmq-types" +version = "0.1.0" +edition = "2021" +description = "Types for the ZMQ Pub/Sub API" +license = "MIT" +authors = ["dimalinux"] +repository = "https://github.com/Cuprate/cuprate/tree/main/zmq/types" + +[dependencies] +serde = { workspace = true, features = ["derive"] } +hex = { workspace = true, features = ["std", "serde"] } +cuprate-types = { workspace = true, features = ["hex"] } + +[dev-dependencies] +serde_json = { workspace = true, features = ["std"] } +assert-json-diff = "2.0.2" + +[lints] +workspace = true diff --git a/zmq/types/src/json_message_types.rs b/zmq/types/src/json_message_types.rs new file mode 100644 index 00000000..2699600f --- /dev/null +++ b/zmq/types/src/json_message_types.rs @@ -0,0 +1,646 @@ +//! Objects for JSON serialization and deserialization in message bodies of +//! the ZMQ pub/sub interface. Handles JSON for the following subscriptions: +//! * `json-full-txpool_add` (`Vec`) +//! * `json-minimal-txpool_add` (`Vec`) +//! * `json-full-chain_main` (`Vec`) +//! * `json-minimal-chain_main` (`ChainMainMin`) +//! * `json-full-miner_data` (`MinerData`) +use cuprate_types::hex::HexBytes; +use serde::{Deserialize, Serialize}; + +/// ZMQ `json-full-txpool_add` packets contain an array of `TxPoolAdd`. +/// +/// Each `TxPoolAdd` object represents a new transaction in the mempool that was +/// not previously seen in a block. Miner coinbase transactions *are not* +/// included. `do-not-relay` transactions *are* included. Values are not +/// republished during a re-org. +#[derive(Debug, Default, Clone, Serialize, Deserialize)] +pub struct TxPoolAdd { + /// transaction version number. `2` indicates Ring CT (all sub-variants). + pub version: u8, + /// if not `0` and less than `500_000_000`, this is the block height when + /// transaction output(s) are spendable; if >= `500_000_000` this is roughly + /// the unix epoch block timestamp when the output(s) are spendable. + pub unlock_time: u64, + /// transaction inputs (key images) with separate rings for each input + pub inputs: Vec, + /// transaction outputs + pub outputs: Vec, + /// extra data for the transaction with variable size, but limited to `1060` + /// bytes (`2120` hex nibbles). + #[serde(with = "hex::serde")] + pub extra: Vec, + /// obsolete, empty array in JSON + signatures: [Obsolete; 0], + /// ring confidential transaction data + pub ringct: PoolRingCt, +} + +/// ZMQ `json-minimal-txpool_add` subscriber messages contain an array of +/// `TxPoolAddMin` JSON objects. See `TxPoolAdd` for information on which +/// transactions are published to subscribers. +#[derive(Debug, Default, Clone, Serialize, Deserialize)] +pub struct TxPoolAddMin { + /// transaction ID + pub id: HexBytes<32>, + /// size of the full transaction blob + pub blob_size: u64, + /// metric used to calculate transaction fee + pub weight: u64, + /// mining fee included in the transaction in piconeros + pub fee: u64, +} + +/// ZMQ `json-full-chain_main` subscriber messages contain an array of +/// `ChainMain` JSON objects. Each `ChainMain` object represents a new block. +/// Push messages only contain more than one block if a re-org occurred. +#[derive(Debug, Serialize, Deserialize)] +pub struct ChainMain { + /// major version of the monero protocol at this block's height + pub major_version: u8, + /// minor version of the monero protocol at this block's height + pub minor_version: u8, + /// epoch time, decided by the miner, at which the block was mined + pub timestamp: u64, + /// block id of the previous block + pub prev_id: HexBytes<32>, + /// cryptographic random one-time number used in mining a Monero block + pub nonce: u32, + /// coinbase transaction information + pub miner_tx: MinerTx, + /// non-coinbase transaction IDs in the block (can be empty) + pub tx_hashes: Vec>, +} + +/// ZMQ `json-minimal-chain_main` subscriber messages contain a single +/// `ChainMainMin` JSON object. Unlike the full version, only the topmost +/// block is sent in the case of a re-org. +#[derive(Debug, Default, Clone, Serialize, Deserialize)] +pub struct ChainMainMin { + /// height of the block + pub first_height: u64, + /// block id of the previous block + pub first_prev_id: HexBytes<32>, + /// block ID of the current block is the 0th entry; additional block IDs + /// will only be included if this is the topmost block of a re-org. + pub ids: Vec>, +} + +/// ZMQ `json-full-miner_data` subscriber messages contain a single +/// `MinerData` object that provides the necessary data to create a +/// custom block template. There is no min version of this object. +#[derive(Debug, Default, Clone, Serialize, Deserialize)] +pub struct MinerData { + /// major version of the monero protocol for the next mined block + pub major_version: u8, + /// height on which to mine + pub height: u64, + /// block id of the most recent block on which to mine the next block + pub prev_id: HexBytes<32>, + /// hash of block to use as seed for Random-X proof-of-work + pub seed_hash: HexBytes<32>, + /// least-significant 64 bits of the 128-bit network difficulty + #[serde(with = "hex_difficulty")] + pub difficulty: u64, + /// median adjusted block size of the latest 100000 blocks + pub median_weight: u64, + /// fixed at `u64::MAX` in perpetuity as Monero has already reached tail emission + pub already_generated_coins: u64, + /// mineable mempool transactions + pub tx_backlog: Vec, +} + +/// Holds a single input for the `TxPoolAdd` `inputs` array. +#[derive(Debug, Default, Clone, Serialize, Deserialize)] +pub struct PoolInput { + pub to_key: ToKey, +} + +/// Same as `PoolInput` (adds an extra JSON name layer) +#[derive(Debug, Default, Clone, Serialize, Deserialize)] +pub struct ToKey { + /// obsolete field (always 0), non-coinbase TX amounts are now encrypted + amount: u64, + /// integer offsets for ring members + pub key_offsets: Vec, + /// key image for the given input + pub key_image: HexBytes<32>, +} + +/// Holds the block height of the coinbase transaction. +#[derive(Debug, Default, Clone, Serialize, Deserialize)] +pub struct MinerInput { + /// namespace layer around the block height + pub r#gen: Gen, +} + +/// Additional namespace layer around the block height in `ChainMain`; gen is +/// another name for a coinbase transaction +#[derive(Debug, Default, Clone, Serialize, Deserialize)] +pub struct Gen { + /// block height when the coinbase transaction was created + pub height: u64, +} + +/// Transaction output data used by both `TxPoolAdd` and `MinerTx` +#[derive(Debug, Default, Clone, Copy, Serialize, Deserialize)] +pub struct Output { + /// zero for non-coinbase transactions which use encrypted amounts or + /// an amount in piconeros for coinbase transactions + pub amount: u64, + /// public key of the output destination + pub to_tagged_key: ToTaggedKey, +} + +/// Holds the public key of an output destination with its view tag. +#[derive(Debug, Default, Clone, Copy, Serialize, Deserialize)] +pub struct ToTaggedKey { + /// public key used to indicate the destination of a transaction output + pub key: HexBytes<32>, + /// 1st byte of a shared secret used to reduce wallet synchronization time + pub view_tag: HexBytes<1>, +} + +/// Ring CT information used inside `TxPoolAdd` +#[derive(Debug, Default, Clone, Serialize, Deserialize)] +pub struct PoolRingCt { + /// ring CT type; `6` is CLSAG Bulletproof Plus + pub r#type: u8, + /// encrypted amount values of the transaction outputs + pub encrypted: Vec, + /// Ring CT commitments, 1 per transaction input + pub commitments: Vec>, + /// mining fee in piconeros + pub fee: u64, + /// data to validate the transaction that can be pruned from older blocks + pub prunable: Prunable, +} + +/// Ring CT information used inside `MinerTx`. Miner coinbase transactions don't +/// use Ring CT, so this only holds a block height. +#[derive(Debug, Default, Clone, Serialize, Deserialize)] +struct MinerRingCt { + /// always zero to indicate that Ring CT is not used + r#type: u8, +} + +/// Holds the encrypted amount of a non-coinbase transaction output. +#[derive(Debug, Default, Clone, Copy, Serialize, Deserialize)] +pub struct Encrypted { + /// obsolete field, but present as zeros in JSON; this does not represent + /// the newer deterministically derived mask + mask: HexBytes<32>, + /// encrypted amount of the transaction output + pub amount: HexBytes<32>, +} + +/// Data needed to validate a transaction that can optionally be pruned from +/// older blocks. +#[derive(Debug, Default, Clone, Serialize, Deserialize)] +pub struct Prunable { + /// obsolete, empty array in JSON + range_proofs: [Obsolete; 0], + /// obsolete, empty array in JSON + bulletproofs: [Obsolete; 0], + /// Bulletproofs+ data used to validate a Ring CT transaction + pub bulletproofs_plus: [BulletproofPlus; 1], + /// obsolete, empty array in JSON + mlsags: [Obsolete; 0], + /// CLSAG signatures; 1 per transaction input + pub clsags: Vec, + /// Ring CT pseudo output commitments; 1 per transaction input (*not* + /// output) + pub pseudo_outs: Vec>, +} + +/// Bulletproofs+ data used to validate the legitimacy of a Ring CT transaction. +#[derive(Debug, Default, Clone, Serialize, Deserialize)] +#[expect(non_snake_case)] +pub struct BulletproofPlus { + pub V: Vec>, + pub A: HexBytes<32>, + pub A1: HexBytes<32>, + pub B: HexBytes<32>, + pub r1: HexBytes<32>, + pub s1: HexBytes<32>, + pub d1: HexBytes<32>, + pub L: Vec>, + pub R: Vec>, +} + +/// Placeholder element type so obsolete fields can be deserialized +/// to the empty vector for backwards compatibility. +#[derive(Debug, Clone, Copy, Serialize, Deserialize)] +struct Obsolete; + +/// CLSAG signature fields +#[expect(non_snake_case)] +#[derive(Debug, Default, Clone, Serialize, Deserialize)] +pub struct Clsag { + pub s: Vec>, + pub c1: HexBytes<32>, + pub D: HexBytes<32>, +} + +/// Part of the new block information in `ChainMain` +#[derive(Debug, Serialize, Deserialize)] +pub struct MinerTx { + /// transaction version number + pub version: u8, + /// block height when the coinbase transaction becomes spendable (currently + /// 60 blocks above the coinbase transaction height) + pub unlock_time: u64, + /// contains the block height in `inputs[0].gen.height` and nothing else as + /// coinbase transactions have no inputs + pub inputs: [MinerInput; 1], + /// transaction outputs + pub outputs: Vec, + /// extra data for the transaction with variable size; not limited to `1060` + /// bytes like the extra field of non-coinbase transactions + #[serde(with = "hex::serde")] + pub extra: Vec, + /// obsolete, empty array in JSON + signatures: [Obsolete; 0], + /// only for JSON compatibility; miners' don't use Ring CT + ringct: MinerRingCt, +} + +/// Holds a transaction entry in the `MinerData` `tx_backlog` field. +#[derive(Debug, Default, Clone, Serialize, Deserialize)] +pub struct TxBacklog { + /// transaction ID + pub id: HexBytes<32>, + /// metric used to calculate transaction fee + pub weight: u64, + /// mining fee in piconeros + pub fee: u64, +} + +mod hex_difficulty { + //! Serializes the u64 difficulty field of `MinerData` in the same ways as + //! monerod. The difficulty value is inside a string, in big-endian hex, and + //! has a 0x prefix with no leading zeros. + use serde::{Deserialize, Deserializer, Serializer}; + + #[expect(clippy::trivially_copy_pass_by_ref)] + pub(super) fn serialize(difficulty: &u64, serializer: S) -> Result + where + S: Serializer, + { + serializer.serialize_str(&format!("0x{difficulty:x}")) + } + + pub(super) fn deserialize<'de, D>(deserializer: D) -> Result + where + D: Deserializer<'de>, + { + let s = String::deserialize(deserializer)?; + let s = s.strip_prefix("0x").unwrap_or(&s); + u64::from_str_radix(s, 16).map_err(serde::de::Error::custom) + } +} + +#[cfg(test)] +mod tests { + use assert_json_diff::assert_json_eq; + use serde_json::{self, json}; + + use super::*; + + #[test] + fn test_txpooladd_json() { + let json1 = json!([ + { + "version": 2, + "unlock_time": 0, + "inputs": [ + { + "to_key": { + "amount": 0, + "key_offsets": [ + 82773133, + 30793552, + 578803, + 620532, + 114291, + 291870, + 111275, + 86455, + 19769, + 1238, + 15164, + 11374, + 5240, + 3547, + 7423, + 4198 + ], + "key_image": "89c060b57bba20c0b795bda4b618749e04eba5b40b30062b071dff6e8dd9071d" + } + } + ], + "outputs": [ + { + "amount": 0, + "to_tagged_key": { + "key": "05b4ff4c3ced6ba078a078af8fee5916512a1893f2b6d9373fb90e0eb4040095", + "view_tag": "7a" + } + }, + { + "amount": 0, + "to_tagged_key": { + "key": "60250376bca49bf24cef45c12738b86347df10954cd35630e81b90bf01e922af", + "view_tag": "b8" + } + } + ], + "extra": "01154b87b3334ce9f99d04635eae4e31252a20ba22acb96ff0764a03dc91d203ed020901be80cbce0723d0b4", + "signatures": [], + "ringct": { + "type": 6, + "encrypted": [ + { + "mask": "0000000000000000000000000000000000000000000000000000000000000000", + "amount": "a956be1858615454000000000000000000000000000000000000000000000000" + }, + { + "mask": "0000000000000000000000000000000000000000000000000000000000000000", + "amount": "72972be61af1210b000000000000000000000000000000000000000000000000" + } + ], + "commitments": [ + "cc2a17e43f0b183235a06e8582fcaaa7c21a07732077e66d4dcfaa0db691ea20", + "04e3cd1d3430bb7a1d9ede5ce9ec0ef2f6f9dd9fd31fb95c9e0b3148f1a660c8" + ], + "fee": 30660000, + "prunable": { + "range_proofs": [], + "bulletproofs": [], + "bulletproofs_plus": [ + { + "V": [ + "0196c1e9ba57ae053ae19c1bfd49e13146bd4b6e49401582f8a5a6f65ae560d0", + "aecd14b0e2d788315023601947c12d7e9227d8a1a0aee41f0b34fe196d96119f" + ], + "A": "8011fb75ba56d16b1ef1193e1fdfdb81e6b83afd726087427163857e8fcdf08e", + "A1": "ab91ab6863fbdee1fb71791e5297d007269f1b2cc050df40628ee7d0a1a5f3cb", + "B": "df1d082111b51d479b7fa72f6363bb731207c9343a528dc05b5798af56702521", + "r1": "2e212ae9ad704611a39b9b242453d2408045b303738b51d6f88f9dba06233401", + "s1": "36be53973fd971edff1f43cc5d04dda78d2b01f4caeaf38bbe195b04e309b30d", + "d1": "592116ca54b2d3ca0e9f222ffcc5fd63d3c992470473911fc70822f37672350a", + "L": [ + "98f1e11d62b90c665a8a96fb1b10332e37a790ea1e01a9e8ec8de74b7b27b0df", + "3a14689f3d743a3be719df9af28ca2f0f398e3a2731d5d6f342d0485bf81a525", + "bcb9e389fd494db66e4c796ff03795daa131426c0776ded6d37bfae51f29623d", + "5aa7e1f2bfcfcd74ac8305ce59a7baf5a901f84f8fbdd3a2d639e4058f35e98b", + "5939aa7ea012f88a26bab20270ce5b164c1880f793dc249ec215a0783b4d4ca7", + "08286f78d1bb0d7fc2efc7a3ac314707a4a1ac9656656d496180e131c1748496", + "7fc1de780305601aab95fda4b005927a4643f222e28407c31ad46cc935b7a27c" + ], + "R": [ + "69b4f329c0a5f8ae05891ac5ac35b947a7442b66e5b5693c99435deac3a62662", + "a193038cb8dc9d22abe6577fe44271c1693176cb636f9d101723670fb5ca5cda", + "90670e7083e503c2989b6548500234740dabf3451b0bd376979e03ca0cb5e50c", + "6ab149089f73799811f631eab272bd6c8f190f38efff4d49577364956d0148bf", + "62f2178cbdc760a0d3787b5fd42161c3c98394c2ff2b88efc039df59d2116e5d", + "536f91da278f730f2524260d2778dc5959d40a5c724dd789d35bbd309eabd933", + "e47c5c8181e692f3ad91733e7d9a52f8b7e3f5016c5e65f789eea367a13f16cd" + ] + } + ], + "mlsags": [], + "clsags": [ + { + "s": [ + "f70840a8d65da85e962d2ce5ed1293ae3de83318b464363db85505d99e317b01", + "b7c1125be139b4ed201ce85b8453920306cac7c5da11e0f8c0fd7702f15c6a06", + "5a04335699f5a816eed1cab79085814dbcf3be5cef51b078b1c3e0210bbba606", + "e4743e114fd6352ea29e0b48ac96688edaba1d5d0634c34301756902eeb1fb0e", + "34aae87ab091082356d2815a7c8e973124245ebc6d163b9f01fbfeb360edcf04", + "d2d0b6ddb44ed42096affec08ea9cd77d2c7cdc5b2e1e964f836d3717640ec00", + "79b34258c8be04ddd955389f7ee3b912286c23492c519a5687b81d770619620e", + "3c889c19693463160d6c7e642c46f5d41db052ee3358c7dcb4826f48bca26607", + "da04927a438fd0d9674e64f0c016f30fde27f251d3466f29dcd5b3d757fec90c", + "f3e08d83b11ca6529bc18748d3f732c325fca8ff79f69f0ed754bcd529898102", + "f00d7125909a9a8cc5283ffc7727fce945e85828459eecb836c7aedca414350e", + "0a635a193af37be1c9519309f25eaf9f37b7bc5892864646d8d2a2187fcec601", + "0c4154d575dff3699bd41f0c354601de6535161755bd2164526076f37e2c6908", + "f7b21e2698333285ea10a95edbe80fe0bb8740c30b35c25bd2002e3693867e02", + "a637f338ff2ed65fa96e5529abc575fc2a35ed1a3f62a9e7be495069d8438800", + "f7c355f1c3a663978c5fe1c9337aabd4085ee537a61eec2c5c1e837cb3728c09" + ], + "c1": "c5dd25e0e32dbefa6ac1d0dc9072620eb97a99224462cdd163287f2b60b9810b", + "D": "c4fa3f939ccf02e4c8842cbd417cf3690421986e558734a0a029f8a86d2791a8" + } + ], + "pseudo_outs": [ + "bcb08920f5476d74294aeb89c8001123bffd2f2ab84e105d553b807674c595ce" + ] + } + } + } + ]); + + let tx_pool_adds: Vec = serde_json::from_value(json1.clone()).unwrap(); + let json2 = serde_json::to_value(&tx_pool_adds).unwrap(); + assert_json_eq!(json1, json2); + } + + #[test] + fn test_txpooladd_min_json() { + let json1 = json!([ + { + "id": "b5086746e805d875cbbbbb49e19aac29d9b75019f656fab8516cdf64ac5cd346", + "blob_size": 1533, + "weight": 1533, + "fee": 30660000 + } + ]); + + let tx_pool_adds: Vec = serde_json::from_value(json1.clone()).unwrap(); + let json2 = serde_json::to_value(&tx_pool_adds).unwrap(); + assert_json_eq!(json1, json2); + } + + #[test] + fn test_chain_main_json() { + let json1 = json!([ + { + "major_version": 16, + "minor_version": 16, + "timestamp": 1726973843, + "prev_id": "ce3731311b7e4c1e58a2fe902dbb5c60bb2c0decc163d5397fa52a260d7f09c1", + "nonce": 537273946, + "miner_tx": { + "version": 2, + "unlock_time": 3242818, + "inputs": [ + { + "gen": { + "height": 3242758 + } + } + ], + "outputs": [ + { + "amount": 618188180000_u64, + "to_tagged_key": { + "key": "83faf44df7e9fb4cf54a8dd6a63868507d1a1896bdb35ea9110d739d5da6cf21", + "view_tag": "38" + } + } + ], + "extra": "010e3356a86dbb339354afbc693408dfe8648bffd0b276e6a431861eb73643d88d02115162e362c98e2d00000000000000000000", + "signatures": [], + "ringct": { + "type": 0 + } + }, + "tx_hashes": [ + "2c1b67d3f10b21270cac116e6d5278dc4024ee2d727e4ad56d6dedb1abc0270c", + "c2cfec0de23229a2ab80ca464cef66fc1cad53647a444f048834ec236c38c867", + "03c7649af2373c0f739d3c2eff9ee1580986b460d2abdd5e2aa332281e52da7e", + "1e0834cc658599e786040bdcd9b589a5e8d975233b72279d04ece1a3dd5572b0", + "ba65c30150e906a8799ee99bb2e6481873e42ed8b025cf967c5798528ddc81b4", + "6fc7b1da1cf433edafb142173e9ac13fe05142a36d8a72e9efdf7a3b94da11d6", + "847c06dcda4540d45cae868d4d031781bd87d9bfa4b2186a611428f52e68ccee", + "79f87a1b2fc17295d2cf25b6a65dd17fd8630829ee50f9c48f15e4a24e72d872", + "32b4f7ce6d864006b274dbd73fc8058151d0fd2dd0bb4b423120e32451fd59eb", + "430fe7fa00b63b68b301a4e4810bef2b5be1f651dba8c360e86eac61227382e7", + "9f8d2bf5e39071abccb336404ea72ab85cb731500a1d386a3bf537b9046df29d", + "f63893b8226ca28c290cb65541d60c1675dda1e2a77a629b6b2d7c3886240b23", + "ee8608b6e80cce14beaf95f747f4da8e40e40a49ad1adc20038843a6da3df3c6", + "05783765c150ed1e46d6380726e7ca1f788305754e553f5f2d49b9f09aaaf88d", + "20b4b95e62f45b72014d6ab14edb0b31e273cdc8c8d106068dd32ef6e92fc0a2", + "9230fb0a9dce8e2ca7e109ebf3480838251691de8ed73ea91f74723c5cf19bac", + "d59cf84a25f56ec0f1352bb05645efe9b9326598c4f7c5bc39a87eb7a20c48fc", + "465deb73c48a460df71861d61666dabb906648035a1fecfd0e988ee37616c655", + "5767bc633729ba4555561510f3db739431b16744234dcd549a0d346eaa6685b1", + "2c8d9af5d5774de96e67835ac5adbc6ca5579125b08bc907b395645eea6410ec", + "d385c884a0687c3360725dd3a3f6acf6f64bf38d8eeea1644d80bc23b13ee870", + "b2bc7e9fa9c1da08a8b6ee58505611c05bc388fd30aece00e9a0041470f7e950", + "69a4a79b50d42d372e91c6608c2652d1d5ddd343526c387ef6cf1e3c158b1765", + "ef508dfa79bbedd226835c42a9d000a64cc4abe0250c9aa55fd968224e2b45c3", + "0413c3b3fc621c472e10a102d77456db506f0df10a909833aed0c6738fb31eeb", + "e0c52d6d649c2f1abce4c6ffce4dd75a23308afbb6abe33af53da232c40caf5f", + "cd1fd68d2a15002ca6236083ff6ae165c8fd922f410da79640a4342fd8ebd1c8", + "ba746f80ca4ee496f4188ba278f1ed69a913238450d52bd2e2f3d3bf6fdd43d3", + "13c964bc13a55621b7bbbfe9a6d703536d951bfa19eedee93dd1286020959021", + "41a6f8d0df227a401a9bd6f5c0fbc21ed89f515ea5c8434a087e8b880080ee1f", + "41c2b5994284790b1ee158f7b87aa1231c14975d6456a91ff6f93c6f81277965", + "7e6b7f169cc6cab88e652771157cf8c2eb6f69dffb6939a79b34c6554fe6c00b", + "619517d9d138bf95c6b77eb801526b8419616de2b8618ccfd3b6d1c10364bc64", + "52cca64fb20fc2f6d06034a1a2d9b5665972ebc2569ec69f8d473caada309add", + "219c106d09da5a27b339ea0f070da090779b31ef9ccfa90d6d25e7388341eff9", + "e07ce6e96e73cff80c9cc4c1b349ad1ef53cff210b876d4e7afd89fcc8b2e5dd", + "e98f2a00b2892cd65c0252d956d88a4bb8024c7db98ca003c127b097f097f276", + "ed81aa398071fe495e37095e51ff50053e132bd11f27ba9c06ac4bf4063b756f", + "667d29a0cefa311e06fcfc22c98ef75edf81deb6c8a812492eb255a049c826db", + "8b16e8cbc1765247456bd67a3106498f686401b7529dc0f6b03360caf8671135", + "013e443e63259748f6d1a5653374826618ba066b7febcf55c829333f0dd9a6c3", + "517a05d82de59a973eb4d343c45558841c9165ccd75ca7c9d2e1a35f80c26c15", + "af74d5dd44cfed8f40f853a6fc405dae23d547482296f8dbbc13c1aed2c3d8c5", + "b5086746e805d875cbbbbb49e19aac29d9b75019f656fab8516cdf64ac5cd346", + "cfcda18d058656797a1272b384774dcfc26a504a24298aa49ba060eb6b4a19e0", + "1f380660a99030cc45f85ba8ee0e0541035c0fde719c84aa692796328974c9dd", + "53127181a0301a27b3a2749dc997556b211d949a99aa34d1c52d5c54220f49d2", + "5d50a66df97f4decc4ecc3f5030589ef966d5af84a995f7fb14f1c02ae9704db", + "cdab9628acdb57c460e292660e7a07caf2ddbcffdfff92f3e5e4fb12119a11ca", + "e740a098a74d7a66a821c4ac3c5f913a82fc7445b5593cc5fa3e48ad1b4589b1", + "760549176fec210cfe0ff58eabbf2670cf33b4cd3942a3b60a98bf8f328a6d01", + "961b0956aa6303ed8ca1687d93ed46b9aa8a0203ec4ce0cbc2e86b364fbfb613", + "b9db041b2c3bfc6b5b0facb638b0b4643eec76b060039a6b11fb43682ed77a97", + "1011c321eb386b9975e8124bdb130790dcf4ac0021da3103cabbf7dfa18ccea7", + "6a9d3d15be4b25bd544d96bb1d7685e53f9484735bb22994feffb9037009aeeb", + "bf20d6193890cf7fdead9e3b60197564c663b5a62eda782a49d4aa7819bb9665", + "472d28f9d25a95e625eb808ff3827e7f6792009e1ba0b3b21951f3058b65a75d", + "e3931b2b66da07f983d2235d9d0b3a3098008458bdc0c1ad4370fae73e1eaa9e", + "e18a0dea6382c95aa4089a971190683b171e9405c06fd4111924144600f3bcf3", + "1a336bcf24026307821b76b9ca18b178c285c591c5df9906e3ffbd2050ccd356", + "8ca2d0e5ae9b9981bb8b76ba0da383c585664b2a2f4e861d58aab00c9b0cc808", + "e1866c27023ccea276034c4d572eab42713132e4fdb2aafa9488f6d74cd49303", + "3674cfafba4cdea5775a72a82e5d553bf180beab456b3cbaa7b41a1574fe1948", + "9bb400dd317425f40176c3094a5573037b0217e0b60761cb66a8fa15b63b36c3", + "c078048028aca3e9bc40f68f4d42ef25c6af2cef4da20bf3be70dd6a23b82d52", + "c28cc85f945085e70259ed02131ae3f8c5992e789c9c75c2c6e257306beaf26e", + "4c2b121795fe2b90fda84813543952382daa29c7b96edd9f96040df13e48e347", + "63c6fba30b5471fd60e715cbaf4448badafde68dbc42c54d96b56dd2c4bf2d15", + "a4240138ecfe736113581f318f261a01992eaa8fa5b7bd6938d9dbeb65aa85d7", + "b9d088a7b21f655d0cf50f8404e874f4d1655fb5565a354d2c0dd6d113619c66", + "9133e7e98a83f6e10a7fd44c104d9124d93e0d3e920f5c160873b394dd3a2fcb", + "953985dbd0ea6f86746e83be144ec2ff2897ef1f3506eede083b893e98dd63ea", + "83af840c4cad46de96c86fcf700ade32e73260d4a16cefa330cb5a722ef59fdf", + "eea3c0c2b016ea0c269f954fd8172c3d118f08103c9842b81b05290c9faf3780", + "ac43a363fdb81fa4f6df1cb06ba49a5f4eeef411957cf2afad55cbc1e79bc4d1", + "ca72cf7bda22aed15c16ca67e7b6cc57109cdc86d4ffe38fd71210a5380fcada", + "477dc1cd62106d9df6b37f8515579a48d01b310387087c08ce7062a8eb5df98d", + "d47b6dcd3b13288825c954df6c6e30eb683d1f79434beaee7172082f8ae74280", + "9c64ef20c69589c56fcc5f3a0d10f6957ecea248e44acb432aaf16a88eeef946", + "d2aa256bfd61bdb64ac38da6cbc3e77fb315bb9fbaf422087c10345377df44f6", + "8b9623e4513594a6eaeb3475ea7d0eb585dd8f6e20e21c316db0b942fada2336", + "860725ed0bd18c744e6b8b02888ad88be1cf23d7153131b220a0f9fbb76976bf", + "387cc6e807efc263a0ad6a30e6313a27d16abef038264d0afa0e6ad943be55da" + ] + } + ]); + + let chain_main: Vec = serde_json::from_value(json1.clone()).unwrap(); + let json2 = serde_json::to_value(&chain_main).unwrap(); + assert_json_eq!(json1, json2); + } + + #[test] + fn test_chain_main_min_json() { + let json1 = json!({ + "first_height": 3242758, + "first_prev_id": "ce3731311b7e4c1e58a2fe902dbb5c60bb2c0decc163d5397fa52a260d7f09c1", + "ids": [ + "ee1238b884e64f7e438223aa8d42d0efc15e7640f1a432448fbad116dc72f1b2" + ] + }); + + let chain_main_min: ChainMainMin = serde_json::from_value(json1.clone()).unwrap(); + let json2 = serde_json::to_value(&chain_main_min).unwrap(); + assert_json_eq!(json1, json2); + } + + #[test] + fn test_miner_data_json() { + let json1 = json!({ + "major_version": 16, + "height": 3242764, + "prev_id": "dc53c24683dca14586fb2909b9aa4a44adb524e010d438e2491e7d8cc1c80831", + "seed_hash": "526577d6e6689ba8736c16ccc76e6ce4ada3b0ceeaa3a2260b96ba188a17d705", + "difficulty": "0x526f2623ce", + "median_weight": 300000, + "already_generated_coins": 18446744073709551615_u64, + "tx_backlog": [ + { + "id": "dbec64651bb4e83d0e9a05c2826bde605a940f12179fab0ab5dc8bc4392c776b", + "weight": 2905, + "fee": 929600000 + }, + { + "id": "ec5728dd1fbd98db1f93d612826e73b95f52cca49f247a6dbc35390f45766a7d", + "weight": 2222, + "fee": 44440000 + }, + { + "id": "41f613b1a470af494e0a705993e305dfaad3e365fcc0b0db0118256fc54559aa", + "weight": 2221, + "fee": 44420000 + }, + { + "id": "34fa33bf96dc2f825fe870e8f5402be6225c1623b345224e0dbc38b6407873de", + "weight": 2217, + "fee": 709440000 + } + ] + }); + + let miner_data: MinerData = serde_json::from_value(json1.clone()).unwrap(); + let json2 = serde_json::to_value(&miner_data).unwrap(); + assert_json_eq!(json1, json2); + } +} diff --git a/zmq/types/src/lib.rs b/zmq/types/src/lib.rs new file mode 100644 index 00000000..3f9562b6 --- /dev/null +++ b/zmq/types/src/lib.rs @@ -0,0 +1 @@ +pub mod json_message_types; From f3c1a5c2aa4629bf69b75268de21fc9112f09405 Mon Sep 17 00:00:00 2001 From: hinto-janai Date: Thu, 21 Nov 2024 17:32:48 -0500 Subject: [PATCH 05/10] Fix #346 (#347) fixes --- cryptonight/src/util.rs | 2 +- storage/blockchain/src/ops/block.rs | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/cryptonight/src/util.rs b/cryptonight/src/util.rs index 7fbf5cb4..de8b70b2 100644 --- a/cryptonight/src/util.rs +++ b/cryptonight/src/util.rs @@ -49,7 +49,7 @@ pub(crate) fn subarray_copy + ?Sized, U: Copy, const LEN: usize>( /// A mutable reference to a fixed-size subarray of type `[U; LEN]`. /// /// # Panics -/// Panics if `start + LEN > array.as_ref().len()`. +/// Panics if `start + LEN > array.as_mut().len()`. #[inline] pub(crate) fn subarray_mut + ?Sized, U, const LEN: usize>( array: &mut T, diff --git a/storage/blockchain/src/ops/block.rs b/storage/blockchain/src/ops/block.rs index 6d32fd81..cc5cb807 100644 --- a/storage/blockchain/src/ops/block.rs +++ b/storage/blockchain/src/ops/block.rs @@ -42,7 +42,7 @@ use crate::{ /// # Panics /// This function will panic if: /// - `block.height > u32::MAX` (not normally possible) -/// - `block.height` is not != [`chain_height`] +/// - `block.height` is != [`chain_height`] // no inline, too big. pub fn add_block( block: &VerifiedBlockInformation, From caa08d5eaa1063919797a830eb0d9358d2a72b80 Mon Sep 17 00:00:00 2001 From: hinto-janai Date: Mon, 25 Nov 2024 15:10:42 -0500 Subject: [PATCH 06/10] benches: initial implementation (#196) * add readme * readme, basic examples * name changes, bin impl * example, docs * book * add `cuprate-criterion-example` * docs, tracing * fix clippy * docs * lib readme * json-rpc benchmarks * add to crates.md * add `fixme` * fix `cargo b` failing this `cfg()` existing makes a regular workspace `cargo b` fail * fix cargo.toml --- Cargo.lock | 313 +++++++++++++++++- Cargo.toml | 95 ++++-- benches/README.md | 6 +- benches/benchmark/bin/Cargo.toml | 43 +++ benches/benchmark/bin/README.md | 27 ++ benches/benchmark/bin/src/log.rs | 29 ++ benches/benchmark/bin/src/main.rs | 49 +++ benches/benchmark/bin/src/print.rs | 38 +++ benches/benchmark/bin/src/run.rs | 36 ++ benches/benchmark/bin/src/timings.rs | 5 + benches/benchmark/example/Cargo.toml | 17 + benches/benchmark/example/README.md | 3 + benches/benchmark/example/src/lib.rs | 42 +++ benches/benchmark/lib/Cargo.toml | 18 + benches/benchmark/lib/README.md | 15 + benches/benchmark/lib/src/benchmark.rs | 45 +++ benches/benchmark/lib/src/lib.rs | 5 + benches/criterion/cuprate-json-rpc/Cargo.toml | 23 ++ .../cuprate-json-rpc/benches/main.rs | 8 + .../cuprate-json-rpc/benches/response.rs | 110 ++++++ benches/criterion/cuprate-json-rpc/src/lib.rs | 2 + benches/criterion/example/Cargo.toml | 21 ++ benches/criterion/example/README.md | 14 + benches/criterion/example/benches/example.rs | 48 +++ benches/criterion/example/benches/main.rs | 10 + benches/criterion/example/src/lib.rs | 13 + books/architecture/src/SUMMARY.md | 13 +- books/architecture/src/appendix/crates.md | 8 + .../src/benchmarking/criterion.md | 1 - .../src/benchmarking/criterion/creating.md | 21 ++ .../src/benchmarking/criterion/intro.md | 4 + .../src/benchmarking/criterion/running.md | 15 + .../src/benchmarking/cuprate/creating.md | 57 ++++ .../src/benchmarking/cuprate/intro.md | 37 +++ .../src/benchmarking/cuprate/running.md | 16 + .../architecture/src/benchmarking/harness.md | 1 - books/architecture/src/benchmarking/intro.md | 23 +- 37 files changed, 1188 insertions(+), 43 deletions(-) create mode 100644 benches/benchmark/bin/Cargo.toml create mode 100644 benches/benchmark/bin/README.md create mode 100644 benches/benchmark/bin/src/log.rs create mode 100644 benches/benchmark/bin/src/main.rs create mode 100644 benches/benchmark/bin/src/print.rs create mode 100644 benches/benchmark/bin/src/run.rs create mode 100644 benches/benchmark/bin/src/timings.rs create mode 100644 benches/benchmark/example/Cargo.toml create mode 100644 benches/benchmark/example/README.md create mode 100644 benches/benchmark/example/src/lib.rs create mode 100644 benches/benchmark/lib/Cargo.toml create mode 100644 benches/benchmark/lib/README.md create mode 100644 benches/benchmark/lib/src/benchmark.rs create mode 100644 benches/benchmark/lib/src/lib.rs create mode 100644 benches/criterion/cuprate-json-rpc/Cargo.toml create mode 100644 benches/criterion/cuprate-json-rpc/benches/main.rs create mode 100644 benches/criterion/cuprate-json-rpc/benches/response.rs create mode 100644 benches/criterion/cuprate-json-rpc/src/lib.rs create mode 100644 benches/criterion/example/Cargo.toml create mode 100644 benches/criterion/example/README.md create mode 100644 benches/criterion/example/benches/example.rs create mode 100644 benches/criterion/example/benches/main.rs create mode 100644 benches/criterion/example/src/lib.rs delete mode 100644 books/architecture/src/benchmarking/criterion.md create mode 100644 books/architecture/src/benchmarking/criterion/creating.md create mode 100644 books/architecture/src/benchmarking/criterion/intro.md create mode 100644 books/architecture/src/benchmarking/criterion/running.md create mode 100644 books/architecture/src/benchmarking/cuprate/creating.md create mode 100644 books/architecture/src/benchmarking/cuprate/intro.md create mode 100644 books/architecture/src/benchmarking/cuprate/running.md delete mode 100644 books/architecture/src/benchmarking/harness.md diff --git a/Cargo.lock b/Cargo.lock index 08c017c4..ac36c565 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -29,6 +29,15 @@ dependencies = [ "zerocopy", ] +[[package]] +name = "aho-corasick" +version = "1.1.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8e60d3430d3a69478ad0993f19238d2df97c507009a52b3c10addcd7f6bcb916" +dependencies = [ + "memchr", +] + [[package]] name = "android-tzdata" version = "0.1.1" @@ -44,6 +53,12 @@ dependencies = [ "libc", ] +[[package]] +name = "anes" +version = "0.1.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4b46cbb362ab8752921c97e041f5e366ee6297bd428a31275b9fcf1e380f7299" + [[package]] name = "anstyle" version = "1.0.10" @@ -347,6 +362,12 @@ dependencies = [ "serde", ] +[[package]] +name = "cast" +version = "0.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "37b2a672a2cb129a2e41c10b1224bb368f9f37a2b16b612598138befd7b37eb5" + [[package]] name = "cc" version = "1.1.31" @@ -380,6 +401,33 @@ dependencies = [ "windows-targets 0.52.6", ] +[[package]] +name = "ciborium" +version = "0.2.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "42e69ffd6f0917f5c029256a24d0161db17cea3997d185db0d35926308770f0e" +dependencies = [ + "ciborium-io", + "ciborium-ll", + "serde", +] + +[[package]] +name = "ciborium-io" +version = "0.2.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "05afea1e0a06c9be33d539b876f1ce3692f4afea2cb41f740e7743225ed1c757" + +[[package]] +name = "ciborium-ll" +version = "0.2.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "57663b653d948a338bfb3eeba9bb2fd5fcfaecb9e199e87e1eda4d9e8b240fd9" +dependencies = [ + "ciborium-io", + "half", +] + [[package]] name = "clap" version = "4.5.20" @@ -478,6 +526,42 @@ dependencies = [ "cfg-if", ] +[[package]] +name = "criterion" +version = "0.5.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f2b12d017a929603d80db1831cd3a24082f8137ce19c69e6447f54f5fc8d692f" +dependencies = [ + "anes", + "cast", + "ciborium", + "clap", + "criterion-plot", + "is-terminal", + "itertools", + "num-traits", + "once_cell", + "oorandom", + "plotters", + "rayon", + "regex", + "serde", + "serde_derive", + "serde_json", + "tinytemplate", + "walkdir", +] + +[[package]] +name = "criterion-plot" +version = "0.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6b50826342786a51a89e2da3a28f1c32b06e387201bc2d19791f622c673706b1" +dependencies = [ + "cast", + "itertools", +] + [[package]] name = "crossbeam" version = "0.8.4" @@ -534,6 +618,12 @@ version = "0.8.20" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "22ec99545bb0ed0ea7bb9b8e1e9122ea386ff8a48c0922e43f36d45ab09e0e80" +[[package]] +name = "crunchy" +version = "0.2.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7a81dae078cea95a014a339291cec439d2f232ebe854a9d672b796c6afafa9b7" + [[package]] name = "crypto-bigint" version = "0.5.5" @@ -583,6 +673,30 @@ dependencies = [ "tokio", ] +[[package]] +name = "cuprate-benchmark" +version = "0.0.0" +dependencies = [ + "cfg-if", + "cuprate-benchmark-example", + "cuprate-benchmark-lib", + "serde", + "serde_json", + "tracing", + "tracing-subscriber", +] + +[[package]] +name = "cuprate-benchmark-example" +version = "0.0.0" +dependencies = [ + "cuprate-benchmark-lib", +] + +[[package]] +name = "cuprate-benchmark-lib" +version = "0.0.0" + [[package]] name = "cuprate-blockchain" version = "0.0.0" @@ -685,6 +799,25 @@ dependencies = [ name = "cuprate-constants" version = "0.1.0" +[[package]] +name = "cuprate-criterion-example" +version = "0.0.0" +dependencies = [ + "criterion", + "function_name", + "serde_json", +] + +[[package]] +name = "cuprate-criterion-json-rpc" +version = "0.0.0" +dependencies = [ + "criterion", + "cuprate-json-rpc", + "function_name", + "serde_json", +] + [[package]] name = "cuprate-cryptonight" version = "0.1.0" @@ -1304,6 +1437,21 @@ dependencies = [ "percent-encoding", ] +[[package]] +name = "function_name" +version = "0.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b1ab577a896d09940b5fe12ec5ae71f9d8211fff62c919c03a3750a9901e98a7" +dependencies = [ + "function_name-proc-macro", +] + +[[package]] +name = "function_name-proc-macro" +version = "0.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "673464e1e314dd67a0fd9544abc99e8eb28d0c7e3b69b033bcff9b2d00b87333" + [[package]] name = "funty" version = "2.0.0" @@ -1453,6 +1601,16 @@ dependencies = [ "tracing", ] +[[package]] +name = "half" +version = "2.4.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6dd08c532ae367adf81c312a4580bc67f1d0fe8bc9c460520283f4c0ff277888" +dependencies = [ + "cfg-if", + "crunchy", +] + [[package]] name = "hashbrown" version = "0.14.5" @@ -1680,6 +1838,26 @@ dependencies = [ "hashbrown 0.15.0", ] +[[package]] +name = "is-terminal" +version = "0.4.12" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f23ff5ef2b80d608d61efee834934d862cd92461afc0560dedf493e4c033738b" +dependencies = [ + "hermit-abi", + "libc", + "windows-sys 0.52.0", +] + +[[package]] +name = "itertools" +version = "0.10.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b0fd2260e829bddf4cb6ea802289de2f86d6a7a690192fbe91b3f46e0f2c8473" +dependencies = [ + "either", +] + [[package]] name = "itoa" version = "1.0.11" @@ -1776,6 +1954,15 @@ version = "0.4.22" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "a7a70ba024b9dc04c27ea2f0c0548feb474ec5c54bba33a7f72f873a39d07b24" +[[package]] +name = "matchers" +version = "0.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8263075bb86c5a1b1427b5ae862e8889656f126e9f77c484496e8b47cf5c5558" +dependencies = [ + "regex-automata 0.1.10", +] + [[package]] name = "matchit" version = "0.7.3" @@ -2031,6 +2218,12 @@ version = "1.20.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "1261fe7e33c73b354eab43b1273a57c8f967d0391e80353e51f764ac02cf6775" +[[package]] +name = "oorandom" +version = "11.1.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0ab1bc2a289d34bd04a330323ac98a1b4bc82c9d9fcb1e66b63caa84da26b575" + [[package]] name = "openssl-probe" version = "0.1.5" @@ -2168,6 +2361,34 @@ version = "0.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "8b870d8c151b6f2fb93e84a13146138f05d02ed11c7e7c54f8826aaaf7c9f184" +[[package]] +name = "plotters" +version = "0.3.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a15b6eccb8484002195a3e44fe65a4ce8e93a625797a063735536fd59cb01cf3" +dependencies = [ + "num-traits", + "plotters-backend", + "plotters-svg", + "wasm-bindgen", + "web-sys", +] + +[[package]] +name = "plotters-backend" +version = "0.3.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "414cec62c6634ae900ea1c56128dfe87cf63e7caece0852ec76aba307cebadb7" + +[[package]] +name = "plotters-svg" +version = "0.3.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "81b30686a7d9c3e010b84284bdd26a29f2138574f52f5eb6f794fc0ad924e705" +dependencies = [ + "plotters-backend", +] + [[package]] name = "ppv-lite86" version = "0.2.20" @@ -2242,7 +2463,7 @@ dependencies = [ "rand", "rand_chacha", "rand_xorshift", - "regex-syntax", + "regex-syntax 0.8.5", "rusty-fork", "tempfile", "unarray", @@ -2408,6 +2629,44 @@ dependencies = [ "syn", ] +[[package]] +name = "regex" +version = "1.10.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b91213439dad192326a0d7c6ee3955910425f441d7038e0d6933b0aec5c4517f" +dependencies = [ + "aho-corasick", + "memchr", + "regex-automata 0.4.7", + "regex-syntax 0.8.5", +] + +[[package]] +name = "regex-automata" +version = "0.1.10" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6c230d73fb8d8c1b9c0b3135c5142a8acee3a0558fb8db5cf1cb65f8d7862132" +dependencies = [ + "regex-syntax 0.6.29", +] + +[[package]] +name = "regex-automata" +version = "0.4.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "38caf58cc5ef2fed281f89292ef23f6365465ed9a41b7a7754eb4e26496c92df" +dependencies = [ + "aho-corasick", + "memchr", + "regex-syntax 0.8.5", +] + +[[package]] +name = "regex-syntax" +version = "0.6.29" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f162c6dd7b008981e4d40210aca20b4bd0f9b60ca9271061b07f78537722f2e1" + [[package]] name = "regex-syntax" version = "0.8.5" @@ -2535,6 +2794,15 @@ version = "1.0.18" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "f3cb5ba0dc43242ce17de99c180e96db90b235b8a9fdc9543c96d2209116bd9f" +[[package]] +name = "same-file" +version = "1.0.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "93fc1dc3aaa9bfed95e02e6eadabb4baf7e3078b0bd1b4d7b6b0b68378900502" +dependencies = [ + "winapi-util", +] + [[package]] name = "schannel" version = "0.1.26" @@ -2889,6 +3157,16 @@ version = "0.5.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "a693d0c8cf16973fac5a93fbe47b8c6452e7097d4fcac49f3d7a18e39c76e62e" +[[package]] +name = "tinytemplate" +version = "1.2.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "be4d6b5f19ff7664e8c98d03e2139cb510db9b0a60b55f8e8709b689d939b6bc" +dependencies = [ + "serde", + "serde_json", +] + [[package]] name = "tinyvec" version = "1.8.0" @@ -3105,10 +3383,14 @@ version = "0.3.18" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "ad0f048c97dbd9faa9b7df56362b8ebcaa52adb06b498c050d2f4e32f90a7a8b" dependencies = [ + "matchers", "nu-ansi-term", + "once_cell", + "regex", "sharded-slab", "smallvec", "thread_local", + "tracing", "tracing-core", "tracing-log", ] @@ -3214,6 +3496,16 @@ dependencies = [ "libc", ] +[[package]] +name = "walkdir" +version = "2.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "29790946404f91d9c5d06f9874efddea1dc06c5efe94541a7d6863108e3a5e4b" +dependencies = [ + "same-file", + "winapi-util", +] + [[package]] name = "want" version = "0.3.1" @@ -3284,6 +3576,16 @@ version = "0.2.95" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "65fc09f10666a9f147042251e0dda9c18f166ff7de300607007e96bdebc1068d" +[[package]] +name = "web-sys" +version = "0.3.69" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "77afa9a11836342370f4817622a2f0f418b134426d91a82dfb48f532d2ec13ef" +dependencies = [ + "js-sys", + "wasm-bindgen", +] + [[package]] name = "webpki-roots" version = "0.26.6" @@ -3309,6 +3611,15 @@ version = "0.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "ac3b87c63620426dd9b991e5ce0329eff545bccbbb34f3be09ff6fb6ab51b7b6" +[[package]] +name = "winapi-util" +version = "0.1.8" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4d4cc384e1e73b93bafa6fb4f1df8c41695c8a91cf9c4c64358067d15a7b6c6b" +dependencies = [ + "windows-sys 0.52.0", +] + [[package]] name = "winapi-x86_64-pc-windows-gnu" version = "0.4.0" diff --git a/Cargo.toml b/Cargo.toml index 1bfd680a..a507631e 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -1,36 +1,57 @@ [workspace] resolver = "2" - members = [ + # Binaries "binaries/cuprated", - "constants", + + # Benchmarks + "benches/benchmark/bin", + "benches/benchmark/lib", + "benches/benchmark/example", + "benches/criterion/example", + "benches/criterion/cuprate-json-rpc", + + # Consensus "consensus", "consensus/context", "consensus/fast-sync", "consensus/rules", - "cryptonight", - "helper", + + # Net "net/epee-encoding", "net/fixed-bytes", "net/levin", "net/wire", + + # P2P "p2p/p2p", "p2p/p2p-core", "p2p/bucket", "p2p/dandelion-tower", "p2p/async-buffer", "p2p/address-book", + + # Storage "storage/blockchain", "storage/service", "storage/txpool", "storage/database", - "pruning", - "test-utils", - "types", + + # RPC "rpc/json-rpc", "rpc/types", "rpc/interface", + + # ZMQ "zmq/types", + + # Misc + "constants", + "cryptonight", + "helper", + "pruning", + "test-utils", + "types", ] [profile.release] @@ -53,34 +74,36 @@ opt-level = 3 [workspace.dependencies] # Cuprate members -cuprate-fast-sync = { path = "consensus/fast-sync", default-features = false } -cuprate-consensus-rules = { path = "consensus/rules", default-features = false } -cuprate-constants = { path = "constants", default-features = false } -cuprate-consensus = { path = "consensus", default-features = false } -cuprate-consensus-context = { path = "consensus/context", default-features = false } -cuprate-cryptonight = { path = "cryptonight", default-features = false } -cuprate-helper = { path = "helper", default-features = false } -cuprate-epee-encoding = { path = "net/epee-encoding", default-features = false } -cuprate-fixed-bytes = { path = "net/fixed-bytes", default-features = false } -cuprate-levin = { path = "net/levin", default-features = false } -cuprate-wire = { path = "net/wire", default-features = false } -cuprate-p2p = { path = "p2p/p2p", default-features = false } -cuprate-p2p-core = { path = "p2p/p2p-core", default-features = false } -cuprate-p2p-bucket = { path = "p2p/p2p-bucket", default-features = false } -cuprate-dandelion-tower = { path = "p2p/dandelion-tower", default-features = false } -cuprate-async-buffer = { path = "p2p/async-buffer", default-features = false } -cuprate-address-book = { path = "p2p/address-book", default-features = false } -cuprate-blockchain = { path = "storage/blockchain", default-features = false } -cuprate-database = { path = "storage/database", default-features = false } -cuprate-database-service = { path = "storage/service", default-features = false } -cuprate-txpool = { path = "storage/txpool", default-features = false } -cuprate-pruning = { path = "pruning", default-features = false } -cuprate-test-utils = { path = "test-utils", default-features = false } -cuprate-types = { path = "types", default-features = false } -cuprate-json-rpc = { path = "rpc/json-rpc", default-features = false } -cuprate-rpc-types = { path = "rpc/types", default-features = false } -cuprate-rpc-interface = { path = "rpc/interface", default-features = false } -cuprate-zmq-types = { path = "zmq/types", default-features = false } +cuprate-benchmark-lib = { path = "benches/benchmark/lib", default-features = false } +cuprate-benchmark-example = { path = "benches/benchmark/example", default-features = false } +cuprate-fast-sync = { path = "consensus/fast-sync", default-features = false } +cuprate-consensus-rules = { path = "consensus/rules", default-features = false } +cuprate-constants = { path = "constants", default-features = false } +cuprate-consensus = { path = "consensus", default-features = false } +cuprate-consensus-context = { path = "consensus/context", default-features = false } +cuprate-cryptonight = { path = "cryptonight", default-features = false } +cuprate-helper = { path = "helper", default-features = false } +cuprate-epee-encoding = { path = "net/epee-encoding", default-features = false } +cuprate-fixed-bytes = { path = "net/fixed-bytes", default-features = false } +cuprate-levin = { path = "net/levin", default-features = false } +cuprate-wire = { path = "net/wire", default-features = false } +cuprate-p2p = { path = "p2p/p2p", default-features = false } +cuprate-p2p-core = { path = "p2p/p2p-core", default-features = false } +cuprate-p2p-bucket = { path = "p2p/p2p-bucket", default-features = false } +cuprate-dandelion-tower = { path = "p2p/dandelion-tower", default-features = false } +cuprate-async-buffer = { path = "p2p/async-buffer", default-features = false } +cuprate-address-book = { path = "p2p/address-book", default-features = false } +cuprate-blockchain = { path = "storage/blockchain", default-features = false } +cuprate-database = { path = "storage/database", default-features = false } +cuprate-database-service = { path = "storage/service", default-features = false } +cuprate-txpool = { path = "storage/txpool", default-features = false } +cuprate-pruning = { path = "pruning", default-features = false } +cuprate-test-utils = { path = "test-utils", default-features = false } +cuprate-types = { path = "types", default-features = false } +cuprate-json-rpc = { path = "rpc/json-rpc", default-features = false } +cuprate-rpc-types = { path = "rpc/types", default-features = false } +cuprate-rpc-interface = { path = "rpc/interface", default-features = false } +cuprate-zmq-types = { path = "zmq/types", default-features = false } # External dependencies anyhow = { version = "1", default-features = false } @@ -125,6 +148,8 @@ tracing-subscriber = { version = "0.3", default-features = false } tracing = { version = "0.1", default-features = false } ## workspace.dev-dependencies +criterion = { version = "0.5" } +function_name = { version = "0.3" } monero-rpc = { git = "https://github.com/Cuprate/serai.git", rev = "d5205ce" } monero-simple-request-rpc = { git = "https://github.com/Cuprate/serai.git", rev = "d5205ce" } tempfile = { version = "3" } diff --git a/benches/README.md b/benches/README.md index 46409041..af6bb932 100644 --- a/benches/README.md +++ b/benches/README.md @@ -1 +1,5 @@ -# TODO +# Benches +This directory contains Cuprate's benchmarks and benchmarking utilities. + +See the [`Benchmarking` section in the Architecture book](https://architecture.cuprate.org/benchmarking/intro.html) +to see how to create and run these benchmarks. \ No newline at end of file diff --git a/benches/benchmark/bin/Cargo.toml b/benches/benchmark/bin/Cargo.toml new file mode 100644 index 00000000..36d0b2c7 --- /dev/null +++ b/benches/benchmark/bin/Cargo.toml @@ -0,0 +1,43 @@ +[package] +name = "cuprate-benchmark" +version = "0.0.0" +edition = "2021" +description = "Cuprate's benchmarking binary" +license = "MIT" +authors = ["hinto-janai"] +repository = "https://github.com/Cuprate/cuprate/tree/main/benches/benchmark/bin" +keywords = ["cuprate", "benchmarking", "binary"] + +[features] +# All new benchmarks should be added here! +all = ["example"] + +# Non-benchmark features. +default = [] +json = [] +trace = [] +debug = [] +warn = [] +info = [] +error = [] + +# Benchmark features. +# New benchmarks should be added here! +example = [ + "dep:cuprate-benchmark-example" +] + +[dependencies] +cuprate-benchmark-lib = { workspace = true } +cuprate-benchmark-example = { workspace = true, optional = true } + +cfg-if = { workspace = true } +serde = { workspace = true, features = ["derive"] } +serde_json = { workspace = true, features = ["std"] } +tracing = { workspace = true, features = ["std", "attributes"] } +tracing-subscriber = { workspace = true, features = ["fmt", "std", "env-filter"] } + +[dev-dependencies] + +[lints] +workspace = true \ No newline at end of file diff --git a/benches/benchmark/bin/README.md b/benches/benchmark/bin/README.md new file mode 100644 index 00000000..ad0700fc --- /dev/null +++ b/benches/benchmark/bin/README.md @@ -0,0 +1,27 @@ +## `cuprate-benchmark` +This crate links all benchmarks together into a single binary that can be run as: `cuprate-benchmark`. + +`cuprate-benchmark` will run all enabled benchmarks sequentially and print data at the end. + +## Benchmarks +Benchmarks are opt-in and enabled via features. + +| Feature | Enables which benchmark crate? | +|----------|--------------------------------| +| example | cuprate-benchmark-example | +| database | cuprate-benchmark-database | + +## Features +These are features that aren't for enabling benchmarks, but rather for other things. + +Since `cuprate-benchmark` is built right before it is ran, +these features almost act like command line arguments. + +| Features | Does what | +|----------|-----------| +| json | Prints JSON timings instead of a markdown table +| trace | Use the `trace` log-level +| debug | Use the `debug` log-level +| warn | Use the `warn` log-level +| info | Use the `info` log-level (default) +| error | Use the `error` log-level \ No newline at end of file diff --git a/benches/benchmark/bin/src/log.rs b/benches/benchmark/bin/src/log.rs new file mode 100644 index 00000000..455f1309 --- /dev/null +++ b/benches/benchmark/bin/src/log.rs @@ -0,0 +1,29 @@ +use cfg_if::cfg_if; +use tracing::{info, instrument, Level}; +use tracing_subscriber::FmtSubscriber; + +/// Initializes the `tracing` logger. +#[instrument] +pub(crate) fn init_logger() { + const LOG_LEVEL: Level = { + cfg_if! { + if #[cfg(feature = "trace")] { + Level::TRACE + } else if #[cfg(feature = "debug")] { + Level::DEBUG + } else if #[cfg(feature = "warn")] { + Level::WARN + } else if #[cfg(feature = "info")] { + Level::INFO + } else if #[cfg(feature = "error")] { + Level::ERROR + } else { + Level::INFO + } + } + }; + + FmtSubscriber::builder().with_max_level(LOG_LEVEL).init(); + + info!("Log level: {LOG_LEVEL}"); +} diff --git a/benches/benchmark/bin/src/main.rs b/benches/benchmark/bin/src/main.rs new file mode 100644 index 00000000..02c480a0 --- /dev/null +++ b/benches/benchmark/bin/src/main.rs @@ -0,0 +1,49 @@ +#![doc = include_str!("../README.md")] +#![allow( + unused_crate_dependencies, + reason = "this crate imports many potentially unused dependencies" +)] + +mod log; +mod print; +mod run; +mod timings; + +use cfg_if::cfg_if; + +/// What `main()` does: +/// 1. Run all enabled benchmarks +/// 2. Record benchmark timings +/// 3. Print timing data +/// +/// To add a new benchmark to be ran here: +/// 1. Copy + paste a `cfg_if` block +/// 2. Change it to your benchmark's feature flag +/// 3. Change it to your benchmark's type +#[allow( + clippy::allow_attributes, + unused_variables, + unused_mut, + unreachable_code, + reason = "clippy does not account for all cfg()s" +)] +fn main() { + log::init_logger(); + + let mut timings = timings::Timings::new(); + + cfg_if! { + if #[cfg(not(any(feature = "example")))] { + println!("No feature specified. Use `--features $BENCHMARK_FEATURE` when building."); + return; + } + } + + cfg_if! { + if #[cfg(feature = "example")] { + run::run_benchmark::(&mut timings); + } + } + + print::print_timings(&timings); +} diff --git a/benches/benchmark/bin/src/print.rs b/benches/benchmark/bin/src/print.rs new file mode 100644 index 00000000..36a5f05a --- /dev/null +++ b/benches/benchmark/bin/src/print.rs @@ -0,0 +1,38 @@ +#![expect(dead_code, reason = "code hidden behind feature flags")] + +use cfg_if::cfg_if; + +use crate::timings::Timings; + +/// Print the final the final markdown table of benchmark timings. +pub(crate) fn print_timings(timings: &Timings) { + println!("\nFinished all benchmarks, printing results:"); + + cfg_if! { + if #[cfg(feature = "json")] { + print_timings_json(timings); + } else { + print_timings_markdown(timings); + } + } +} + +/// Default timing formatting. +pub(crate) fn print_timings_markdown(timings: &Timings) { + let mut s = String::new(); + s.push_str("| Benchmark | Time (seconds) |\n"); + s.push_str("|------------------------------------|----------------|"); + + #[expect(clippy::iter_over_hash_type)] + for (k, v) in timings { + s += &format!("\n| {k:<34} | {v:<14} |"); + } + + println!("\n{s}"); +} + +/// Enabled via `json` feature. +pub(crate) fn print_timings_json(timings: &Timings) { + let json = serde_json::to_string_pretty(timings).unwrap(); + println!("\n{json}"); +} diff --git a/benches/benchmark/bin/src/run.rs b/benches/benchmark/bin/src/run.rs new file mode 100644 index 00000000..05a220f9 --- /dev/null +++ b/benches/benchmark/bin/src/run.rs @@ -0,0 +1,36 @@ +use tracing::{info, instrument, trace}; + +use cuprate_benchmark_lib::Benchmark; + +use crate::timings::Timings; + +/// Run a [`Benchmark`] and record its timing. +#[instrument(skip_all)] +pub(crate) fn run_benchmark(timings: &mut Timings) { + // Get the benchmark name. + let name = B::name(); + trace!("Running benchmark: {name}"); + + // Setup the benchmark input. + let input = B::SETUP(); + + // Sleep before running the benchmark. + trace!("Pre-benchmark, sleeping for: {:?}", B::POST_SLEEP_DURATION); + std::thread::sleep(B::PRE_SLEEP_DURATION); + + // Run/time the benchmark. + let now = std::time::Instant::now(); + B::MAIN(input); + let time = now.elapsed().as_secs_f32(); + + // Print the benchmark timings. + info!("{name:>34} ... {time}"); + assert!( + timings.insert(name, time).is_none(), + "There were 2 benchmarks with the same name - this collides the final output: {name}", + ); + + // Sleep for a cooldown period after the benchmark run. + trace!("Post-benchmark, sleeping for: {:?}", B::POST_SLEEP_DURATION); + std::thread::sleep(B::POST_SLEEP_DURATION); +} diff --git a/benches/benchmark/bin/src/timings.rs b/benches/benchmark/bin/src/timings.rs new file mode 100644 index 00000000..34a07952 --- /dev/null +++ b/benches/benchmark/bin/src/timings.rs @@ -0,0 +1,5 @@ +/// Benchmark timing data. +/// +/// - Key = benchmark name +/// - Value = benchmark time in seconds +pub(crate) type Timings = std::collections::HashMap<&'static str, f32>; diff --git a/benches/benchmark/example/Cargo.toml b/benches/benchmark/example/Cargo.toml new file mode 100644 index 00000000..5728bcd4 --- /dev/null +++ b/benches/benchmark/example/Cargo.toml @@ -0,0 +1,17 @@ +[package] +name = "cuprate-benchmark-example" +version = "0.0.0" +edition = "2021" +description = "Example showcasing Cuprate's benchmarking harness" +license = "MIT" +authors = ["hinto-janai"] +repository = "https://github.com/Cuprate/cuprate/tree/main/benches/benchmark/example" +keywords = ["cuprate", "benchmarking", "example"] + +[dependencies] +cuprate-benchmark-lib = { path = "../lib" } + +[dev-dependencies] + +[lints] +workspace = true \ No newline at end of file diff --git a/benches/benchmark/example/README.md b/benches/benchmark/example/README.md new file mode 100644 index 00000000..be6b7167 --- /dev/null +++ b/benches/benchmark/example/README.md @@ -0,0 +1,3 @@ +## `cuprate-benchmark-example` +This crate contains a short example benchmark that shows how to implement and use +`cuprate-benchmark-lib` so that it can be ran by `cuprate-benchmark`. \ No newline at end of file diff --git a/benches/benchmark/example/src/lib.rs b/benches/benchmark/example/src/lib.rs new file mode 100644 index 00000000..cc704a7f --- /dev/null +++ b/benches/benchmark/example/src/lib.rs @@ -0,0 +1,42 @@ +#![doc = include_str!("../README.md")] + +use std::hint::black_box; + +use cuprate_benchmark_lib::Benchmark; + +/// Marker struct that implements [`Benchmark`] +pub struct Example; + +/// The input to our benchmark function. +pub type ExampleBenchmarkInput = u64; + +/// The setup function that creates the input. +pub const fn example_benchmark_setup() -> ExampleBenchmarkInput { + 1 +} + +/// The main benchmarking function. +#[expect(clippy::unit_arg)] +pub fn example_benchmark_main(input: ExampleBenchmarkInput) { + // In this case, we're simply benchmarking the + // performance of simple arithmetic on the input data. + + fn math(input: ExampleBenchmarkInput, number: u64) { + let x = input; + let x = black_box(x * number); + let x = black_box(x / number); + let x = black_box(x + number); + let _ = black_box(x - number); + } + + for number in 1..100_000_000 { + black_box(math(input, number)); + } +} + +// This implementation will be run by `cuprate-benchmark`. +impl Benchmark for Example { + type Input = ExampleBenchmarkInput; + const SETUP: fn() -> Self::Input = example_benchmark_setup; + const MAIN: fn(Self::Input) = example_benchmark_main; +} diff --git a/benches/benchmark/lib/Cargo.toml b/benches/benchmark/lib/Cargo.toml new file mode 100644 index 00000000..b0771f09 --- /dev/null +++ b/benches/benchmark/lib/Cargo.toml @@ -0,0 +1,18 @@ +[package] +name = "cuprate-benchmark-lib" +version = "0.0.0" +edition = "2021" +description = "Cuprate's benchmarking library" +license = "MIT" +authors = ["hinto-janai"] +repository = "https://github.com/Cuprate/cuprate/tree/main/benches/benchmark/lib" +keywords = ["cuprate", "benchmarking", "library"] + +[features] + +[dependencies] + +[dev-dependencies] + +[lints] +workspace = true \ No newline at end of file diff --git a/benches/benchmark/lib/README.md b/benches/benchmark/lib/README.md new file mode 100644 index 00000000..9ea79ae4 --- /dev/null +++ b/benches/benchmark/lib/README.md @@ -0,0 +1,15 @@ +## `cuprate-benchmark-lib` +This crate is the glue between +[`cuprate-benchmark`](https://github.com/Cuprate/cuprate/tree/benches/benches/benchmark/bin) +and all the benchmark crates. + +It defines the [`crate::Benchmark`] trait, which is the behavior of all benchmarks. + +See the [`cuprate-benchmark-example`](https://github.com/Cuprate/cuprate/tree/benches/benches/benchmark/example) +crate to see an example implementation of this trait. + +After implementing this trait, a few steps must +be done such that the `cuprate-benchmark` binary +can actually run your benchmark crate; see the +[`Benchmarking` section in the Architecture book](https://architecture.cuprate.org/benchmarking/intro.html) +to see how to do this. \ No newline at end of file diff --git a/benches/benchmark/lib/src/benchmark.rs b/benches/benchmark/lib/src/benchmark.rs new file mode 100644 index 00000000..4dca5504 --- /dev/null +++ b/benches/benchmark/lib/src/benchmark.rs @@ -0,0 +1,45 @@ +//! Benchmarking trait. + +use std::time::Duration; + +/// A benchmarking function and its inputs. +pub trait Benchmark { + /// The benchmark's name. + /// + /// This is automatically implemented + /// as the name of the [`Self`] type. + // + // FIXME: use `const` instead of `fn` when stable + // + fn name() -> &'static str { + std::any::type_name::() + } + + /// Input to the main benchmarking function. + /// + /// This is passed to [`Self::MAIN`]. + type Input; + + /// Setup function to generate the input. + /// + /// This function is not timed. + const SETUP: fn() -> Self::Input; + + /// The main function to benchmark. + /// + /// The start of the timer begins right before + /// this function is called and ends after the + /// function returns. + const MAIN: fn(Self::Input); + + /// `cuprate-benchmark` will sleep for this [`Duration`] after + /// creating the [`Self::Input`], but before starting [`Self::MAIN`]. + /// + /// 1 second by default. + const PRE_SLEEP_DURATION: Duration = Duration::from_secs(1); + + /// `cuprate-benchmark` will sleep for this [`Duration`] after [`Self::MAIN`]. + /// + /// 1 second by default. + const POST_SLEEP_DURATION: Duration = Duration::from_secs(1); +} diff --git a/benches/benchmark/lib/src/lib.rs b/benches/benchmark/lib/src/lib.rs new file mode 100644 index 00000000..a6bec820 --- /dev/null +++ b/benches/benchmark/lib/src/lib.rs @@ -0,0 +1,5 @@ +#![doc = include_str!("../README.md")] + +mod benchmark; + +pub use benchmark::Benchmark; diff --git a/benches/criterion/cuprate-json-rpc/Cargo.toml b/benches/criterion/cuprate-json-rpc/Cargo.toml new file mode 100644 index 00000000..a0cae64e --- /dev/null +++ b/benches/criterion/cuprate-json-rpc/Cargo.toml @@ -0,0 +1,23 @@ +[package] +name = "cuprate-criterion-json-rpc" +version = "0.0.0" +edition = "2021" +description = "Criterion benchmarking for cuprate-json-rpc" +license = "MIT" +authors = ["hinto-janai"] +repository = "https://github.com/Cuprate/cuprate/tree/main/benches/criterion/cuprate-json-rpc" +keywords = ["cuprate", "json-rpc", "criterion", "benchmark"] + +[dependencies] +cuprate-json-rpc = { workspace = true } + +criterion = { workspace = true } +function_name = { workspace = true } +serde_json = { workspace = true, features = ["default"] } + +[[bench]] +name = "main" +harness = false + +[lints] +workspace = true \ No newline at end of file diff --git a/benches/criterion/cuprate-json-rpc/benches/main.rs b/benches/criterion/cuprate-json-rpc/benches/main.rs new file mode 100644 index 00000000..a7249430 --- /dev/null +++ b/benches/criterion/cuprate-json-rpc/benches/main.rs @@ -0,0 +1,8 @@ +//! Benchmarks for `cuprate-json-rpc`. +#![allow(unused_crate_dependencies)] + +mod response; + +criterion::criterion_main! { + response::serde, +} diff --git a/benches/criterion/cuprate-json-rpc/benches/response.rs b/benches/criterion/cuprate-json-rpc/benches/response.rs new file mode 100644 index 00000000..908a9f49 --- /dev/null +++ b/benches/criterion/cuprate-json-rpc/benches/response.rs @@ -0,0 +1,110 @@ +//! Benchmarks for [`Response`]. +#![allow(unused_attributes, unused_crate_dependencies)] + +use criterion::{black_box, criterion_group, criterion_main, Criterion}; +use function_name::named; +use serde_json::{from_str, to_string_pretty}; + +use cuprate_json_rpc::{Id, Response}; + +// `serde` benchmarks on `Response`. +// +// These are benchmarked as `Response` has a custom serde implementation. +criterion_group! { + name = serde; + config = Criterion::default(); + targets = + response_from_str_u8, + response_from_str_u64, + response_from_str_string_5_len, + response_from_str_string_10_len, + response_from_str_string_100_len, + response_from_str_string_500_len, + response_to_string_pretty_u8, + response_to_string_pretty_u64, + response_to_string_pretty_string_5_len, + response_to_string_pretty_string_10_len, + response_to_string_pretty_string_100_len, + response_to_string_pretty_string_500_len, + response_from_str_bad_field_1, + response_from_str_bad_field_5, + response_from_str_bad_field_10, + response_from_str_bad_field_100, + response_from_str_missing_field, +} +criterion_main!(serde); + +/// Generate `from_str` deserialization benchmark functions for [`Response`]. +macro_rules! impl_from_str_benchmark { + ( + $( + $fn_name:ident => $request_type:ty => $request_string:literal, + )* + ) => { + $( + #[named] + fn $fn_name(c: &mut Criterion) { + let request_string = $request_string; + + c.bench_function(function_name!(), |b| { + b.iter(|| { + let _r = from_str::>( + black_box(request_string) + ); + }); + }); + } + )* + }; +} + +impl_from_str_benchmark! { + response_from_str_u8 => u8 => r#"{"jsonrpc":"2.0","id":123,"result":0}"#, + response_from_str_u64 => u64 => r#"{"jsonrpc":"2.0","id":123,"result":0}"#, + response_from_str_string_5_len => String => r#"{"jsonrpc":"2.0","id":123,"result":"hello"}"#, + response_from_str_string_10_len => String => r#"{"jsonrpc":"2.0","id":123,"result":"hellohello"}"#, + response_from_str_string_100_len => String => r#"{"jsonrpc":"2.0","id":123,"result":"helloworldhelloworldhelloworldhelloworldhelloworldhelloworldhelloworldhelloworldhelloworldhelloworld"}"#, + response_from_str_string_500_len => String => r#"{"jsonrpc":"2.0","id":123,"result":"helloworldhelloworldhelloworldhelloworldhelloworldhelloworldhelloworldhelloworldhelloworldhelloworldhelloworldhelloworldhelloworldhelloworldhelloworldhelloworldhelloworldhelloworldhelloworldhelloworldhelloworldhelloworldhelloworldhelloworldhelloworldhelloworldhelloworldhelloworldhelloworldhelloworldhelloworldhelloworldhelloworldhelloworldhelloworldhelloworldhelloworldhelloworldhelloworldhelloworldhelloworldhelloworldhelloworldhelloworldhelloworldhelloworldhelloworldhelloworldhelloworldhelloworld"}"#, + + // The custom serde currently looks at all fields. + // These are for testing the performance if the serde + // has to parse through a bunch of unrelated fields. + response_from_str_bad_field_1 => u8 => r#"{"bad_field":0,"jsonrpc":"2.0","id":123,"result":0}"#, + response_from_str_bad_field_5 => u8 => r#"{"bad_field_1":0,"bad_field_2":0,"bad_field_3":0,"bad_field_4":0,"bad_field_5":0,"jsonrpc":"2.0","id":123,"result":0}"#, + response_from_str_bad_field_10 => u8 => r#"{"bad_field_1":0,"bad_field_2":0,"bad_field_3":0,"bad_field_4":0,"bad_field_5":0,"bad_field_6":0,"bad_field_7":0,"bad_field_8":0,"bad_field_9":0,"bad_field_10":0,"jsonrpc":"2.0","id":123,"result":0}"#, + response_from_str_bad_field_100 => u8 => r#"{"1":0,"2":0,"3":0,"4":0,"5":0,"6":0,"7":0,"8":0,"9":0,"10":0,"11":0,"12":0,"13":0,"14":0,"15":0,"16":0,"17":0,"18":0,"19":0,"20":0,"21":0,"22":0,"23":0,"24":0,"25":0,"26":0,"27":0,"28":0,"29":0,"30":0,"31":0,"32":0,"33":0,"34":0,"35":0,"36":0,"37":0,"38":0,"39":0,"40":0,"41":0,"42":0,"43":0,"44":0,"45":0,"46":0,"47":0,"48":0,"49":0,"50":0,"51":0,"52":0,"53":0,"54":0,"55":0,"56":0,"57":0,"58":0,"59":0,"60":0,"61":0,"62":0,"63":0,"64":0,"65":0,"66":0,"67":0,"68":0,"69":0,"70":0,"71":0,"72":0,"73":0,"74":0,"75":0,"76":0,"77":0,"78":0,"79":0,"80":0,"81":0,"82":0,"83":0,"84":0,"85":0,"86":0,"87":0,"88":0,"89":0,"90":0,"91":0,"92":0,"93":0,"94":0,"95":0,"96":0,"97":0,"98":0,"99":0,"100":0,"jsonrpc":"2.0","id":123,"result":0}"#, + + // These are missing the `jsonrpc` field. + response_from_str_missing_field => u8 => r#"{"id":123,"result":0}"#, +} + +/// Generate `to_string_pretty` serialization benchmark functions for [`Response`]. +macro_rules! impl_to_string_pretty_benchmark { + ( + $( + $fn_name:ident => $request_constructor:expr, + )* + ) => { + $( + #[named] + fn $fn_name(c: &mut Criterion) { + let request = $request_constructor; + + c.bench_function(function_name!(), |b| { + b.iter(|| { + let _s = to_string_pretty(black_box(&request)).unwrap(); + }); + }); + } + )* + }; +} + +impl_to_string_pretty_benchmark! { + response_to_string_pretty_u8 => Response::::ok(Id::Null, 0), + response_to_string_pretty_u64 => Response::::ok(Id::Null, 0), + response_to_string_pretty_string_5_len => Response::ok(Id::Null, String::from("hello")), + response_to_string_pretty_string_10_len => Response::ok(Id::Null, String::from("hellohello")), + response_to_string_pretty_string_100_len => Response::ok(Id::Null, String::from("helloworldhelloworldhelloworldhelloworldhelloworldhelloworldhelloworldhelloworldhelloworldhelloworld")), + response_to_string_pretty_string_500_len => Response::ok(Id::Null, String::from("helloworldhelloworldhelloworldhelloworldhelloworldhelloworldhelloworldhelloworldhelloworldhelloworldhelloworldhelloworldhelloworldhelloworldhelloworldhelloworldhelloworldhelloworldhelloworldhelloworldhelloworldhelloworldhelloworldhelloworldhelloworldhelloworldhelloworldhelloworldhelloworldhelloworldhelloworldhelloworldhelloworldhelloworldhelloworldhelloworldhelloworldhelloworldhelloworldhelloworldhelloworldhelloworldhelloworldhelloworldhelloworldhelloworldhelloworldhelloworldhelloworldhelloworld")), +} diff --git a/benches/criterion/cuprate-json-rpc/src/lib.rs b/benches/criterion/cuprate-json-rpc/src/lib.rs new file mode 100644 index 00000000..b29887aa --- /dev/null +++ b/benches/criterion/cuprate-json-rpc/src/lib.rs @@ -0,0 +1,2 @@ +//! Benchmark lib for `cuprate-json-rpc`. +#![allow(unused_crate_dependencies, reason = "used in benchmarks")] diff --git a/benches/criterion/example/Cargo.toml b/benches/criterion/example/Cargo.toml new file mode 100644 index 00000000..43e60119 --- /dev/null +++ b/benches/criterion/example/Cargo.toml @@ -0,0 +1,21 @@ +[package] +name = "cuprate-criterion-example" +version = "0.0.0" +edition = "2021" +description = "Criterion benchmarking example for Cuprate" +license = "MIT" +authors = ["hinto-janai"] +repository = "https://github.com/Cuprate/cuprate/tree/main/benches/criterion/example" +keywords = ["cuprate", "criterion", "benchmark", "example"] + +[dependencies] +criterion = { workspace = true } +function_name = { workspace = true } +serde_json = { workspace = true, features = ["default"] } + +[[bench]] +name = "main" +harness = false + +[lints] +workspace = true \ No newline at end of file diff --git a/benches/criterion/example/README.md b/benches/criterion/example/README.md new file mode 100644 index 00000000..cf1983ff --- /dev/null +++ b/benches/criterion/example/README.md @@ -0,0 +1,14 @@ +## `cuprate-criterion-example` +An example of using Criterion for benchmarking Cuprate crates. + +Consider copy+pasting this crate to use as a base when creating new Criterion benchmark crates. + +## `src/` +Benchmark crates have a `benches/` ran by `cargo bench`, but they are also crates themselves, +as in, they have a `src` folder that `benches/` can pull code from. + +The `src` directories in these benchmarking crates are usually filled with +helper functions, types, etc, that are used repeatedly in the benchmarks. + +## `benches/` +These are the actual benchmarks ran by `cargo bench`. diff --git a/benches/criterion/example/benches/example.rs b/benches/criterion/example/benches/example.rs new file mode 100644 index 00000000..7ea8e9a1 --- /dev/null +++ b/benches/criterion/example/benches/example.rs @@ -0,0 +1,48 @@ +//! Benchmarks. +#![allow(unused_attributes, unused_crate_dependencies)] + +use criterion::{black_box, criterion_group, criterion_main, BenchmarkId, Criterion, Throughput}; +use function_name::named; + +use cuprate_criterion_example::SomeHardToCreateObject; + +// This is how you register criterion benchmarks. +criterion_group! { + name = benches; + config = Criterion::default(); + targets = benchmark_1, benchmark_range, +} +criterion_main!(benches); + +/// Benchmark a single input. +/// +/// +#[named] +fn benchmark_1(c: &mut Criterion) { + // It is recommended to use `function_name!()` as a benchmark + // identifier instead of manually re-typing the function name. + c.bench_function(function_name!(), |b| { + b.iter(|| { + black_box(SomeHardToCreateObject::from(1)); + }); + }); +} + +/// Benchmark a range of inputs. +/// +/// +#[named] +fn benchmark_range(c: &mut Criterion) { + let mut group = c.benchmark_group(function_name!()); + + for i in 0..4 { + group.throughput(Throughput::Elements(i)); + group.bench_with_input(BenchmarkId::from_parameter(i), &i, |b, &i| { + b.iter(|| { + black_box(SomeHardToCreateObject::from(i)); + }); + }); + } + + group.finish(); +} diff --git a/benches/criterion/example/benches/main.rs b/benches/criterion/example/benches/main.rs new file mode 100644 index 00000000..d4f0bf80 --- /dev/null +++ b/benches/criterion/example/benches/main.rs @@ -0,0 +1,10 @@ +//! Benchmarks examples. +#![allow(unused_crate_dependencies)] + +// All modules within `benches/` are `mod`ed here. +mod example; + +// And all the Criterion benchmarks are registered like so: +criterion::criterion_main! { + example::benches, +} diff --git a/benches/criterion/example/src/lib.rs b/benches/criterion/example/src/lib.rs new file mode 100644 index 00000000..0f732a47 --- /dev/null +++ b/benches/criterion/example/src/lib.rs @@ -0,0 +1,13 @@ +#![doc = include_str!("../README.md")] // See the README for crate documentation. +#![allow(unused_crate_dependencies, reason = "used in benchmarks")] + +/// Shared type that all benchmarks can use. +#[expect(dead_code)] +pub struct SomeHardToCreateObject(u64); + +impl From for SomeHardToCreateObject { + /// Shared function that all benchmarks can use. + fn from(value: u64) -> Self { + Self(value) + } +} diff --git a/books/architecture/src/SUMMARY.md b/books/architecture/src/SUMMARY.md index 0961d8fc..a99d099f 100644 --- a/books/architecture/src/SUMMARY.md +++ b/books/architecture/src/SUMMARY.md @@ -143,9 +143,16 @@ --- -- [⚪️ Benchmarking](benchmarking/intro.md) - - [⚪️ Criterion](benchmarking/criterion.md) - - [⚪️ Harness](benchmarking/harness.md) +- [🟢 Benchmarking](benchmarking/intro.md) + - [🟢 Criterion](benchmarking/criterion/intro.md) + - [🟢 Creating](benchmarking/criterion/creating.md) + - [🟢 Running](benchmarking/criterion/running.md) + - [🟢 `cuprate-benchmark`](benchmarking/cuprate/intro.md) + - [🟢 Creating](benchmarking/cuprate/creating.md) + - [🟢 Running](benchmarking/cuprate/running.md) + +--- + - [⚪️ Testing](testing/intro.md) - [⚪️ Monero data](testing/monero-data.md) - [⚪️ RPC client](testing/rpc-client.md) diff --git a/books/architecture/src/appendix/crates.md b/books/architecture/src/appendix/crates.md index a0dff481..5124180f 100644 --- a/books/architecture/src/appendix/crates.md +++ b/books/architecture/src/appendix/crates.md @@ -68,3 +68,11 @@ cargo doc --open --package cuprate-blockchain | [`cuprate-helper`](https://doc.cuprate.org/cuprate_helper) | [`helper/`](https://github.com/Cuprate/cuprate/tree/main/helper) | Kitchen-sink helper crate for Cuprate | [`cuprate-test-utils`](https://doc.cuprate.org/cuprate_test_utils) | [`test-utils/`](https://github.com/Cuprate/cuprate/tree/main/test-utils) | Testing utilities for Cuprate | [`cuprate-types`](https://doc.cuprate.org/cuprate_types) | [`types/`](https://github.com/Cuprate/cuprate/tree/main/types) | Shared types across Cuprate + +## Benchmarks +| Crate | In-tree path | Purpose | +|-------|--------------|---------| +| [`cuprate-benchmark`](https://doc.cuprate.org/cuprate_benchmark) | [`benches/benchmark/bin/`](https://github.com/Cuprate/cuprate/tree/main/benches/benchmark/bin) | Cuprate benchmarking binary +| [`cuprate-benchmark-lib`](https://doc.cuprate.org/cuprate_benchmark_lib) | [`benches/benchmark/lib/`](https://github.com/Cuprate/cuprate/tree/main/benches/benchmark/lib) | Cuprate benchmarking library +| `cuprate-benchmark-*` | [`benches/benchmark/cuprate-*`](https://github.com/Cuprate/cuprate/tree/main/benches/benchmark/) | Benchmark for a Cuprate crate that uses `cuprate-benchmark` +| `cuprate-criterion-*` | [`benches/criterion/cuprate-*`](https://github.com/Cuprate/cuprate/tree/main/benches/criterion) | Benchmark for a Cuprate crate that uses [Criterion](https://bheisler.github.io/criterion.rs/book) \ No newline at end of file diff --git a/books/architecture/src/benchmarking/criterion.md b/books/architecture/src/benchmarking/criterion.md deleted file mode 100644 index e9d61e65..00000000 --- a/books/architecture/src/benchmarking/criterion.md +++ /dev/null @@ -1 +0,0 @@ -# ⚪️ Criterion diff --git a/books/architecture/src/benchmarking/criterion/creating.md b/books/architecture/src/benchmarking/criterion/creating.md new file mode 100644 index 00000000..01009042 --- /dev/null +++ b/books/architecture/src/benchmarking/criterion/creating.md @@ -0,0 +1,21 @@ +# Creating +Creating a new Criterion-based benchmarking crate for one of Cuprate's crates is relatively simple, +although, it requires knowledge of how to use Criterion first: + +1. Read the `Getting Started` section of +2. Copy [`benches/criterion/example`](https://github.com/Cuprate/cuprate/tree/main/benches/criterion/example) as base +3. Get started + +## Naming +New benchmark crates using Criterion should: +- Be in [`benches/criterion/`](https://github.com/Cuprate/cuprate/tree/main/benches/criterion/) +- Be in the `cuprate-criterion-$CRATE_NAME` format + +For a real example, see: +[`cuprate-criterion-json-rpc`](https://github.com/Cuprate/cuprate/tree/main/benches/criterion/cuprate-json-rpc). + +## Workspace +Finally, make sure to add the benchmark crate to the workspace +[`Cargo.toml`](https://github.com/Cuprate/cuprate/blob/main/Cargo.toml) file. + +Your benchmark is now ready to be ran. \ No newline at end of file diff --git a/books/architecture/src/benchmarking/criterion/intro.md b/books/architecture/src/benchmarking/criterion/intro.md new file mode 100644 index 00000000..b7a79b21 --- /dev/null +++ b/books/architecture/src/benchmarking/criterion/intro.md @@ -0,0 +1,4 @@ +# Criterion +Each sub-directory in [`benches/criterion/`](https://github.com/Cuprate/cuprate/tree/main/benches/criterion) is a crate that uses [Criterion](https://bheisler.github.io/criterion.rs/book) for timing single functions and/or groups of functions. + +They are generally be small in scope. \ No newline at end of file diff --git a/books/architecture/src/benchmarking/criterion/running.md b/books/architecture/src/benchmarking/criterion/running.md new file mode 100644 index 00000000..14067f67 --- /dev/null +++ b/books/architecture/src/benchmarking/criterion/running.md @@ -0,0 +1,15 @@ +# Running +To run all Criterion benchmarks, run this from the repository root: +```bash +cargo bench +``` + +To run specific package(s), use: +```bash +cargo bench --package $CRITERION_BENCHMARK_CRATE_NAME +``` + +For example: +```bash +cargo bench --package cuprate-criterion-json-rpc +``` \ No newline at end of file diff --git a/books/architecture/src/benchmarking/cuprate/creating.md b/books/architecture/src/benchmarking/cuprate/creating.md new file mode 100644 index 00000000..76eab789 --- /dev/null +++ b/books/architecture/src/benchmarking/cuprate/creating.md @@ -0,0 +1,57 @@ +# Creating +New benchmarks are plugged into `cuprate-benchmark` by: +1. Implementing `cuprate_benchmark_lib::Benchmark` +1. Registering the benchmark in the `cuprate_benchmark` binary + +See [`benches/benchmark/example`](https://github.com/Cuprate/cuprate/tree/main/benches/benchmark/example) +for an example. + +## Creating the benchmark crate +Before plugging into `cuprate-benchmark`, your actual benchmark crate must be created: + +1. Create a new crate inside `benches/benchmark` (consider copying `benches/benchmark/example` as a base) +1. Pull in `cuprate_benchmark_lib` as a dependency +1. Create a benchmark +1. Implement `cuprate_benchmark_lib::Benchmark` + +New benchmark crates using `cuprate-database` should: +- Be in [`benches/benchmark/`](https://github.com/Cuprate/cuprate/tree/main/benches/benchmark/) +- Be in the `cuprate-benchmark-$CRATE_NAME` format + +For a real example, see: +[`cuprate-benchmark-database`](https://github.com/Cuprate/cuprate/tree/main/benches/benchmark/cuprate-database). + +## `cuprate_benchmark_lib::Benchmark` +This is the trait that standardizes all benchmarks ran under `cuprate-benchmark`. + +It must be implemented by your benchmarking crate. + +See `cuprate-benchmark-lib` crate documentation for a user-guide: . + +## Adding a feature to `cuprate-benchmark` +After your benchmark's behavior is defined, it must be registered +in the binary that is actually ran: `cuprate-benchmark`. + +If your benchmark is new, add a new crate feature to [`cuprate-benchmark`'s Cargo.toml file](https://github.com/Cuprate/cuprate/tree/main/benches/benchmark/bin/Cargo.toml) with an optional dependency to your benchmarking crate. + +Please remember to edit the feature table in the +[`README.md`](https://github.com/Cuprate/cuprate/tree/main/benches/benchmark/bin/README.md) as well! + +## Adding to `cuprate-benchmark`'s `main()` +After adding your crate's feature, add a conditional line that run the benchmark +if the feature is enabled to the `main()` function: + +For example, if your crate's name is `egg`: +```rust +cfg_if! { + if #[cfg(feature = "egg")] { + run::run_benchmark::(&mut timings); + } +} +``` + +## Workspace +Finally, make sure to add the benchmark crate to the workspace +[`Cargo.toml`](https://github.com/Cuprate/cuprate/blob/main/Cargo.toml) file. + +Your benchmark is now ready to be ran. \ No newline at end of file diff --git a/books/architecture/src/benchmarking/cuprate/intro.md b/books/architecture/src/benchmarking/cuprate/intro.md new file mode 100644 index 00000000..25efb460 --- /dev/null +++ b/books/architecture/src/benchmarking/cuprate/intro.md @@ -0,0 +1,37 @@ +# cuprate-benchmark +Cuprate has 2 custom crates for general benchmarking: +- `cuprate-benchmark`; the actual binary crate ran +- `cuprate-benchmark-lib`; the library that other crates hook into + +The abstract purpose of `cuprate-benchmark` is very simple: +1. Set-up the benchmark +1. Start timer +1. Run benchmark +1. Output data + +`cuprate-benchmark` runs the benchmarks found in [`benches/benchmark/cuprate-*`](https://github.com/Cuprate/cuprate/tree/main/benches/benchmark). + +`cuprate-benchmark-lib` defines the `Benchmark` trait that all +benchmark crates implement to "plug-in" to the benchmarking harness. + +## Diagram +A diagram displaying the relation between `cuprate-benchmark` and related crates. + +``` + ┌─────────────────────┐ + │ cuprate_benchmark │ + │ (actual binary ran) │ + └──────────┬──────────┘ + ┌──────────────────┴───────────────────┐ + │ cuprate_benchmark_lib │ + │ ┌───────────────────────────────────┐│ + │ │ trait Benchmark ││ + │ └───────────────────────────────────┘│ + └──────────────────┬───────────────────┘ +┌───────────────────────────┐ │ ┌───────────────────────────┐ +│ cuprate_benchmark_example ├──┼───┤ cuprate_benchmark_* │ +└───────────────────────────┘ │ └───────────────────────────┘ +┌───────────────────────────┐ │ ┌───────────────────────────┐ +│ cuprate_benchmark_* ├──┴───┤ cuprate_benchmark_* │ +└───────────────────────────┘ └───────────────────────────┘ +``` \ No newline at end of file diff --git a/books/architecture/src/benchmarking/cuprate/running.md b/books/architecture/src/benchmarking/cuprate/running.md new file mode 100644 index 00000000..b776163e --- /dev/null +++ b/books/architecture/src/benchmarking/cuprate/running.md @@ -0,0 +1,16 @@ +# Running +`cuprate-benchmark` benchmarks are ran with this command: +```bash +cargo run --release --package cuprate-benchmark --features $BENCHMARK_CRATE_FEATURE +``` + +For example, to run the example benchmark: +```bash +cargo run --release --package cuprate-benchmark --features example +``` + +Use the `all` feature to run all benchmarks: +```bash +# Run all benchmarks +cargo run --release --package cuprate-benchmark --features all +``` diff --git a/books/architecture/src/benchmarking/harness.md b/books/architecture/src/benchmarking/harness.md deleted file mode 100644 index 6f82b523..00000000 --- a/books/architecture/src/benchmarking/harness.md +++ /dev/null @@ -1 +0,0 @@ -# ⚪️ Harness diff --git a/books/architecture/src/benchmarking/intro.md b/books/architecture/src/benchmarking/intro.md index f043a0ba..e6ab6b12 100644 --- a/books/architecture/src/benchmarking/intro.md +++ b/books/architecture/src/benchmarking/intro.md @@ -1 +1,22 @@ -# ⚪️ Benchmarking +# Benchmarking +Cuprate has 2 types of benchmarks: +- [Criterion](https://bheisler.github.io/criterion.rs/book/user_guide/advanced_configuration.html) benchmarks +- `cuprate-benchmark` benchmarks + +Criterion is used for micro benchmarks; they time single functions, groups of functions, and generally are small in scope. + +`cuprate-benchmark` and [`cuprate-benchmark-lib`](https://doc.cuprate.org/cuprate_benchmark_lib) are custom in-house crates Cuprate uses for macro benchmarks; these test sub-systems, sections of a sub-system, or otherwise larger or more complicated code that isn't well-suited for micro benchmarks. + +## File layout and purpose +All benchmarking related files are in the [`benches/`](https://github.com/Cuprate/cuprate/tree/main/benches) folder. + +This directory is organized like such: + +| Directory | Purpose | +|-------------------------------|---------| +| [`benches/criterion/`](https://github.com/Cuprate/cuprate/tree/main/benches/criterion) | Criterion (micro) benchmarks +| `benches/criterion/cuprate-*` | Criterion benchmarks for the crate with the same name +| [`benches/benchmark/`](https://github.com/Cuprate/cuprate/tree/main/benches/benchmark) | Cuprate's custom benchmarking files +| [`benches/benchmark/bin`](https://github.com/Cuprate/cuprate/tree/main/benches/benchmark/bin) | The `cuprate-benchmark` crate; the actual binary run that links all benchmarks +| [`benches/benchmark/lib`](https://github.com/Cuprate/cuprate/tree/main/benches/benchmark/lib) | The `cuprate-benchmark-lib` crate; the benchmarking framework all benchmarks plug into +| `benches/benchmark/cuprate-*` | `cuprate-benchmark` benchmarks for the crate with the same name From 01150ab84c1d24147bb45c61dffc45a70956aba3 Mon Sep 17 00:00:00 2001 From: hinto-janai Date: Wed, 27 Nov 2024 18:04:58 -0500 Subject: [PATCH 07/10] rpc/types: fix epee deserialization for `GetBlocksResponse` (#345) * header + flatten * fix optional values * `fn error() -> String` -> `error!() -> &'static str` * extract out `PoolInfo` * fix cargo hack --- rpc/types/src/bin.rs | 292 +++----------------------------- rpc/types/src/misc/misc.rs | 4 +- rpc/types/src/misc/mod.rs | 2 + rpc/types/src/misc/pool_info.rs | 171 +++++++++++++++++++ rpc/types/src/misc/tx_entry.rs | 5 +- 5 files changed, 199 insertions(+), 275 deletions(-) create mode 100644 rpc/types/src/misc/pool_info.rs diff --git a/rpc/types/src/bin.rs b/rpc/types/src/bin.rs index 7b941918..414214ca 100644 --- a/rpc/types/src/bin.rs +++ b/rpc/types/src/bin.rs @@ -9,26 +9,19 @@ use cuprate_fixed_bytes::ByteArrayVec; use serde::{Deserialize, Serialize}; #[cfg(feature = "epee")] -use cuprate_epee_encoding::{ - container_as_blob::ContainerAsBlob, - epee_object, error, - macros::bytes::{Buf, BufMut}, - read_epee_value, write_field, EpeeObject, EpeeObjectBuilder, -}; +use cuprate_epee_encoding::container_as_blob::ContainerAsBlob; use cuprate_types::BlockCompleteEntry; use crate::{ base::AccessResponseBase, - macros::{define_request, define_request_and_response, define_request_and_response_doc}, - misc::{BlockOutputIndices, GetOutputsOut, OutKeyBin, PoolTxInfo, Status}, + macros::define_request_and_response, + misc::{BlockOutputIndices, GetOutputsOut, OutKeyBin, PoolInfo}, rpc_call::RpcCallValue, }; #[cfg(any(feature = "epee", feature = "serde"))] use crate::defaults::{default_false, default_zero}; -#[cfg(feature = "epee")] -use crate::misc::PoolInfoExtent; //---------------------------------------------------------------------------------------------------- Definitions define_request_and_response! { @@ -115,15 +108,14 @@ define_request_and_response! { } } -//---------------------------------------------------------------------------------------------------- GetBlocks -define_request! { - #[doc = define_request_and_response_doc!( - "response" => GetBlocksResponse, - get_blocksbin, - cc73fe71162d564ffda8e549b79a350bca53c454, - core_rpc_server_commands_defs, h, 162, 262, - )] - GetBlocksRequest { +define_request_and_response! { + get_blocksbin, + cc73fe71162d564ffda8e549b79a350bca53c454 => + core_rpc_server_commands_defs.h => 162..=262, + + GetBlocks, + + Request { requested_info: u8 = default_zero::(), "default_zero", // FIXME: This is a `std::list` in `monerod` because...? block_ids: ByteArrayVec<32>, @@ -131,259 +123,17 @@ define_request! { prune: bool, no_miner_tx: bool = default_false(), "default_false", pool_info_since: u64 = default_zero::(), "default_zero", - } -} + }, -#[doc = define_request_and_response_doc!( - "request" => GetBlocksRequest, - get_blocksbin, - cc73fe71162d564ffda8e549b79a350bca53c454, - core_rpc_server_commands_defs, h, 162, 262, -)] -/// -/// This response's variant depends upon [`PoolInfoExtent`]. -#[cfg_attr(feature = "serde", derive(Serialize, Deserialize))] -#[derive(Clone, Debug, PartialEq, Eq, PartialOrd, Ord, Hash)] -pub enum GetBlocksResponse { - /// Will always serialize a [`PoolInfoExtent::None`] field. - PoolInfoNone(GetBlocksResponsePoolInfoNone), - /// Will always serialize a [`PoolInfoExtent::Incremental`] field. - PoolInfoIncremental(GetBlocksResponsePoolInfoIncremental), - /// Will always serialize a [`PoolInfoExtent::Full`] field. - PoolInfoFull(GetBlocksResponsePoolInfoFull), -} - -impl Default for GetBlocksResponse { - fn default() -> Self { - Self::PoolInfoNone(GetBlocksResponsePoolInfoNone::default()) - } -} - -/// Data within [`GetBlocksResponse::PoolInfoNone`]. -#[cfg_attr(feature = "serde", derive(Serialize, Deserialize))] -#[derive(Clone, Default, Debug, PartialEq, Eq, PartialOrd, Ord, Hash)] -pub struct GetBlocksResponsePoolInfoNone { - pub status: Status, - pub untrusted: bool, - pub blocks: Vec, - pub start_height: u64, - pub current_height: u64, - pub output_indices: Vec, - pub daemon_time: u64, -} - -#[cfg(feature = "epee")] -epee_object! { - GetBlocksResponsePoolInfoNone, - status: Status, - untrusted: bool, - blocks: Vec, - start_height: u64, - current_height: u64, - output_indices: Vec, - daemon_time: u64, -} - -/// Data within [`GetBlocksResponse::PoolInfoIncremental`]. -#[cfg_attr(feature = "serde", derive(Serialize, Deserialize))] -#[derive(Clone, Default, Debug, PartialEq, Eq, PartialOrd, Ord, Hash)] -pub struct GetBlocksResponsePoolInfoIncremental { - pub status: Status, - pub untrusted: bool, - pub blocks: Vec, - pub start_height: u64, - pub current_height: u64, - pub output_indices: Vec, - pub daemon_time: u64, - pub added_pool_txs: Vec, - pub remaining_added_pool_txids: ByteArrayVec<32>, - pub removed_pool_txids: ByteArrayVec<32>, -} - -#[cfg(feature = "epee")] -epee_object! { - GetBlocksResponsePoolInfoIncremental, - status: Status, - untrusted: bool, - blocks: Vec, - start_height: u64, - current_height: u64, - output_indices: Vec, - daemon_time: u64, - added_pool_txs: Vec, - remaining_added_pool_txids: ByteArrayVec<32>, - removed_pool_txids: ByteArrayVec<32>, -} - -/// Data within [`GetBlocksResponse::PoolInfoFull`]. -#[cfg_attr(feature = "serde", derive(Serialize, Deserialize))] -#[derive(Clone, Default, Debug, PartialEq, Eq, PartialOrd, Ord, Hash)] -pub struct GetBlocksResponsePoolInfoFull { - pub status: Status, - pub untrusted: bool, - pub blocks: Vec, - pub start_height: u64, - pub current_height: u64, - pub output_indices: Vec, - pub daemon_time: u64, - pub added_pool_txs: Vec, - pub remaining_added_pool_txids: ByteArrayVec<32>, -} - -#[cfg(feature = "epee")] -epee_object! { - GetBlocksResponsePoolInfoFull, - status: Status, - untrusted: bool, - blocks: Vec, - start_height: u64, - current_height: u64, - output_indices: Vec, - daemon_time: u64, - added_pool_txs: Vec, - remaining_added_pool_txids: ByteArrayVec<32>, -} - -#[cfg(feature = "epee")] -/// [`EpeeObjectBuilder`] for [`GetBlocksResponse`]. -/// -/// Not for public usage. -#[derive(Clone, Debug, Default, PartialEq, Eq, PartialOrd, Ord, Hash)] -#[cfg_attr(feature = "serde", derive(Serialize, Deserialize))] -pub struct __GetBlocksResponseEpeeBuilder { - pub status: Option, - pub untrusted: Option, - pub blocks: Option>, - pub start_height: Option, - pub current_height: Option, - pub output_indices: Option>, - pub daemon_time: Option, - pub pool_info_extent: Option, - pub added_pool_txs: Option>, - pub remaining_added_pool_txids: Option>, - pub removed_pool_txids: Option>, -} - -#[cfg(feature = "epee")] -impl EpeeObjectBuilder for __GetBlocksResponseEpeeBuilder { - fn add_field(&mut self, name: &str, r: &mut B) -> error::Result { - macro_rules! read_epee_field { - ($($field:ident),*) => { - match name { - $( - stringify!($field) => { self.$field = Some(read_epee_value(r)?); }, - )* - _ => return Ok(false), - } - }; - } - - read_epee_field! { - status, - untrusted, - blocks, - start_height, - current_height, - output_indices, - daemon_time, - pool_info_extent, - added_pool_txs, - remaining_added_pool_txids, - removed_pool_txids - } - - Ok(true) - } - - fn finish(self) -> error::Result { - const ELSE: error::Error = error::Error::Format("Required field was not found!"); - - let status = self.status.ok_or(ELSE)?; - let untrusted = self.untrusted.ok_or(ELSE)?; - let blocks = self.blocks.ok_or(ELSE)?; - let start_height = self.start_height.ok_or(ELSE)?; - let current_height = self.current_height.ok_or(ELSE)?; - let output_indices = self.output_indices.ok_or(ELSE)?; - let daemon_time = self.daemon_time.ok_or(ELSE)?; - let pool_info_extent = self.pool_info_extent.ok_or(ELSE)?; - - let this = match pool_info_extent { - PoolInfoExtent::None => { - GetBlocksResponse::PoolInfoNone(GetBlocksResponsePoolInfoNone { - status, - untrusted, - blocks, - start_height, - current_height, - output_indices, - daemon_time, - }) - } - PoolInfoExtent::Incremental => { - GetBlocksResponse::PoolInfoIncremental(GetBlocksResponsePoolInfoIncremental { - status, - untrusted, - blocks, - start_height, - current_height, - output_indices, - daemon_time, - added_pool_txs: self.added_pool_txs.ok_or(ELSE)?, - remaining_added_pool_txids: self.remaining_added_pool_txids.ok_or(ELSE)?, - removed_pool_txids: self.removed_pool_txids.ok_or(ELSE)?, - }) - } - PoolInfoExtent::Full => { - GetBlocksResponse::PoolInfoFull(GetBlocksResponsePoolInfoFull { - status, - untrusted, - blocks, - start_height, - current_height, - output_indices, - daemon_time, - added_pool_txs: self.added_pool_txs.ok_or(ELSE)?, - remaining_added_pool_txids: self.remaining_added_pool_txids.ok_or(ELSE)?, - }) - } - }; - - Ok(this) - } -} - -#[cfg(feature = "epee")] -impl EpeeObject for GetBlocksResponse { - type Builder = __GetBlocksResponseEpeeBuilder; - - fn number_of_fields(&self) -> u64 { - // [`PoolInfoExtent`] + inner struct fields. - let inner_fields = match self { - Self::PoolInfoNone(s) => s.number_of_fields(), - Self::PoolInfoIncremental(s) => s.number_of_fields(), - Self::PoolInfoFull(s) => s.number_of_fields(), - }; - - 1 + inner_fields - } - - fn write_fields(self, w: &mut B) -> error::Result<()> { - match self { - Self::PoolInfoNone(s) => { - s.write_fields(w)?; - write_field(PoolInfoExtent::None.to_u8(), "pool_info_extent", w)?; - } - Self::PoolInfoIncremental(s) => { - s.write_fields(w)?; - write_field(PoolInfoExtent::Incremental.to_u8(), "pool_info_extent", w)?; - } - Self::PoolInfoFull(s) => { - s.write_fields(w)?; - write_field(PoolInfoExtent::Full.to_u8(), "pool_info_extent", w)?; - } - } - - Ok(()) + // TODO: add `top_block_hash` field + // + AccessResponseBase { + blocks: Vec, + start_height: u64, + current_height: u64, + output_indices: Vec, + daemon_time: u64, + pool_info: PoolInfo, } } diff --git a/rpc/types/src/misc/misc.rs b/rpc/types/src/misc/misc.rs index 8f7467ba..67ec756d 100644 --- a/rpc/types/src/misc/misc.rs +++ b/rpc/types/src/misc/misc.rs @@ -11,11 +11,11 @@ use serde::{Deserialize, Serialize}; #[cfg(feature = "epee")] use cuprate_epee_encoding::epee_object; -use crate::macros::monero_definition_link; - #[cfg(any(feature = "epee", feature = "serde"))] use crate::defaults::default_zero; +use crate::macros::monero_definition_link; + //---------------------------------------------------------------------------------------------------- Macros /// This macro (local to this file) defines all the misc types. /// diff --git a/rpc/types/src/misc/mod.rs b/rpc/types/src/misc/mod.rs index e09f8477..49767560 100644 --- a/rpc/types/src/misc/mod.rs +++ b/rpc/types/src/misc/mod.rs @@ -17,6 +17,7 @@ mod distribution; mod key_image_spent_status; #[expect(clippy::module_inception)] mod misc; +mod pool_info; mod pool_info_extent; mod status; mod tx_entry; @@ -30,6 +31,7 @@ pub use misc::{ OutputDistributionData, Peer, PoolTxInfo, PublicNode, SetBan, Span, SpentKeyImageInfo, SyncInfoPeer, TxBacklogEntry, TxInfo, TxOutputIndices, TxpoolHisto, TxpoolStats, }; +pub use pool_info::PoolInfo; pub use pool_info_extent::PoolInfoExtent; pub use status::Status; pub use tx_entry::TxEntry; diff --git a/rpc/types/src/misc/pool_info.rs b/rpc/types/src/misc/pool_info.rs new file mode 100644 index 00000000..e9ba8753 --- /dev/null +++ b/rpc/types/src/misc/pool_info.rs @@ -0,0 +1,171 @@ +#[cfg(feature = "serde")] +use serde::{Deserialize, Serialize}; + +#[cfg(feature = "epee")] +use crate::misc::PoolInfoExtent; +#[cfg(feature = "epee")] +use cuprate_epee_encoding::{ + epee_object, error, + macros::bytes::{Buf, BufMut}, + read_epee_value, write_field, EpeeObject, EpeeObjectBuilder, +}; + +use cuprate_fixed_bytes::ByteArrayVec; + +use crate::misc::PoolTxInfo; + +//---------------------------------------------------------------------------------------------------- PoolInfo +#[doc = crate::macros::monero_definition_link!( + cc73fe71162d564ffda8e549b79a350bca53c454, + "rpc/core_rpc_server_commands_defs.h", + 223..=228 +)] +/// Used in [`crate::bin::GetBlocksResponse`]. +#[cfg_attr(feature = "serde", derive(Serialize, Deserialize))] +#[derive(Clone, Debug, Default, PartialEq, Eq, PartialOrd, Ord, Hash)] +#[repr(u8)] +pub enum PoolInfo { + #[default] + None, + Incremental(PoolInfoIncremental), + Full(PoolInfoFull), +} + +//---------------------------------------------------------------------------------------------------- Internal data +/// Data within [`PoolInfo::Incremental`]. +#[cfg_attr(feature = "serde", derive(Serialize, Deserialize))] +#[derive(Clone, Default, Debug, PartialEq, Eq, PartialOrd, Ord, Hash)] +pub struct PoolInfoIncremental { + pub added_pool_txs: Vec, + pub remaining_added_pool_txids: ByteArrayVec<32>, + pub removed_pool_txids: ByteArrayVec<32>, +} + +#[cfg(feature = "epee")] +epee_object! { + PoolInfoIncremental, + added_pool_txs: Vec, + remaining_added_pool_txids: ByteArrayVec<32>, + removed_pool_txids: ByteArrayVec<32>, +} + +/// Data within [`PoolInfo::Full`]. +#[cfg_attr(feature = "serde", derive(Serialize, Deserialize))] +#[derive(Clone, Default, Debug, PartialEq, Eq, PartialOrd, Ord, Hash)] +pub struct PoolInfoFull { + pub added_pool_txs: Vec, + pub remaining_added_pool_txids: ByteArrayVec<32>, +} + +#[cfg(feature = "epee")] +epee_object! { + PoolInfoFull, + added_pool_txs: Vec, + remaining_added_pool_txids: ByteArrayVec<32>, +} + +//---------------------------------------------------------------------------------------------------- PoolInfo epee impl +#[cfg(feature = "epee")] +/// [`EpeeObjectBuilder`] for [`GetBlocksResponse`]. +/// +/// Not for public usage. +#[derive(Clone, Debug, Default, PartialEq, Eq, PartialOrd, Ord, Hash)] +#[cfg_attr(feature = "serde", derive(Serialize, Deserialize))] +pub struct __PoolInfoEpeeBuilder { + /// This is a distinct field in `monerod`, + /// which as represented in this library with [`PoolInfo`]'s `u8` tag. + pub pool_info_extent: Option, + + pub added_pool_txs: Option>, + pub remaining_added_pool_txids: Option>, + pub removed_pool_txids: Option>, +} + +// Custom epee implementation. +// +// HACK/INVARIANT: +// If any data within [`PoolInfo`] changes, the below code should be changed as well. +#[cfg(feature = "epee")] +impl EpeeObjectBuilder for __PoolInfoEpeeBuilder { + fn add_field(&mut self, name: &str, r: &mut B) -> error::Result { + macro_rules! read_epee_field { + ($($field:ident),*) => { + match name { + $( + stringify!($field) => { self.$field = Some(read_epee_value(r)?); }, + )* + _ => return Ok(false), + } + }; + } + + read_epee_field! { + pool_info_extent, + added_pool_txs, + remaining_added_pool_txids, + removed_pool_txids + } + + Ok(true) + } + + fn finish(self) -> error::Result { + // INVARIANT: + // `monerod` omits serializing the field itself when a container is empty, + // `unwrap_or_default()` is used over `error()` in these cases. + // Some of the uses are when values have default fallbacks: `pool_info_extent`. + + let pool_info_extent = self.pool_info_extent.unwrap_or_default(); + let this = match pool_info_extent { + PoolInfoExtent::None => PoolInfo::None, + PoolInfoExtent::Incremental => PoolInfo::Incremental(PoolInfoIncremental { + added_pool_txs: self.added_pool_txs.unwrap_or_default(), + remaining_added_pool_txids: self.remaining_added_pool_txids.unwrap_or_default(), + removed_pool_txids: self.removed_pool_txids.unwrap_or_default(), + }), + PoolInfoExtent::Full => PoolInfo::Full(PoolInfoFull { + added_pool_txs: self.added_pool_txs.unwrap_or_default(), + remaining_added_pool_txids: self.remaining_added_pool_txids.unwrap_or_default(), + }), + }; + + Ok(this) + } +} + +#[cfg(feature = "epee")] +impl EpeeObject for PoolInfo { + type Builder = __PoolInfoEpeeBuilder; + + fn number_of_fields(&self) -> u64 { + // Inner struct fields. + let inner_fields = match self { + Self::None => 0, + Self::Incremental(s) => s.number_of_fields(), + Self::Full(s) => s.number_of_fields(), + }; + + // [`PoolInfoExtent`] + inner struct fields + 1 + inner_fields + } + + fn write_fields(self, w: &mut B) -> error::Result<()> { + const FIELD: &str = "pool_info_extent"; + + match self { + Self::None => { + write_field(PoolInfoExtent::None.to_u8(), FIELD, w)?; + } + Self::Incremental(s) => { + s.write_fields(w)?; + write_field(PoolInfoExtent::Incremental.to_u8(), FIELD, w)?; + } + Self::Full(s) => { + s.write_fields(w)?; + write_field(PoolInfoExtent::Full.to_u8(), FIELD, w)?; + } + } + + Ok(()) + } +} diff --git a/rpc/types/src/misc/tx_entry.rs b/rpc/types/src/misc/tx_entry.rs index 86d02075..59dd4604 100644 --- a/rpc/types/src/misc/tx_entry.rs +++ b/rpc/types/src/misc/tx_entry.rs @@ -2,8 +2,6 @@ //---------------------------------------------------------------------------------------------------- Use #[cfg(feature = "serde")] -use crate::serde::{serde_false, serde_true}; -#[cfg(feature = "serde")] use serde::{Deserialize, Serialize}; #[cfg(feature = "epee")] @@ -13,6 +11,9 @@ use cuprate_epee_encoding::{ EpeeObject, EpeeObjectBuilder, }; +#[cfg(feature = "serde")] +use crate::serde::{serde_false, serde_true}; + //---------------------------------------------------------------------------------------------------- TxEntry #[doc = crate::macros::monero_definition_link!( cc73fe71162d564ffda8e549b79a350bca53c454, From 38541dbfda781277d48dd218bb676aede2f8b8a2 Mon Sep 17 00:00:00 2001 From: hinto-janai Date: Thu, 28 Nov 2024 14:53:59 -0500 Subject: [PATCH 08/10] workspace: add/fix 1.83 lints (#353) * 1.83 `cargo clippy --fix` * fix type complexity, add `DbResult` * clippy fix * redb fix * Update consensus/context/src/difficulty.rs Co-authored-by: hinto-janai --------- Co-authored-by: Boog900 --- Cargo.toml | 4 ++ .../cuprate-json-rpc/benches/response.rs | 2 +- binaries/cuprated/src/statics.rs | 2 +- consensus/context/src/difficulty.rs | 4 +- consensus/fast-sync/src/create.rs | 4 +- net/epee-encoding/src/macros.rs | 6 +- p2p/async-buffer/src/lib.rs | 4 +- p2p/dandelion-tower/src/tests/mod.rs | 1 + p2p/p2p-core/src/lib.rs | 1 - .../src/block_downloader/download_batch.rs | 4 +- p2p/p2p/src/broadcast.rs | 1 + pruning/src/lib.rs | 2 +- rpc/interface/src/route/bin.rs | 2 +- rpc/interface/src/route/other.rs | 2 +- rpc/types/src/json.rs | 2 +- rpc/types/src/macros.rs | 10 +-- rpc/types/src/misc/misc.rs | 2 +- rpc/types/src/other.rs | 2 +- storage/blockchain/src/ops/alt_block/block.rs | 17 +++-- storage/blockchain/src/ops/alt_block/chain.rs | 6 +- storage/blockchain/src/ops/alt_block/tx.rs | 6 +- storage/blockchain/src/ops/block.rs | 23 +++---- storage/blockchain/src/ops/blockchain.rs | 10 ++- storage/blockchain/src/ops/key_image.rs | 8 +-- storage/blockchain/src/ops/macros.rs | 2 +- storage/blockchain/src/ops/output.rs | 29 ++++----- storage/blockchain/src/ops/property.rs | 7 +-- storage/blockchain/src/ops/tx.rs | 22 +++---- storage/blockchain/src/service/read.rs | 14 ++--- storage/blockchain/src/service/types.rs | 4 +- storage/blockchain/src/service/write.rs | 4 +- storage/database/src/backend/heed/database.rs | 63 ++++++++----------- storage/database/src/backend/heed/env.rs | 19 +++--- .../database/src/backend/heed/transaction.rs | 10 +-- storage/database/src/backend/redb/database.rs | 57 ++++++++--------- storage/database/src/backend/redb/env.rs | 19 +++--- storage/database/src/backend/redb/storable.rs | 10 ++- .../database/src/backend/redb/transaction.rs | 8 +-- storage/database/src/config/sync_mode.rs | 1 - storage/database/src/database.rs | 44 +++++++------ storage/database/src/env.rs | 39 ++++++------ storage/database/src/error.rs | 3 + storage/database/src/lib.rs | 2 +- storage/database/src/table.rs | 1 - storage/database/src/tables.rs | 16 ++--- storage/database/src/transaction.rs | 10 +-- storage/service/src/service/read.rs | 10 +-- storage/service/src/service/write.rs | 15 +++-- storage/txpool/src/ops/key_images.rs | 4 +- storage/txpool/src/ops/tx_read.rs | 6 +- storage/txpool/src/ops/tx_write.rs | 7 +-- storage/txpool/src/service/read.rs | 4 +- storage/txpool/src/service/types.rs | 4 +- storage/txpool/src/service/write.rs | 20 +++--- test-utils/src/data/constants.rs | 2 +- test-utils/src/rpc/data/macros.rs | 4 +- 56 files changed, 269 insertions(+), 316 deletions(-) diff --git a/Cargo.toml b/Cargo.toml index a507631e..3cc3ab18 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -279,6 +279,9 @@ rest_pat_in_fully_bound_structs = "deny" redundant_type_annotations = "deny" infinite_loop = "deny" zero_repeat_side_effects = "deny" +non_zero_suggestions = "deny" +manual_is_power_of_two = "deny" +used_underscore_items = "deny" # Warm cast_possible_truncation = "deny" @@ -371,6 +374,7 @@ unused_lifetimes = "deny" unused_macro_rules = "deny" ambiguous_glob_imports = "deny" unused_unsafe = "deny" +rust_2024_compatibility = "deny" # Warm let_underscore = { level = "deny", priority = -1 } diff --git a/benches/criterion/cuprate-json-rpc/benches/response.rs b/benches/criterion/cuprate-json-rpc/benches/response.rs index 908a9f49..890958ec 100644 --- a/benches/criterion/cuprate-json-rpc/benches/response.rs +++ b/benches/criterion/cuprate-json-rpc/benches/response.rs @@ -82,7 +82,7 @@ impl_from_str_benchmark! { macro_rules! impl_to_string_pretty_benchmark { ( $( - $fn_name:ident => $request_constructor:expr, + $fn_name:ident => $request_constructor:expr_2021, )* ) => { $( diff --git a/binaries/cuprated/src/statics.rs b/binaries/cuprated/src/statics.rs index 9839608f..2d7338da 100644 --- a/binaries/cuprated/src/statics.rs +++ b/binaries/cuprated/src/statics.rs @@ -13,7 +13,7 @@ use std::{ macro_rules! define_init_lazylock_statics { ($( $( #[$attr:meta] )* - $name:ident: $t:ty = $init_fn:expr; + $name:ident: $t:ty = $init_fn:expr_2021; )*) => { /// Initialize global static `LazyLock` data. pub fn init_lazylock_statics() { diff --git a/consensus/context/src/difficulty.rs b/consensus/context/src/difficulty.rs index 1b61eb9e..3bbcb059 100644 --- a/consensus/context/src/difficulty.rs +++ b/consensus/context/src/difficulty.rs @@ -328,8 +328,8 @@ fn next_difficulty( time_span = 1; } - // TODO: do checked operations here and unwrap so we don't silently overflow? - (windowed_work * u128::from(hf.block_time().as_secs()) + time_span - 1) / time_span + // TODO: do `checked_mul` here and unwrap so we don't silently overflow? + (windowed_work * u128::from(hf.block_time().as_secs())).div_ceil(time_span) } /// Get the start and end of the window to calculate difficulty. diff --git a/consensus/fast-sync/src/create.rs b/consensus/fast-sync/src/create.rs index 8c47b8e5..9410f60a 100644 --- a/consensus/fast-sync/src/create.rs +++ b/consensus/fast-sync/src/create.rs @@ -9,7 +9,7 @@ use clap::Parser; use tower::{Service, ServiceExt}; use cuprate_blockchain::{ - config::ConfigBuilder, cuprate_database::RuntimeError, service::BlockchainReadHandle, + config::ConfigBuilder, cuprate_database::DbResult, service::BlockchainReadHandle, }; use cuprate_types::{ blockchain::{BlockchainReadRequest, BlockchainResponse}, @@ -23,7 +23,7 @@ const BATCH_SIZE: usize = 512; async fn read_batch( handle: &mut BlockchainReadHandle, height_from: usize, -) -> Result, RuntimeError> { +) -> DbResult> { let mut block_ids = Vec::::with_capacity(BATCH_SIZE); for height in height_from..(height_from + BATCH_SIZE) { diff --git a/net/epee-encoding/src/macros.rs b/net/epee-encoding/src/macros.rs index 38dcc45d..bb1afefd 100644 --- a/net/epee-encoding/src/macros.rs +++ b/net/epee-encoding/src/macros.rs @@ -76,14 +76,14 @@ macro_rules! epee_object { // All this does is return the second (right) arg if present otherwise the left is returned. ( @internal_try_right_then_left - $a:expr, $b:expr + $a:expr_2021, $b:expr_2021 ) => { $b }; ( @internal_try_right_then_left - $a:expr, + $a:expr_2021, ) => { $a }; @@ -122,7 +122,7 @@ macro_rules! epee_object { // ------------------------------------------------------------------------ Entry Point ( $obj:ident, - $($field: ident $(($alt_name: literal))?: $ty:ty $(as $ty_as:ty )? $(= $default:expr)? $(=> $read_fn:expr, $write_fn:expr, $should_write_fn:expr)?, )* + $($field: ident $(($alt_name: literal))?: $ty:ty $(as $ty_as:ty )? $(= $default:expr_2021)? $(=> $read_fn:expr_2021, $write_fn:expr_2021, $should_write_fn:expr_2021)?, )* $(!flatten: $flat_field: ident: $flat_ty:ty ,)* ) => { diff --git a/p2p/async-buffer/src/lib.rs b/p2p/async-buffer/src/lib.rs index 0e2ced24..8174481e 100644 --- a/p2p/async-buffer/src/lib.rs +++ b/p2p/async-buffer/src/lib.rs @@ -157,7 +157,7 @@ pub struct BufferSinkSend<'a, T> { item: Option, } -impl<'a, T> Future for BufferSinkSend<'a, T> { +impl Future for BufferSinkSend<'_, T> { type Output = Result<(), BufferError>; fn poll(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { @@ -183,7 +183,7 @@ pub struct BufferSinkReady<'a, T> { size_needed: usize, } -impl<'a, T> Future for BufferSinkReady<'a, T> { +impl Future for BufferSinkReady<'_, T> { type Output = (); fn poll(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { diff --git a/p2p/dandelion-tower/src/tests/mod.rs b/p2p/dandelion-tower/src/tests/mod.rs index 601ee252..ae9bee1e 100644 --- a/p2p/dandelion-tower/src/tests/mod.rs +++ b/p2p/dandelion-tower/src/tests/mod.rs @@ -12,6 +12,7 @@ use crate::{ OutboundPeer, State, }; +#[expect(clippy::type_complexity)] pub(crate) fn mock_discover_svc() -> ( impl Stream< Item = Result< diff --git a/p2p/p2p-core/src/lib.rs b/p2p/p2p-core/src/lib.rs index 26e10686..e5746932 100644 --- a/p2p/p2p-core/src/lib.rs +++ b/p2p/p2p-core/src/lib.rs @@ -121,7 +121,6 @@ pub trait NetZoneAddress: /// /// - TODO: IP zone banning? /// - TODO: rename this to Host. - type BanID: Debug + Hash + Eq + Clone + Copy + Send + 'static; /// Changes the port of this address to `port`. diff --git a/p2p/p2p/src/block_downloader/download_batch.rs b/p2p/p2p/src/block_downloader/download_batch.rs index ef621ce8..7b6e4c96 100644 --- a/p2p/p2p/src/block_downloader/download_batch.rs +++ b/p2p/p2p/src/block_downloader/download_batch.rs @@ -146,9 +146,9 @@ fn deserialize_batch( // Check the height lines up as expected. // This must happen after the hash check. - if !block + if block .number() - .is_some_and(|height| height == expected_height) + .is_none_or(|height| height != expected_height) { tracing::warn!( "Invalid chain, expected height: {expected_height}, got height: {:?}", diff --git a/p2p/p2p/src/broadcast.rs b/p2p/p2p/src/broadcast.rs index fc73efbc..38aba323 100644 --- a/p2p/p2p/src/broadcast.rs +++ b/p2p/p2p/src/broadcast.rs @@ -57,6 +57,7 @@ impl Default for BroadcastConfig { /// - The [`BroadcastSvc`] /// - A function that takes in [`InternalPeerID`]s and produces [`BroadcastMessageStream`]s to give to **outbound** peers. /// - A function that takes in [`InternalPeerID`]s and produces [`BroadcastMessageStream`]s to give to **inbound** peers. +#[expect(clippy::type_complexity)] pub(crate) fn init_broadcast_channels( config: BroadcastConfig, ) -> ( diff --git a/pruning/src/lib.rs b/pruning/src/lib.rs index cd31598a..e49aedb1 100644 --- a/pruning/src/lib.rs +++ b/pruning/src/lib.rs @@ -327,7 +327,7 @@ impl DecompressedPruningSeed { /// /// This function will also error if `block_height` > `blockchain_height` /// - pub fn get_next_unpruned_block( + pub const fn get_next_unpruned_block( &self, block_height: usize, blockchain_height: usize, diff --git a/rpc/interface/src/route/bin.rs b/rpc/interface/src/route/bin.rs index f7e3a01c..2fd9963c 100644 --- a/rpc/interface/src/route/bin.rs +++ b/rpc/interface/src/route/bin.rs @@ -68,7 +68,7 @@ macro_rules! generate_endpoints_with_no_input { /// - [`generate_endpoints_with_input`] /// - [`generate_endpoints_with_no_input`] macro_rules! generate_endpoints_inner { - ($variant:ident, $handler:ident, $request:expr) => { + ($variant:ident, $handler:ident, $request:expr_2021) => { paste::paste! { { // Check if restricted. diff --git a/rpc/interface/src/route/other.rs b/rpc/interface/src/route/other.rs index 3ff84487..19a58d93 100644 --- a/rpc/interface/src/route/other.rs +++ b/rpc/interface/src/route/other.rs @@ -71,7 +71,7 @@ macro_rules! generate_endpoints_with_no_input { /// - [`generate_endpoints_with_input`] /// - [`generate_endpoints_with_no_input`] macro_rules! generate_endpoints_inner { - ($variant:ident, $handler:ident, $request:expr) => { + ($variant:ident, $handler:ident, $request:expr_2021) => { paste::paste! { { // Check if restricted. diff --git a/rpc/types/src/json.rs b/rpc/types/src/json.rs index cb55e64a..a454cab4 100644 --- a/rpc/types/src/json.rs +++ b/rpc/types/src/json.rs @@ -37,7 +37,7 @@ macro_rules! serde_doc_test { ( // `const` string from `cuprate_test_utils::rpc::data` // v - $cuprate_test_utils_rpc_const:ident => $expected:expr + $cuprate_test_utils_rpc_const:ident => $expected:expr_2021 // ^ // Expected value as an expression ) => { diff --git a/rpc/types/src/macros.rs b/rpc/types/src/macros.rs index 85f4272e..db1b5d8d 100644 --- a/rpc/types/src/macros.rs +++ b/rpc/types/src/macros.rs @@ -77,7 +77,7 @@ macro_rules! define_request_and_response { $( #[$request_field_attr:meta] )* // Field attribute. $request_field:ident: $request_field_type:ty // field_name: field type $(as $request_field_type_as:ty)? // (optional) alternative type (de)serialization - $(= $request_field_type_default:expr, $request_field_type_default_string:literal)?, // (optional) default value + $(= $request_field_type_default:expr_2021, $request_field_type_default_string:literal)?, // (optional) default value )* }, @@ -89,7 +89,7 @@ macro_rules! define_request_and_response { $( #[$response_field_attr:meta] )* $response_field:ident: $response_field_type:ty $(as $response_field_type_as:ty)? - $(= $response_field_type_default:expr, $response_field_type_default_string:literal)?, + $(= $response_field_type_default:expr_2021, $response_field_type_default_string:literal)?, )* } ) => { paste::paste! { @@ -229,7 +229,7 @@ macro_rules! define_request { // field_name: FieldType $field:ident: $field_type:ty $(as $field_as:ty)? - $(= $field_default:expr, $field_default_string:literal)?, + $(= $field_default:expr_2021, $field_default_string:literal)?, // The $field_default is an optional extra token that represents // a default value to pass to [`cuprate_epee_encoding::epee_object`], // see it for usage. @@ -286,7 +286,7 @@ macro_rules! define_response { $( #[$field_attr:meta] )* $field:ident: $field_type:ty $(as $field_as:ty)? - $(= $field_default:expr, $field_default_string:literal)?, + $(= $field_default:expr_2021, $field_default_string:literal)?, )* } ) => { @@ -323,7 +323,7 @@ macro_rules! define_response { $( #[$field_attr:meta] )* $field:ident: $field_type:ty $(as $field_as:ty)? - $(= $field_default:expr, $field_default_string:literal)?, + $(= $field_default:expr_2021, $field_default_string:literal)?, )* } ) => { diff --git a/rpc/types/src/misc/misc.rs b/rpc/types/src/misc/misc.rs index 67ec756d..2d88f2a4 100644 --- a/rpc/types/src/misc/misc.rs +++ b/rpc/types/src/misc/misc.rs @@ -37,7 +37,7 @@ macro_rules! define_struct_and_impl_epee { $( $( #[$field_attr:meta] )* // Field attributes // Field name => the type => optional `epee_object` default value. - $field_name:ident: $field_type:ty $(= $field_default:expr)?, + $field_name:ident: $field_type:ty $(= $field_default:expr_2021)?, )* } ) => { diff --git a/rpc/types/src/other.rs b/rpc/types/src/other.rs index 3694041c..d5cbe82b 100644 --- a/rpc/types/src/other.rs +++ b/rpc/types/src/other.rs @@ -65,7 +65,7 @@ macro_rules! serde_doc_test { ( // `const` string from `cuprate_test_utils::rpc::data` // v - $cuprate_test_utils_rpc_const:ident => $expected:expr + $cuprate_test_utils_rpc_const:ident => $expected:expr_2021 // ^ // Expected value as an expression ) => { diff --git a/storage/blockchain/src/ops/alt_block/block.rs b/storage/blockchain/src/ops/alt_block/block.rs index 6bd01cb3..480bd7d8 100644 --- a/storage/blockchain/src/ops/alt_block/block.rs +++ b/storage/blockchain/src/ops/alt_block/block.rs @@ -1,7 +1,7 @@ use bytemuck::TransparentWrapper; use monero_serai::block::{Block, BlockHeader}; -use cuprate_database::{DatabaseRo, DatabaseRw, RuntimeError, StorableVec}; +use cuprate_database::{DatabaseRo, DatabaseRw, DbResult, StorableVec}; use cuprate_helper::map::{combine_low_high_bits_to_u128, split_u128_into_low_high_bits}; use cuprate_types::{AltBlockInformation, Chain, ChainId, ExtendedBlockHeader, HardFork}; @@ -21,7 +21,7 @@ use crate::{ pub fn flush_alt_blocks<'a, E: cuprate_database::EnvInner<'a>>( env_inner: &E, tx_rw: &mut E::Rw<'_>, -) -> Result<(), RuntimeError> { +) -> DbResult<()> { use crate::tables::{ AltBlockBlobs, AltBlockHeights, AltBlocksInfo, AltChainInfos, AltTransactionBlobs, AltTransactionInfos, @@ -47,10 +47,7 @@ pub fn flush_alt_blocks<'a, E: cuprate_database::EnvInner<'a>>( /// - `alt_block.height` is == `0` /// - `alt_block.txs.len()` != `alt_block.block.transactions.len()` /// -pub fn add_alt_block( - alt_block: &AltBlockInformation, - tables: &mut impl TablesMut, -) -> Result<(), RuntimeError> { +pub fn add_alt_block(alt_block: &AltBlockInformation, tables: &mut impl TablesMut) -> DbResult<()> { let alt_block_height = AltBlockHeight { chain_id: alt_block.chain_id.into(), height: alt_block.height, @@ -100,7 +97,7 @@ pub fn add_alt_block( pub fn get_alt_block( alt_block_height: &AltBlockHeight, tables: &impl Tables, -) -> Result { +) -> DbResult { let block_info = tables.alt_blocks_info().get(alt_block_height)?; let block_blob = tables.alt_block_blobs().get(alt_block_height)?.0; @@ -111,7 +108,7 @@ pub fn get_alt_block( .transactions .iter() .map(|tx_hash| get_alt_transaction(tx_hash, tables)) - .collect::>()?; + .collect::>()?; Ok(AltBlockInformation { block, @@ -141,7 +138,7 @@ pub fn get_alt_block_hash( block_height: &BlockHeight, alt_chain: ChainId, tables: &impl Tables, -) -> Result { +) -> DbResult { let alt_chains = tables.alt_chain_infos(); // First find what [`ChainId`] this block would be stored under. @@ -188,7 +185,7 @@ pub fn get_alt_block_hash( pub fn get_alt_block_extended_header_from_height( height: &AltBlockHeight, table: &impl Tables, -) -> Result { +) -> DbResult { let block_info = table.alt_blocks_info().get(height)?; let block_blob = table.alt_block_blobs().get(height)?.0; diff --git a/storage/blockchain/src/ops/alt_block/chain.rs b/storage/blockchain/src/ops/alt_block/chain.rs index 5b5f3cb1..676fd7f2 100644 --- a/storage/blockchain/src/ops/alt_block/chain.rs +++ b/storage/blockchain/src/ops/alt_block/chain.rs @@ -1,6 +1,6 @@ use std::cmp::{max, min}; -use cuprate_database::{DatabaseRo, DatabaseRw, RuntimeError}; +use cuprate_database::{DatabaseRo, DatabaseRw, DbResult, RuntimeError}; use cuprate_types::{Chain, ChainId}; use crate::{ @@ -21,7 +21,7 @@ pub fn update_alt_chain_info( alt_block_height: &AltBlockHeight, prev_hash: &BlockHash, tables: &mut impl TablesMut, -) -> Result<(), RuntimeError> { +) -> DbResult<()> { let parent_chain = match tables.alt_block_heights().get(prev_hash) { Ok(alt_parent_height) => Chain::Alt(alt_parent_height.chain_id.into()), Err(RuntimeError::KeyNotFound) => Chain::Main, @@ -74,7 +74,7 @@ pub fn get_alt_chain_history_ranges( range: std::ops::Range, alt_chain: ChainId, alt_chain_infos: &impl DatabaseRo, -) -> Result)>, RuntimeError> { +) -> DbResult)>> { let mut ranges = Vec::with_capacity(5); let mut i = range.end; diff --git a/storage/blockchain/src/ops/alt_block/tx.rs b/storage/blockchain/src/ops/alt_block/tx.rs index 4185c6cb..b410fed9 100644 --- a/storage/blockchain/src/ops/alt_block/tx.rs +++ b/storage/blockchain/src/ops/alt_block/tx.rs @@ -1,7 +1,7 @@ use bytemuck::TransparentWrapper; use monero_serai::transaction::Transaction; -use cuprate_database::{DatabaseRo, DatabaseRw, RuntimeError, StorableVec}; +use cuprate_database::{DatabaseRo, DatabaseRw, DbResult, RuntimeError, StorableVec}; use cuprate_types::VerifiedTransactionInformation; use crate::{ @@ -22,7 +22,7 @@ use crate::{ pub fn add_alt_transaction_blob( tx: &VerifiedTransactionInformation, tables: &mut impl TablesMut, -) -> Result<(), RuntimeError> { +) -> DbResult<()> { tables.alt_transaction_infos_mut().put( &tx.tx_hash, &AltTransactionInfo { @@ -51,7 +51,7 @@ pub fn add_alt_transaction_blob( pub fn get_alt_transaction( tx_hash: &TxHash, tables: &impl Tables, -) -> Result { +) -> DbResult { let tx_info = tables.alt_transaction_infos().get(tx_hash)?; let tx_blob = match tables.alt_transaction_blobs().get(tx_hash) { diff --git a/storage/blockchain/src/ops/block.rs b/storage/blockchain/src/ops/block.rs index cc5cb807..5e541878 100644 --- a/storage/blockchain/src/ops/block.rs +++ b/storage/blockchain/src/ops/block.rs @@ -8,7 +8,7 @@ use monero_serai::{ }; use cuprate_database::{ - RuntimeError, StorableVec, {DatabaseRo, DatabaseRw}, + DbResult, RuntimeError, StorableVec, {DatabaseRo, DatabaseRw}, }; use cuprate_helper::{ map::{combine_low_high_bits_to_u128, split_u128_into_low_high_bits}, @@ -44,10 +44,7 @@ use crate::{ /// - `block.height > u32::MAX` (not normally possible) /// - `block.height` is != [`chain_height`] // no inline, too big. -pub fn add_block( - block: &VerifiedBlockInformation, - tables: &mut impl TablesMut, -) -> Result<(), RuntimeError> { +pub fn add_block(block: &VerifiedBlockInformation, tables: &mut impl TablesMut) -> DbResult<()> { //------------------------------------------------------ Check preconditions first // Cast height to `u32` for storage (handled at top of function). @@ -153,7 +150,7 @@ pub fn add_block( pub fn pop_block( move_to_alt_chain: Option, tables: &mut impl TablesMut, -) -> Result<(BlockHeight, BlockHash, Block), RuntimeError> { +) -> DbResult<(BlockHeight, BlockHash, Block)> { //------------------------------------------------------ Block Info // Remove block data from tables. let (block_height, block_info) = tables.block_infos_mut().pop_last()?; @@ -195,7 +192,7 @@ pub fn pop_block( tx, }) }) - .collect::, RuntimeError>>()?; + .collect::>>()?; alt_block::add_alt_block( &AltBlockInformation { @@ -239,7 +236,7 @@ pub fn pop_block( pub fn get_block_extended_header( block_hash: &BlockHash, tables: &impl Tables, -) -> Result { +) -> DbResult { get_block_extended_header_from_height(&tables.block_heights().get(block_hash)?, tables) } @@ -253,7 +250,7 @@ pub fn get_block_extended_header( pub fn get_block_extended_header_from_height( block_height: &BlockHeight, tables: &impl Tables, -) -> Result { +) -> DbResult { let block_info = tables.block_infos().get(block_height)?; let block_header_blob = tables.block_header_blobs().get(block_height)?.0; let block_header = BlockHeader::read(&mut block_header_blob.as_slice())?; @@ -279,7 +276,7 @@ pub fn get_block_extended_header_from_height( #[inline] pub fn get_block_extended_header_top( tables: &impl Tables, -) -> Result<(ExtendedBlockHeader, BlockHeight), RuntimeError> { +) -> DbResult<(ExtendedBlockHeader, BlockHeight)> { let height = chain_height(tables.block_heights())?.saturating_sub(1); let header = get_block_extended_header_from_height(&height, tables)?; Ok((header, height)) @@ -292,7 +289,7 @@ pub fn get_block_extended_header_top( pub fn get_block_info( block_height: &BlockHeight, table_block_infos: &impl DatabaseRo, -) -> Result { +) -> DbResult { table_block_infos.get(block_height) } @@ -302,7 +299,7 @@ pub fn get_block_info( pub fn get_block_height( block_hash: &BlockHash, table_block_heights: &impl DatabaseRo, -) -> Result { +) -> DbResult { table_block_heights.get(block_hash) } @@ -317,7 +314,7 @@ pub fn get_block_height( pub fn block_exists( block_hash: &BlockHash, table_block_heights: &impl DatabaseRo, -) -> Result { +) -> DbResult { table_block_heights.contains(block_hash) } diff --git a/storage/blockchain/src/ops/blockchain.rs b/storage/blockchain/src/ops/blockchain.rs index 04f8b26d..71633635 100644 --- a/storage/blockchain/src/ops/blockchain.rs +++ b/storage/blockchain/src/ops/blockchain.rs @@ -1,7 +1,7 @@ //! Blockchain functions - chain height, generated coins, etc. //---------------------------------------------------------------------------------------------------- Import -use cuprate_database::{DatabaseRo, RuntimeError}; +use cuprate_database::{DatabaseRo, DbResult, RuntimeError}; use crate::{ ops::macros::doc_error, @@ -22,9 +22,7 @@ use crate::{ /// So the height of a new block would be `chain_height()`. #[doc = doc_error!()] #[inline] -pub fn chain_height( - table_block_heights: &impl DatabaseRo, -) -> Result { +pub fn chain_height(table_block_heights: &impl DatabaseRo) -> DbResult { #[expect(clippy::cast_possible_truncation, reason = "we enforce 64-bit")] table_block_heights.len().map(|height| height as usize) } @@ -45,7 +43,7 @@ pub fn chain_height( #[inline] pub fn top_block_height( table_block_heights: &impl DatabaseRo, -) -> Result { +) -> DbResult { match table_block_heights.len()? { 0 => Err(RuntimeError::KeyNotFound), #[expect(clippy::cast_possible_truncation, reason = "we enforce 64-bit")] @@ -70,7 +68,7 @@ pub fn top_block_height( pub fn cumulative_generated_coins( block_height: &BlockHeight, table_block_infos: &impl DatabaseRo, -) -> Result { +) -> DbResult { match table_block_infos.get(block_height) { Ok(block_info) => Ok(block_info.cumulative_generated_coins), Err(RuntimeError::KeyNotFound) if block_height == &0 => Ok(0), diff --git a/storage/blockchain/src/ops/key_image.rs b/storage/blockchain/src/ops/key_image.rs index 19444d6b..5f179129 100644 --- a/storage/blockchain/src/ops/key_image.rs +++ b/storage/blockchain/src/ops/key_image.rs @@ -1,7 +1,7 @@ //! Key image functions. //---------------------------------------------------------------------------------------------------- Import -use cuprate_database::{DatabaseRo, DatabaseRw, RuntimeError}; +use cuprate_database::{DatabaseRo, DatabaseRw, DbResult}; use crate::{ ops::macros::{doc_add_block_inner_invariant, doc_error}, @@ -17,7 +17,7 @@ use crate::{ pub fn add_key_image( key_image: &KeyImage, table_key_images: &mut impl DatabaseRw, -) -> Result<(), RuntimeError> { +) -> DbResult<()> { table_key_images.put(key_image, &()) } @@ -28,7 +28,7 @@ pub fn add_key_image( pub fn remove_key_image( key_image: &KeyImage, table_key_images: &mut impl DatabaseRw, -) -> Result<(), RuntimeError> { +) -> DbResult<()> { table_key_images.delete(key_image) } @@ -38,7 +38,7 @@ pub fn remove_key_image( pub fn key_image_exists( key_image: &KeyImage, table_key_images: &impl DatabaseRo, -) -> Result { +) -> DbResult { table_key_images.contains(key_image) } diff --git a/storage/blockchain/src/ops/macros.rs b/storage/blockchain/src/ops/macros.rs index 18ec5068..9c6ef7dd 100644 --- a/storage/blockchain/src/ops/macros.rs +++ b/storage/blockchain/src/ops/macros.rs @@ -8,7 +8,7 @@ macro_rules! doc_error { () => { r#"# Errors -This function returns [`RuntimeError::KeyNotFound`] if the input (if applicable) doesn't exist or other `RuntimeError`'s on database errors."# +This function returns [`cuprate_database::RuntimeError::KeyNotFound`] if the input (if applicable) doesn't exist or other `RuntimeError`'s on database errors."# }; } pub(super) use doc_error; diff --git a/storage/blockchain/src/ops/output.rs b/storage/blockchain/src/ops/output.rs index 14c209ab..96d94bb1 100644 --- a/storage/blockchain/src/ops/output.rs +++ b/storage/blockchain/src/ops/output.rs @@ -5,7 +5,7 @@ use curve25519_dalek::edwards::CompressedEdwardsY; use monero_serai::transaction::Timelock; use cuprate_database::{ - RuntimeError, {DatabaseRo, DatabaseRw}, + DbResult, RuntimeError, {DatabaseRo, DatabaseRw}, }; use cuprate_helper::crypto::compute_zero_commitment; use cuprate_helper::map::u64_to_timelock; @@ -30,7 +30,7 @@ pub fn add_output( amount: Amount, output: &Output, tables: &mut impl TablesMut, -) -> Result { +) -> DbResult { // FIXME: this would be much better expressed with a // `btree_map::Entry`-like API, fix `trait DatabaseRw`. let num_outputs = match tables.num_outputs().get(&amount) { @@ -61,7 +61,7 @@ pub fn add_output( pub fn remove_output( pre_rct_output_id: &PreRctOutputId, tables: &mut impl TablesMut, -) -> Result<(), RuntimeError> { +) -> DbResult<()> { // Decrement the amount index by 1, or delete the entry out-right. // FIXME: this would be much better expressed with a // `btree_map::Entry`-like API, fix `trait DatabaseRw`. @@ -86,7 +86,7 @@ pub fn remove_output( pub fn get_output( pre_rct_output_id: &PreRctOutputId, table_outputs: &impl DatabaseRo, -) -> Result { +) -> DbResult { table_outputs.get(pre_rct_output_id) } @@ -95,7 +95,7 @@ pub fn get_output( /// This returns the amount of pre-RCT outputs currently stored. #[doc = doc_error!()] #[inline] -pub fn get_num_outputs(table_outputs: &impl DatabaseRo) -> Result { +pub fn get_num_outputs(table_outputs: &impl DatabaseRo) -> DbResult { table_outputs.len() } @@ -110,7 +110,7 @@ pub fn get_num_outputs(table_outputs: &impl DatabaseRo) -> Result, -) -> Result { +) -> DbResult { let amount_index = get_rct_num_outputs(table_rct_outputs)?; table_rct_outputs.put(&amount_index, rct_output)?; Ok(amount_index) @@ -123,7 +123,7 @@ pub fn add_rct_output( pub fn remove_rct_output( amount_index: &AmountIndex, table_rct_outputs: &mut impl DatabaseRw, -) -> Result<(), RuntimeError> { +) -> DbResult<()> { table_rct_outputs.delete(amount_index) } @@ -133,7 +133,7 @@ pub fn remove_rct_output( pub fn get_rct_output( amount_index: &AmountIndex, table_rct_outputs: &impl DatabaseRo, -) -> Result { +) -> DbResult { table_rct_outputs.get(amount_index) } @@ -142,9 +142,7 @@ pub fn get_rct_output( /// This returns the amount of RCT outputs currently stored. #[doc = doc_error!()] #[inline] -pub fn get_rct_num_outputs( - table_rct_outputs: &impl DatabaseRo, -) -> Result { +pub fn get_rct_num_outputs(table_rct_outputs: &impl DatabaseRo) -> DbResult { table_rct_outputs.len() } @@ -155,7 +153,7 @@ pub fn output_to_output_on_chain( output: &Output, amount: Amount, table_tx_unlock_time: &impl DatabaseRo, -) -> Result { +) -> DbResult { let commitment = compute_zero_commitment(amount); let time_lock = if output @@ -191,7 +189,7 @@ pub fn output_to_output_on_chain( pub fn rct_output_to_output_on_chain( rct_output: &RctOutput, table_tx_unlock_time: &impl DatabaseRo, -) -> Result { +) -> DbResult { // INVARIANT: Commitments stored are valid when stored by the database. let commitment = CompressedEdwardsY::from_slice(&rct_output.commitment) .unwrap() @@ -223,10 +221,7 @@ pub fn rct_output_to_output_on_chain( /// /// Note that this still support RCT outputs, in that case, [`PreRctOutputId::amount`] should be `0`. #[doc = doc_error!()] -pub fn id_to_output_on_chain( - id: &PreRctOutputId, - tables: &impl Tables, -) -> Result { +pub fn id_to_output_on_chain(id: &PreRctOutputId, tables: &impl Tables) -> DbResult { // v2 transactions. if id.amount == 0 { let rct_output = get_rct_output(&id.amount_index, tables.rct_outputs())?; diff --git a/storage/blockchain/src/ops/property.rs b/storage/blockchain/src/ops/property.rs index 7810000a..3dbb9509 100644 --- a/storage/blockchain/src/ops/property.rs +++ b/storage/blockchain/src/ops/property.rs @@ -3,10 +3,9 @@ //! SOMEDAY: the database `properties` table is not yet implemented. //---------------------------------------------------------------------------------------------------- Import +use cuprate_database::DbResult; use cuprate_pruning::PruningSeed; -use cuprate_database::RuntimeError; - use crate::ops::macros::doc_error; //---------------------------------------------------------------------------------------------------- Free Functions @@ -20,7 +19,7 @@ use crate::ops::macros::doc_error; /// // SOMEDAY /// ``` #[inline] -pub const fn get_blockchain_pruning_seed() -> Result { +pub const fn get_blockchain_pruning_seed() -> DbResult { // SOMEDAY: impl pruning. // We need a DB properties table. Ok(PruningSeed::NotPruned) @@ -36,7 +35,7 @@ pub const fn get_blockchain_pruning_seed() -> Result /// // SOMEDAY /// ``` #[inline] -pub const fn db_version() -> Result { +pub const fn db_version() -> DbResult { // SOMEDAY: We need a DB properties table. Ok(crate::constants::DATABASE_VERSION) } diff --git a/storage/blockchain/src/ops/tx.rs b/storage/blockchain/src/ops/tx.rs index 5a60ad53..0312f215 100644 --- a/storage/blockchain/src/ops/tx.rs +++ b/storage/blockchain/src/ops/tx.rs @@ -4,7 +4,7 @@ use bytemuck::TransparentWrapper; use monero_serai::transaction::{Input, Timelock, Transaction}; -use cuprate_database::{DatabaseRo, DatabaseRw, RuntimeError, StorableVec}; +use cuprate_database::{DatabaseRo, DatabaseRw, DbResult, RuntimeError, StorableVec}; use cuprate_helper::crypto::compute_zero_commitment; use crate::{ @@ -52,7 +52,7 @@ pub fn add_tx( tx_hash: &TxHash, block_height: &BlockHeight, tables: &mut impl TablesMut, -) -> Result { +) -> DbResult { let tx_id = get_num_tx(tables.tx_ids_mut())?; //------------------------------------------------------ Transaction data @@ -129,7 +129,7 @@ pub fn add_tx( )? .amount_index) }) - .collect::, RuntimeError>>()?, + .collect::>>()?, Transaction::V2 { prefix, proofs } => prefix .outputs .iter() @@ -186,10 +186,7 @@ pub fn add_tx( /// #[doc = doc_error!()] #[inline] -pub fn remove_tx( - tx_hash: &TxHash, - tables: &mut impl TablesMut, -) -> Result<(TxId, Transaction), RuntimeError> { +pub fn remove_tx(tx_hash: &TxHash, tables: &mut impl TablesMut) -> DbResult<(TxId, Transaction)> { //------------------------------------------------------ Transaction data let tx_id = tables.tx_ids_mut().take(tx_hash)?; let tx_blob = tables.tx_blobs_mut().take(&tx_id)?; @@ -267,7 +264,7 @@ pub fn get_tx( tx_hash: &TxHash, table_tx_ids: &impl DatabaseRo, table_tx_blobs: &impl DatabaseRo, -) -> Result { +) -> DbResult { get_tx_from_id(&table_tx_ids.get(tx_hash)?, table_tx_blobs) } @@ -277,7 +274,7 @@ pub fn get_tx( pub fn get_tx_from_id( tx_id: &TxId, table_tx_blobs: &impl DatabaseRo, -) -> Result { +) -> DbResult { let tx_blob = table_tx_blobs.get(tx_id)?.0; Ok(Transaction::read(&mut tx_blob.as_slice())?) } @@ -294,7 +291,7 @@ pub fn get_tx_from_id( /// - etc #[doc = doc_error!()] #[inline] -pub fn get_num_tx(table_tx_ids: &impl DatabaseRo) -> Result { +pub fn get_num_tx(table_tx_ids: &impl DatabaseRo) -> DbResult { table_tx_ids.len() } @@ -304,10 +301,7 @@ pub fn get_num_tx(table_tx_ids: &impl DatabaseRo) -> Result, -) -> Result { +pub fn tx_exists(tx_hash: &TxHash, table_tx_ids: &impl DatabaseRo) -> DbResult { table_tx_ids.contains(tx_hash) } diff --git a/storage/blockchain/src/service/read.rs b/storage/blockchain/src/service/read.rs index e3c01802..76577590 100644 --- a/storage/blockchain/src/service/read.rs +++ b/storage/blockchain/src/service/read.rs @@ -21,7 +21,7 @@ use rayon::{ }; use thread_local::ThreadLocal; -use cuprate_database::{ConcreteEnv, DatabaseRo, Env, EnvInner, RuntimeError}; +use cuprate_database::{ConcreteEnv, DatabaseRo, DbResult, Env, EnvInner, RuntimeError}; use cuprate_database_service::{init_thread_pool, DatabaseReadService, ReaderThreads}; use cuprate_helper::map::combine_low_high_bits_to_u128; use cuprate_types::{ @@ -305,7 +305,7 @@ fn block_extended_header_in_range( let tables = get_tables!(env_inner, tx_ro, tables)?.as_ref(); get_block_extended_header_from_height(&block_height, tables) }) - .collect::, RuntimeError>>()?, + .collect::>>()?, Chain::Alt(chain_id) => { let ranges = { let tx_ro = tx_ro.get_or_try(|| env_inner.tx_ro())?; @@ -381,7 +381,7 @@ fn outputs(env: &ConcreteEnv, outputs: HashMap>) -> // The 2nd mapping function. // This is pulled out from the below `map()` for readability. - let inner_map = |amount, amount_index| -> Result<(AmountIndex, OutputOnChain), RuntimeError> { + let inner_map = |amount, amount_index| -> DbResult<(AmountIndex, OutputOnChain)> { let tx_ro = tx_ro.get_or_try(|| env_inner.tx_ro())?; let tables = get_tables!(env_inner, tx_ro, tables)?.as_ref(); @@ -404,10 +404,10 @@ fn outputs(env: &ConcreteEnv, outputs: HashMap>) -> amount_index_set .into_par_iter() .map(|amount_index| inner_map(amount, amount_index)) - .collect::, RuntimeError>>()?, + .collect::>>()?, )) }) - .collect::>, RuntimeError>>()?; + .collect::>>>()?; Ok(BlockchainResponse::Outputs(map)) } @@ -456,7 +456,7 @@ fn number_outputs_with_amount(env: &ConcreteEnv, amounts: Vec) -> Respon } } }) - .collect::, RuntimeError>>()?; + .collect::>>()?; Ok(BlockchainResponse::NumberOutputsWithAmount(map)) } @@ -522,7 +522,7 @@ fn compact_chain_history(env: &ConcreteEnv) -> ResponseResult { .map(compact_history_index_to_height_offset::) .map_while(|i| top_block_height.checked_sub(i)) .map(|height| Ok(get_block_info(&height, &table_block_infos)?.block_hash)) - .collect::, RuntimeError>>()?; + .collect::>>()?; if compact_history_genesis_not_included::(top_block_height) { block_ids.push(get_block_info(&0, &table_block_infos)?.block_hash); diff --git a/storage/blockchain/src/service/types.rs b/storage/blockchain/src/service/types.rs index 9cd86e9c..190e9f6f 100644 --- a/storage/blockchain/src/service/types.rs +++ b/storage/blockchain/src/service/types.rs @@ -1,7 +1,7 @@ //! Database service type aliases. //---------------------------------------------------------------------------------------------------- Use -use cuprate_database::RuntimeError; +use cuprate_database::DbResult; use cuprate_database_service::{DatabaseReadService, DatabaseWriteHandle}; use cuprate_types::blockchain::{ BlockchainReadRequest, BlockchainResponse, BlockchainWriteRequest, @@ -11,7 +11,7 @@ use cuprate_types::blockchain::{ /// The actual type of the response. /// /// Either our [`BlockchainResponse`], or a database error occurred. -pub(super) type ResponseResult = Result; +pub(super) type ResponseResult = DbResult; /// The blockchain database write service. pub type BlockchainWriteHandle = DatabaseWriteHandle; diff --git a/storage/blockchain/src/service/write.rs b/storage/blockchain/src/service/write.rs index 07162d2a..84c2538f 100644 --- a/storage/blockchain/src/service/write.rs +++ b/storage/blockchain/src/service/write.rs @@ -2,7 +2,7 @@ //---------------------------------------------------------------------------------------------------- Import use std::sync::Arc; -use cuprate_database::{ConcreteEnv, DatabaseRo, Env, EnvInner, RuntimeError, TxRw}; +use cuprate_database::{ConcreteEnv, DatabaseRo, DbResult, Env, EnvInner, TxRw}; use cuprate_database_service::DatabaseWriteHandle; use cuprate_types::{ blockchain::{BlockchainResponse, BlockchainWriteRequest}, @@ -36,7 +36,7 @@ pub fn init_write_service(env: Arc) -> BlockchainWriteHandle { fn handle_blockchain_request( env: &ConcreteEnv, req: &BlockchainWriteRequest, -) -> Result { +) -> DbResult { match req { BlockchainWriteRequest::WriteBlock(block) => write_block(env, block), BlockchainWriteRequest::WriteAltBlock(alt_block) => write_alt_block(env, alt_block), diff --git a/storage/database/src/backend/heed/database.rs b/storage/database/src/backend/heed/database.rs index c985d0de..15f16b45 100644 --- a/storage/database/src/backend/heed/database.rs +++ b/storage/database/src/backend/heed/database.rs @@ -6,7 +6,7 @@ use std::{cell::RefCell, ops::RangeBounds}; use crate::{ backend::heed::types::HeedDb, database::{DatabaseIter, DatabaseRo, DatabaseRw}, - error::RuntimeError, + error::{DbResult, RuntimeError}, table::Table, }; @@ -54,16 +54,13 @@ fn get( db: &HeedDb, tx_ro: &heed::RoTxn<'_>, key: &T::Key, -) -> Result { +) -> DbResult { db.get(tx_ro, key)?.ok_or(RuntimeError::KeyNotFound) } /// Shared [`DatabaseRo::len()`]. #[inline] -fn len( - db: &HeedDb, - tx_ro: &heed::RoTxn<'_>, -) -> Result { +fn len(db: &HeedDb, tx_ro: &heed::RoTxn<'_>) -> DbResult { Ok(db.len(tx_ro)?) } @@ -72,7 +69,7 @@ fn len( fn first( db: &HeedDb, tx_ro: &heed::RoTxn<'_>, -) -> Result<(T::Key, T::Value), RuntimeError> { +) -> DbResult<(T::Key, T::Value)> { db.first(tx_ro)?.ok_or(RuntimeError::KeyNotFound) } @@ -81,16 +78,13 @@ fn first( fn last( db: &HeedDb, tx_ro: &heed::RoTxn<'_>, -) -> Result<(T::Key, T::Value), RuntimeError> { +) -> DbResult<(T::Key, T::Value)> { db.last(tx_ro)?.ok_or(RuntimeError::KeyNotFound) } /// Shared [`DatabaseRo::is_empty()`]. #[inline] -fn is_empty( - db: &HeedDb, - tx_ro: &heed::RoTxn<'_>, -) -> Result { +fn is_empty(db: &HeedDb, tx_ro: &heed::RoTxn<'_>) -> DbResult { Ok(db.is_empty(tx_ro)?) } @@ -100,7 +94,7 @@ impl DatabaseIter for HeedTableRo<'_, T> { fn get_range<'a, Range>( &'a self, range: Range, - ) -> Result> + 'a, RuntimeError> + ) -> DbResult> + 'a> where Range: RangeBounds + 'a, { @@ -108,24 +102,17 @@ impl DatabaseIter for HeedTableRo<'_, T> { } #[inline] - fn iter( - &self, - ) -> Result> + '_, RuntimeError> - { + fn iter(&self) -> DbResult> + '_> { Ok(self.db.iter(self.tx_ro)?.map(|res| Ok(res?))) } #[inline] - fn keys( - &self, - ) -> Result> + '_, RuntimeError> { + fn keys(&self) -> DbResult> + '_> { Ok(self.db.iter(self.tx_ro)?.map(|res| Ok(res?.0))) } #[inline] - fn values( - &self, - ) -> Result> + '_, RuntimeError> { + fn values(&self) -> DbResult> + '_> { Ok(self.db.iter(self.tx_ro)?.map(|res| Ok(res?.1))) } } @@ -134,27 +121,27 @@ impl DatabaseIter for HeedTableRo<'_, T> { // SAFETY: `HeedTableRo: !Send` as it holds a reference to `heed::RoTxn: Send + !Sync`. unsafe impl DatabaseRo for HeedTableRo<'_, T> { #[inline] - fn get(&self, key: &T::Key) -> Result { + fn get(&self, key: &T::Key) -> DbResult { get::(&self.db, self.tx_ro, key) } #[inline] - fn len(&self) -> Result { + fn len(&self) -> DbResult { len::(&self.db, self.tx_ro) } #[inline] - fn first(&self) -> Result<(T::Key, T::Value), RuntimeError> { + fn first(&self) -> DbResult<(T::Key, T::Value)> { first::(&self.db, self.tx_ro) } #[inline] - fn last(&self) -> Result<(T::Key, T::Value), RuntimeError> { + fn last(&self) -> DbResult<(T::Key, T::Value)> { last::(&self.db, self.tx_ro) } #[inline] - fn is_empty(&self) -> Result { + fn is_empty(&self) -> DbResult { is_empty::(&self.db, self.tx_ro) } } @@ -164,45 +151,45 @@ unsafe impl DatabaseRo for HeedTableRo<'_, T> { // `HeedTableRw`'s write transaction is `!Send`. unsafe impl DatabaseRo for HeedTableRw<'_, '_, T> { #[inline] - fn get(&self, key: &T::Key) -> Result { + fn get(&self, key: &T::Key) -> DbResult { get::(&self.db, &self.tx_rw.borrow(), key) } #[inline] - fn len(&self) -> Result { + fn len(&self) -> DbResult { len::(&self.db, &self.tx_rw.borrow()) } #[inline] - fn first(&self) -> Result<(T::Key, T::Value), RuntimeError> { + fn first(&self) -> DbResult<(T::Key, T::Value)> { first::(&self.db, &self.tx_rw.borrow()) } #[inline] - fn last(&self) -> Result<(T::Key, T::Value), RuntimeError> { + fn last(&self) -> DbResult<(T::Key, T::Value)> { last::(&self.db, &self.tx_rw.borrow()) } #[inline] - fn is_empty(&self) -> Result { + fn is_empty(&self) -> DbResult { is_empty::(&self.db, &self.tx_rw.borrow()) } } impl DatabaseRw for HeedTableRw<'_, '_, T> { #[inline] - fn put(&mut self, key: &T::Key, value: &T::Value) -> Result<(), RuntimeError> { + fn put(&mut self, key: &T::Key, value: &T::Value) -> DbResult<()> { Ok(self.db.put(&mut self.tx_rw.borrow_mut(), key, value)?) } #[inline] - fn delete(&mut self, key: &T::Key) -> Result<(), RuntimeError> { + fn delete(&mut self, key: &T::Key) -> DbResult<()> { self.db.delete(&mut self.tx_rw.borrow_mut(), key)?; Ok(()) } #[inline] - fn take(&mut self, key: &T::Key) -> Result { + fn take(&mut self, key: &T::Key) -> DbResult { // LMDB/heed does not return the value on deletion. // So, fetch it first - then delete. let value = get::(&self.db, &self.tx_rw.borrow(), key)?; @@ -216,7 +203,7 @@ impl DatabaseRw for HeedTableRw<'_, '_, T> { } #[inline] - fn pop_first(&mut self) -> Result<(T::Key, T::Value), RuntimeError> { + fn pop_first(&mut self) -> DbResult<(T::Key, T::Value)> { let tx_rw = &mut self.tx_rw.borrow_mut(); // Get the value first... @@ -235,7 +222,7 @@ impl DatabaseRw for HeedTableRw<'_, '_, T> { } #[inline] - fn pop_last(&mut self) -> Result<(T::Key, T::Value), RuntimeError> { + fn pop_last(&mut self) -> DbResult<(T::Key, T::Value)> { let tx_rw = &mut self.tx_rw.borrow_mut(); // Get the value first... diff --git a/storage/database/src/backend/heed/env.rs b/storage/database/src/backend/heed/env.rs index 568379e5..b603013b 100644 --- a/storage/database/src/backend/heed/env.rs +++ b/storage/database/src/backend/heed/env.rs @@ -18,7 +18,7 @@ use crate::{ config::{Config, SyncMode}, database::{DatabaseIter, DatabaseRo, DatabaseRw}, env::{Env, EnvInner}, - error::{InitError, RuntimeError}, + error::{DbResult, InitError, RuntimeError}, key::{Key, KeyCompare}, resize::ResizeAlgorithm, table::Table, @@ -203,7 +203,7 @@ impl Env for ConcreteEnv { &self.config } - fn sync(&self) -> Result<(), RuntimeError> { + fn sync(&self) -> DbResult<()> { Ok(self.env.read().unwrap().force_sync()?) } @@ -253,12 +253,12 @@ where type Rw<'a> = RefCell>; #[inline] - fn tx_ro(&self) -> Result, RuntimeError> { + fn tx_ro(&self) -> DbResult> { Ok(self.read_txn()?) } #[inline] - fn tx_rw(&self) -> Result, RuntimeError> { + fn tx_rw(&self) -> DbResult> { Ok(RefCell::new(self.write_txn()?)) } @@ -266,7 +266,7 @@ where fn open_db_ro( &self, tx_ro: &Self::Ro<'_>, - ) -> Result + DatabaseIter, RuntimeError> { + ) -> DbResult + DatabaseIter> { // Open up a read-only database using our table's const metadata. // // INVARIANT: LMDB caches the ordering / comparison function from [`EnvInner::create_db`], @@ -281,10 +281,7 @@ where } #[inline] - fn open_db_rw( - &self, - tx_rw: &Self::Rw<'_>, - ) -> Result, RuntimeError> { + fn open_db_rw(&self, tx_rw: &Self::Rw<'_>) -> DbResult> { // Open up a read/write database using our table's const metadata. // // INVARIANT: LMDB caches the ordering / comparison function from [`EnvInner::create_db`], @@ -296,7 +293,7 @@ where }) } - fn create_db(&self, tx_rw: &Self::Rw<'_>) -> Result<(), RuntimeError> { + fn create_db(&self, tx_rw: &Self::Rw<'_>) -> DbResult<()> { // Create a database using our: // - [`Table`]'s const metadata. // - (potentially) our [`Key`] comparison function @@ -328,7 +325,7 @@ where } #[inline] - fn clear_db(&self, tx_rw: &mut Self::Rw<'_>) -> Result<(), RuntimeError> { + fn clear_db(&self, tx_rw: &mut Self::Rw<'_>) -> DbResult<()> { let tx_rw = tx_rw.get_mut(); // Open the table. We don't care about flags or key diff --git a/storage/database/src/backend/heed/transaction.rs b/storage/database/src/backend/heed/transaction.rs index d32f3707..b7c0f54d 100644 --- a/storage/database/src/backend/heed/transaction.rs +++ b/storage/database/src/backend/heed/transaction.rs @@ -4,31 +4,31 @@ use std::cell::RefCell; //---------------------------------------------------------------------------------------------------- Import use crate::{ - error::RuntimeError, + error::DbResult, transaction::{TxRo, TxRw}, }; //---------------------------------------------------------------------------------------------------- TxRo impl TxRo<'_> for heed::RoTxn<'_> { - fn commit(self) -> Result<(), RuntimeError> { + fn commit(self) -> DbResult<()> { Ok(heed::RoTxn::commit(self)?) } } //---------------------------------------------------------------------------------------------------- TxRw impl TxRo<'_> for RefCell> { - fn commit(self) -> Result<(), RuntimeError> { + fn commit(self) -> DbResult<()> { TxRw::commit(self) } } impl TxRw<'_> for RefCell> { - fn commit(self) -> Result<(), RuntimeError> { + fn commit(self) -> DbResult<()> { Ok(heed::RwTxn::commit(self.into_inner())?) } /// This function is infallible. - fn abort(self) -> Result<(), RuntimeError> { + fn abort(self) -> DbResult<()> { heed::RwTxn::abort(self.into_inner()); Ok(()) } diff --git a/storage/database/src/backend/redb/database.rs b/storage/database/src/backend/redb/database.rs index dafb2417..0be58ef0 100644 --- a/storage/database/src/backend/redb/database.rs +++ b/storage/database/src/backend/redb/database.rs @@ -11,7 +11,7 @@ use crate::{ types::{RedbTableRo, RedbTableRw}, }, database::{DatabaseIter, DatabaseRo, DatabaseRw}, - error::RuntimeError, + error::{DbResult, RuntimeError}, table::Table, }; @@ -25,7 +25,7 @@ use crate::{ fn get( db: &impl ReadableTable, StorableRedb>, key: &T::Key, -) -> Result { +) -> DbResult { Ok(db.get(key)?.ok_or(RuntimeError::KeyNotFound)?.value()) } @@ -33,7 +33,7 @@ fn get( #[inline] fn len( db: &impl ReadableTable, StorableRedb>, -) -> Result { +) -> DbResult { Ok(db.len()?) } @@ -41,7 +41,7 @@ fn len( #[inline] fn first( db: &impl ReadableTable, StorableRedb>, -) -> Result<(T::Key, T::Value), RuntimeError> { +) -> DbResult<(T::Key, T::Value)> { let (key, value) = db.first()?.ok_or(RuntimeError::KeyNotFound)?; Ok((key.value(), value.value())) } @@ -50,7 +50,7 @@ fn first( #[inline] fn last( db: &impl ReadableTable, StorableRedb>, -) -> Result<(T::Key, T::Value), RuntimeError> { +) -> DbResult<(T::Key, T::Value)> { let (key, value) = db.last()?.ok_or(RuntimeError::KeyNotFound)?; Ok((key.value(), value.value())) } @@ -59,7 +59,7 @@ fn last( #[inline] fn is_empty( db: &impl ReadableTable, StorableRedb>, -) -> Result { +) -> DbResult { Ok(db.is_empty()?) } @@ -69,7 +69,7 @@ impl DatabaseIter for RedbTableRo { fn get_range<'a, Range>( &'a self, range: Range, - ) -> Result> + 'a, RuntimeError> + ) -> DbResult> + 'a> where Range: RangeBounds + 'a, { @@ -80,10 +80,7 @@ impl DatabaseIter for RedbTableRo { } #[inline] - fn iter( - &self, - ) -> Result> + '_, RuntimeError> - { + fn iter(&self) -> DbResult> + '_> { Ok(ReadableTable::iter(self)?.map(|result| { let (key, value) = result?; Ok((key.value(), value.value())) @@ -91,9 +88,7 @@ impl DatabaseIter for RedbTableRo { } #[inline] - fn keys( - &self, - ) -> Result> + '_, RuntimeError> { + fn keys(&self) -> DbResult> + '_> { Ok(ReadableTable::iter(self)?.map(|result| { let (key, _value) = result?; Ok(key.value()) @@ -101,9 +96,7 @@ impl DatabaseIter for RedbTableRo { } #[inline] - fn values( - &self, - ) -> Result> + '_, RuntimeError> { + fn values(&self) -> DbResult> + '_> { Ok(ReadableTable::iter(self)?.map(|result| { let (_key, value) = result?; Ok(value.value()) @@ -115,27 +108,27 @@ impl DatabaseIter for RedbTableRo { // SAFETY: Both `redb`'s transaction and table types are `Send + Sync`. unsafe impl DatabaseRo for RedbTableRo { #[inline] - fn get(&self, key: &T::Key) -> Result { + fn get(&self, key: &T::Key) -> DbResult { get::(self, key) } #[inline] - fn len(&self) -> Result { + fn len(&self) -> DbResult { len::(self) } #[inline] - fn first(&self) -> Result<(T::Key, T::Value), RuntimeError> { + fn first(&self) -> DbResult<(T::Key, T::Value)> { first::(self) } #[inline] - fn last(&self) -> Result<(T::Key, T::Value), RuntimeError> { + fn last(&self) -> DbResult<(T::Key, T::Value)> { last::(self) } #[inline] - fn is_empty(&self) -> Result { + fn is_empty(&self) -> DbResult { is_empty::(self) } } @@ -144,27 +137,27 @@ unsafe impl DatabaseRo for RedbTableRo // SAFETY: Both `redb`'s transaction and table types are `Send + Sync`. unsafe impl DatabaseRo for RedbTableRw<'_, T::Key, T::Value> { #[inline] - fn get(&self, key: &T::Key) -> Result { + fn get(&self, key: &T::Key) -> DbResult { get::(self, key) } #[inline] - fn len(&self) -> Result { + fn len(&self) -> DbResult { len::(self) } #[inline] - fn first(&self) -> Result<(T::Key, T::Value), RuntimeError> { + fn first(&self) -> DbResult<(T::Key, T::Value)> { first::(self) } #[inline] - fn last(&self) -> Result<(T::Key, T::Value), RuntimeError> { + fn last(&self) -> DbResult<(T::Key, T::Value)> { last::(self) } #[inline] - fn is_empty(&self) -> Result { + fn is_empty(&self) -> DbResult { is_empty::(self) } } @@ -173,19 +166,19 @@ impl DatabaseRw for RedbTableRw<'_, T::Key, T::Value> { // `redb` returns the value after function calls so we end with Ok(()) instead. #[inline] - fn put(&mut self, key: &T::Key, value: &T::Value) -> Result<(), RuntimeError> { + fn put(&mut self, key: &T::Key, value: &T::Value) -> DbResult<()> { redb::Table::insert(self, key, value)?; Ok(()) } #[inline] - fn delete(&mut self, key: &T::Key) -> Result<(), RuntimeError> { + fn delete(&mut self, key: &T::Key) -> DbResult<()> { redb::Table::remove(self, key)?; Ok(()) } #[inline] - fn take(&mut self, key: &T::Key) -> Result { + fn take(&mut self, key: &T::Key) -> DbResult { if let Some(value) = redb::Table::remove(self, key)? { Ok(value.value()) } else { @@ -194,13 +187,13 @@ impl DatabaseRw for RedbTableRw<'_, T::Key, T::Value> { } #[inline] - fn pop_first(&mut self) -> Result<(T::Key, T::Value), RuntimeError> { + fn pop_first(&mut self) -> DbResult<(T::Key, T::Value)> { let (key, value) = redb::Table::pop_first(self)?.ok_or(RuntimeError::KeyNotFound)?; Ok((key.value(), value.value())) } #[inline] - fn pop_last(&mut self) -> Result<(T::Key, T::Value), RuntimeError> { + fn pop_last(&mut self) -> DbResult<(T::Key, T::Value)> { let (key, value) = redb::Table::pop_last(self)?.ok_or(RuntimeError::KeyNotFound)?; Ok((key.value(), value.value())) } diff --git a/storage/database/src/backend/redb/env.rs b/storage/database/src/backend/redb/env.rs index a405ea72..4bd49d68 100644 --- a/storage/database/src/backend/redb/env.rs +++ b/storage/database/src/backend/redb/env.rs @@ -6,7 +6,7 @@ use crate::{ config::{Config, SyncMode}, database::{DatabaseIter, DatabaseRo, DatabaseRw}, env::{Env, EnvInner}, - error::{InitError, RuntimeError}, + error::{DbResult, InitError, RuntimeError}, table::Table, TxRw, }; @@ -105,7 +105,7 @@ impl Env for ConcreteEnv { &self.config } - fn sync(&self) -> Result<(), RuntimeError> { + fn sync(&self) -> DbResult<()> { // `redb`'s syncs are tied with write transactions, // so just create one, don't do anything and commit. let mut tx_rw = self.env.begin_write()?; @@ -127,12 +127,12 @@ where type Rw<'a> = redb::WriteTransaction; #[inline] - fn tx_ro(&self) -> Result { + fn tx_ro(&self) -> DbResult { Ok(self.0.begin_read()?) } #[inline] - fn tx_rw(&self) -> Result { + fn tx_rw(&self) -> DbResult { // `redb` has sync modes on the TX level, unlike heed, // which sets it at the Environment level. // @@ -146,7 +146,7 @@ where fn open_db_ro( &self, tx_ro: &Self::Ro<'_>, - ) -> Result + DatabaseIter, RuntimeError> { + ) -> DbResult + DatabaseIter> { // Open up a read-only database using our `T: Table`'s const metadata. let table: redb::TableDefinition<'static, StorableRedb, StorableRedb> = redb::TableDefinition::new(T::NAME); @@ -155,10 +155,7 @@ where } #[inline] - fn open_db_rw( - &self, - tx_rw: &Self::Rw<'_>, - ) -> Result, RuntimeError> { + fn open_db_rw(&self, tx_rw: &Self::Rw<'_>) -> DbResult> { // Open up a read/write database using our `T: Table`'s const metadata. let table: redb::TableDefinition<'static, StorableRedb, StorableRedb> = redb::TableDefinition::new(T::NAME); @@ -168,14 +165,14 @@ where Ok(tx_rw.open_table(table)?) } - fn create_db(&self, tx_rw: &redb::WriteTransaction) -> Result<(), RuntimeError> { + fn create_db(&self, tx_rw: &redb::WriteTransaction) -> DbResult<()> { // INVARIANT: `redb` creates tables if they don't exist. self.open_db_rw::(tx_rw)?; Ok(()) } #[inline] - fn clear_db(&self, tx_rw: &mut redb::WriteTransaction) -> Result<(), RuntimeError> { + fn clear_db(&self, tx_rw: &mut redb::WriteTransaction) -> DbResult<()> { let table: redb::TableDefinition< 'static, StorableRedb<::Key>, diff --git a/storage/database/src/backend/redb/storable.rs b/storage/database/src/backend/redb/storable.rs index abf2e71b..f0412efb 100644 --- a/storage/database/src/backend/redb/storable.rs +++ b/storage/database/src/backend/redb/storable.rs @@ -34,8 +34,14 @@ impl redb::Value for StorableRedb where T: Storable + 'static, { - type SelfType<'a> = T where Self: 'a; - type AsBytes<'a> = &'a [u8] where Self: 'a; + type SelfType<'a> + = T + where + Self: 'a; + type AsBytes<'a> + = &'a [u8] + where + Self: 'a; #[inline] fn fixed_width() -> Option { diff --git a/storage/database/src/backend/redb/transaction.rs b/storage/database/src/backend/redb/transaction.rs index 5048851d..8d93986d 100644 --- a/storage/database/src/backend/redb/transaction.rs +++ b/storage/database/src/backend/redb/transaction.rs @@ -2,14 +2,14 @@ //---------------------------------------------------------------------------------------------------- Import use crate::{ - error::RuntimeError, + error::DbResult, transaction::{TxRo, TxRw}, }; //---------------------------------------------------------------------------------------------------- TxRo impl TxRo<'_> for redb::ReadTransaction { /// This function is infallible. - fn commit(self) -> Result<(), RuntimeError> { + fn commit(self) -> DbResult<()> { // `redb`'s read transactions cleanup automatically when all references are dropped. // // There is `close()`: @@ -22,11 +22,11 @@ impl TxRo<'_> for redb::ReadTransaction { //---------------------------------------------------------------------------------------------------- TxRw impl TxRw<'_> for redb::WriteTransaction { - fn commit(self) -> Result<(), RuntimeError> { + fn commit(self) -> DbResult<()> { Ok(self.commit()?) } - fn abort(self) -> Result<(), RuntimeError> { + fn abort(self) -> DbResult<()> { Ok(self.abort()?) } } diff --git a/storage/database/src/config/sync_mode.rs b/storage/database/src/config/sync_mode.rs index 5a0cba52..dbb34e7f 100644 --- a/storage/database/src/config/sync_mode.rs +++ b/storage/database/src/config/sync_mode.rs @@ -9,7 +9,6 @@ //! based on these values. //---------------------------------------------------------------------------------------------------- Import - #[cfg(feature = "serde")] use serde::{Deserialize, Serialize}; diff --git a/storage/database/src/database.rs b/storage/database/src/database.rs index 6fbb7aaa..c019972b 100644 --- a/storage/database/src/database.rs +++ b/storage/database/src/database.rs @@ -3,7 +3,10 @@ //---------------------------------------------------------------------------------------------------- Import use std::ops::RangeBounds; -use crate::{error::RuntimeError, table::Table}; +use crate::{ + error::{DbResult, RuntimeError}, + table::Table, +}; //---------------------------------------------------------------------------------------------------- DatabaseIter /// Generic post-fix documentation for `DatabaseIter` methods. @@ -48,27 +51,22 @@ pub trait DatabaseIter { fn get_range<'a, Range>( &'a self, range: Range, - ) -> Result> + 'a, RuntimeError> + ) -> DbResult> + 'a> where Range: RangeBounds + 'a; /// Get an [`Iterator`] that returns the `(key, value)` types for this database. #[doc = doc_iter!()] #[expect(clippy::iter_not_returning_iterator)] - fn iter( - &self, - ) -> Result> + '_, RuntimeError>; + fn iter(&self) -> DbResult> + '_>; /// Get an [`Iterator`] that returns _only_ the `key` type for this database. #[doc = doc_iter!()] - fn keys(&self) - -> Result> + '_, RuntimeError>; + fn keys(&self) -> DbResult> + '_>; /// Get an [`Iterator`] that returns _only_ the `value` type for this database. #[doc = doc_iter!()] - fn values( - &self, - ) -> Result> + '_, RuntimeError>; + fn values(&self) -> DbResult> + '_>; } //---------------------------------------------------------------------------------------------------- DatabaseRo @@ -76,7 +74,7 @@ pub trait DatabaseIter { macro_rules! doc_database { () => { r"# Errors -This will return [`RuntimeError::KeyNotFound`] if: +This will return [`crate::RuntimeError::KeyNotFound`] if: - Input does not exist OR - Database is empty" }; @@ -111,7 +109,7 @@ This will return [`RuntimeError::KeyNotFound`] if: pub unsafe trait DatabaseRo { /// Get the value corresponding to a key. #[doc = doc_database!()] - fn get(&self, key: &T::Key) -> Result; + fn get(&self, key: &T::Key) -> DbResult; /// Returns `true` if the database contains a value for the specified key. /// @@ -120,7 +118,7 @@ pub unsafe trait DatabaseRo { /// as in that case, `Ok(false)` will be returned. /// /// Other errors may still occur. - fn contains(&self, key: &T::Key) -> Result { + fn contains(&self, key: &T::Key) -> DbResult { match self.get(key) { Ok(_) => Ok(true), Err(RuntimeError::KeyNotFound) => Ok(false), @@ -132,21 +130,21 @@ pub unsafe trait DatabaseRo { /// /// # Errors /// This will never return [`RuntimeError::KeyNotFound`]. - fn len(&self) -> Result; + fn len(&self) -> DbResult; /// Returns the first `(key, value)` pair in the database. #[doc = doc_database!()] - fn first(&self) -> Result<(T::Key, T::Value), RuntimeError>; + fn first(&self) -> DbResult<(T::Key, T::Value)>; /// Returns the last `(key, value)` pair in the database. #[doc = doc_database!()] - fn last(&self) -> Result<(T::Key, T::Value), RuntimeError>; + fn last(&self) -> DbResult<(T::Key, T::Value)>; /// Returns `true` if the database contains no `(key, value)` pairs. /// /// # Errors /// This can only return [`RuntimeError::Io`] on errors. - fn is_empty(&self) -> Result; + fn is_empty(&self) -> DbResult; } //---------------------------------------------------------------------------------------------------- DatabaseRw @@ -161,7 +159,7 @@ pub trait DatabaseRw: DatabaseRo { #[doc = doc_database!()] /// /// This will never [`RuntimeError::KeyExists`]. - fn put(&mut self, key: &T::Key, value: &T::Value) -> Result<(), RuntimeError>; + fn put(&mut self, key: &T::Key, value: &T::Value) -> DbResult<()>; /// Delete a key-value pair in the database. /// @@ -170,7 +168,7 @@ pub trait DatabaseRw: DatabaseRo { #[doc = doc_database!()] /// /// This will never [`RuntimeError::KeyExists`]. - fn delete(&mut self, key: &T::Key) -> Result<(), RuntimeError>; + fn delete(&mut self, key: &T::Key) -> DbResult<()>; /// Delete and return a key-value pair in the database. /// @@ -178,7 +176,7 @@ pub trait DatabaseRw: DatabaseRo { /// it will serialize the `T::Value` and return it. /// #[doc = doc_database!()] - fn take(&mut self, key: &T::Key) -> Result; + fn take(&mut self, key: &T::Key) -> DbResult; /// Fetch the value, and apply a function to it - or delete the entry. /// @@ -192,7 +190,7 @@ pub trait DatabaseRw: DatabaseRo { /// - If `f` returns `None`, the entry will be [`DatabaseRw::delete`]d /// #[doc = doc_database!()] - fn update(&mut self, key: &T::Key, mut f: F) -> Result<(), RuntimeError> + fn update(&mut self, key: &T::Key, mut f: F) -> DbResult<()> where F: FnMut(T::Value) -> Option, { @@ -207,10 +205,10 @@ pub trait DatabaseRw: DatabaseRo { /// Removes and returns the first `(key, value)` pair in the database. /// #[doc = doc_database!()] - fn pop_first(&mut self) -> Result<(T::Key, T::Value), RuntimeError>; + fn pop_first(&mut self) -> DbResult<(T::Key, T::Value)>; /// Removes and returns the last `(key, value)` pair in the database. /// #[doc = doc_database!()] - fn pop_last(&mut self) -> Result<(T::Key, T::Value), RuntimeError>; + fn pop_last(&mut self) -> DbResult<(T::Key, T::Value)>; } diff --git a/storage/database/src/env.rs b/storage/database/src/env.rs index 1ae6aa1f..56b92cbd 100644 --- a/storage/database/src/env.rs +++ b/storage/database/src/env.rs @@ -6,7 +6,7 @@ use std::num::NonZeroUsize; use crate::{ config::Config, database::{DatabaseIter, DatabaseRo, DatabaseRw}, - error::{InitError, RuntimeError}, + error::{DbResult, InitError}, resize::ResizeAlgorithm, table::Table, transaction::{TxRo, TxRw}, @@ -39,7 +39,7 @@ pub trait Env: Sized { /// /// # Invariant /// If this is `false`, that means this [`Env`] - /// must _never_ return a [`RuntimeError::ResizeNeeded`]. + /// must _never_ return a [`crate::RuntimeError::ResizeNeeded`]. /// /// If this is `true`, [`Env::resize_map`] & [`Env::current_map_size`] /// _must_ be re-implemented, as it just panics by default. @@ -88,7 +88,7 @@ pub trait Env: Sized { /// This will error if the database file could not be opened. /// /// This is the only [`Env`] function that will return - /// an [`InitError`] instead of a [`RuntimeError`]. + /// an [`InitError`] instead of a [`crate::RuntimeError`]. fn open(config: Config) -> Result; /// Return the [`Config`] that this database was [`Env::open`]ed with. @@ -107,7 +107,7 @@ pub trait Env: Sized { /// /// # Errors /// If there is a synchronization error, this should return an error. - fn sync(&self) -> Result<(), RuntimeError>; + fn sync(&self) -> DbResult<()>; /// Resize the database's memory map to a /// new (bigger) size using a [`ResizeAlgorithm`]. @@ -218,14 +218,14 @@ pub trait EnvInner<'env> { /// Create a read-only transaction. /// /// # Errors - /// This will only return [`RuntimeError::Io`] if it errors. - fn tx_ro(&self) -> Result, RuntimeError>; + /// This will only return [`crate::RuntimeError::Io`] if it errors. + fn tx_ro(&self) -> DbResult>; /// Create a read/write transaction. /// /// # Errors - /// This will only return [`RuntimeError::Io`] if it errors. - fn tx_rw(&self) -> Result, RuntimeError>; + /// This will only return [`crate::RuntimeError::Io`] if it errors. + fn tx_rw(&self) -> DbResult>; /// Open a database in read-only mode. /// @@ -269,17 +269,17 @@ pub trait EnvInner<'env> { /// ``` /// /// # Errors - /// This will only return [`RuntimeError::Io`] on normal errors. + /// This will only return [`crate::RuntimeError::Io`] on normal errors. /// /// If the specified table is not created upon before this function is called, - /// this will return [`RuntimeError::TableNotFound`]. + /// this will return [`crate::RuntimeError::TableNotFound`]. /// /// # Invariant #[doc = doc_heed_create_db_invariant!()] fn open_db_ro( &self, tx_ro: &Self::Ro<'_>, - ) -> Result + DatabaseIter, RuntimeError>; + ) -> DbResult + DatabaseIter>; /// Open a database in read/write mode. /// @@ -293,25 +293,22 @@ pub trait EnvInner<'env> { /// passed as a generic to this function. /// /// # Errors - /// This will only return [`RuntimeError::Io`] on errors. + /// This will only return [`crate::RuntimeError::Io`] on errors. /// /// # Invariant #[doc = doc_heed_create_db_invariant!()] - fn open_db_rw( - &self, - tx_rw: &Self::Rw<'_>, - ) -> Result, RuntimeError>; + fn open_db_rw(&self, tx_rw: &Self::Rw<'_>) -> DbResult>; /// Create a database table. /// /// This will create the database [`Table`] passed as a generic to this function. /// /// # Errors - /// This will only return [`RuntimeError::Io`] on errors. + /// This will only return [`crate::RuntimeError::Io`] on errors. /// /// # Invariant #[doc = doc_heed_create_db_invariant!()] - fn create_db(&self, tx_rw: &Self::Rw<'_>) -> Result<(), RuntimeError>; + fn create_db(&self, tx_rw: &Self::Rw<'_>) -> DbResult<()>; /// Clear all `(key, value)`'s from a database table. /// @@ -322,9 +319,9 @@ pub trait EnvInner<'env> { /// function's effects can be aborted using [`TxRw::abort`]. /// /// # Errors - /// This will return [`RuntimeError::Io`] on normal errors. + /// This will return [`crate::RuntimeError::Io`] on normal errors. /// /// If the specified table is not created upon before this function is called, - /// this will return [`RuntimeError::TableNotFound`]. - fn clear_db(&self, tx_rw: &mut Self::Rw<'_>) -> Result<(), RuntimeError>; + /// this will return [`crate::RuntimeError::TableNotFound`]. + fn clear_db(&self, tx_rw: &mut Self::Rw<'_>) -> DbResult<()>; } diff --git a/storage/database/src/error.rs b/storage/database/src/error.rs index 3471ac74..82f80b9a 100644 --- a/storage/database/src/error.rs +++ b/storage/database/src/error.rs @@ -7,6 +7,9 @@ use std::fmt::Debug; /// Alias for a thread-safe boxed error. type BoxError = Box; +/// [`Result`] with [`RuntimeError`] as the error. +pub type DbResult = Result; + //---------------------------------------------------------------------------------------------------- InitError /// Errors that occur during ([`Env::open`](crate::env::Env::open)). /// diff --git a/storage/database/src/lib.rs b/storage/database/src/lib.rs index 45bfc53c..8e48fca0 100644 --- a/storage/database/src/lib.rs +++ b/storage/database/src/lib.rs @@ -50,7 +50,7 @@ pub use constants::{ }; pub use database::{DatabaseIter, DatabaseRo, DatabaseRw}; pub use env::{Env, EnvInner}; -pub use error::{InitError, RuntimeError}; +pub use error::{DbResult, InitError, RuntimeError}; pub use key::{Key, KeyCompare}; pub use storable::{Storable, StorableBytes, StorableStr, StorableVec}; pub use table::Table; diff --git a/storage/database/src/table.rs b/storage/database/src/table.rs index 3ad0e793..6d0daa20 100644 --- a/storage/database/src/table.rs +++ b/storage/database/src/table.rs @@ -1,7 +1,6 @@ //! Database table abstraction; `trait Table`. //---------------------------------------------------------------------------------------------------- Import - use crate::{key::Key, storable::Storable}; //---------------------------------------------------------------------------------------------------- Table diff --git a/storage/database/src/tables.rs b/storage/database/src/tables.rs index 83a00e16..56203ad0 100644 --- a/storage/database/src/tables.rs +++ b/storage/database/src/tables.rs @@ -211,7 +211,7 @@ macro_rules! define_tables { /// /// # Errors /// This returns errors on regular database errors. - fn all_tables_empty(&self) -> Result; + fn all_tables_empty(&self) -> $crate::DbResult; } /// Object containing all opened [`Table`](cuprate_database::Table)s in read + iter mode. @@ -293,7 +293,7 @@ macro_rules! define_tables { } )* - fn all_tables_empty(&self) -> Result { + fn all_tables_empty(&self) -> $crate::DbResult { $( if !$crate::DatabaseRo::is_empty(&self.$index)? { return Ok(false); @@ -369,7 +369,7 @@ macro_rules! define_tables { /// /// # Errors /// This will only return [`cuprate_database::RuntimeError::Io`] if it errors. - fn open_tables(&self, tx_ro: &Self::Ro<'_>) -> Result; + fn open_tables(&self, tx_ro: &Self::Ro<'_>) -> $crate::DbResult; /// Open all tables in read-write mode. /// @@ -378,7 +378,7 @@ macro_rules! define_tables { /// /// # Errors /// This will only return [`cuprate_database::RuntimeError::Io`] on errors. - fn open_tables_mut(&self, tx_rw: &Self::Rw<'_>) -> Result; + fn open_tables_mut(&self, tx_rw: &Self::Rw<'_>) -> $crate::DbResult; /// Create all database tables. /// @@ -386,7 +386,7 @@ macro_rules! define_tables { /// /// # Errors /// This will only return [`cuprate_database::RuntimeError::Io`] on errors. - fn create_tables(&self, tx_rw: &Self::Rw<'_>) -> Result<(), $crate::RuntimeError>; + fn create_tables(&self, tx_rw: &Self::Rw<'_>) -> $crate::DbResult<()>; } impl<'env, Ei> OpenTables<'env> for Ei @@ -396,19 +396,19 @@ macro_rules! define_tables { type Ro<'tx> = >::Ro<'tx>; type Rw<'tx> = >::Rw<'tx>; - fn open_tables(&self, tx_ro: &Self::Ro<'_>) -> Result { + fn open_tables(&self, tx_ro: &Self::Ro<'_>) -> $crate::DbResult { Ok(($( Self::open_db_ro::<[<$table:camel>]>(self, tx_ro)?, )*)) } - fn open_tables_mut(&self, tx_rw: &Self::Rw<'_>) -> Result { + fn open_tables_mut(&self, tx_rw: &Self::Rw<'_>) -> $crate::DbResult { Ok(($( Self::open_db_rw::<[<$table:camel>]>(self, tx_rw)?, )*)) } - fn create_tables(&self, tx_rw: &Self::Rw<'_>) -> Result<(), $crate::RuntimeError> { + fn create_tables(&self, tx_rw: &Self::Rw<'_>) -> $crate::DbResult<()> { let result = Ok(($( Self::create_db::<[<$table:camel>]>(self, tx_rw), )*)); diff --git a/storage/database/src/transaction.rs b/storage/database/src/transaction.rs index 8f33983d..16d1c518 100644 --- a/storage/database/src/transaction.rs +++ b/storage/database/src/transaction.rs @@ -1,7 +1,7 @@ //! Database transaction abstraction; `trait TxRo`, `trait TxRw`. //---------------------------------------------------------------------------------------------------- Import -use crate::error::RuntimeError; +use crate::error::DbResult; //---------------------------------------------------------------------------------------------------- TxRo /// Read-only database transaction. @@ -16,7 +16,7 @@ pub trait TxRo<'tx> { /// /// # Errors /// This operation will always return `Ok(())` with the `redb` backend. - fn commit(self) -> Result<(), RuntimeError>; + fn commit(self) -> DbResult<()>; } //---------------------------------------------------------------------------------------------------- TxRw @@ -32,12 +32,12 @@ pub trait TxRw<'tx> { /// This operation will always return `Ok(())` with the `redb` backend. /// /// If `Env::MANUAL_RESIZE == true`, - /// [`RuntimeError::ResizeNeeded`] may be returned. - fn commit(self) -> Result<(), RuntimeError>; + /// [`crate::RuntimeError::ResizeNeeded`] may be returned. + fn commit(self) -> DbResult<()>; /// Abort the transaction, erasing any writes that have occurred. /// /// # Errors /// This operation will always return `Ok(())` with the `heed` backend. - fn abort(self) -> Result<(), RuntimeError>; + fn abort(self) -> DbResult<()>; } diff --git a/storage/service/src/service/read.rs b/storage/service/src/service/read.rs index 0ab68539..187ffa4c 100644 --- a/storage/service/src/service/read.rs +++ b/storage/service/src/service/read.rs @@ -7,7 +7,7 @@ use futures::channel::oneshot; use rayon::ThreadPool; use tower::Service; -use cuprate_database::{ConcreteEnv, RuntimeError}; +use cuprate_database::{ConcreteEnv, DbResult, RuntimeError}; use cuprate_helper::asynch::InfallibleOneshotReceiver; /// The [`rayon::ThreadPool`] service. @@ -24,7 +24,7 @@ pub struct DatabaseReadService { pool: Arc, /// The function used to handle request. - inner_handler: Arc Result + Send + Sync + 'static>, + inner_handler: Arc DbResult + Send + Sync + 'static>, } // Deriving [`Clone`] means `Req` & `Res` need to be `Clone`, even if they aren't. @@ -51,7 +51,7 @@ where pub fn new( env: Arc, pool: Arc, - req_handler: impl Fn(&ConcreteEnv, Req) -> Result + Send + Sync + 'static, + req_handler: impl Fn(&ConcreteEnv, Req) -> DbResult + Send + Sync + 'static, ) -> Self { let inner_handler = Arc::new(move |req| req_handler(&env, req)); @@ -69,9 +69,9 @@ where { type Response = Res; type Error = RuntimeError; - type Future = InfallibleOneshotReceiver>; + type Future = InfallibleOneshotReceiver>; - fn poll_ready(&mut self, _: &mut Context<'_>) -> Poll> { + fn poll_ready(&mut self, _: &mut Context<'_>) -> Poll> { Poll::Ready(Ok(())) } diff --git a/storage/service/src/service/write.rs b/storage/service/src/service/write.rs index 607c4aa6..6bcd7255 100644 --- a/storage/service/src/service/write.rs +++ b/storage/service/src/service/write.rs @@ -6,7 +6,7 @@ use std::{ use futures::channel::oneshot; -use cuprate_database::{ConcreteEnv, Env, RuntimeError}; +use cuprate_database::{ConcreteEnv, DbResult, Env, RuntimeError}; use cuprate_helper::asynch::InfallibleOneshotReceiver; //---------------------------------------------------------------------------------------------------- Constants @@ -26,8 +26,7 @@ pub struct DatabaseWriteHandle { /// Sender channel to the database write thread-pool. /// /// We provide the response channel for the thread-pool. - pub(super) sender: - crossbeam::channel::Sender<(Req, oneshot::Sender>)>, + pub(super) sender: crossbeam::channel::Sender<(Req, oneshot::Sender>)>, } impl Clone for DatabaseWriteHandle { @@ -48,7 +47,7 @@ where #[inline(never)] // Only called once. pub fn init( env: Arc, - inner_handler: impl Fn(&ConcreteEnv, &Req) -> Result + Send + 'static, + inner_handler: impl Fn(&ConcreteEnv, &Req) -> DbResult + Send + 'static, ) -> Self { // Initialize `Request/Response` channels. let (sender, receiver) = crossbeam::channel::unbounded(); @@ -66,10 +65,10 @@ where impl tower::Service for DatabaseWriteHandle { type Response = Res; type Error = RuntimeError; - type Future = InfallibleOneshotReceiver>; + type Future = InfallibleOneshotReceiver>; #[inline] - fn poll_ready(&mut self, _: &mut Context<'_>) -> Poll> { + fn poll_ready(&mut self, _: &mut Context<'_>) -> Poll> { Poll::Ready(Ok(())) } @@ -89,8 +88,8 @@ impl tower::Service for DatabaseWriteHandle { /// The main function of the writer thread. fn database_writer( env: &ConcreteEnv, - receiver: &crossbeam::channel::Receiver<(Req, oneshot::Sender>)>, - inner_handler: impl Fn(&ConcreteEnv, &Req) -> Result, + receiver: &crossbeam::channel::Receiver<(Req, oneshot::Sender>)>, + inner_handler: impl Fn(&ConcreteEnv, &Req) -> DbResult, ) where Req: Send + 'static, Res: Debug + Send + 'static, diff --git a/storage/txpool/src/ops/key_images.rs b/storage/txpool/src/ops/key_images.rs index 04aa1b44..76cae141 100644 --- a/storage/txpool/src/ops/key_images.rs +++ b/storage/txpool/src/ops/key_images.rs @@ -1,7 +1,7 @@ //! Tx-pool key image ops. use monero_serai::transaction::Input; -use cuprate_database::{DatabaseRw, RuntimeError}; +use cuprate_database::{DatabaseRw, DbResult}; use crate::{ops::TxPoolWriteError, tables::SpentKeyImages, types::TransactionHash}; @@ -34,7 +34,7 @@ pub(super) fn add_tx_key_images( pub(super) fn remove_tx_key_images( inputs: &[Input], kis_table: &mut impl DatabaseRw, -) -> Result<(), RuntimeError> { +) -> DbResult<()> { for ki in inputs.iter().map(ki_from_input) { kis_table.delete(&ki)?; } diff --git a/storage/txpool/src/ops/tx_read.rs b/storage/txpool/src/ops/tx_read.rs index 55690750..24101f77 100644 --- a/storage/txpool/src/ops/tx_read.rs +++ b/storage/txpool/src/ops/tx_read.rs @@ -5,7 +5,7 @@ use std::sync::Mutex; use monero_serai::transaction::Transaction; -use cuprate_database::{DatabaseRo, RuntimeError}; +use cuprate_database::{DatabaseRo, DbResult}; use cuprate_types::{TransactionVerificationData, TxVersion}; use crate::{ @@ -17,7 +17,7 @@ use crate::{ pub fn get_transaction_verification_data( tx_hash: &TransactionHash, tables: &impl Tables, -) -> Result { +) -> DbResult { let tx_blob = tables.transaction_blobs().get(tx_hash)?.0; let tx_info = tables.transaction_infos().get(tx_hash)?; @@ -45,7 +45,7 @@ pub fn get_transaction_verification_data( pub fn in_stem_pool( tx_hash: &TransactionHash, tx_infos: &impl DatabaseRo, -) -> Result { +) -> DbResult { Ok(tx_infos .get(tx_hash)? .flags diff --git a/storage/txpool/src/ops/tx_write.rs b/storage/txpool/src/ops/tx_write.rs index dc5ab463..8f426fb7 100644 --- a/storage/txpool/src/ops/tx_write.rs +++ b/storage/txpool/src/ops/tx_write.rs @@ -4,7 +4,7 @@ use bytemuck::TransparentWrapper; use monero_serai::transaction::{NotPruned, Transaction}; -use cuprate_database::{DatabaseRw, RuntimeError, StorableVec}; +use cuprate_database::{DatabaseRw, DbResult, StorableVec}; use cuprate_types::TransactionVerificationData; use crate::{ @@ -67,10 +67,7 @@ pub fn add_transaction( } /// Removes a transaction from the transaction pool. -pub fn remove_transaction( - tx_hash: &TransactionHash, - tables: &mut impl TablesMut, -) -> Result<(), RuntimeError> { +pub fn remove_transaction(tx_hash: &TransactionHash, tables: &mut impl TablesMut) -> DbResult<()> { // Remove the tx blob from table 0. let tx_blob = tables.transaction_blobs_mut().take(tx_hash)?.0; diff --git a/storage/txpool/src/service/read.rs b/storage/txpool/src/service/read.rs index 0de1e7d0..44a29b3c 100644 --- a/storage/txpool/src/service/read.rs +++ b/storage/txpool/src/service/read.rs @@ -11,7 +11,7 @@ use std::{ use rayon::ThreadPool; -use cuprate_database::{ConcreteEnv, DatabaseRo, Env, EnvInner, RuntimeError}; +use cuprate_database::{ConcreteEnv, DatabaseRo, DbResult, Env, EnvInner, RuntimeError}; use cuprate_database_service::{init_thread_pool, DatabaseReadService, ReaderThreads}; use crate::{ @@ -137,7 +137,7 @@ fn filter_known_tx_blob_hashes( // A closure that returns `true` if a tx with a certain blob hash is unknown. // This also fills in `stem_tx_hashes`. - let mut tx_unknown = |blob_hash| -> Result { + let mut tx_unknown = |blob_hash| -> DbResult { match tx_blob_hashes.get(&blob_hash) { Ok(tx_hash) => { if in_stem_pool(&tx_hash, &tx_infos)? { diff --git a/storage/txpool/src/service/types.rs b/storage/txpool/src/service/types.rs index 5c6b97ce..af1ca98b 100644 --- a/storage/txpool/src/service/types.rs +++ b/storage/txpool/src/service/types.rs @@ -2,7 +2,7 @@ //! //! Only used internally for our [`tower::Service`] impls. -use cuprate_database::RuntimeError; +use cuprate_database::DbResult; use cuprate_database_service::{DatabaseReadService, DatabaseWriteHandle}; use crate::service::interface::{ @@ -12,7 +12,7 @@ use crate::service::interface::{ /// The actual type of the response. /// /// Either our [`TxpoolReadResponse`], or a database error occurred. -pub(super) type ReadResponseResult = Result; +pub(super) type ReadResponseResult = DbResult; /// The transaction pool database write service. pub type TxpoolWriteHandle = DatabaseWriteHandle; diff --git a/storage/txpool/src/service/write.rs b/storage/txpool/src/service/write.rs index 13ab81fa..23c5a8a4 100644 --- a/storage/txpool/src/service/write.rs +++ b/storage/txpool/src/service/write.rs @@ -1,6 +1,8 @@ use std::sync::Arc; -use cuprate_database::{ConcreteEnv, DatabaseRo, DatabaseRw, Env, EnvInner, RuntimeError, TxRw}; +use cuprate_database::{ + ConcreteEnv, DatabaseRo, DatabaseRw, DbResult, Env, EnvInner, RuntimeError, TxRw, +}; use cuprate_database_service::DatabaseWriteHandle; use cuprate_types::TransactionVerificationData; @@ -25,7 +27,7 @@ pub(super) fn init_write_service(env: Arc) -> TxpoolWriteHandle { fn handle_txpool_request( env: &ConcreteEnv, req: &TxpoolWriteRequest, -) -> Result { +) -> DbResult { match req { TxpoolWriteRequest::AddTransaction { tx, state_stem } => { add_transaction(env, tx, *state_stem) @@ -50,7 +52,7 @@ fn add_transaction( env: &ConcreteEnv, tx: &TransactionVerificationData, state_stem: bool, -) -> Result { +) -> DbResult { let env_inner = env.env_inner(); let tx_rw = env_inner.tx_rw()?; @@ -83,7 +85,7 @@ fn add_transaction( fn remove_transaction( env: &ConcreteEnv, tx_hash: &TransactionHash, -) -> Result { +) -> DbResult { let env_inner = env.env_inner(); let tx_rw = env_inner.tx_rw()?; @@ -105,10 +107,7 @@ fn remove_transaction( } /// [`TxpoolWriteRequest::Promote`] -fn promote( - env: &ConcreteEnv, - tx_hash: &TransactionHash, -) -> Result { +fn promote(env: &ConcreteEnv, tx_hash: &TransactionHash) -> DbResult { let env_inner = env.env_inner(); let tx_rw = env_inner.tx_rw()?; @@ -134,10 +133,7 @@ fn promote( } /// [`TxpoolWriteRequest::NewBlock`] -fn new_block( - env: &ConcreteEnv, - spent_key_images: &[KeyImage], -) -> Result { +fn new_block(env: &ConcreteEnv, spent_key_images: &[KeyImage]) -> DbResult { let env_inner = env.env_inner(); let tx_rw = env_inner.tx_rw()?; diff --git a/test-utils/src/data/constants.rs b/test-utils/src/data/constants.rs index fff04416..78413edf 100644 --- a/test-utils/src/data/constants.rs +++ b/test-utils/src/data/constants.rs @@ -104,7 +104,7 @@ macro_rules! const_tx_blob { hash: $hash:literal, // Transaction hash data_path: $data_path:literal, // Path to the transaction blob version: $version:literal, // Transaction version - timelock: $timelock:expr, // Transaction's timelock (use the real type `Timelock`) + timelock: $timelock:expr_2021, // Transaction's timelock (use the real type `Timelock`) input_len: $input_len:literal, // Amount of inputs output_len: $output_len:literal, // Amount of outputs ) => { diff --git a/test-utils/src/rpc/data/macros.rs b/test-utils/src/rpc/data/macros.rs index 63a214c6..5f87c53a 100644 --- a/test-utils/src/rpc/data/macros.rs +++ b/test-utils/src/rpc/data/macros.rs @@ -25,11 +25,11 @@ macro_rules! define_request_and_response { // The request type (and any doc comments, derives, etc). $( #[$request_attr:meta] )* - Request = $request:expr; + Request = $request:expr_2021; // The response type (and any doc comments, derives, etc). $( #[$response_attr:meta] )* - Response = $response:expr; + Response = $response:expr_2021; ) => { paste::paste! { #[doc = $crate::rpc::data::macros::define_request_and_response_doc!( "response" => [<$name:upper _RESPONSE>], From ecd077b402cd907e9ffc23eb5904a3cab7e479de Mon Sep 17 00:00:00 2001 From: Boog900 Date: Tue, 3 Dec 2024 15:17:21 +0000 Subject: [PATCH 09/10] cuprated: config & args (#304) * init config * split sections * finish initial config. * fix clap * misc changes * fix doc * fix test & clippy * fix test 2 * try fix windows * testing * testing 2 * fix windows test * fix windows: the remix. * review comments * fix imports * rename & fix default config file * fix cargo hack * enable serde on `cuprate-helper` * changes from matrix chats * fix ci * fix doc * fix doc test * move Cuprated.toml * remove default.rs * `size` -> `bytes` * `addressbook_path` -> `address_book_path` * fix config output * fix ci * Update binaries/cuprated/src/config/args.rs Co-authored-by: hinto-janai --------- Co-authored-by: hinto-janai --- Cargo.lock | 40 +++- Cargo.toml | 2 + binaries/cuprated/Cargo.toml | 59 +++--- binaries/cuprated/Cuprated.toml | 67 +++++++ binaries/cuprated/src/config.rs | 158 ++++++++++++++++ binaries/cuprated/src/config/args.rs | 55 ++++++ binaries/cuprated/src/config/fs.rs | 21 +++ binaries/cuprated/src/config/p2p.rs | 178 ++++++++++++++++++ binaries/cuprated/src/config/storage.rs | 67 +++++++ .../cuprated/src/config/tracing_config.rs | 42 +++++ binaries/cuprated/src/constants.rs | 8 + binaries/cuprated/src/main.rs | 2 + helper/Cargo.toml | 2 + helper/src/fs.rs | 95 ++++++---- helper/src/network.rs | 34 +++- p2p/address-book/Cargo.toml | 2 +- p2p/address-book/src/book/tests.rs | 2 +- p2p/address-book/src/lib.rs | 9 +- p2p/address-book/src/store.rs | 11 +- p2p/p2p/src/block_downloader.rs | 16 +- p2p/p2p/src/block_downloader/tests.rs | 8 +- storage/blockchain/Cargo.toml | 2 +- storage/blockchain/README.md | 2 +- storage/blockchain/src/config.rs | 74 +++++--- storage/blockchain/src/ops/mod.rs | 2 +- storage/blockchain/src/service/mod.rs | 2 +- storage/blockchain/src/service/tests.rs | 3 +- storage/blockchain/src/tests.rs | 4 +- storage/txpool/Cargo.toml | 2 +- storage/txpool/README.md | 2 +- storage/txpool/src/config.rs | 84 +++++---- storage/txpool/src/ops.rs | 2 +- storage/txpool/src/service.rs | 2 +- 33 files changed, 888 insertions(+), 171 deletions(-) create mode 100644 binaries/cuprated/Cuprated.toml create mode 100644 binaries/cuprated/src/config/args.rs create mode 100644 binaries/cuprated/src/config/fs.rs create mode 100644 binaries/cuprated/src/config/p2p.rs create mode 100644 binaries/cuprated/src/config/storage.rs create mode 100644 binaries/cuprated/src/config/tracing_config.rs diff --git a/Cargo.lock b/Cargo.lock index ac36c565..0d55c8aa 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -446,6 +446,7 @@ checksum = "19bc80abd44e4bed93ca373a0704ccbd1b710dc5749406201bb018272808dc54" dependencies = [ "anstyle", "clap_lex", + "terminal_size", ] [[package]] @@ -933,6 +934,7 @@ dependencies = [ "libc", "monero-serai", "rayon", + "serde", "tokio", "windows", ] @@ -1188,7 +1190,6 @@ dependencies = [ "cuprate-consensus", "cuprate-consensus-context", "cuprate-consensus-rules", - "cuprate-constants", "cuprate-cryptonight", "cuprate-dandelion-tower", "cuprate-database", @@ -1230,6 +1231,7 @@ dependencies = [ "tokio", "tokio-stream", "tokio-util", + "toml", "tower 0.5.1 (git+https://github.com/Cuprate/tower.git?rev=6c7faf0)", "tracing", "tracing-subscriber", @@ -2904,6 +2906,15 @@ dependencies = [ "serde", ] +[[package]] +name = "serde_spanned" +version = "0.6.8" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "87607cb1398ed59d48732e575a4c28a7a8ebf2454b964fe3f224f2afc07909e1" +dependencies = [ + "serde", +] + [[package]] name = "serde_urlencoded" version = "0.7.1" @@ -3121,6 +3132,16 @@ dependencies = [ "windows-sys 0.59.0", ] +[[package]] +name = "terminal_size" +version = "0.4.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4f599bd7ca042cfdf8f4512b277c02ba102247820f9d9d4a9f521f496751a6ef" +dependencies = [ + "rustix", + "windows-sys 0.59.0", +] + [[package]] name = "thiserror" version = "1.0.66" @@ -3262,11 +3283,26 @@ dependencies = [ "tracing", ] +[[package]] +name = "toml" +version = "0.8.19" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a1ed1f98e3fdc28d6d910e6737ae6ab1a93bf1985935a1193e68f93eeb68d24e" +dependencies = [ + "serde", + "serde_spanned", + "toml_datetime", + "toml_edit", +] + [[package]] name = "toml_datetime" version = "0.6.8" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "0dd7358ecb8fc2f8d014bf86f6f638ce72ba252a2c3a2572f2a795f1d23efb41" +dependencies = [ + "serde", +] [[package]] name = "toml_edit" @@ -3275,6 +3311,8 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "4ae48d6208a266e853d946088ed816055e556cc6028c5e8e2b84d9fa5dd7c7f5" dependencies = [ "indexmap", + "serde", + "serde_spanned", "toml_datetime", "winnow", ] diff --git a/Cargo.toml b/Cargo.toml index 3cc3ab18..9be1528c 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -55,6 +55,7 @@ members = [ ] [profile.release] +panic = "abort" lto = true # Build with LTO strip = "none" # Keep panic stack traces codegen-units = 1 # Optimize for binary speed over compile times @@ -144,6 +145,7 @@ tokio-util = { version = "0.7", default-features = false } tokio-stream = { version = "0.1", default-features = false } tokio = { version = "1", default-features = false } tower = { git = "https://github.com/Cuprate/tower.git", rev = "6c7faf0", default-features = false } # +toml = { version = "0.8", default-features = false } tracing-subscriber = { version = "0.3", default-features = false } tracing = { version = "0.1", default-features = false } diff --git a/binaries/cuprated/Cargo.toml b/binaries/cuprated/Cargo.toml index 9ebdd780..acf8827b 100644 --- a/binaries/cuprated/Cargo.toml +++ b/binaries/cuprated/Cargo.toml @@ -2,7 +2,7 @@ name = "cuprated" version = "0.0.1" edition = "2021" -description = "The Cuprate Monero Rust node." +description = "The Cuprate Rust Monero node." license = "AGPL-3.0-only" authors = ["Boog900", "hinto-janai", "SyntheticBird45"] repository = "https://github.com/Cuprate/cuprate/tree/main/binaries/cuprated" @@ -12,29 +12,29 @@ repository = "https://github.com/Cuprate/cuprate/tree/main/binaries/cuprated" cuprate-consensus = { workspace = true } cuprate-fast-sync = { workspace = true } cuprate-consensus-context = { workspace = true } -cuprate-consensus-rules = { workspace = true } -cuprate-constants = { workspace = true } -cuprate-cryptonight = { workspace = true } -cuprate-helper = { workspace = true } -cuprate-epee-encoding = { workspace = true } -cuprate-fixed-bytes = { workspace = true } -cuprate-levin = { workspace = true } -cuprate-wire = { workspace = true } -cuprate-p2p = { workspace = true } -cuprate-p2p-core = { workspace = true } -cuprate-dandelion-tower = { workspace = true, features = ["txpool"] } -cuprate-async-buffer = { workspace = true } -cuprate-address-book = { workspace = true } -cuprate-blockchain = { workspace = true } -cuprate-database-service = { workspace = true } -cuprate-txpool = { workspace = true } -cuprate-database = { workspace = true } -cuprate-pruning = { workspace = true } -cuprate-test-utils = { workspace = true } -cuprate-types = { workspace = true } -cuprate-json-rpc = { workspace = true } -cuprate-rpc-interface = { workspace = true } -cuprate-rpc-types = { workspace = true } +cuprate-consensus-rules = { workspace = true } +cuprate-cryptonight = { workspace = true } +cuprate-helper = { workspace = true, features = ["serde"] } +cuprate-epee-encoding = { workspace = true } +cuprate-fixed-bytes = { workspace = true } +cuprate-levin = { workspace = true } +cuprate-wire = { workspace = true } +cuprate-p2p = { workspace = true } +cuprate-p2p-core = { workspace = true } +cuprate-dandelion-tower = { workspace = true, features = ["txpool"] } +cuprate-async-buffer = { workspace = true } +cuprate-address-book = { workspace = true } +cuprate-blockchain = { workspace = true } +cuprate-database-service = { workspace = true, features = ["serde"] } +cuprate-txpool = { workspace = true } +cuprate-database = { workspace = true, features = ["serde"] } +cuprate-pruning = { workspace = true } +cuprate-test-utils = { workspace = true } +cuprate-types = { workspace = true } +cuprate-json-rpc = { workspace = true } +cuprate-rpc-interface = { workspace = true } +cuprate-rpc-types = { workspace = true } + # TODO: after v1.0.0, remove unneeded dependencies. anyhow = { workspace = true } @@ -44,7 +44,7 @@ borsh = { workspace = true } bytemuck = { workspace = true } bytes = { workspace = true } cfg-if = { workspace = true } -clap = { workspace = true, features = ["cargo"] } +clap = { workspace = true, features = ["cargo", "help", "wrap_help"] } chrono = { workspace = true } crypto-bigint = { workspace = true } crossbeam = { workspace = true } @@ -71,15 +71,10 @@ thread_local = { workspace = true } tokio-util = { workspace = true } tokio-stream = { workspace = true } tokio = { workspace = true } +toml = { workspace = true, features = ["parse", "display"]} tower = { workspace = true } tracing-subscriber = { workspace = true, features = ["std", "fmt", "default"] } -tracing = { workspace = true } +tracing = { workspace = true, features = ["default"] } [lints] workspace = true - -[profile.dev] -panic = "abort" - -[profile.release] -panic = "abort" diff --git a/binaries/cuprated/Cuprated.toml b/binaries/cuprated/Cuprated.toml new file mode 100644 index 00000000..d248ce1f --- /dev/null +++ b/binaries/cuprated/Cuprated.toml @@ -0,0 +1,67 @@ +# ____ _ +# / ___| _ _ __ _ __ __ _| |_ ___ +# | | | | | | '_ \| '__/ _` | __/ _ \ +# | |__| |_| | |_) | | | (_| | || __/ +# \____\__,_| .__/|_| \__,_|\__\___| +# |_| +# + +## The network to run on, valid values: "Mainnet", "Testnet", "Stagenet". +network = "Mainnet" + +## Tracing config. +[tracing] +## The minimum level for log events to be displayed. +level = "info" + +## Clear-net config. +[p2p.clear_net] +## The number of outbound connections we should make and maintain. +outbound_connections = 64 +## The number of extra connections we should make under load from the rest of Cuprate, i.e. when syncing. +extra_outbound_connections = 8 +## The maximum number of incoming we should allow. +max_inbound_connections = 128 +## The percent of outbound connections that should be to nodes we have not connected to before. +gray_peers_percent = 0.7 +## The port to accept connections on, if left `0` no connections will be accepted. +p2p_port = 0 +## The IP address to listen to connections on. +listen_on = "0.0.0.0" + +## The Clear-net addressbook config. +[p2p.clear_net.address_book_config] +## The size of the white peer list, which contains peers we have made a connection to before. +max_white_list_length = 1_000 +## The size of the gray peer list, which contains peers we have not made a connection to before. +max_gray_list_length = 5_000 +## The amount of time between address book saves. +peer_save_period = { secs = 90, nanos = 0 } + +## The block downloader config. +[p2p.block_downloader] +## The size of the buffer of sequential blocks waiting to be verified and added to the chain (bytes). +buffer_bytes = 50_000_000 +## The size of the queue of blocks which are waiting for a parent block to be downloaded (bytes). +in_progress_queue_bytes = 50_000_000 +## The target size of a batch of blocks (bytes), must not exceed 100MB. +target_batch_bytes= 5_000_000 +## The amount of time between checking the pool of connected peers for free peers to download blocks. +check_client_pool_interval = { secs = 30, nanos = 0 } + +## Storage config +[storage] +## The amount of reader threads to spawn. +reader_threads = "OnePerThread" + +## Txpool storage config. +[storage.txpool] +## The database sync mode for the txpool. +sync_mode = "Async" +## The maximum size of all the txs in the pool (bytes). +max_txpool_byte_size = 100_000_000 + +## Blockchain storage config. +[storage.blockchain] +## The database sync mode for the blockchain. +sync_mode = "Async" diff --git a/binaries/cuprated/src/config.rs b/binaries/cuprated/src/config.rs index d613c1fc..c6267a69 100644 --- a/binaries/cuprated/src/config.rs +++ b/binaries/cuprated/src/config.rs @@ -1 +1,159 @@ //! cuprated config +use std::{ + fs::{read_to_string, File}, + io, + path::Path, + time::Duration, +}; + +use clap::Parser; +use serde::{Deserialize, Serialize}; + +use cuprate_consensus::ContextConfig; +use cuprate_helper::{ + fs::{CUPRATE_CONFIG_DIR, DEFAULT_CONFIG_FILE_NAME}, + network::Network, +}; +use cuprate_p2p::block_downloader::BlockDownloaderConfig; +use cuprate_p2p_core::{ClearNet, ClearNetServerCfg}; + +mod args; +mod fs; +mod p2p; +mod storage; +mod tracing_config; + +use crate::config::fs::FileSystemConfig; +use p2p::P2PConfig; +use storage::StorageConfig; +use tracing_config::TracingConfig; + +/// Reads the args & config file, returning a [`Config`]. +pub fn read_config_and_args() -> Config { + let args = args::Args::parse(); + args.do_quick_requests(); + + let config: Config = if let Some(config_file) = &args.config_file { + // If a config file was set in the args try to read it and exit if we can't. + match Config::read_from_path(config_file) { + Ok(config) => config, + Err(e) => { + eprintln!("Failed to read config from file: {e}"); + std::process::exit(1); + } + } + } else { + // First attempt to read the config file from the current directory. + std::env::current_dir() + .map(|path| path.join(DEFAULT_CONFIG_FILE_NAME)) + .map_err(Into::into) + .and_then(Config::read_from_path) + .inspect_err(|e| tracing::debug!("Failed to read config from current dir: {e}")) + // otherwise try the main config directory. + .or_else(|_| { + let file = CUPRATE_CONFIG_DIR.join(DEFAULT_CONFIG_FILE_NAME); + Config::read_from_path(file) + }) + .inspect_err(|e| { + tracing::debug!("Failed to read config from config dir: {e}"); + eprintln!("Failed to find/read config file, using default config."); + }) + .unwrap_or_default() + }; + + args.apply_args(config) +} + +/// The config for all of Cuprate. +#[derive(Default, Deserialize, Serialize)] +#[serde(deny_unknown_fields, default)] +pub struct Config { + /// The network we should run on. + network: Network, + + /// [`tracing`] config. + tracing: TracingConfig, + + /// The P2P network config. + p2p: P2PConfig, + + /// The storage config. + storage: StorageConfig, + + fs: FileSystemConfig, +} + +impl Config { + /// Attempts to read a config file in [`toml`] format from the given [`Path`]. + /// + /// # Errors + /// + /// Will return an [`Err`] if the file cannot be read or if the file is not a valid [`toml`] config. + fn read_from_path(file: impl AsRef) -> Result { + let file_text = read_to_string(file.as_ref())?; + + Ok(toml::from_str(&file_text) + .inspect(|_| eprintln!("Using config at: {}", file.as_ref().to_string_lossy())) + .inspect_err(|e| { + eprintln!("{e}"); + eprintln!( + "Failed to parse config file at: {}", + file.as_ref().to_string_lossy() + ); + })?) + } + + /// Returns the current [`Network`] we are running on. + pub const fn network(&self) -> Network { + self.network + } + + /// The [`ClearNet`], [`cuprate_p2p::P2PConfig`]. + pub fn clearnet_p2p_config(&self) -> cuprate_p2p::P2PConfig { + cuprate_p2p::P2PConfig { + network: self.network, + seeds: p2p::clear_net_seed_nodes(self.network), + outbound_connections: self.p2p.clear_net.general.outbound_connections, + extra_outbound_connections: self.p2p.clear_net.general.extra_outbound_connections, + max_inbound_connections: self.p2p.clear_net.general.max_inbound_connections, + gray_peers_percent: self.p2p.clear_net.general.gray_peers_percent, + server_config: Some(ClearNetServerCfg { + ip: self.p2p.clear_net.listen_on, + }), + p2p_port: self.p2p.clear_net.general.p2p_port, + // TODO: set this if a public RPC server is set. + rpc_port: 0, + address_book_config: self + .p2p + .clear_net + .general + .address_book_config(&self.fs.cache_directory, self.network), + } + } + + /// The [`ContextConfig`]. + pub const fn context_config(&self) -> ContextConfig { + match self.network { + Network::Mainnet => ContextConfig::main_net(), + Network::Stagenet => ContextConfig::stage_net(), + Network::Testnet => ContextConfig::test_net(), + } + } + + /// The [`cuprate_blockchain`] config. + pub fn blockchain_config(&self) -> cuprate_blockchain::config::Config { + let blockchain = &self.storage.blockchain; + + // We don't set reader threads as we manually make the reader threadpool. + cuprate_blockchain::config::ConfigBuilder::default() + .network(self.network) + .data_directory(self.fs.data_directory.clone()) + .sync_mode(blockchain.shared.sync_mode) + .build() + } + + /// The [`BlockDownloaderConfig`]. + pub fn block_downloader_config(&self) -> BlockDownloaderConfig { + self.p2p.block_downloader.clone().into() + } +} diff --git a/binaries/cuprated/src/config/args.rs b/binaries/cuprated/src/config/args.rs new file mode 100644 index 00000000..c4c2f9fd --- /dev/null +++ b/binaries/cuprated/src/config/args.rs @@ -0,0 +1,55 @@ +use std::{io::Write, path::PathBuf, process::exit}; + +use clap::builder::TypedValueParser; + +use cuprate_helper::network::Network; + +use crate::{config::Config, constants::EXAMPLE_CONFIG}; + +/// Cuprate Args. +#[derive(clap::Parser, Debug)] +#[command(version, about)] +pub struct Args { + /// The network to run on. + #[arg( + long, + default_value_t = Network::Mainnet, + value_parser = clap::builder::PossibleValuesParser::new(["mainnet", "testnet", "stagenet"]) + .map(|s| s.parse::().unwrap()), + )] + pub network: Network, + /// The amount of outbound clear-net connections to maintain. + #[arg(long)] + pub outbound_connections: Option, + /// The PATH of the `cuprated` config file. + #[arg(long)] + pub config_file: Option, + /// Generate a config file and print it to stdout. + #[arg(long)] + pub generate_config: bool, +} + +impl Args { + /// Complete any quick requests asked for in [`Args`]. + /// + /// May cause the process to [`exit`]. + pub fn do_quick_requests(&self) { + if self.generate_config { + println!("{EXAMPLE_CONFIG}"); + exit(0); + } + } + + /// Apply the [`Args`] to the given [`Config`]. + /// + /// This may exit the program if a config value was set that requires an early exit. + pub const fn apply_args(&self, mut config: Config) -> Config { + config.network = self.network; + + if let Some(outbound_connections) = self.outbound_connections { + config.p2p.clear_net.general.outbound_connections = outbound_connections; + } + + config + } +} diff --git a/binaries/cuprated/src/config/fs.rs b/binaries/cuprated/src/config/fs.rs new file mode 100644 index 00000000..f8f61307 --- /dev/null +++ b/binaries/cuprated/src/config/fs.rs @@ -0,0 +1,21 @@ +use std::path::PathBuf; + +use serde::{Deserialize, Serialize}; + +use cuprate_helper::fs::{CUPRATE_CACHE_DIR, CUPRATE_DATA_DIR}; + +#[derive(Deserialize, Serialize)] +#[serde(deny_unknown_fields, default)] +pub struct FileSystemConfig { + pub data_directory: PathBuf, + pub cache_directory: PathBuf, +} + +impl Default for FileSystemConfig { + fn default() -> Self { + Self { + data_directory: CUPRATE_DATA_DIR.to_path_buf(), + cache_directory: CUPRATE_CACHE_DIR.to_path_buf(), + } + } +} diff --git a/binaries/cuprated/src/config/p2p.rs b/binaries/cuprated/src/config/p2p.rs new file mode 100644 index 00000000..51f8d0d6 --- /dev/null +++ b/binaries/cuprated/src/config/p2p.rs @@ -0,0 +1,178 @@ +use std::{ + net::{IpAddr, Ipv4Addr, Ipv6Addr, SocketAddr}, + path::Path, + time::Duration, +}; + +use serde::{Deserialize, Serialize}; + +use cuprate_helper::{fs::address_book_path, network::Network}; + +/// P2P config. +#[derive(Default, Deserialize, Serialize)] +#[serde(deny_unknown_fields, default)] +pub struct P2PConfig { + /// Clear-net config. + pub clear_net: ClearNetConfig, + /// Block downloader config. + pub block_downloader: BlockDownloaderConfig, +} + +#[derive(Clone, Deserialize, Serialize)] +#[serde(deny_unknown_fields, default)] +pub struct BlockDownloaderConfig { + /// The size in bytes of the buffer between the block downloader and the place which + /// is consuming the downloaded blocks. + pub buffer_bytes: usize, + /// The size of the in progress queue (in bytes) at which we stop requesting more blocks. + pub in_progress_queue_bytes: usize, + /// The [`Duration`] between checking the client pool for free peers. + pub check_client_pool_interval: Duration, + /// The target size of a single batch of blocks (in bytes). + pub target_batch_bytes: usize, +} + +impl From for cuprate_p2p::block_downloader::BlockDownloaderConfig { + fn from(value: BlockDownloaderConfig) -> Self { + Self { + buffer_bytes: value.buffer_bytes, + in_progress_queue_bytes: value.in_progress_queue_bytes, + check_client_pool_interval: value.check_client_pool_interval, + target_batch_bytes: value.target_batch_bytes, + initial_batch_len: 1, + } + } +} + +impl Default for BlockDownloaderConfig { + fn default() -> Self { + Self { + buffer_bytes: 50_000_000, + in_progress_queue_bytes: 50_000_000, + check_client_pool_interval: Duration::from_secs(30), + target_batch_bytes: 5_000_000, + } + } +} + +/// The config values for P2P clear-net. +#[derive(Deserialize, Serialize)] +#[serde(deny_unknown_fields, default)] +pub struct ClearNetConfig { + /// The server config. + pub listen_on: IpAddr, + #[serde(flatten)] + pub general: SharedNetConfig, +} + +impl Default for ClearNetConfig { + fn default() -> Self { + Self { + listen_on: IpAddr::V4(Ipv4Addr::UNSPECIFIED), + general: Default::default(), + } + } +} + +/// Network config values shared between all network zones. +#[derive(Deserialize, Serialize)] +#[serde(deny_unknown_fields, default)] +pub struct SharedNetConfig { + /// The number of outbound connections to make and try keep. + pub outbound_connections: usize, + /// The amount of extra connections we can make if we are under load from the rest of Cuprate. + pub extra_outbound_connections: usize, + /// The maximum amount of inbound connections + pub max_inbound_connections: usize, + /// The percent of connections that should be to peers we haven't connected to before. + pub gray_peers_percent: f64, + /// port to use to accept p2p connections. + pub p2p_port: u16, + /// The address book config. + address_book_config: AddressBookConfig, +} + +impl SharedNetConfig { + /// Returns the [`AddressBookConfig`]. + pub fn address_book_config( + &self, + cache_dir: &Path, + network: Network, + ) -> cuprate_address_book::AddressBookConfig { + cuprate_address_book::AddressBookConfig { + max_white_list_length: self.address_book_config.max_white_list_length, + max_gray_list_length: self.address_book_config.max_gray_list_length, + peer_store_directory: address_book_path(cache_dir, network), + peer_save_period: self.address_book_config.peer_save_period, + } + } +} + +impl Default for SharedNetConfig { + fn default() -> Self { + Self { + outbound_connections: 64, + extra_outbound_connections: 8, + max_inbound_connections: 128, + gray_peers_percent: 0.7, + p2p_port: 0, + address_book_config: AddressBookConfig::default(), + } + } +} + +#[derive(Deserialize, Serialize)] +#[serde(deny_unknown_fields, default)] +pub struct AddressBookConfig { + max_white_list_length: usize, + max_gray_list_length: usize, + peer_save_period: Duration, +} + +impl Default for AddressBookConfig { + fn default() -> Self { + Self { + max_white_list_length: 1_000, + max_gray_list_length: 5_000, + peer_save_period: Duration::from_secs(30), + } + } +} + +/// Seed nodes for [`ClearNet`](cuprate_p2p_core::ClearNet). +pub fn clear_net_seed_nodes(network: Network) -> Vec { + let seeds = match network { + Network::Mainnet => [ + "176.9.0.187:18080", + "88.198.163.90:18080", + "66.85.74.134:18080", + "51.79.173.165:18080", + "192.99.8.110:18080", + "37.187.74.171:18080", + "77.172.183.193:18080", + ] + .as_slice(), + Network::Stagenet => [ + "176.9.0.187:38080", + "51.79.173.165:38080", + "192.99.8.110:38080", + "37.187.74.171:38080", + "77.172.183.193:38080", + ] + .as_slice(), + Network::Testnet => [ + "176.9.0.187:28080", + "51.79.173.165:28080", + "192.99.8.110:28080", + "37.187.74.171:28080", + "77.172.183.193:28080", + ] + .as_slice(), + }; + + seeds + .iter() + .map(|s| s.parse()) + .collect::>() + .unwrap() +} diff --git a/binaries/cuprated/src/config/storage.rs b/binaries/cuprated/src/config/storage.rs new file mode 100644 index 00000000..b3e3c1f4 --- /dev/null +++ b/binaries/cuprated/src/config/storage.rs @@ -0,0 +1,67 @@ +use std::path::PathBuf; + +use serde::{Deserialize, Serialize}; + +use cuprate_database::config::SyncMode; +use cuprate_database_service::ReaderThreads; +use cuprate_helper::fs::CUPRATE_DATA_DIR; + +/// The storage config. +#[derive(Default, Deserialize, Serialize)] +#[serde(deny_unknown_fields, default)] +pub struct StorageConfig { + /// The amount of reader threads to spawn between the tx-pool and blockchain. + pub reader_threads: ReaderThreads, + /// The tx-pool config. + pub txpool: TxpoolConfig, + /// The blockchain config. + pub blockchain: BlockchainConfig, +} + +/// The blockchain config. +#[derive(Deserialize, Serialize)] +#[serde(deny_unknown_fields, default)] +pub struct BlockchainConfig { + #[serde(flatten)] + pub shared: SharedStorageConfig, +} + +impl Default for BlockchainConfig { + fn default() -> Self { + Self { + shared: SharedStorageConfig { + sync_mode: SyncMode::Async, + }, + } + } +} + +/// The tx-pool config. +#[derive(Deserialize, Serialize)] +#[serde(deny_unknown_fields, default)] +pub struct TxpoolConfig { + #[serde(flatten)] + pub shared: SharedStorageConfig, + + /// The maximum size of the tx-pool. + pub max_txpool_byte_size: usize, +} + +impl Default for TxpoolConfig { + fn default() -> Self { + Self { + shared: SharedStorageConfig { + sync_mode: SyncMode::Async, + }, + max_txpool_byte_size: 100_000_000, + } + } +} + +/// Config values shared between the tx-pool and blockchain. +#[derive(Default, Deserialize, Serialize)] +#[serde(deny_unknown_fields, default)] +pub struct SharedStorageConfig { + /// The [`SyncMode`] of the database. + pub sync_mode: SyncMode, +} diff --git a/binaries/cuprated/src/config/tracing_config.rs b/binaries/cuprated/src/config/tracing_config.rs new file mode 100644 index 00000000..859d516a --- /dev/null +++ b/binaries/cuprated/src/config/tracing_config.rs @@ -0,0 +1,42 @@ +use serde::{Deserialize, Serialize}; +use tracing::level_filters::LevelFilter; + +/// [`tracing`] config. +#[derive(Deserialize, Serialize)] +#[serde(deny_unknown_fields, default)] +pub struct TracingConfig { + /// The default minimum log level. + #[serde(with = "level_filter_serde")] + level: LevelFilter, +} + +impl Default for TracingConfig { + fn default() -> Self { + Self { + level: LevelFilter::INFO, + } + } +} + +mod level_filter_serde { + use std::str::FromStr; + + use serde::{Deserialize, Deserializer, Serializer}; + use tracing::level_filters::LevelFilter; + + #[expect(clippy::trivially_copy_pass_by_ref, reason = "serde")] + pub fn serialize(level_filter: &LevelFilter, s: S) -> Result + where + S: Serializer, + { + s.serialize_str(&level_filter.to_string()) + } + + pub fn deserialize<'de, D>(d: D) -> Result + where + D: Deserializer<'de>, + { + let s = String::deserialize(d)?; + LevelFilter::from_str(&s).map_err(serde::de::Error::custom) + } +} diff --git a/binaries/cuprated/src/constants.rs b/binaries/cuprated/src/constants.rs index 2f3c7bb6..057e8bd0 100644 --- a/binaries/cuprated/src/constants.rs +++ b/binaries/cuprated/src/constants.rs @@ -18,9 +18,12 @@ pub const VERSION_BUILD: &str = if cfg!(debug_assertions) { pub const PANIC_CRITICAL_SERVICE_ERROR: &str = "A service critical to Cuprate's function returned an unexpected error."; +pub const EXAMPLE_CONFIG: &str = include_str!("../Cuprated.toml"); + #[cfg(test)] mod test { use super::*; + use crate::config::Config; #[test] fn version() { @@ -35,4 +38,9 @@ mod test { assert_eq!(VERSION_BUILD, "0.0.1-release"); } } + + #[test] + fn generate_config_text_is_valid() { + let config: Config = toml::from_str(EXAMPLE_CONFIG).unwrap(); + } } diff --git a/binaries/cuprated/src/main.rs b/binaries/cuprated/src/main.rs index d5c832e2..617c5b61 100644 --- a/binaries/cuprated/src/main.rs +++ b/binaries/cuprated/src/main.rs @@ -29,6 +29,8 @@ fn main() { // Initialize global static `LazyLock` data. statics::init_lazylock_statics(); + let _config = config::read_config_and_args(); + // TODO: everything else. todo!() } diff --git a/helper/Cargo.toml b/helper/Cargo.toml index 1b3158f1..716beb34 100644 --- a/helper/Cargo.toml +++ b/helper/Cargo.toml @@ -35,6 +35,8 @@ futures = { workspace = true, optional = true, features = ["std"] } monero-serai = { workspace = true, optional = true } rayon = { workspace = true, optional = true } +serde = { workspace = true, optional = true, features = ["derive"] } + # This is kinda a stupid work around. # [thread] needs to activate one of these libs (windows|libc) # although it depends on what target we're building for. diff --git a/helper/src/fs.rs b/helper/src/fs.rs index 5d62a644..f694f62d 100644 --- a/helper/src/fs.rs +++ b/helper/src/fs.rs @@ -28,7 +28,12 @@ //! - //---------------------------------------------------------------------------------------------------- Use -use std::{path::PathBuf, sync::LazyLock}; +use std::{ + path::{Path, PathBuf}, + sync::LazyLock, +}; + +use crate::network::Network; //---------------------------------------------------------------------------------------------------- Const /// Cuprate's main directory. @@ -58,6 +63,9 @@ pub const CUPRATE_DIR: &str = { } }; +/// The default name of Cuprate's config file. +pub const DEFAULT_CONFIG_FILE_NAME: &str = "Cuprated.toml"; + //---------------------------------------------------------------------------------------------------- Directories /// Create a `LazyLock` for common PATHs used by Cuprate. /// @@ -150,32 +158,61 @@ impl_path_lazylock! { CUPRATE_DATA_DIR, data_dir, "", +} - /// Cuprate's blockchain directory. - /// - /// This is the PATH used for any Cuprate blockchain files. - /// - /// | OS | PATH | - /// |---------|----------------------------------------------------------------| - /// | Windows | `C:\Users\Alice\AppData\Roaming\Cuprate\blockchain\` | - /// | macOS | `/Users/Alice/Library/Application Support/Cuprate/blockchain/` | - /// | Linux | `/home/alice/.local/share/cuprate/blockchain/` | - CUPRATE_BLOCKCHAIN_DIR, - data_dir, - "blockchain", +/// Joins the [`Network`] to the [`Path`]. +/// +/// This will keep the path the same for [`Network::Mainnet`]. +fn path_with_network(path: &Path, network: Network) -> PathBuf { + match network { + Network::Mainnet => path.to_path_buf(), + network => path.join(network.to_string()), + } +} - /// Cuprate's transaction pool directory. - /// - /// This is the PATH used for any Cuprate txpool files. - /// - /// | OS | PATH | - /// |---------|------------------------------------------------------------| - /// | Windows | `C:\Users\Alice\AppData\Roaming\Cuprate\txpool\` | - /// | macOS | `/Users/Alice/Library/Application Support/Cuprate/txpool/` | - /// | Linux | `/home/alice/.local/share/cuprate/txpool/` | - CUPRATE_TXPOOL_DIR, - data_dir, - "txpool", +/// Cuprate's blockchain directory. +/// +/// This is the PATH used for any Cuprate blockchain files. +/// +/// ```rust +/// use cuprate_helper::{network::Network, fs::{CUPRATE_DATA_DIR, blockchain_path}}; +/// +/// assert_eq!(blockchain_path(&**CUPRATE_DATA_DIR, Network::Mainnet).as_path(), CUPRATE_DATA_DIR.join("blockchain")); +/// assert_eq!(blockchain_path(&**CUPRATE_DATA_DIR, Network::Stagenet).as_path(), CUPRATE_DATA_DIR.join(Network::Stagenet.to_string()).join("blockchain")); +/// assert_eq!(blockchain_path(&**CUPRATE_DATA_DIR, Network::Testnet).as_path(), CUPRATE_DATA_DIR.join(Network::Testnet.to_string()).join("blockchain")); +/// ``` +pub fn blockchain_path(data_dir: &Path, network: Network) -> PathBuf { + path_with_network(data_dir, network).join("blockchain") +} + +/// Cuprate's txpool directory. +/// +/// This is the PATH used for any Cuprate txpool files. +/// +/// ```rust +/// use cuprate_helper::{network::Network, fs::{CUPRATE_DATA_DIR, txpool_path}}; +/// +/// assert_eq!(txpool_path(&**CUPRATE_DATA_DIR, Network::Mainnet).as_path(), CUPRATE_DATA_DIR.join("txpool")); +/// assert_eq!(txpool_path(&**CUPRATE_DATA_DIR, Network::Stagenet).as_path(), CUPRATE_DATA_DIR.join(Network::Stagenet.to_string()).join("txpool")); +/// assert_eq!(txpool_path(&**CUPRATE_DATA_DIR, Network::Testnet).as_path(), CUPRATE_DATA_DIR.join(Network::Testnet.to_string()).join("txpool")); +/// ``` +pub fn txpool_path(data_dir: &Path, network: Network) -> PathBuf { + path_with_network(data_dir, network).join("txpool") +} + +/// Cuprate's address-book directory. +/// +/// This is the PATH used for any Cuprate address-book files. +/// +/// ```rust +/// use cuprate_helper::{network::Network, fs::{CUPRATE_CACHE_DIR, address_book_path}}; +/// +/// assert_eq!(address_book_path(&**CUPRATE_CACHE_DIR, Network::Mainnet).as_path(), CUPRATE_CACHE_DIR.join("addressbook")); +/// assert_eq!(address_book_path(&**CUPRATE_CACHE_DIR, Network::Stagenet).as_path(), CUPRATE_CACHE_DIR.join(Network::Stagenet.to_string()).join("addressbook")); +/// assert_eq!(address_book_path(&**CUPRATE_CACHE_DIR, Network::Testnet).as_path(), CUPRATE_CACHE_DIR.join(Network::Testnet.to_string()).join("addressbook")); +/// ``` +pub fn address_book_path(cache_dir: &Path, network: Network) -> PathBuf { + path_with_network(cache_dir, network).join("addressbook") } //---------------------------------------------------------------------------------------------------- Tests @@ -197,29 +234,21 @@ mod test { (&*CUPRATE_CACHE_DIR, ""), (&*CUPRATE_CONFIG_DIR, ""), (&*CUPRATE_DATA_DIR, ""), - (&*CUPRATE_BLOCKCHAIN_DIR, ""), - (&*CUPRATE_TXPOOL_DIR, ""), ]; if cfg!(target_os = "windows") { array[0].1 = r"AppData\Local\Cuprate"; array[1].1 = r"AppData\Roaming\Cuprate"; array[2].1 = r"AppData\Roaming\Cuprate"; - array[3].1 = r"AppData\Roaming\Cuprate\blockchain"; - array[4].1 = r"AppData\Roaming\Cuprate\txpool"; } else if cfg!(target_os = "macos") { array[0].1 = "Library/Caches/Cuprate"; array[1].1 = "Library/Application Support/Cuprate"; array[2].1 = "Library/Application Support/Cuprate"; - array[3].1 = "Library/Application Support/Cuprate/blockchain"; - array[4].1 = "Library/Application Support/Cuprate/txpool"; } else { // Assumes Linux. array[0].1 = ".cache/cuprate"; array[1].1 = ".config/cuprate"; array[2].1 = ".local/share/cuprate"; - array[3].1 = ".local/share/cuprate/blockchain"; - array[4].1 = ".local/share/cuprate/txpool"; }; for (path, expected) in array { diff --git a/helper/src/network.rs b/helper/src/network.rs index f3224b33..d2961170 100644 --- a/helper/src/network.rs +++ b/helper/src/network.rs @@ -5,6 +5,12 @@ //! into it's own crate. //! //! `#[no_std]` compatible. +// TODO: move to types crate. + +use core::{ + fmt::{Display, Formatter}, + str::FromStr, +}; const MAINNET_NETWORK_ID: [u8; 16] = [ 0x12, 0x30, 0xF1, 0x71, 0x61, 0x04, 0x41, 0x61, 0x17, 0x31, 0x00, 0x82, 0x16, 0xA1, 0xA1, 0x10, @@ -17,7 +23,8 @@ const STAGENET_NETWORK_ID: [u8; 16] = [ ]; /// An enum representing every Monero network. -#[derive(Debug, Clone, Copy, Default)] +#[derive(Debug, Clone, Copy, Default, Ord, PartialOrd, Eq, PartialEq)] +#[cfg_attr(feature = "serde", derive(serde::Deserialize, serde::Serialize))] pub enum Network { /// Mainnet #[default] @@ -38,3 +45,28 @@ impl Network { } } } + +#[derive(Debug, PartialEq, Eq)] +pub struct ParseNetworkError; + +impl FromStr for Network { + type Err = ParseNetworkError; + + fn from_str(s: &str) -> Result { + match s { + "mainnet" | "Mainnet" => Ok(Self::Mainnet), + "testnet" | "Testnet" => Ok(Self::Testnet), + "stagenet" | "Stagenet" => Ok(Self::Stagenet), + _ => Err(ParseNetworkError), + } + } +} +impl Display for Network { + fn fmt(&self, f: &mut Formatter<'_>) -> core::fmt::Result { + f.write_str(match self { + Self::Mainnet => "mainnet", + Self::Testnet => "testnet", + Self::Stagenet => "stagenet", + }) + } +} diff --git a/p2p/address-book/Cargo.toml b/p2p/address-book/Cargo.toml index a88819f7..d57cfded 100644 --- a/p2p/address-book/Cargo.toml +++ b/p2p/address-book/Cargo.toml @@ -23,7 +23,7 @@ indexmap = { workspace = true, features = ["std"] } rand = { workspace = true, features = ["std", "std_rng"] } -borsh = { workspace = true, features = ["derive", "std"]} +borsh = { workspace = true, features = ["derive", "std"] } [dev-dependencies] cuprate-test-utils = { workspace = true } diff --git a/p2p/address-book/src/book/tests.rs b/p2p/address-book/src/book/tests.rs index 216fcfac..b2c4c493 100644 --- a/p2p/address-book/src/book/tests.rs +++ b/p2p/address-book/src/book/tests.rs @@ -15,7 +15,7 @@ fn test_cfg() -> AddressBookConfig { AddressBookConfig { max_white_list_length: 100, max_gray_list_length: 500, - peer_store_file: PathBuf::new(), + peer_store_directory: PathBuf::new(), peer_save_period: Duration::from_secs(60), } } diff --git a/p2p/address-book/src/lib.rs b/p2p/address-book/src/lib.rs index c0903485..054be462 100644 --- a/p2p/address-book/src/lib.rs +++ b/p2p/address-book/src/lib.rs @@ -29,8 +29,8 @@ pub struct AddressBookConfig { /// /// Gray peers are peers we are yet to make a connection to. pub max_gray_list_length: usize, - /// The location to store the address book. - pub peer_store_file: PathBuf, + /// The location to store the peer store files. + pub peer_store_directory: PathBuf, /// The amount of time between saving the address book to disk. pub peer_save_period: Duration, } @@ -63,11 +63,6 @@ pub enum AddressBookError { pub async fn init_address_book( cfg: AddressBookConfig, ) -> Result, std::io::Error> { - tracing::info!( - "Loading peers from file: {} ", - cfg.peer_store_file.display() - ); - let (white_list, gray_list) = match store::read_peers_from_disk::(&cfg).await { Ok(res) => res, Err(e) if e.kind() == ErrorKind::NotFound => (vec![], vec![]), diff --git a/p2p/address-book/src/store.rs b/p2p/address-book/src/store.rs index 9abf0c3e..47994ae5 100644 --- a/p2p/address-book/src/store.rs +++ b/p2p/address-book/src/store.rs @@ -39,7 +39,9 @@ pub(crate) fn save_peers_to_disk( }) .unwrap(); - let file = cfg.peer_store_file.clone(); + let file = cfg + .peer_store_directory + .join(format!("{}_p2p_state", Z::NAME)); spawn_blocking(move || fs::write(&file, &data)) } @@ -52,7 +54,12 @@ pub(crate) async fn read_peers_from_disk( ), std::io::Error, > { - let file = cfg.peer_store_file.clone(); + let file = cfg + .peer_store_directory + .join(format!("{}_p2p_state", Z::NAME)); + + tracing::info!("Loading peers from file: {} ", file.display()); + let data = spawn_blocking(move || fs::read(file)).await.unwrap()?; let de_ser: DeserPeerDataV1 = from_slice(&data)?; diff --git a/p2p/p2p/src/block_downloader.rs b/p2p/p2p/src/block_downloader.rs index faac4d5d..db103000 100644 --- a/p2p/p2p/src/block_downloader.rs +++ b/p2p/p2p/src/block_downloader.rs @@ -62,15 +62,15 @@ pub struct BlockBatch { pub struct BlockDownloaderConfig { /// The size in bytes of the buffer between the block downloader and the place which /// is consuming the downloaded blocks. - pub buffer_size: usize, + pub buffer_bytes: usize, /// The size of the in progress queue (in bytes) at which we stop requesting more blocks. - pub in_progress_queue_size: usize, + pub in_progress_queue_bytes: usize, /// The [`Duration`] between checking the client pool for free peers. pub check_client_pool_interval: Duration, /// The target size of a single batch of blocks (in bytes). - pub target_batch_size: usize, + pub target_batch_bytes: usize, /// The initial amount of blocks to request (in number of blocks) - pub initial_batch_size: usize, + pub initial_batch_len: usize, } /// An error that occurred in the [`BlockDownloader`]. @@ -145,7 +145,7 @@ where + 'static, C::Future: Send + 'static, { - let (buffer_appender, buffer_stream) = cuprate_async_buffer::new_buffer(config.buffer_size); + let (buffer_appender, buffer_stream) = cuprate_async_buffer::new_buffer(config.buffer_bytes); let block_downloader = BlockDownloader::new(peer_set, our_chain_svc, buffer_appender, config); @@ -242,7 +242,7 @@ where Self { peer_set, our_chain_svc, - amount_of_blocks_to_request: config.initial_batch_size, + amount_of_blocks_to_request: config.initial_batch_len, amount_of_blocks_to_request_updated_at: 0, amount_of_empty_chain_entries: 0, block_download_tasks: JoinSet::new(), @@ -381,7 +381,7 @@ where } // If our ready queue is too large send duplicate requests for the blocks we are waiting on. - if self.block_queue.size() >= self.config.in_progress_queue_size { + if self.block_queue.size() >= self.config.in_progress_queue_bytes { return self.request_inflight_batch_again(client); } @@ -565,7 +565,7 @@ where self.amount_of_blocks_to_request = calculate_next_block_batch_size( block_batch.size, block_batch.blocks.len(), - self.config.target_batch_size, + self.config.target_batch_bytes, ); tracing::debug!( diff --git a/p2p/p2p/src/block_downloader/tests.rs b/p2p/p2p/src/block_downloader/tests.rs index 6799482d..707b858d 100644 --- a/p2p/p2p/src/block_downloader/tests.rs +++ b/p2p/p2p/src/block_downloader/tests.rs @@ -66,11 +66,11 @@ proptest! { genesis: *blockchain.blocks.first().unwrap().0 }, BlockDownloaderConfig { - buffer_size: 1_000, - in_progress_queue_size: 10_000, + buffer_bytes: 1_000, + in_progress_queue_bytes: 10_000, check_client_pool_interval: Duration::from_secs(5), - target_batch_size: 5_000, - initial_batch_size: 1, + target_batch_bytes: 5_000, + initial_batch_len: 1, }); let blocks = stream.map(|blocks| blocks.blocks).concat().await; diff --git a/storage/blockchain/Cargo.toml b/storage/blockchain/Cargo.toml index 6fd973cd..92b4374b 100644 --- a/storage/blockchain/Cargo.toml +++ b/storage/blockchain/Cargo.toml @@ -15,7 +15,7 @@ default = ["heed"] heed = ["cuprate-database/heed"] redb = ["cuprate-database/redb"] redb-memory = ["cuprate-database/redb-memory"] -serde = ["dep:serde", "cuprate-database/serde", "cuprate-database-service/serde"] +serde = ["dep:serde", "cuprate-database/serde", "cuprate-database-service/serde", "cuprate-helper/serde"] [dependencies] cuprate-database = { workspace = true } diff --git a/storage/blockchain/README.md b/storage/blockchain/README.md index 3f97a3d6..8ab8b432 100644 --- a/storage/blockchain/README.md +++ b/storage/blockchain/README.md @@ -76,7 +76,7 @@ use cuprate_blockchain::{ let tmp_dir = tempfile::tempdir()?; let db_dir = tmp_dir.path().to_owned(); let config = ConfigBuilder::new() - .db_directory(db_dir.into()) + .data_directory(db_dir.into()) .build(); // Initialize the database environment. diff --git a/storage/blockchain/src/config.rs b/storage/blockchain/src/config.rs index e4b76068..4bef2cb0 100644 --- a/storage/blockchain/src/config.rs +++ b/storage/blockchain/src/config.rs @@ -25,7 +25,7 @@ //! //! let config = ConfigBuilder::new() //! // Use a custom database directory. -//! .db_directory(db_dir.into()) +//! .data_directory(db_dir.into()) //! // Use as many reader threads as possible (when using `service`). //! .reader_threads(ReaderThreads::OnePerThread) //! // Use the fastest sync mode. @@ -41,13 +41,16 @@ //! ``` //---------------------------------------------------------------------------------------------------- Import -use std::{borrow::Cow, path::Path}; +use std::{borrow::Cow, path::PathBuf}; #[cfg(feature = "serde")] use serde::{Deserialize, Serialize}; use cuprate_database::{config::SyncMode, resize::ResizeAlgorithm}; -use cuprate_helper::fs::CUPRATE_BLOCKCHAIN_DIR; +use cuprate_helper::{ + fs::{blockchain_path, CUPRATE_DATA_DIR}, + network::Network, +}; // re-exports pub use cuprate_database_service::ReaderThreads; @@ -59,8 +62,9 @@ pub use cuprate_database_service::ReaderThreads; #[derive(Debug, Clone, PartialEq, PartialOrd)] #[cfg_attr(feature = "serde", derive(Serialize, Deserialize))] pub struct ConfigBuilder { - /// [`Config::db_directory`]. - db_directory: Option>, + network: Network, + + data_dir: Option, /// [`Config::cuprate_database_config`]. db_config: cuprate_database::config::ConfigBuilder, @@ -76,10 +80,12 @@ impl ConfigBuilder { /// after this function to use default values. pub fn new() -> Self { Self { - db_directory: None, - db_config: cuprate_database::config::ConfigBuilder::new(Cow::Borrowed( - &*CUPRATE_BLOCKCHAIN_DIR, - )), + network: Network::default(), + data_dir: None, + db_config: cuprate_database::config::ConfigBuilder::new(Cow::Owned(blockchain_path( + &CUPRATE_DATA_DIR, + Network::Mainnet, + ))), reader_threads: None, } } @@ -87,21 +93,21 @@ impl ConfigBuilder { /// Build into a [`Config`]. /// /// # Default values - /// If [`ConfigBuilder::db_directory`] was not called, - /// the default [`CUPRATE_BLOCKCHAIN_DIR`] will be used. + /// If [`ConfigBuilder::data_directory`] was not called, + /// [`blockchain_path`] with [`CUPRATE_DATA_DIR`] [`Network::Mainnet`] will be used. /// /// For all other values, [`Default::default`] is used. pub fn build(self) -> Config { // INVARIANT: all PATH safety checks are done // in `helper::fs`. No need to do them here. - let db_directory = self - .db_directory - .unwrap_or_else(|| Cow::Borrowed(&*CUPRATE_BLOCKCHAIN_DIR)); + let data_dir = self + .data_dir + .unwrap_or_else(|| CUPRATE_DATA_DIR.to_path_buf()); let reader_threads = self.reader_threads.unwrap_or_default(); let db_config = self .db_config - .db_directory(db_directory) + .db_directory(Cow::Owned(blockchain_path(&data_dir, self.network))) .reader_threads(reader_threads.as_threads()) .build(); @@ -111,10 +117,17 @@ impl ConfigBuilder { } } - /// Set a custom database directory (and file) [`Path`]. + /// Change the network this blockchain database is for. #[must_use] - pub fn db_directory(mut self, db_directory: Cow<'static, Path>) -> Self { - self.db_directory = Some(db_directory); + pub const fn network(mut self, network: Network) -> Self { + self.network = network; + self + } + + /// Set a custom database directory (and file) [`PathBuf`]. + #[must_use] + pub fn data_directory(mut self, db_directory: PathBuf) -> Self { + self.data_dir = Some(db_directory); self } @@ -145,9 +158,7 @@ impl ConfigBuilder { /// Good default for testing, and resource-available machines. #[must_use] pub fn fast(mut self) -> Self { - self.db_config = - cuprate_database::config::ConfigBuilder::new(Cow::Borrowed(&*CUPRATE_BLOCKCHAIN_DIR)) - .fast(); + self.db_config = self.db_config.fast(); self.reader_threads = Some(ReaderThreads::OnePerThread); self @@ -159,9 +170,7 @@ impl ConfigBuilder { /// Good default for resource-limited machines, e.g. a cheap VPS. #[must_use] pub fn low_power(mut self) -> Self { - self.db_config = - cuprate_database::config::ConfigBuilder::new(Cow::Borrowed(&*CUPRATE_BLOCKCHAIN_DIR)) - .low_power(); + self.db_config = self.db_config.low_power(); self.reader_threads = Some(ReaderThreads::One); self @@ -170,10 +179,13 @@ impl ConfigBuilder { impl Default for ConfigBuilder { fn default() -> Self { - let db_directory = Cow::Borrowed(&**CUPRATE_BLOCKCHAIN_DIR); Self { - db_directory: Some(db_directory.clone()), - db_config: cuprate_database::config::ConfigBuilder::new(db_directory), + network: Network::default(), + data_dir: Some(CUPRATE_DATA_DIR.to_path_buf()), + db_config: cuprate_database::config::ConfigBuilder::new(Cow::Owned(blockchain_path( + &CUPRATE_DATA_DIR, + Network::default(), + ))), reader_threads: Some(ReaderThreads::default()), } } @@ -201,7 +213,7 @@ impl Config { /// Create a new [`Config`] with sane default settings. /// /// The [`cuprate_database::config::Config::db_directory`] - /// will be set to [`CUPRATE_BLOCKCHAIN_DIR`]. + /// will be set to [`blockchain_path`] with [`CUPRATE_DATA_DIR`] [`Network::Mainnet`]. /// /// All other values will be [`Default::default`]. /// @@ -213,14 +225,14 @@ impl Config { /// resize::ResizeAlgorithm, /// DATABASE_DATA_FILENAME, /// }; - /// use cuprate_helper::fs::*; + /// use cuprate_helper::{fs::*, network::Network}; /// /// use cuprate_blockchain::config::*; /// /// let config = Config::new(); /// - /// assert_eq!(config.db_config.db_directory(), &*CUPRATE_BLOCKCHAIN_DIR); - /// assert!(config.db_config.db_file().starts_with(&*CUPRATE_BLOCKCHAIN_DIR)); + /// assert_eq!(config.db_config.db_directory().as_ref(), blockchain_path(&CUPRATE_DATA_DIR, Network::Mainnet).as_path()); + /// assert!(config.db_config.db_file().starts_with(&*CUPRATE_DATA_DIR)); /// assert!(config.db_config.db_file().ends_with(DATABASE_DATA_FILENAME)); /// assert_eq!(config.db_config.sync_mode, SyncMode::default()); /// assert_eq!(config.db_config.resize_algorithm, ResizeAlgorithm::default()); diff --git a/storage/blockchain/src/ops/mod.rs b/storage/blockchain/src/ops/mod.rs index 285aa244..96b25aff 100644 --- a/storage/blockchain/src/ops/mod.rs +++ b/storage/blockchain/src/ops/mod.rs @@ -71,7 +71,7 @@ //! let tmp_dir = tempfile::tempdir()?; //! let db_dir = tmp_dir.path().to_owned(); //! let config = ConfigBuilder::new() -//! .db_directory(db_dir.into()) +//! .data_directory(db_dir.into()) //! .build(); //! //! // Initialize the database environment. diff --git a/storage/blockchain/src/service/mod.rs b/storage/blockchain/src/service/mod.rs index c5eb80c2..d6a811bd 100644 --- a/storage/blockchain/src/service/mod.rs +++ b/storage/blockchain/src/service/mod.rs @@ -77,7 +77,7 @@ //! let tmp_dir = tempfile::tempdir()?; //! let db_dir = tmp_dir.path().to_owned(); //! let config = ConfigBuilder::new() -//! .db_directory(db_dir.into()) +//! .data_directory(db_dir.into()) //! .build(); //! //! // Initialize the database thread-pool. diff --git a/storage/blockchain/src/service/tests.rs b/storage/blockchain/src/service/tests.rs index 719f3613..38db6652 100644 --- a/storage/blockchain/src/service/tests.rs +++ b/storage/blockchain/src/service/tests.rs @@ -7,7 +7,6 @@ //---------------------------------------------------------------------------------------------------- Use use std::{ - borrow::Cow, collections::{HashMap, HashSet}, sync::Arc, }; @@ -46,7 +45,7 @@ fn init_service() -> ( ) { let tempdir = tempfile::tempdir().unwrap(); let config = ConfigBuilder::new() - .db_directory(Cow::Owned(tempdir.path().into())) + .data_directory(tempdir.path().into()) .low_power() .build(); let (reader, writer, env) = init(config).unwrap(); diff --git a/storage/blockchain/src/tests.rs b/storage/blockchain/src/tests.rs index 1fe20637..4192f812 100644 --- a/storage/blockchain/src/tests.rs +++ b/storage/blockchain/src/tests.rs @@ -5,7 +5,7 @@ //! - only used internally //---------------------------------------------------------------------------------------------------- Import -use std::{borrow::Cow, fmt::Debug}; +use std::fmt::Debug; use pretty_assertions::assert_eq; @@ -74,7 +74,7 @@ impl AssertTableLen { pub(crate) fn tmp_concrete_env() -> (impl Env, tempfile::TempDir) { let tempdir = tempfile::tempdir().unwrap(); let config = ConfigBuilder::new() - .db_directory(Cow::Owned(tempdir.path().into())) + .data_directory(tempdir.path().into()) .low_power() .build(); let env = crate::open(config).unwrap(); diff --git a/storage/txpool/Cargo.toml b/storage/txpool/Cargo.toml index c9082655..0fb43b24 100644 --- a/storage/txpool/Cargo.toml +++ b/storage/txpool/Cargo.toml @@ -15,7 +15,7 @@ default = ["heed"] heed = ["cuprate-database/heed"] redb = ["cuprate-database/redb"] redb-memory = ["cuprate-database/redb-memory"] -serde = ["dep:serde", "cuprate-database/serde", "cuprate-database-service/serde"] +serde = ["dep:serde", "cuprate-database/serde", "cuprate-database-service/serde", "cuprate-helper/serde"] [dependencies] cuprate-database = { workspace = true, features = ["heed"] } diff --git a/storage/txpool/README.md b/storage/txpool/README.md index d14f445b..ca4f737c 100644 --- a/storage/txpool/README.md +++ b/storage/txpool/README.md @@ -78,7 +78,7 @@ use cuprate_txpool::{ let tmp_dir = tempfile::tempdir()?; let db_dir = tmp_dir.path().to_owned(); let config = ConfigBuilder::new() - .db_directory(db_dir.into()) + .data_directory(db_dir.into()) .build(); // Initialize the database environment. diff --git a/storage/txpool/src/config.rs b/storage/txpool/src/config.rs index 1ef0d734..724ae21a 100644 --- a/storage/txpool/src/config.rs +++ b/storage/txpool/src/config.rs @@ -1,15 +1,18 @@ //! The transaction pool [`Config`]. -use std::{borrow::Cow, path::Path}; +use std::{borrow::Cow, path::PathBuf}; + +#[cfg(feature = "serde")] +use serde::{Deserialize, Serialize}; use cuprate_database::{ config::{Config as DbConfig, SyncMode}, resize::ResizeAlgorithm, }; use cuprate_database_service::ReaderThreads; -use cuprate_helper::fs::CUPRATE_TXPOOL_DIR; - -#[cfg(feature = "serde")] -use serde::{Deserialize, Serialize}; +use cuprate_helper::{ + fs::{txpool_path, CUPRATE_DATA_DIR}, + network::Network, +}; /// The default transaction pool weight limit. const DEFAULT_TXPOOL_WEIGHT_LIMIT: usize = 600 * 1024 * 1024; @@ -21,8 +24,9 @@ const DEFAULT_TXPOOL_WEIGHT_LIMIT: usize = 600 * 1024 * 1024; #[derive(Debug, Clone, PartialEq, PartialOrd)] #[cfg_attr(feature = "serde", derive(Serialize, Deserialize))] pub struct ConfigBuilder { - /// [`Config::db_directory`]. - db_directory: Option>, + network: Network, + + data_dir: Option, /// [`Config::cuprate_database_config`]. db_config: cuprate_database::config::ConfigBuilder, @@ -41,10 +45,12 @@ impl ConfigBuilder { /// after this function to use default values. pub fn new() -> Self { Self { - db_directory: None, - db_config: cuprate_database::config::ConfigBuilder::new(Cow::Borrowed( - &*CUPRATE_TXPOOL_DIR, - )), + network: Network::default(), + data_dir: None, + db_config: cuprate_database::config::ConfigBuilder::new(Cow::Owned(txpool_path( + &CUPRATE_DATA_DIR, + Network::Mainnet, + ))), reader_threads: None, max_txpool_weight: None, } @@ -53,16 +59,16 @@ impl ConfigBuilder { /// Build into a [`Config`]. /// /// # Default values - /// If [`ConfigBuilder::db_directory`] was not called, - /// the default [`CUPRATE_TXPOOL_DIR`] will be used. + /// If [`ConfigBuilder::data_directory`] was not called, + /// [`txpool_path`] with [`CUPRATE_DATA_DIR`] and [`Network::Mainnet`] will be used. /// /// For all other values, [`Default::default`] is used. pub fn build(self) -> Config { // INVARIANT: all PATH safety checks are done // in `helper::fs`. No need to do them here. - let db_directory = self - .db_directory - .unwrap_or_else(|| Cow::Borrowed(&*CUPRATE_TXPOOL_DIR)); + let data_dir = self + .data_dir + .unwrap_or_else(|| CUPRATE_DATA_DIR.to_path_buf()); let reader_threads = self.reader_threads.unwrap_or_default(); @@ -72,7 +78,7 @@ impl ConfigBuilder { let db_config = self .db_config - .db_directory(db_directory) + .db_directory(Cow::Owned(txpool_path(&data_dir, self.network))) .reader_threads(reader_threads.as_threads()) .build(); @@ -83,6 +89,13 @@ impl ConfigBuilder { } } + /// Change the network this database is for. + #[must_use] + pub const fn network(mut self, network: Network) -> Self { + self.network = network; + self + } + /// Sets a new maximum weight for the transaction pool. #[must_use] pub const fn max_txpool_weight(mut self, max_txpool_weight: usize) -> Self { @@ -90,10 +103,10 @@ impl ConfigBuilder { self } - /// Set a custom database directory (and file) [`Path`]. + /// Set a custom data directory [`PathBuf`]. #[must_use] - pub fn db_directory(mut self, db_directory: Cow<'static, Path>) -> Self { - self.db_directory = Some(db_directory); + pub fn data_directory(mut self, db_directory: PathBuf) -> Self { + self.data_dir = Some(db_directory); self } @@ -124,9 +137,7 @@ impl ConfigBuilder { /// Good default for testing, and resource-available machines. #[must_use] pub fn fast(mut self) -> Self { - self.db_config = - cuprate_database::config::ConfigBuilder::new(Cow::Borrowed(&*CUPRATE_TXPOOL_DIR)) - .fast(); + self.db_config = self.db_config.fast(); self.reader_threads = Some(ReaderThreads::OnePerThread); self @@ -138,9 +149,7 @@ impl ConfigBuilder { /// Good default for resource-limited machines, e.g. a cheap VPS. #[must_use] pub fn low_power(mut self) -> Self { - self.db_config = - cuprate_database::config::ConfigBuilder::new(Cow::Borrowed(&*CUPRATE_TXPOOL_DIR)) - .low_power(); + self.db_config = self.db_config.low_power(); self.reader_threads = Some(ReaderThreads::One); self @@ -149,10 +158,13 @@ impl ConfigBuilder { impl Default for ConfigBuilder { fn default() -> Self { - let db_directory = Cow::Borrowed(CUPRATE_TXPOOL_DIR.as_path()); Self { - db_directory: Some(db_directory.clone()), - db_config: cuprate_database::config::ConfigBuilder::new(db_directory), + network: Network::default(), + data_dir: Some(CUPRATE_DATA_DIR.to_path_buf()), + db_config: cuprate_database::config::ConfigBuilder::new(Cow::Owned(txpool_path( + &CUPRATE_DATA_DIR, + Network::Mainnet, + ))), reader_threads: Some(ReaderThreads::default()), max_txpool_weight: Some(DEFAULT_TXPOOL_WEIGHT_LIMIT), } @@ -184,7 +196,7 @@ impl Config { /// Create a new [`Config`] with sane default settings. /// /// The [`DbConfig::db_directory`] - /// will be set to [`CUPRATE_TXPOOL_DIR`]. + /// will be set to [`txpool_path`] with [`CUPRATE_DATA_DIR`] and [`Network::Mainnet`]. /// /// All other values will be [`Default::default`]. /// @@ -197,25 +209,21 @@ impl Config { /// DATABASE_DATA_FILENAME, /// }; /// use cuprate_database_service::ReaderThreads; - /// use cuprate_helper::fs::*; + /// use cuprate_helper::{fs::*, network::Network}; /// /// use cuprate_txpool::Config; /// /// let config = Config::new(); /// - /// assert_eq!(config.db_config.db_directory(), &*CUPRATE_TXPOOL_DIR); - /// assert!(config.db_config.db_file().starts_with(&*CUPRATE_TXPOOL_DIR)); + /// assert_eq!(config.db_config.db_directory(), txpool_path(&CUPRATE_DATA_DIR, Network::Mainnet).as_path()); + /// assert!(config.db_config.db_file().starts_with(&*CUPRATE_DATA_DIR)); /// assert!(config.db_config.db_file().ends_with(DATABASE_DATA_FILENAME)); /// assert_eq!(config.db_config.sync_mode, SyncMode::default()); /// assert_eq!(config.db_config.resize_algorithm, ResizeAlgorithm::default()); /// assert_eq!(config.reader_threads, ReaderThreads::default()); /// ``` pub fn new() -> Self { - Self { - db_config: DbConfig::new(Cow::Borrowed(&*CUPRATE_TXPOOL_DIR)), - reader_threads: ReaderThreads::default(), - max_txpool_weight: 0, - } + ConfigBuilder::new().build() } } diff --git a/storage/txpool/src/ops.rs b/storage/txpool/src/ops.rs index 289a8bbf..badc4f6d 100644 --- a/storage/txpool/src/ops.rs +++ b/storage/txpool/src/ops.rs @@ -51,7 +51,7 @@ //! let tmp_dir = tempfile::tempdir()?; //! let db_dir = tmp_dir.path().to_owned(); //! let config = ConfigBuilder::new() -//! .db_directory(db_dir.into()) +//! .data_directory(db_dir.into()) //! .build(); //! //! // Initialize the database environment. diff --git a/storage/txpool/src/service.rs b/storage/txpool/src/service.rs index a82de5bf..03ce2f03 100644 --- a/storage/txpool/src/service.rs +++ b/storage/txpool/src/service.rs @@ -83,7 +83,7 @@ //! let tmp_dir = tempfile::tempdir()?; //! let db_dir = tmp_dir.path().to_owned(); //! let config = ConfigBuilder::new() -//! .db_directory(db_dir.into()) +//! .data_directory(db_dir.into()) //! .build(); //! //! // Initialize the database thread-pool. From 7b8756fa80e386fb04173d8220c15c86bf9f9888 Mon Sep 17 00:00:00 2001 From: Boog900 Date: Tue, 3 Dec 2024 20:21:05 +0000 Subject: [PATCH 10/10] cuprated: P2P protocol request handler (#303) * add cuprated skeleton * fmt and add deny exception * add main chain batch handler * add blockchain init * very rough block manager * misc changes * move more config values * add new tables & types * add function to fully add an alt block * resolve current todo!s * add new requests * WIP: starting re-orgs * add last service request * commit Cargo.lock * add test * more docs + cleanup + alt blocks request * clippy + fmt * document types * move tx_fee to helper * more doc updates * fmt * fix imports * remove config files * fix merge errors * fix generated coins * handle more p2p requests + alt blocks * clean up handler code * add function for incoming blocks * add docs to handler functions * broadcast new blocks + add commands * add fluffy block handler * fix new block handling * small cleanup * increase outbound peer count * fix merge * clean up the blockchain manger * add more docs + cleanup imports * fix typo * fix doc * remove unrelated changes * add `get_objects` handler * add `get_chain` handler * add `fluffy_missing_txs` handler * add `new_fluffy_block` handler * improve interface globals * manger -> manager * enums instead of bools * move chain service to separate file * more review fixes * sort imports + docs * init dandelion integration * add dandelion start function * finish incoming tx handler * Add tx blob hash table * Add missing txpool requests * handle duplicate stem txs * check txpool on incoming block * add request to remove tx in new blocks from the pool * tell the txpool about incoming blocks * fix merge * typos * remove blockchain height from txpool * fix merge * fix merge * handle incoming txs in p2p request handler * Allow `IncomingTxHandler` to be given later * add p2p clearnet init * fix build * misc changes * doc updates * more doc updates * sort imports * review changes * Result -> DbResult * use `NonZero` * review fixes * remove `rust-2024-compatibility` lint --- Cargo.lock | 1 + Cargo.toml | 1 - binaries/cuprated/src/p2p.rs | 49 ++ .../cuprated/src/p2p/core_sync_service.rs | 49 ++ binaries/cuprated/src/p2p/request_handler.rs | 421 ++++++++++++++++++ binaries/cuprated/src/txpool.rs | 2 +- binaries/cuprated/src/txpool/incoming_tx.rs | 5 + net/wire/src/p2p/protocol.rs | 2 +- p2p/p2p-core/src/protocol.rs | 4 + p2p/p2p-core/src/protocol/try_from.rs | 3 + p2p/p2p/src/constants.rs | 6 +- storage/blockchain/Cargo.toml | 1 + storage/blockchain/src/ops/block.rs | 68 ++- storage/blockchain/src/ops/blockchain.rs | 42 +- storage/blockchain/src/service/read.rs | 174 ++++++-- types/src/blockchain.rs | 66 ++- types/src/lib.rs | 4 +- types/src/types.rs | 7 + 18 files changed, 861 insertions(+), 44 deletions(-) create mode 100644 binaries/cuprated/src/p2p/core_sync_service.rs diff --git a/Cargo.lock b/Cargo.lock index 0d55c8aa..c8701f35 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -704,6 +704,7 @@ version = "0.0.0" dependencies = [ "bitflags 2.6.0", "bytemuck", + "bytes", "cuprate-constants", "cuprate-database", "cuprate-database-service", diff --git a/Cargo.toml b/Cargo.toml index 9be1528c..18130577 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -376,7 +376,6 @@ unused_lifetimes = "deny" unused_macro_rules = "deny" ambiguous_glob_imports = "deny" unused_unsafe = "deny" -rust_2024_compatibility = "deny" # Warm let_underscore = { level = "deny", priority = -1 } diff --git a/binaries/cuprated/src/p2p.rs b/binaries/cuprated/src/p2p.rs index cdf1cef7..aeb98b60 100644 --- a/binaries/cuprated/src/p2p.rs +++ b/binaries/cuprated/src/p2p.rs @@ -1,8 +1,57 @@ //! P2P //! //! Will handle initiating the P2P and contains a protocol request handler. +use futures::{FutureExt, TryFutureExt}; +use tokio::sync::oneshot; +use tower::ServiceExt; +use cuprate_blockchain::service::BlockchainReadHandle; +use cuprate_consensus::BlockChainContextService; +use cuprate_p2p::{NetworkInterface, P2PConfig}; +use cuprate_p2p_core::ClearNet; +use cuprate_txpool::service::TxpoolReadHandle; + +use crate::txpool::IncomingTxHandler; + +mod core_sync_service; mod network_address; pub mod request_handler; pub use network_address::CrossNetworkInternalPeerId; + +/// Starts the P2P clearnet network, returning a [`NetworkInterface`] to interact with it. +/// +/// A [`oneshot::Sender`] is also returned to provide the [`IncomingTxHandler`], until this is provided network +/// handshakes can not be completed. +pub async fn start_clearnet_p2p( + blockchain_read_handle: BlockchainReadHandle, + blockchain_context_service: BlockChainContextService, + txpool_read_handle: TxpoolReadHandle, + config: P2PConfig, +) -> Result< + ( + NetworkInterface, + oneshot::Sender, + ), + tower::BoxError, +> { + let (incoming_tx_handler_tx, incoming_tx_handler_rx) = oneshot::channel(); + + let request_handler_maker = request_handler::P2pProtocolRequestHandlerMaker { + blockchain_read_handle, + blockchain_context_service: blockchain_context_service.clone(), + txpool_read_handle, + incoming_tx_handler: None, + incoming_tx_handler_fut: incoming_tx_handler_rx.shared(), + }; + + Ok(( + cuprate_p2p::initialize_network( + request_handler_maker.map_response(|s| s.map_err(Into::into)), + core_sync_service::CoreSyncService(blockchain_context_service), + config, + ) + .await?, + incoming_tx_handler_tx, + )) +} diff --git a/binaries/cuprated/src/p2p/core_sync_service.rs b/binaries/cuprated/src/p2p/core_sync_service.rs new file mode 100644 index 00000000..d3c3ca1e --- /dev/null +++ b/binaries/cuprated/src/p2p/core_sync_service.rs @@ -0,0 +1,49 @@ +use std::task::{Context, Poll}; + +use futures::{future::BoxFuture, FutureExt, TryFutureExt}; +use tower::Service; + +use cuprate_consensus::{ + BlockChainContextRequest, BlockChainContextResponse, BlockChainContextService, +}; +use cuprate_helper::{cast::usize_to_u64, map::split_u128_into_low_high_bits}; +use cuprate_p2p_core::services::{CoreSyncDataRequest, CoreSyncDataResponse}; +use cuprate_wire::CoreSyncData; + +/// The core sync service. +#[derive(Clone)] +pub struct CoreSyncService(pub BlockChainContextService); + +impl Service for CoreSyncService { + type Response = CoreSyncDataResponse; + type Error = tower::BoxError; + type Future = BoxFuture<'static, Result>; + + fn poll_ready(&mut self, cx: &mut Context<'_>) -> Poll> { + self.0.poll_ready(cx) + } + + fn call(&mut self, _: CoreSyncDataRequest) -> Self::Future { + self.0 + .call(BlockChainContextRequest::Context) + .map_ok(|res| { + let BlockChainContextResponse::Context(context) = res else { + unreachable!() + }; + + let context = context.unchecked_blockchain_context(); + let (cumulative_difficulty, cumulative_difficulty_top64) = + split_u128_into_low_high_bits(context.cumulative_difficulty); + + CoreSyncDataResponse(CoreSyncData { + cumulative_difficulty, + cumulative_difficulty_top64, + current_height: usize_to_u64(context.chain_height), + pruning_seed: 0, + top_id: context.top_hash, + top_version: context.current_hf.as_u8(), + }) + }) + .boxed() + } +} diff --git a/binaries/cuprated/src/p2p/request_handler.rs b/binaries/cuprated/src/p2p/request_handler.rs index 8b137891..7d72fa37 100644 --- a/binaries/cuprated/src/p2p/request_handler.rs +++ b/binaries/cuprated/src/p2p/request_handler.rs @@ -1 +1,422 @@ +use std::{ + collections::HashSet, + future::{ready, Ready}, + hash::Hash, + task::{Context, Poll}, +}; +use bytes::Bytes; +use futures::{ + future::{BoxFuture, Shared}, + FutureExt, +}; +use monero_serai::{block::Block, transaction::Transaction}; +use tokio::sync::{broadcast, oneshot, watch}; +use tokio_stream::wrappers::WatchStream; +use tower::{Service, ServiceExt}; + +use cuprate_blockchain::service::BlockchainReadHandle; +use cuprate_consensus::{ + transactions::new_tx_verification_data, BlockChainContextRequest, BlockChainContextResponse, + BlockChainContextService, +}; +use cuprate_dandelion_tower::TxState; +use cuprate_fixed_bytes::ByteArrayVec; +use cuprate_helper::cast::u64_to_usize; +use cuprate_helper::{ + asynch::rayon_spawn_async, + cast::usize_to_u64, + map::{combine_low_high_bits_to_u128, split_u128_into_low_high_bits}, +}; +use cuprate_p2p::constants::{ + MAX_BLOCKS_IDS_IN_CHAIN_ENTRY, MAX_BLOCK_BATCH_LEN, MAX_TRANSACTION_BLOB_SIZE, MEDIUM_BAN, +}; +use cuprate_p2p_core::{ + client::{InternalPeerID, PeerInformation}, + NetZoneAddress, NetworkZone, ProtocolRequest, ProtocolResponse, +}; +use cuprate_txpool::service::TxpoolReadHandle; +use cuprate_types::{ + blockchain::{BlockchainReadRequest, BlockchainResponse}, + BlockCompleteEntry, TransactionBlobs, TxsInBlock, +}; +use cuprate_wire::protocol::{ + ChainRequest, ChainResponse, FluffyMissingTransactionsRequest, GetObjectsRequest, + GetObjectsResponse, NewFluffyBlock, NewTransactions, +}; + +use crate::{ + blockchain::interface::{self as blockchain_interface, IncomingBlockError}, + constants::PANIC_CRITICAL_SERVICE_ERROR, + p2p::CrossNetworkInternalPeerId, + txpool::{IncomingTxError, IncomingTxHandler, IncomingTxs}, +}; + +/// The P2P protocol request handler [`MakeService`](tower::MakeService). +#[derive(Clone)] +pub struct P2pProtocolRequestHandlerMaker { + pub blockchain_read_handle: BlockchainReadHandle, + pub blockchain_context_service: BlockChainContextService, + pub txpool_read_handle: TxpoolReadHandle, + + /// The [`IncomingTxHandler`], wrapped in an [`Option`] as there is a cyclic reference between [`P2pProtocolRequestHandlerMaker`] + /// and the [`IncomingTxHandler`]. + pub incoming_tx_handler: Option, + + /// A [`Future`](std::future::Future) that produces the [`IncomingTxHandler`]. + pub incoming_tx_handler_fut: Shared>, +} + +impl Service> for P2pProtocolRequestHandlerMaker +where + InternalPeerID: Into, +{ + type Response = P2pProtocolRequestHandler; + type Error = tower::BoxError; + type Future = Ready>; + + fn poll_ready(&mut self, cx: &mut Context<'_>) -> Poll> { + if self.incoming_tx_handler.is_none() { + return self + .incoming_tx_handler_fut + .poll_unpin(cx) + .map(|incoming_tx_handler| { + self.incoming_tx_handler = Some(incoming_tx_handler?); + Ok(()) + }); + } + + Poll::Ready(Ok(())) + } + + fn call(&mut self, peer_information: PeerInformation) -> Self::Future { + let Some(incoming_tx_handler) = self.incoming_tx_handler.clone() else { + panic!("poll_ready was not called or did not return `Poll::Ready`") + }; + + // TODO: check sync info? + + let blockchain_read_handle = self.blockchain_read_handle.clone(); + let txpool_read_handle = self.txpool_read_handle.clone(); + + ready(Ok(P2pProtocolRequestHandler { + peer_information, + blockchain_read_handle, + blockchain_context_service: self.blockchain_context_service.clone(), + txpool_read_handle, + incoming_tx_handler, + })) + } +} + +/// The P2P protocol request handler. +#[derive(Clone)] +pub struct P2pProtocolRequestHandler { + peer_information: PeerInformation, + blockchain_read_handle: BlockchainReadHandle, + blockchain_context_service: BlockChainContextService, + txpool_read_handle: TxpoolReadHandle, + incoming_tx_handler: IncomingTxHandler, +} + +impl Service for P2pProtocolRequestHandler +where + InternalPeerID: Into, +{ + type Response = ProtocolResponse; + type Error = anyhow::Error; + type Future = BoxFuture<'static, Result>; + + fn poll_ready(&mut self, _: &mut Context<'_>) -> Poll> { + Poll::Ready(Ok(())) + } + + fn call(&mut self, request: ProtocolRequest) -> Self::Future { + match request { + ProtocolRequest::GetObjects(r) => { + get_objects(r, self.blockchain_read_handle.clone()).boxed() + } + ProtocolRequest::GetChain(r) => { + get_chain(r, self.blockchain_read_handle.clone()).boxed() + } + ProtocolRequest::FluffyMissingTxs(r) => { + fluffy_missing_txs(r, self.blockchain_read_handle.clone()).boxed() + } + ProtocolRequest::NewBlock(_) => ready(Err(anyhow::anyhow!( + "Peer sent a full block when we support fluffy blocks" + ))) + .boxed(), + ProtocolRequest::NewFluffyBlock(r) => new_fluffy_block( + self.peer_information.clone(), + r, + self.blockchain_read_handle.clone(), + self.txpool_read_handle.clone(), + ) + .boxed(), + ProtocolRequest::NewTransactions(r) => new_transactions( + self.peer_information.clone(), + r, + self.blockchain_context_service.clone(), + self.incoming_tx_handler.clone(), + ) + .boxed(), + ProtocolRequest::GetTxPoolCompliment(_) => ready(Ok(ProtocolResponse::NA)).boxed(), // TODO: should we support this? + } + } +} + +//---------------------------------------------------------------------------------------------------- Handler functions + +/// [`ProtocolRequest::GetObjects`] +async fn get_objects( + request: GetObjectsRequest, + mut blockchain_read_handle: BlockchainReadHandle, +) -> anyhow::Result { + if request.blocks.len() > MAX_BLOCK_BATCH_LEN { + anyhow::bail!("Peer requested more blocks than allowed.") + } + + let block_hashes: Vec<[u8; 32]> = (&request.blocks).into(); + // deallocate the backing `Bytes`. + drop(request); + + let BlockchainResponse::BlockCompleteEntries { + blocks, + missing_hashes, + blockchain_height, + } = blockchain_read_handle + .ready() + .await? + .call(BlockchainReadRequest::BlockCompleteEntries(block_hashes)) + .await? + else { + unreachable!(); + }; + + Ok(ProtocolResponse::GetObjects(GetObjectsResponse { + blocks, + missed_ids: ByteArrayVec::from(missing_hashes), + current_blockchain_height: usize_to_u64(blockchain_height), + })) +} + +/// [`ProtocolRequest::GetChain`] +async fn get_chain( + request: ChainRequest, + mut blockchain_read_handle: BlockchainReadHandle, +) -> anyhow::Result { + if request.block_ids.len() > MAX_BLOCKS_IDS_IN_CHAIN_ENTRY { + anyhow::bail!("Peer sent too many block hashes in chain request.") + } + + let block_hashes: Vec<[u8; 32]> = (&request.block_ids).into(); + let want_pruned_data = request.prune; + // deallocate the backing `Bytes`. + drop(request); + + let BlockchainResponse::NextChainEntry { + start_height, + chain_height, + block_ids, + block_weights, + cumulative_difficulty, + first_block_blob, + } = blockchain_read_handle + .ready() + .await? + .call(BlockchainReadRequest::NextChainEntry(block_hashes, 10_000)) + .await? + else { + unreachable!(); + }; + + let Some(start_height) = start_height else { + anyhow::bail!("The peers chain has a different genesis block than ours."); + }; + + let (cumulative_difficulty_low64, cumulative_difficulty_top64) = + split_u128_into_low_high_bits(cumulative_difficulty); + + Ok(ProtocolResponse::GetChain(ChainResponse { + start_height: usize_to_u64(std::num::NonZero::get(start_height)), + total_height: usize_to_u64(chain_height), + cumulative_difficulty_low64, + cumulative_difficulty_top64, + m_block_ids: ByteArrayVec::from(block_ids), + first_block: first_block_blob.map_or(Bytes::new(), Bytes::from), + // only needed when pruned + m_block_weights: if want_pruned_data { + block_weights.into_iter().map(usize_to_u64).collect() + } else { + vec![] + }, + })) +} + +/// [`ProtocolRequest::FluffyMissingTxs`] +async fn fluffy_missing_txs( + mut request: FluffyMissingTransactionsRequest, + mut blockchain_read_handle: BlockchainReadHandle, +) -> anyhow::Result { + let tx_indexes = std::mem::take(&mut request.missing_tx_indices); + let block_hash: [u8; 32] = *request.block_hash; + let current_blockchain_height = request.current_blockchain_height; + + // deallocate the backing `Bytes`. + drop(request); + + let BlockchainResponse::TxsInBlock(res) = blockchain_read_handle + .ready() + .await? + .call(BlockchainReadRequest::TxsInBlock { + block_hash, + tx_indexes, + }) + .await? + else { + unreachable!(); + }; + + let Some(TxsInBlock { block, txs }) = res else { + anyhow::bail!("The peer requested txs out of range."); + }; + + Ok(ProtocolResponse::NewFluffyBlock(NewFluffyBlock { + b: BlockCompleteEntry { + block: Bytes::from(block), + txs: TransactionBlobs::Normal(txs.into_iter().map(Bytes::from).collect()), + pruned: false, + // only needed for pruned blocks. + block_weight: 0, + }, + current_blockchain_height, + })) +} + +/// [`ProtocolRequest::NewFluffyBlock`] +async fn new_fluffy_block( + peer_information: PeerInformation, + request: NewFluffyBlock, + mut blockchain_read_handle: BlockchainReadHandle, + mut txpool_read_handle: TxpoolReadHandle, +) -> anyhow::Result { + // TODO: check context service here and ignore the block? + let current_blockchain_height = request.current_blockchain_height; + + peer_information + .core_sync_data + .lock() + .unwrap() + .current_height = current_blockchain_height; + + let (block, txs) = rayon_spawn_async(move || -> Result<_, anyhow::Error> { + let block = Block::read(&mut request.b.block.as_ref())?; + + let tx_blobs = request + .b + .txs + .take_normal() + .ok_or(anyhow::anyhow!("Peer sent pruned txs in fluffy block"))?; + + let txs = tx_blobs + .into_iter() + .map(|tx_blob| { + if tx_blob.len() > MAX_TRANSACTION_BLOB_SIZE { + anyhow::bail!("Peer sent a transaction over the size limit."); + } + + let tx = Transaction::read(&mut tx_blob.as_ref())?; + + Ok((tx.hash(), tx)) + }) + .collect::>()?; + + // The backing `Bytes` will be deallocated when this closure returns. + + Ok((block, txs)) + }) + .await?; + + let res = blockchain_interface::handle_incoming_block( + block, + txs, + &mut blockchain_read_handle, + &mut txpool_read_handle, + ) + .await; + + match res { + Ok(_) => Ok(ProtocolResponse::NA), + Err(IncomingBlockError::UnknownTransactions(block_hash, missing_tx_indices)) => Ok( + ProtocolResponse::FluffyMissingTransactionsRequest(FluffyMissingTransactionsRequest { + block_hash: block_hash.into(), + current_blockchain_height, + missing_tx_indices: missing_tx_indices.into_iter().map(usize_to_u64).collect(), + }), + ), + Err(IncomingBlockError::Orphan) => { + // Block's parent was unknown, could be syncing? + Ok(ProtocolResponse::NA) + } + Err(e) => Err(e.into()), + } +} + +/// [`ProtocolRequest::NewTransactions`] +async fn new_transactions( + peer_information: PeerInformation, + request: NewTransactions, + mut blockchain_context_service: BlockChainContextService, + mut incoming_tx_handler: IncomingTxHandler, +) -> anyhow::Result +where + A: NetZoneAddress, + InternalPeerID: Into, +{ + let BlockChainContextResponse::Context(context) = blockchain_context_service + .ready() + .await + .expect(PANIC_CRITICAL_SERVICE_ERROR) + .call(BlockChainContextRequest::Context) + .await + .expect(PANIC_CRITICAL_SERVICE_ERROR) + else { + unreachable!() + }; + + let context = context.unchecked_blockchain_context(); + + // If we are more than 2 blocks behind the peer then ignore the txs - we are probably still syncing. + if usize_to_u64(context.chain_height + 2) + < peer_information + .core_sync_data + .lock() + .unwrap() + .current_height + { + return Ok(ProtocolResponse::NA); + } + + let state = if request.dandelionpp_fluff { + TxState::Fluff + } else { + TxState::Stem { + from: peer_information.id.into(), + } + }; + + // Drop all the data except the stuff we still need. + let NewTransactions { txs, .. } = request; + + let res = incoming_tx_handler + .ready() + .await + .expect(PANIC_CRITICAL_SERVICE_ERROR) + .call(IncomingTxs { txs, state }) + .await; + + match res { + Ok(()) => Ok(ProtocolResponse::NA), + Err(e) => Err(e.into()), + } +} diff --git a/binaries/cuprated/src/txpool.rs b/binaries/cuprated/src/txpool.rs index 9592c2bf..20769567 100644 --- a/binaries/cuprated/src/txpool.rs +++ b/binaries/cuprated/src/txpool.rs @@ -12,4 +12,4 @@ mod dandelion; mod incoming_tx; mod txs_being_handled; -pub use incoming_tx::IncomingTxHandler; +pub use incoming_tx::{IncomingTxError, IncomingTxHandler, IncomingTxs}; diff --git a/binaries/cuprated/src/txpool/incoming_tx.rs b/binaries/cuprated/src/txpool/incoming_tx.rs index e2041598..bf7684e4 100644 --- a/binaries/cuprated/src/txpool/incoming_tx.rs +++ b/binaries/cuprated/src/txpool/incoming_tx.rs @@ -43,9 +43,13 @@ use crate::{ }; /// An error that can happen handling an incoming tx. +#[derive(Debug, thiserror::Error)] pub enum IncomingTxError { + #[error("Error parsing tx: {0}")] Parse(std::io::Error), + #[error(transparent)] Consensus(ExtendedConsensusError), + #[error("Duplicate tx in message")] DuplicateTransaction, } @@ -67,6 +71,7 @@ pub(super) 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. +#[derive(Clone)] pub struct IncomingTxHandler { /// A store of txs currently being handled in incoming tx requests. pub(super) txs_being_handled: TxsBeingHandled, diff --git a/net/wire/src/p2p/protocol.rs b/net/wire/src/p2p/protocol.rs index 1d1d45ab..cc4b49d3 100644 --- a/net/wire/src/p2p/protocol.rs +++ b/net/wire/src/p2p/protocol.rs @@ -159,7 +159,7 @@ epee_object!( current_blockchain_height: u64, ); -/// A request for Txs we are missing from our `TxPool` +/// A request for txs we are missing from an incoming block. #[derive(Debug, Clone, PartialEq, Eq)] pub struct FluffyMissingTransactionsRequest { /// The Block we are missing the Txs in diff --git a/p2p/p2p-core/src/protocol.rs b/p2p/p2p-core/src/protocol.rs index 7d8d431b..82aac824 100644 --- a/p2p/p2p-core/src/protocol.rs +++ b/p2p/p2p-core/src/protocol.rs @@ -116,6 +116,7 @@ pub enum ProtocolResponse { GetChain(ChainResponse), NewFluffyBlock(NewFluffyBlock), NewTransactions(NewTransactions), + FluffyMissingTransactionsRequest(FluffyMissingTransactionsRequest), NA, } @@ -139,6 +140,9 @@ impl PeerResponse { ProtocolResponse::GetChain(_) => MessageID::GetChain, ProtocolResponse::NewFluffyBlock(_) => MessageID::NewBlock, ProtocolResponse::NewTransactions(_) => MessageID::NewFluffyBlock, + ProtocolResponse::FluffyMissingTransactionsRequest(_) => { + MessageID::FluffyMissingTxs + } ProtocolResponse::NA => return None, }, diff --git a/p2p/p2p-core/src/protocol/try_from.rs b/p2p/p2p-core/src/protocol/try_from.rs index d3a7260f..2dfc41db 100644 --- a/p2p/p2p-core/src/protocol/try_from.rs +++ b/p2p/p2p-core/src/protocol/try_from.rs @@ -71,6 +71,9 @@ impl TryFrom for ProtocolMessage { ProtocolResponse::NewFluffyBlock(val) => Self::NewFluffyBlock(val), ProtocolResponse::GetChain(val) => Self::ChainEntryResponse(val), ProtocolResponse::GetObjects(val) => Self::GetObjectsResponse(val), + ProtocolResponse::FluffyMissingTransactionsRequest(val) => { + Self::FluffyMissingTransactionsRequest(val) + } ProtocolResponse::NA => return Err(MessageConversionError), }) } diff --git a/p2p/p2p/src/constants.rs b/p2p/p2p/src/constants.rs index f70d64c9..a81557c2 100644 --- a/p2p/p2p/src/constants.rs +++ b/p2p/p2p/src/constants.rs @@ -52,7 +52,7 @@ pub(crate) const INITIAL_CHAIN_REQUESTS_TO_SEND: usize = 3; /// The enforced maximum amount of blocks to request in a batch. /// /// Requesting more than this will cause the peer to disconnect and potentially lead to bans. -pub(crate) const MAX_BLOCK_BATCH_LEN: usize = 100; +pub const MAX_BLOCK_BATCH_LEN: usize = 100; /// The timeout that the block downloader will use for requests. pub(crate) const BLOCK_DOWNLOADER_REQUEST_TIMEOUT: Duration = Duration::from_secs(30); @@ -61,13 +61,13 @@ pub(crate) const BLOCK_DOWNLOADER_REQUEST_TIMEOUT: Duration = Duration::from_sec /// be less than. /// /// ref: -pub(crate) const MAX_TRANSACTION_BLOB_SIZE: usize = 1_000_000; +pub const MAX_TRANSACTION_BLOB_SIZE: usize = 1_000_000; /// The maximum amount of block IDs allowed in a chain entry response. /// /// ref: // TODO: link to the protocol book when this section is added. -pub(crate) const MAX_BLOCKS_IDS_IN_CHAIN_ENTRY: usize = 25_000; +pub const MAX_BLOCKS_IDS_IN_CHAIN_ENTRY: usize = 25_000; /// The amount of failures downloading a specific batch before we stop attempting to download it. pub(crate) const MAX_DOWNLOAD_FAILURES: usize = 5; diff --git a/storage/blockchain/Cargo.toml b/storage/blockchain/Cargo.toml index 92b4374b..c9359248 100644 --- a/storage/blockchain/Cargo.toml +++ b/storage/blockchain/Cargo.toml @@ -34,6 +34,7 @@ serde = { workspace = true, optional = true } tower = { workspace = true } thread_local = { workspace = true } rayon = { workspace = true } +bytes = { workspace = true } [dev-dependencies] cuprate-constants = { workspace = true } diff --git a/storage/blockchain/src/ops/block.rs b/storage/blockchain/src/ops/block.rs index 5e541878..2dc88aa4 100644 --- a/storage/blockchain/src/ops/block.rs +++ b/storage/blockchain/src/ops/block.rs @@ -2,21 +2,23 @@ //---------------------------------------------------------------------------------------------------- Import use bytemuck::TransparentWrapper; +use bytes::Bytes; use monero_serai::{ block::{Block, BlockHeader}, transaction::Transaction, }; use cuprate_database::{ - DbResult, RuntimeError, StorableVec, {DatabaseRo, DatabaseRw}, + DbResult, RuntimeError, StorableVec, {DatabaseIter, DatabaseRo, DatabaseRw}, }; +use cuprate_helper::cast::usize_to_u64; use cuprate_helper::{ map::{combine_low_high_bits_to_u128, split_u128_into_low_high_bits}, tx::tx_fee, }; use cuprate_types::{ - AltBlockInformation, ChainId, ExtendedBlockHeader, HardFork, VerifiedBlockInformation, - VerifiedTransactionInformation, + AltBlockInformation, BlockCompleteEntry, ChainId, ExtendedBlockHeader, HardFork, + TransactionBlobs, VerifiedBlockInformation, VerifiedTransactionInformation, }; use crate::{ @@ -27,7 +29,7 @@ use crate::{ output::get_rct_num_outputs, tx::{add_tx, remove_tx}, }, - tables::{BlockHeights, BlockInfos, Tables, TablesMut}, + tables::{BlockHeights, BlockInfos, Tables, TablesIter, TablesMut}, types::{BlockHash, BlockHeight, BlockInfo}, }; @@ -222,6 +224,64 @@ pub fn pop_block( Ok((block_height, block_info.block_hash, block)) } +//---------------------------------------------------------------------------------------------------- `get_block_blob_with_tx_indexes` +/// Retrieve a block's raw bytes, the index of the miner transaction and the number of non miner-txs in the block. +/// +#[doc = doc_error!()] +pub fn get_block_blob_with_tx_indexes( + block_height: &BlockHeight, + tables: &impl Tables, +) -> Result<(Vec, u64, usize), RuntimeError> { + let miner_tx_idx = tables.block_infos().get(block_height)?.mining_tx_index; + + let block_txs = tables.block_txs_hashes().get(block_height)?.0; + let numb_txs = block_txs.len(); + + // Get the block header + let mut block = tables.block_header_blobs().get(block_height)?.0; + + // Add the miner tx to the blob. + let mut miner_tx_blob = tables.tx_blobs().get(&miner_tx_idx)?.0; + block.append(&mut miner_tx_blob); + + // Add the blocks tx hashes. + monero_serai::io::write_varint(&block_txs.len(), &mut block) + .expect("The number of txs per block will not exceed u64::MAX"); + + let block_txs_bytes = bytemuck::must_cast_slice(&block_txs); + block.extend_from_slice(block_txs_bytes); + + Ok((block, miner_tx_idx, numb_txs)) +} + +//---------------------------------------------------------------------------------------------------- `get_block_extended_header_*` +/// Retrieve a [`BlockCompleteEntry`] from the database. +/// +#[doc = doc_error!()] +pub fn get_block_complete_entry( + block_hash: &BlockHash, + tables: &impl TablesIter, +) -> Result { + let block_height = tables.block_heights().get(block_hash)?; + let (block_blob, miner_tx_idx, numb_non_miner_txs) = + get_block_blob_with_tx_indexes(&block_height, tables)?; + + let first_tx_idx = miner_tx_idx + 1; + + let tx_blobs = tables + .tx_blobs_iter() + .get_range(first_tx_idx..(usize_to_u64(numb_non_miner_txs) + first_tx_idx))? + .map(|tx_blob| Ok(Bytes::from(tx_blob?.0))) + .collect::>()?; + + Ok(BlockCompleteEntry { + block: Bytes::from(block_blob), + txs: TransactionBlobs::Normal(tx_blobs), + pruned: false, + block_weight: 0, + }) +} + //---------------------------------------------------------------------------------------------------- `get_block_extended_header_*` /// Retrieve a [`ExtendedBlockHeader`] from the database. /// diff --git a/storage/blockchain/src/ops/blockchain.rs b/storage/blockchain/src/ops/blockchain.rs index 71633635..54dd752a 100644 --- a/storage/blockchain/src/ops/blockchain.rs +++ b/storage/blockchain/src/ops/blockchain.rs @@ -4,9 +4,9 @@ use cuprate_database::{DatabaseRo, DbResult, RuntimeError}; use crate::{ - ops::macros::doc_error, + ops::{block::block_exists, macros::doc_error}, tables::{BlockHeights, BlockInfos}, - types::BlockHeight, + types::{BlockHash, BlockHeight}, }; //---------------------------------------------------------------------------------------------------- Free Functions @@ -76,6 +76,44 @@ pub fn cumulative_generated_coins( } } +/// Find the split point between our chain and a list of [`BlockHash`]s from another chain. +/// +/// This function accepts chains in chronological and reverse chronological order, however +/// if the wrong order is specified the return value is meaningless. +/// +/// For chronologically ordered chains this will return the index of the first unknown, for reverse +/// chronologically ordered chains this will return the index of the first known. +/// +/// If all blocks are known for chronologically ordered chains or unknown for reverse chronologically +/// ordered chains then the length of the chain will be returned. +#[doc = doc_error!()] +#[inline] +pub fn find_split_point( + block_ids: &[BlockHash], + chronological_order: bool, + table_block_heights: &impl DatabaseRo, +) -> Result { + let mut err = None; + + // Do a binary search to find the first unknown/known block in the batch. + let idx = block_ids.partition_point(|block_id| { + match block_exists(block_id, table_block_heights) { + Ok(exists) => exists == chronological_order, + Err(e) => { + err.get_or_insert(e); + // if this happens the search is scrapped, just return `false` back. + false + } + } + }); + + if let Some(e) = err { + return Err(e); + } + + Ok(idx) +} + //---------------------------------------------------------------------------------------------------- Tests #[cfg(test)] mod test { diff --git a/storage/blockchain/src/service/read.rs b/storage/blockchain/src/service/read.rs index 76577590..84b1b921 100644 --- a/storage/blockchain/src/service/read.rs +++ b/storage/blockchain/src/service/read.rs @@ -10,23 +10,26 @@ //---------------------------------------------------------------------------------------------------- Import use std::{ + cmp::min, collections::{HashMap, HashSet}, sync::Arc, }; use rayon::{ - iter::{IntoParallelIterator, ParallelIterator}, + iter::{Either, IntoParallelIterator, ParallelIterator}, prelude::*, ThreadPool, }; use thread_local::ThreadLocal; -use cuprate_database::{ConcreteEnv, DatabaseRo, DbResult, Env, EnvInner, RuntimeError}; +use cuprate_database::{ + ConcreteEnv, DatabaseIter, DatabaseRo, DbResult, Env, EnvInner, RuntimeError, +}; use cuprate_database_service::{init_thread_pool, DatabaseReadService, ReaderThreads}; use cuprate_helper::map::combine_low_high_bits_to_u128; use cuprate_types::{ blockchain::{BlockchainReadRequest, BlockchainResponse}, - Chain, ChainId, ExtendedBlockHeader, OutputHistogramInput, OutputOnChain, + Chain, ChainId, ExtendedBlockHeader, OutputHistogramInput, OutputOnChain, TxsInBlock, }; use crate::{ @@ -36,9 +39,10 @@ use crate::{ get_alt_chain_history_ranges, }, block::{ - block_exists, get_block_extended_header_from_height, get_block_height, get_block_info, + block_exists, get_block_blob_with_tx_indexes, get_block_complete_entry, + get_block_extended_header_from_height, get_block_height, get_block_info, }, - blockchain::{cumulative_generated_coins, top_block_height}, + blockchain::{cumulative_generated_coins, find_split_point, top_block_height}, key_image::key_image_exists, output::id_to_output_on_chain, }, @@ -46,7 +50,7 @@ use crate::{ free::{compact_history_genesis_not_included, compact_history_index_to_height_offset}, types::{BlockchainReadHandle, ResponseResult}, }, - tables::{AltBlockHeights, BlockHeights, BlockInfos, OpenTables, Tables}, + tables::{AltBlockHeights, BlockHeights, BlockInfos, OpenTables, Tables, TablesIter}, types::{ AltBlockHeight, Amount, AmountIndex, BlockHash, BlockHeight, KeyImage, PreRctOutputId, }, @@ -100,6 +104,7 @@ fn map_request( /* SOMEDAY: pre-request handling, run some code for each request? */ match request { + R::BlockCompleteEntries(block_hashes) => block_complete_entries(env, block_hashes), R::BlockExtendedHeader(block) => block_extended_header(env, block), R::BlockHash(block, chain) => block_hash(env, block, chain), R::FindBlock(block_hash) => find_block(env, block_hash), @@ -113,7 +118,12 @@ fn map_request( R::NumberOutputsWithAmount(vec) => number_outputs_with_amount(env, vec), R::KeyImagesSpent(set) => key_images_spent(env, set), R::CompactChainHistory => compact_chain_history(env), + R::NextChainEntry(block_hashes, amount) => next_chain_entry(env, &block_hashes, amount), R::FindFirstUnknown(block_ids) => find_first_unknown(env, &block_ids), + R::TxsInBlock { + block_hash, + tx_indexes, + } => txs_in_block(env, block_hash, tx_indexes), R::AltBlocksInChain(chain_id) => alt_blocks_in_chain(env, chain_id), R::Block { height } => block(env, height), R::BlockByHash(hash) => block_by_hash(env, hash), @@ -198,6 +208,38 @@ macro_rules! get_tables { // TODO: The overhead of parallelism may be too much for every request, perfomace test to find optimal // amount of parallelism. +/// [`BlockchainReadRequest::BlockCompleteEntries`]. +fn block_complete_entries(env: &ConcreteEnv, block_hashes: Vec) -> ResponseResult { + // Prepare tx/tables in `ThreadLocal`. + let env_inner = env.env_inner(); + let tx_ro = thread_local(env); + let tables = thread_local(env); + + let (missing_hashes, blocks) = block_hashes + .into_par_iter() + .map(|block_hash| { + let tx_ro = tx_ro.get_or_try(|| env_inner.tx_ro())?; + let tables = get_tables!(env_inner, tx_ro, tables)?.as_ref(); + + match get_block_complete_entry(&block_hash, tables) { + Err(RuntimeError::KeyNotFound) => Ok(Either::Left(block_hash)), + res => res.map(Either::Right), + } + }) + .collect::>()?; + + let tx_ro = tx_ro.get_or_try(|| env_inner.tx_ro())?; + let tables = get_tables!(env_inner, tx_ro, tables)?.as_ref(); + + let blockchain_height = crate::ops::blockchain::chain_height(tables.block_heights())?; + + Ok(BlockchainResponse::BlockCompleteEntries { + blocks, + missing_hashes, + blockchain_height, + }) +} + /// [`BlockchainReadRequest::BlockExtendedHeader`]. #[inline] fn block_extended_header(env: &ConcreteEnv, block_height: BlockHeight) -> ResponseResult { @@ -335,7 +377,7 @@ fn block_extended_header_in_range( } }) }) - .collect::, _>>()? + .collect::>>()? } }; @@ -534,6 +576,76 @@ fn compact_chain_history(env: &ConcreteEnv) -> ResponseResult { }) } +/// [`BlockchainReadRequest::NextChainEntry`] +/// +/// # Invariant +/// `block_ids` must be sorted in reverse chronological block order, or else +/// the returned result is unspecified and meaningless, as this function +/// performs a binary search. +fn next_chain_entry( + env: &ConcreteEnv, + block_ids: &[BlockHash], + next_entry_size: usize, +) -> ResponseResult { + // Single-threaded, no `ThreadLocal` required. + let env_inner = env.env_inner(); + let tx_ro = env_inner.tx_ro()?; + + let tables = env_inner.open_tables(&tx_ro)?; + let table_block_heights = tables.block_heights(); + let table_block_infos = tables.block_infos_iter(); + + let idx = find_split_point(block_ids, false, table_block_heights)?; + + // This will happen if we have a different genesis block. + if idx == block_ids.len() { + return Ok(BlockchainResponse::NextChainEntry { + start_height: None, + chain_height: 0, + block_ids: vec![], + block_weights: vec![], + cumulative_difficulty: 0, + first_block_blob: None, + }); + } + + // The returned chain entry must overlap with one of the blocks we were told about. + let first_known_block_hash = block_ids[idx]; + let first_known_height = table_block_heights.get(&first_known_block_hash)?; + + let chain_height = crate::ops::blockchain::chain_height(table_block_heights)?; + let last_height_in_chain_entry = min(first_known_height + next_entry_size, chain_height); + + let (block_ids, block_weights) = table_block_infos + .get_range(first_known_height..last_height_in_chain_entry)? + .map(|block_info| { + let block_info = block_info?; + + Ok((block_info.block_hash, block_info.weight)) + }) + .collect::, Vec<_>)>>()?; + + let top_block_info = table_block_infos.get(&(chain_height - 1))?; + + let first_block_blob = if block_ids.len() >= 2 { + Some(get_block_blob_with_tx_indexes(&(first_known_height + 1), &tables)?.0) + } else { + None + }; + + Ok(BlockchainResponse::NextChainEntry { + start_height: std::num::NonZero::new(first_known_height), + chain_height, + block_ids, + block_weights, + cumulative_difficulty: combine_low_high_bits_to_u128( + top_block_info.cumulative_difficulty_low, + top_block_info.cumulative_difficulty_high, + ), + first_block_blob, + }) +} + /// [`BlockchainReadRequest::FindFirstUnknown`] /// /// # Invariant @@ -546,24 +658,7 @@ fn find_first_unknown(env: &ConcreteEnv, block_ids: &[BlockHash]) -> ResponseRes let table_block_heights = env_inner.open_db_ro::(&tx_ro)?; - let mut err = None; - - // Do a binary search to find the first unknown block in the batch. - let idx = - block_ids.partition_point( - |block_id| match block_exists(block_id, &table_block_heights) { - Ok(exists) => exists, - Err(e) => { - err.get_or_insert(e); - // if this happens the search is scrapped, just return `false` back. - false - } - }, - ); - - if let Some(e) = err { - return Err(e); - } + let idx = find_split_point(block_ids, true, &table_block_heights)?; Ok(if idx == block_ids.len() { BlockchainResponse::FindFirstUnknown(None) @@ -576,6 +671,33 @@ fn find_first_unknown(env: &ConcreteEnv, block_ids: &[BlockHash]) -> ResponseRes }) } +/// [`BlockchainReadRequest::TxsInBlock`] +fn txs_in_block(env: &ConcreteEnv, block_hash: [u8; 32], missing_txs: Vec) -> ResponseResult { + // Single-threaded, no `ThreadLocal` required. + let env_inner = env.env_inner(); + let tx_ro = env_inner.tx_ro()?; + let tables = env_inner.open_tables(&tx_ro)?; + + let block_height = tables.block_heights().get(&block_hash)?; + + let (block, miner_tx_index, numb_txs) = get_block_blob_with_tx_indexes(&block_height, &tables)?; + let first_tx_index = miner_tx_index + 1; + + if numb_txs < missing_txs.len() { + return Ok(BlockchainResponse::TxsInBlock(None)); + } + + let txs = missing_txs + .into_iter() + .map(|index_offset| Ok(tables.tx_blobs().get(&(first_tx_index + index_offset))?.0)) + .collect::>()?; + + Ok(BlockchainResponse::TxsInBlock(Some(TxsInBlock { + block, + txs, + }))) +} + /// [`BlockchainReadRequest::AltBlocksInChain`] fn alt_blocks_in_chain(env: &ConcreteEnv, chain_id: ChainId) -> ResponseResult { // Prepare tx/tables in `ThreadLocal`. @@ -613,7 +735,7 @@ fn alt_blocks_in_chain(env: &ConcreteEnv, chain_id: ChainId) -> ResponseResult { ) }) }) - .collect::>()?; + .collect::>()?; Ok(BlockchainResponse::AltBlocksInChain(blocks)) } diff --git a/types/src/blockchain.rs b/types/src/blockchain.rs index c39c0bd8..7518935d 100644 --- a/types/src/blockchain.rs +++ b/types/src/blockchain.rs @@ -11,9 +11,9 @@ use std::{ use monero_serai::block::Block; use crate::{ - types::{Chain, ExtendedBlockHeader, OutputOnChain, VerifiedBlockInformation}, - AltBlockInformation, ChainId, ChainInfo, CoinbaseTxSum, OutputHistogramEntry, - OutputHistogramInput, + types::{Chain, ExtendedBlockHeader, OutputOnChain, TxsInBlock, VerifiedBlockInformation}, + AltBlockInformation, BlockCompleteEntry, ChainId, ChainInfo, CoinbaseTxSum, + OutputHistogramEntry, OutputHistogramInput, }; //---------------------------------------------------------------------------------------------------- ReadRequest @@ -27,6 +27,11 @@ use crate::{ /// See `Response` for the expected responses per `Request`. #[derive(Debug, Clone, PartialEq, Eq)] pub enum BlockchainReadRequest { + /// Request [`BlockCompleteEntry`]s. + /// + /// The input is the block hashes. + BlockCompleteEntries(Vec<[u8; 32]>), + /// Request a block's extended header. /// /// The input is the block's height. @@ -96,6 +101,16 @@ pub enum BlockchainReadRequest { /// A request for the compact chain history. CompactChainHistory, + /// A request for the next chain entry. + /// + /// Input is a list of block hashes and the amount of block hashes to return in the next chain entry. + /// + /// # Invariant + /// The [`Vec`] containing the block IDs must be sorted in reverse chronological block + /// order, or else the returned response is unspecified and meaningless, + /// as this request performs a binary search + NextChainEntry(Vec<[u8; 32]>, usize), + /// A request to find the first unknown block ID in a list of block IDs. /// /// # Invariant @@ -104,6 +119,16 @@ pub enum BlockchainReadRequest { /// as this request performs a binary search. FindFirstUnknown(Vec<[u8; 32]>), + /// A request for transactions from a specific block. + TxsInBlock { + /// The block to get transactions from. + block_hash: [u8; 32], + /// The indexes of the transactions from the block. + /// This is not the global index of the txs, instead it is the local index as they appear in + /// the block. + tx_indexes: Vec, + }, + /// A request for all alt blocks in the chain with the given [`ChainId`]. AltBlocksInChain(ChainId), @@ -182,6 +207,16 @@ pub enum BlockchainWriteRequest { #[expect(clippy::large_enum_variant)] pub enum BlockchainResponse { //------------------------------------------------------ Reads + /// Response to [`BlockchainReadRequest::BlockCompleteEntries`]. + BlockCompleteEntries { + /// The [`BlockCompleteEntry`]s that we had. + blocks: Vec, + /// The hashes of blocks that were requested, but we don't have. + missing_hashes: Vec<[u8; 32]>, + /// Our blockchain height. + blockchain_height: usize, + }, + /// Response to [`BlockchainReadRequest::BlockExtendedHeader`]. /// /// Inner value is the extended headed of the requested block. @@ -248,6 +283,24 @@ pub enum BlockchainResponse { cumulative_difficulty: u128, }, + /// Response to [`BlockchainReadRequest::NextChainEntry`]. + /// + /// If all blocks were unknown `start_height` will be [`None`], the other fields will be meaningless. + NextChainEntry { + /// The start height of this entry, [`None`] if we could not find the split point. + start_height: Option>, + /// The current chain height. + chain_height: usize, + /// The next block hashes in the entry. + block_ids: Vec<[u8; 32]>, + /// The block weights of the next blocks. + block_weights: Vec, + /// The current cumulative difficulty of our chain. + cumulative_difficulty: u128, + /// The block blob of the 2nd block in `block_ids`, if there is one. + first_block_blob: Option>, + }, + /// Response to [`BlockchainReadRequest::FindFirstUnknown`]. /// /// Contains the index of the first unknown block and its expected height. @@ -255,7 +308,12 @@ pub enum BlockchainResponse { /// This will be [`None`] if all blocks were known. FindFirstUnknown(Option<(usize, usize)>), - /// Response to [`BlockchainReadRequest::AltBlocksInChain`]. + /// The response for [`BlockchainReadRequest::TxsInBlock`]. + /// + /// Will return [`None`] if the request contained an index out of range. + TxsInBlock(Option), + + /// The response for [`BlockchainReadRequest::AltBlocksInChain`]. /// /// Contains all the alt blocks in the alt-chain in chronological order. AltBlocksInChain(Vec), diff --git a/types/src/lib.rs b/types/src/lib.rs index a5a04f9d..7aaf0b9e 100644 --- a/types/src/lib.rs +++ b/types/src/lib.rs @@ -26,8 +26,8 @@ pub use transaction_verification_data::{ pub use types::{ AddAuxPow, AltBlockInformation, AuxPow, Chain, ChainId, ChainInfo, CoinbaseTxSum, ExtendedBlockHeader, FeeEstimate, HardForkInfo, MinerData, MinerDataTxBacklogEntry, - OutputHistogramEntry, OutputHistogramInput, OutputOnChain, VerifiedBlockInformation, - VerifiedTransactionInformation, + OutputHistogramEntry, OutputHistogramInput, OutputOnChain, TxsInBlock, + VerifiedBlockInformation, VerifiedTransactionInformation, }; //---------------------------------------------------------------------------------------------------- Feature-gated diff --git a/types/src/types.rs b/types/src/types.rs index 720ad0a7..8a5b5aad 100644 --- a/types/src/types.rs +++ b/types/src/types.rs @@ -259,6 +259,13 @@ pub struct AddAuxPow { pub aux_pow: Vec, } +/// The inner response for a request for txs in a block. +#[derive(Clone, Debug, PartialEq, Eq)] +pub struct TxsInBlock { + pub block: Vec, + pub txs: Vec>, +} + //---------------------------------------------------------------------------------------------------- Tests #[cfg(test)] mod test {