From 216886efb02cca5795e97f78a0035675a7716c67 Mon Sep 17 00:00:00 2001 From: Rodrigo Araujo Date: Tue, 13 Jun 2023 10:30:49 -0700 Subject: [PATCH 01/18] feat: Add initial new tx pool sync service --- Cargo.lock | 1 + crates/fuel-core/src/p2p_test_helpers.rs | 51 ++---- crates/fuel-core/src/service.rs | 35 +--- .../fuel-core/src/service/adapters/txpool.rs | 66 +++----- crates/fuel-core/src/service/sub_services.rs | 24 +-- crates/services/p2p/src/behavior.rs | 46 ++--- crates/services/p2p/src/p2p_service.rs | 125 ++++---------- crates/services/p2p/src/service.rs | 75 ++------ crates/services/src/service.rs | 15 +- crates/services/txpool/Cargo.toml | 1 + crates/services/txpool/src/ports.rs | 29 ++-- crates/services/txpool/src/service.rs | 160 +++++++++++++----- tests/tests/sync.rs | 21 +-- 13 files changed, 259 insertions(+), 390 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 513d79cfc88..14a1bf2ee01 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -2998,6 +2998,7 @@ dependencies = [ "async-trait", "fuel-core-chain-config", "fuel-core-metrics", + "fuel-core-p2p", "fuel-core-services", "fuel-core-storage", "fuel-core-trace", diff --git a/crates/fuel-core/src/p2p_test_helpers.rs b/crates/fuel-core/src/p2p_test_helpers.rs index 0352694159d..ab4393bc14b 100644 --- a/crates/fuel-core/src/p2p_test_helpers.rs +++ b/crates/fuel-core/src/p2p_test_helpers.rs @@ -4,60 +4,29 @@ use crate::{ chain_config::ChainConfig, database::Database, p2p::Multiaddr, - service::{ - genesis::maybe_initialize_state, - Config, - FuelService, - ServiceTrait, - }, + service::{genesis::maybe_initialize_state, Config, FuelService, ServiceTrait}, }; use fuel_core_p2p::{ - codecs::postcard::PostcardCodec, - network_service::FuelP2PService, - PeerId, -}; -use fuel_core_poa::{ - ports::BlockImporter, - Trigger, -}; -use fuel_core_storage::{ - tables::Transactions, - StorageAsRef, + codecs::postcard::PostcardCodec, network_service::FuelP2PService, PeerId, }; +use fuel_core_poa::{ports::BlockImporter, Trigger}; +use fuel_core_storage::{tables::Transactions, StorageAsRef}; use fuel_core_types::{ - fuel_asm::{ - op, - RegId, - }, + fuel_asm::{op, RegId}, fuel_crypto::SecretKey, fuel_tx::{ - ConsensusParameters, - Input, - Transaction, - TransactionBuilder, - TxId, - UniqueIdentifier, - UtxoId, - }, - fuel_types::{ - Address, - Bytes32, + ConsensusParameters, Input, Transaction, TransactionBuilder, TxId, + UniqueIdentifier, UtxoId, }, + fuel_types::{Address, Bytes32}, secrecy::Secret, }; use futures::StreamExt; use itertools::Itertools; -use rand::{ - rngs::StdRng, - Rng, - SeedableRng, -}; +use rand::{rngs::StdRng, Rng, SeedableRng}; use std::{ collections::HashMap, - ops::{ - Index, - IndexMut, - }, + ops::{Index, IndexMut}, sync::Arc, time::Duration, }; diff --git a/crates/fuel-core/src/service.rs b/crates/fuel-core/src/service.rs index 8282885d720..05db466fb2c 100644 --- a/crates/fuel-core/src/service.rs +++ b/crates/fuel-core/src/service.rs @@ -1,22 +1,11 @@ -use crate::{ - database::Database, - service::adapters::P2PAdapter, -}; +use crate::{database::Database, service::adapters::P2PAdapter}; use fuel_core_services::{ - RunnableService, - RunnableTask, - ServiceRunner, - State, - StateWatcher, + RunnableService, RunnableTask, ServiceRunner, State, StateWatcher, }; use std::net::SocketAddr; use tracing::log::warn; -pub use config::{ - Config, - DbType, - VMConfig, -}; +pub use config::{Config, DbType, VMConfig}; pub use fuel_core_services::Service as ServiceTrait; pub use fuel_core_consensus_module::RelayerVerifierConfig; @@ -273,19 +262,9 @@ impl RunnableTask for Task { #[cfg(test)] mod tests { - use crate::service::{ - Config, - Task, - }; - use fuel_core_services::{ - RunnableService, - RunnableTask, - State, - }; - use std::{ - thread::sleep, - time::Duration, - }; + use crate::service::{Config, Task}; + use fuel_core_services::{RunnableService, RunnableTask, State}; + use std::{thread::sleep, time::Duration}; #[tokio::test] async fn run_start_and_stop() { @@ -305,7 +284,7 @@ mod tests { task.sub_services()[i].stop_and_await().await.unwrap(); assert!(!task.run(&mut watcher).await.unwrap()); } else { - break + break; } i += 1; } diff --git a/crates/fuel-core/src/service/adapters/txpool.rs b/crates/fuel-core/src/service/adapters/txpool.rs index 3dd9ba9c089..4ab690db592 100644 --- a/crates/fuel-core/src/service/adapters/txpool.rs +++ b/crates/fuel-core/src/service/adapters/txpool.rs @@ -1,54 +1,32 @@ use crate::{ database::Database, - service::adapters::{ - BlockImporterAdapter, - P2PAdapter, + service::{ + adapters::{BlockImporterAdapter, P2PAdapter}, + sub_services::TxPoolService, }, }; +use fuel_core_p2p::PeerId; use fuel_core_services::stream::BoxStream; use fuel_core_storage::{ not_found, - tables::{ - Coins, - ContractsRawCode, - Messages, - SpentMessages, - }, - Result as StorageResult, - StorageAsRef, + tables::{Coins, ContractsRawCode, Messages, SpentMessages}, + Result as StorageResult, StorageAsRef, }; -use fuel_core_txpool::ports::BlockImporter; +use fuel_core_txpool::ports::{BlockImporter, TxPoolSyncPort}; use fuel_core_types::{ - entities::{ - coins::coin::CompressedCoin, - message::Message, - }, - fuel_tx::{ - Transaction, - UtxoId, - }, - fuel_types::{ - BlockHeight, - ContractId, - Nonce, - }, + entities::{coins::coin::CompressedCoin, message::Message}, + fuel_tx::{Transaction, UtxoId}, + fuel_types::{BlockHeight, ContractId, Nonce}, services::{ block_importer::ImportResult, - p2p::{ - GossipsubMessageAcceptance, - GossipsubMessageInfo, - TransactionGossipData, - }, + p2p::{GossipsubMessageAcceptance, GossipsubMessageInfo, TransactionGossipData}, }, }; use std::sync::Arc; impl BlockImporter for BlockImporterAdapter { fn block_events(&self) -> BoxStream> { - use tokio_stream::{ - wrappers::BroadcastStream, - StreamExt, - }; + use tokio_stream::{wrappers::BroadcastStream, StreamExt}; Box::pin( BroadcastStream::new(self.block_importer.subscribe()) .filter_map(|result| result.ok()), @@ -56,6 +34,21 @@ impl BlockImporter for BlockImporterAdapter { } } +#[cfg(feature = "p2p")] +impl TxPoolSyncPort for P2PAdapter { + fn new_connection(&self) -> BoxStream { + use tokio_stream::{wrappers::BroadcastStream, StreamExt}; + if let Some(service) = &self.service { + Box::pin( + BroadcastStream::new(service.subscribe_to_connections()) + .filter_map(|result| result.ok()), + ) + } else { + fuel_core_services::stream::IntoBoxStream::into_boxed(tokio_stream::pending()) + } + } +} + #[cfg(feature = "p2p")] impl fuel_core_txpool::ports::PeerToPeer for P2PAdapter { type GossipedTransaction = TransactionGossipData; @@ -69,10 +62,7 @@ impl fuel_core_txpool::ports::PeerToPeer for P2PAdapter { } fn gossiped_transaction_events(&self) -> BoxStream { - use tokio_stream::{ - wrappers::BroadcastStream, - StreamExt, - }; + use tokio_stream::{wrappers::BroadcastStream, StreamExt}; if let Some(service) = &self.service { Box::pin( BroadcastStream::new(service.subscribe_tx()) diff --git a/crates/fuel-core/src/service/sub_services.rs b/crates/fuel-core/src/service/sub_services.rs index cca69abdc7b..6537a17ca0d 100644 --- a/crates/fuel-core/src/service/sub_services.rs +++ b/crates/fuel-core/src/service/sub_services.rs @@ -1,23 +1,17 @@ #![allow(clippy::let_unit_value)] use super::adapters::P2PAdapter; +use crate::txpool::service::new_txpool_syncing_service; use crate::{ database::Database, fuel_core_graphql_api::Config as GraphQLConfig, schema::build_schema, service::{ adapters::{ - BlockImporterAdapter, - BlockProducerAdapter, - ExecutorAdapter, - MaybeRelayerAdapter, - PoAAdapter, - TxPoolAdapter, - VerifierAdapter, + BlockImporterAdapter, BlockProducerAdapter, ExecutorAdapter, + MaybeRelayerAdapter, PoAAdapter, TxPoolAdapter, VerifierAdapter, }, - Config, - SharedState, - SubServices, + Config, SharedState, SubServices, }, }; use fuel_core_poa::Trigger; @@ -112,6 +106,15 @@ pub fn init_sub_services( ); let tx_pool_adapter = TxPoolAdapter::new(txpool.shared.clone()); + let genesis = database.get_genesis()?; + let p2p_config = config.p2p.clone().expect("p2p config is required"); + let p2p_config = p2p_config.init(genesis)?; + let tx_pool_sync = new_txpool_syncing_service( + config.txpool.clone(), + p2p_config, + p2p_adapter.clone(), + ); + let block_producer = fuel_core_producer::Producer { config: config.block_producer.clone(), db: database.clone(), @@ -218,6 +221,7 @@ pub fn init_sub_services( if let Some(network) = network.take() { services.push(Box::new(network)); services.push(Box::new(sync)); + services.push(Box::new(tx_pool_sync)); } } diff --git a/crates/services/p2p/src/behavior.rs b/crates/services/p2p/src/behavior.rs index 4ce5e3d77e6..5e68b91726d 100644 --- a/crates/services/p2p/src/behavior.rs +++ b/crates/services/p2p/src/behavior.rs @@ -1,50 +1,24 @@ use crate::{ codecs::NetworkCodec, config::Config, - discovery::{ - DiscoveryBehaviour, - DiscoveryConfig, - DiscoveryEvent, - }, - gossipsub::{ - config::build_gossipsub_behaviour, - topics::GossipTopic, - }, - peer_report::{ - PeerReportBehaviour, - PeerReportEvent, - }, - request_response::messages::{ - NetworkResponse, - RequestMessage, - }, + discovery::{DiscoveryBehaviour, DiscoveryConfig, DiscoveryEvent}, + gossipsub::{config::build_gossipsub_behaviour, topics::GossipTopic}, + peer_report::{PeerReportBehaviour, PeerReportEvent}, + request_response::messages::{NetworkResponse, RequestMessage}, }; use fuel_core_types::fuel_types::BlockHeight; use libp2p::{ gossipsub::{ - error::PublishError, - Gossipsub, - GossipsubEvent, - MessageAcceptance, - MessageId, + error::PublishError, Gossipsub, GossipsubEvent, MessageAcceptance, MessageId, }, request_response::{ - ProtocolSupport, - RequestId, - RequestResponse, - RequestResponseConfig, - RequestResponseEvent, - ResponseChannel, + ProtocolSupport, RequestId, RequestResponse, RequestResponseConfig, + RequestResponseEvent, ResponseChannel, }, swarm::NetworkBehaviour, - Multiaddr, - PeerId, -}; -use tracing::{ - debug, - error, - log::warn, + Multiaddr, PeerId, }; +use tracing::{debug, error, log::warn}; #[derive(Debug)] pub enum FuelBehaviourEvent { @@ -171,7 +145,7 @@ impl FuelBehaviour { Ok(true) => { debug!(target: "fuel-p2p", "Sent a report for MessageId: {} from PeerId: {}", msg_id, propagation_source); if should_check_score { - return self.gossipsub.peer_score(propagation_source) + return self.gossipsub.peer_score(propagation_source); } } Ok(false) => { diff --git a/crates/services/p2p/src/p2p_service.rs b/crates/services/p2p/src/p2p_service.rs index 632b2351fca..4a7873ac2dc 100644 --- a/crates/services/p2p/src/p2p_service.rs +++ b/crates/services/p2p/src/p2p_service.rs @@ -1,72 +1,37 @@ use crate::{ - behavior::{ - FuelBehaviour, - FuelBehaviourEvent, - }, + behavior::{FuelBehaviour, FuelBehaviourEvent}, codecs::NetworkCodec, - config::{ - build_transport, - Config, - }, + config::{build_transport, Config}, gossipsub::{ - messages::{ - GossipsubBroadcastRequest, - GossipsubMessage as FuelGossipsubMessage, - }, + messages::{GossipsubBroadcastRequest, GossipsubMessage as FuelGossipsubMessage}, topics::GossipsubTopics, }, - peer_manager::{ - PeerManager, - Punisher, - }, + peer_manager::{PeerManager, Punisher}, peer_report::PeerReportEvent, request_response::messages::{ - NetworkResponse, - OutboundResponse, - RequestError, - RequestMessage, - ResponseChannelItem, - ResponseError, - ResponseMessage, + NetworkResponse, OutboundResponse, RequestError, RequestMessage, + ResponseChannelItem, ResponseError, ResponseMessage, }, }; use fuel_core_metrics::p2p_metrics::P2P_METRICS; use fuel_core_types::{ - fuel_types::BlockHeight, - services::p2p::peer_reputation::AppScore, + fuel_types::BlockHeight, services::p2p::peer_reputation::AppScore, }; use futures::prelude::*; use libp2p::{ gossipsub::{ - error::PublishError, - GossipsubEvent, - MessageAcceptance, - MessageId, - TopicHash, + error::PublishError, GossipsubEvent, MessageAcceptance, MessageId, TopicHash, }, multiaddr::Protocol, request_response::{ - RequestId, - RequestResponseEvent, - RequestResponseMessage, - ResponseChannel, + RequestId, RequestResponseEvent, RequestResponseMessage, ResponseChannel, }, - swarm::{ - AddressScore, - ConnectionLimits, - SwarmBuilder, - SwarmEvent, - }, - Multiaddr, - PeerId, - Swarm, + swarm::{AddressScore, ConnectionLimits, SwarmBuilder, SwarmEvent}, + Multiaddr, PeerId, Swarm, }; use rand::seq::IteratorRandom; use std::collections::HashMap; -use tracing::{ - debug, - warn, -}; +use tracing::{debug, warn}; impl Punisher for Swarm> { fn ban_peer(&mut self, peer_id: PeerId) { @@ -286,7 +251,7 @@ impl FuelP2PService { let peers_count = self.peer_manager.total_peers_connected(); if peers_count == 0 { - return Err(RequestError::NoPeersConnected) + return Err(RequestError::NoPeersConnected); } let mut range = rand::thread_rng(); @@ -323,16 +288,16 @@ impl FuelP2PService { .is_err() { debug!("Failed to send ResponseMessage for {:?}", request_id); - return Err(ResponseError::SendingResponseFailed) + return Err(ResponseError::SendingResponseFailed); } } (Ok(_), None) => { debug!("ResponseChannel for {:?} does not exist!", request_id); - return Err(ResponseError::ResponseChannelDoesNotExist) + return Err(ResponseError::ResponseChannelDoesNotExist); } (Err(e), _) => { debug!("Failed to convert to IntermediateResponse with {:?}", e); - return Err(ResponseError::ConversionToIntermediateFailed) + return Err(ResponseError::ConversionToIntermediateFailed); } } @@ -508,7 +473,7 @@ impl FuelP2PService { return Some(FuelP2PEvent::PeerInfoUpdated { peer_id, block_height, - }) + }); } PeerReportEvent::PeerConnected { peer_id, @@ -522,14 +487,14 @@ impl FuelP2PService { ) { let _ = self.swarm.disconnect_peer_id(peer_id); } else if initial_connection { - return Some(FuelP2PEvent::PeerConnected(peer_id)) + return Some(FuelP2PEvent::PeerConnected(peer_id)); } } PeerReportEvent::PeerDisconnected { peer_id } => { if self.peer_manager.handle_peer_disconnect(peer_id) { let _ = self.swarm.dial(peer_id); } - return Some(FuelP2PEvent::PeerDisconnected(peer_id)) + return Some(FuelP2PEvent::PeerDisconnected(peer_id)); } } } @@ -545,7 +510,7 @@ impl FuelP2PService { return Some(FuelP2PEvent::RequestMessage { request_id, request_message: request, - }) + }); } RequestResponseMessage::Response { request_id, @@ -632,70 +597,45 @@ mod tests { codecs::postcard::PostcardCodec, config::Config, gossipsub::{ - messages::{ - GossipsubBroadcastRequest, - GossipsubMessage, - }, + messages::{GossipsubBroadcastRequest, GossipsubMessage}, topics::{ - GossipTopic, - CON_VOTE_GOSSIP_TOPIC, - NEW_BLOCK_GOSSIP_TOPIC, + GossipTopic, CON_VOTE_GOSSIP_TOPIC, NEW_BLOCK_GOSSIP_TOPIC, NEW_TX_GOSSIP_TOPIC, }, }, p2p_service::FuelP2PEvent, peer_manager::PeerInfo, request_response::messages::{ - OutboundResponse, - RequestMessage, - ResponseChannelItem, + OutboundResponse, RequestMessage, ResponseChannelItem, }, service::to_message_acceptance, }; use fuel_core_types::{ blockchain::{ block::Block, - consensus::{ - poa::PoAConsensus, - Consensus, - ConsensusVote, - }, + consensus::{poa::PoAConsensus, Consensus, ConsensusVote}, header::PartialBlockHeader, primitives::BlockId, - SealedBlock, - SealedBlockHeader, + SealedBlock, SealedBlockHeader, }, fuel_tx::Transaction, services::p2p::GossipsubMessageAcceptance, }; use futures::StreamExt; use libp2p::{ - gossipsub::{ - error::PublishError, - Topic, - }, + gossipsub::{error::PublishError, Topic}, identity::Keypair, multiaddr::Protocol, swarm::SwarmEvent, - Multiaddr, - PeerId, + Multiaddr, PeerId, }; use std::{ collections::HashSet, - net::{ - IpAddr, - Ipv4Addr, - SocketAddrV4, - TcpListener, - }, + net::{IpAddr, Ipv4Addr, SocketAddrV4, TcpListener}, sync::Arc, time::Duration, }; - use tokio::sync::{ - mpsc, - oneshot, - watch, - }; + use tokio::sync::{mpsc, oneshot, watch}; use tracing_attributes::instrument; /// helper function for building FuelP2PService @@ -792,7 +732,7 @@ mod tests { match fuel_p2p_service.swarm.select_next_some().await { SwarmEvent::NewListenAddr { .. } => { // listener address registered, we are good to go - break + break; } SwarmEvent::Behaviour(_) => {} other_event => { @@ -1126,10 +1066,7 @@ mod tests { #[tokio::test] #[instrument] async fn nodes_cannot_connect_due_to_different_checksum() { - use libp2p::{ - swarm::DialError, - TransportError, - }; + use libp2p::{swarm::DialError, TransportError}; // Node A let mut p2p_config = Config::default_initialized("nodes_cannot_connect_due_to_different_checksum"); diff --git a/crates/services/p2p/src/service.rs b/crates/services/p2p/src/service.rs index 344ea46fdd6..7f1d240c182 100644 --- a/crates/services/p2p/src/service.rs +++ b/crates/services/p2p/src/service.rs @@ -1,72 +1,32 @@ use crate::{ - codecs::{ - postcard::PostcardCodec, - NetworkCodec, - }, + codecs::{postcard::PostcardCodec, NetworkCodec}, config::Config, - gossipsub::messages::{ - GossipsubBroadcastRequest, - GossipsubMessage, - }, - p2p_service::{ - FuelP2PEvent, - FuelP2PService, - }, - ports::{ - BlockHeightImporter, - P2pDb, - }, - request_response::messages::{ - OutboundResponse, - RequestMessage, - ResponseChannelItem, - }, + gossipsub::messages::{GossipsubBroadcastRequest, GossipsubMessage}, + p2p_service::{FuelP2PEvent, FuelP2PService}, + ports::{BlockHeightImporter, P2pDb}, + request_response::messages::{OutboundResponse, RequestMessage, ResponseChannelItem}, }; use anyhow::anyhow; use fuel_core_services::{ - stream::BoxStream, - RunnableService, - RunnableTask, - ServiceRunner, - StateWatcher, + stream::BoxStream, RunnableService, RunnableTask, ServiceRunner, StateWatcher, }; use fuel_core_types::{ blockchain::{ - block::Block, - consensus::ConsensusVote, - primitives::BlockId, - SealedBlock, + block::Block, consensus::ConsensusVote, primitives::BlockId, SealedBlock, SealedBlockHeader, }, fuel_tx::Transaction, fuel_types::BlockHeight, services::p2p::{ - peer_reputation::{ - AppScore, - PeerReport, - }, - BlockHeightHeartbeatData, - GossipData, - GossipsubMessageAcceptance, - GossipsubMessageInfo, - PeerId as FuelPeerId, - TransactionGossipData, + peer_reputation::{AppScore, PeerReport}, + BlockHeightHeartbeatData, GossipData, GossipsubMessageAcceptance, + GossipsubMessageInfo, PeerId as FuelPeerId, TransactionGossipData, }, }; use futures::StreamExt; -use libp2p::{ - gossipsub::MessageAcceptance, - PeerId, -}; -use std::{ - fmt::Debug, - sync::Arc, -}; -use tokio::sync::{ - broadcast, - mpsc, - oneshot, -}; +use libp2p::{gossipsub::MessageAcceptance, PeerId}; +use std::{fmt::Debug, sync::Arc}; +use tokio::sync::{broadcast, mpsc, oneshot}; use tracing::warn; pub type Service = ServiceRunner>; @@ -451,6 +411,10 @@ impl SharedState { self.reserved_peers_broadcast.subscribe() } + pub fn subscribe_to_connections(&self) -> broadcast::Receiver { + todo!("Implement subscribe_to_connections") + } + pub fn report_peer( &self, peer_id: FuelPeerId, @@ -532,10 +496,7 @@ pub mod tests { use fuel_core_types::{ blockchain::{ block::Block, - consensus::{ - poa::PoAConsensus, - Consensus, - }, + consensus::{poa::PoAConsensus, Consensus}, }, fuel_types::BlockHeight, }; diff --git a/crates/services/src/service.rs b/crates/services/src/service.rs index 4c2e6d49261..865db6e2dc7 100644 --- a/crates/services/src/service.rs +++ b/crates/services/src/service.rs @@ -1,7 +1,4 @@ -use crate::state::{ - State, - StateWatcher, -}; +use crate::state::{State, StateWatcher}; use anyhow::anyhow; use futures::FutureExt; use tokio::sync::watch; @@ -159,7 +156,7 @@ where loop { let state = start.borrow().clone(); if !state.starting() { - return Ok(state) + return Ok(state); } start.changed().await?; } @@ -169,7 +166,7 @@ where loop { let state = stop.borrow().clone(); if state.stopped() { - return Ok(state) + return Ok(state); } stop.changed().await?; } @@ -303,7 +300,7 @@ where // If the state after update is not `Starting` then return to stop the service. if !state.borrow().starting() { - return + return; } // We can panic here, because it is inside of the task. @@ -331,7 +328,7 @@ where Ok(Ok(should_continue)) => { if !should_continue { tracing::debug!("stopping"); - break + break; } tracing::debug!("run loop"); } @@ -342,7 +339,7 @@ where Err(panic) => { tracing::debug!("got a panic"); got_panic = Some(panic); - break + break; } } } diff --git a/crates/services/txpool/Cargo.toml b/crates/services/txpool/Cargo.toml index f05b7671ffd..c60da18bc66 100644 --- a/crates/services/txpool/Cargo.toml +++ b/crates/services/txpool/Cargo.toml @@ -17,6 +17,7 @@ fuel-core-chain-config = { workspace = true } fuel-core-metrics = { workspace = true } fuel-core-services = { workspace = true } fuel-core-storage = { workspace = true } +fuel-core-p2p = { workspace = true } fuel-core-types = { workspace = true } parking_lot = { workspace = true } tokio = { workspace = true, default-features = false, features = ["sync"] } diff --git a/crates/services/txpool/src/ports.rs b/crates/services/txpool/src/ports.rs index de51f429e93..48d0dbe4557 100644 --- a/crates/services/txpool/src/ports.rs +++ b/crates/services/txpool/src/ports.rs @@ -1,27 +1,13 @@ +use fuel_core_p2p::PeerId; use fuel_core_services::stream::BoxStream; use fuel_core_storage::Result as StorageResult; use fuel_core_types::{ - entities::{ - coins::coin::CompressedCoin, - message::Message, - }, - fuel_tx::{ - Transaction, - UtxoId, - }, - fuel_types::{ - BlockHeight, - Bytes32, - ContractId, - Nonce, - }, + entities::{coins::coin::CompressedCoin, message::Message}, + fuel_tx::{Transaction, UtxoId}, + fuel_types::{BlockHeight, Bytes32, ContractId, Nonce}, services::{ block_importer::ImportResult, - p2p::{ - GossipsubMessageAcceptance, - GossipsubMessageInfo, - NetworkData, - }, + p2p::{GossipsubMessageAcceptance, GossipsubMessageInfo, NetworkData}, txpool::TransactionStatus, }, }; @@ -62,3 +48,8 @@ pub trait TxPoolDb: Send + Sync { fn transaction_status(&self, tx_id: &Bytes32) -> StorageResult; } + +pub trait TxPoolSyncPort: Send + Sync { + /// Streams new connections to the node. + fn new_connection(&self) -> BoxStream; +} diff --git a/crates/services/txpool/src/service.rs b/crates/services/txpool/src/service.rs index 729eddf560b..003cb509503 100644 --- a/crates/services/txpool/src/service.rs +++ b/crates/services/txpool/src/service.rs @@ -1,67 +1,41 @@ use crate::{ - ports::{ - BlockImporter, - PeerToPeer, - TxPoolDb, - }, + ports::{BlockImporter, PeerToPeer, TxPoolDb, TxPoolSyncPort}, transaction_selector::select_transactions, - Config, - Error as TxPoolError, - TxInfo, - TxPool, + Config, Error as TxPoolError, TxInfo, TxPool, +}; + +use fuel_core_p2p::{ + codecs::postcard::PostcardCodec, network_service::FuelP2PEvent, + network_service::FuelP2PService, PeerId, }; use fuel_core_services::{ - stream::BoxStream, - RunnableService, - RunnableTask, - ServiceRunner, - StateWatcher, + stream::BoxStream, RunnableService, RunnableTask, ServiceRunner, StateWatcher, }; use fuel_core_types::{ - fuel_tx::{ - ConsensusParameters, - Transaction, - TxId, - UniqueIdentifier, - }, - fuel_types::{ - BlockHeight, - Bytes32, - }, + fuel_tx::{ConsensusParameters, Transaction, TxId, UniqueIdentifier}, + fuel_types::{BlockHeight, Bytes32}, services::{ block_importer::ImportResult, p2p::{ - GossipData, - GossipsubMessageAcceptance, - GossipsubMessageInfo, + GossipData, GossipsubMessageAcceptance, GossipsubMessageInfo, TransactionGossipData, }, - txpool::{ - ArcPoolTx, - Error, - InsertionResult, - TransactionStatus, - }, + txpool::{ArcPoolTx, Error, InsertionResult, TransactionStatus}, }, tai64::Tai64, }; use parking_lot::Mutex as ParkingMutex; use std::sync::Arc; -use tokio::{ - sync::broadcast, - time::MissedTickBehavior, -}; +use tokio::{sync::broadcast, time::MissedTickBehavior}; use tokio_stream::StreamExt; use update_sender::UpdateSender; -use self::update_sender::{ - MpscChannel, - TxStatusStream, -}; +use self::update_sender::{MpscChannel, TxStatusStream}; mod update_sender; pub type Service = ServiceRunner>; +pub type TxPoolSyncService = ServiceRunner>; #[derive(Clone)] pub struct TxStatusChange { @@ -127,6 +101,90 @@ impl Clone for SharedState { } } +pub struct TxPoolSyncSharedState { + p2p: Arc, +} + +impl Clone for TxPoolSyncSharedState { + fn clone(&self) -> Self { + Self { + p2p: self.p2p.clone(), + } + } +} + +pub struct TxPoolSyncTask { + shared: TxPoolSyncSharedState, + ttl_timer: tokio::time::Interval, +} + +#[async_trait::async_trait] +impl RunnableService for TxPoolSyncTask +where + P2P: TxPoolSyncPort + Send + Sync, +{ + const NAME: &'static str = "TxPoolSync"; + + type SharedData = TxPoolSyncSharedState; + type Task = TxPoolSyncTask; + type TaskParams = (); + + fn shared_data(&self) -> Self::SharedData { + self.shared.clone() + } + + async fn into_task( + mut self, + _: &StateWatcher, + _: Self::TaskParams, + ) -> anyhow::Result { + self.ttl_timer.reset(); + Ok(self) + } +} + +#[async_trait::async_trait] +impl RunnableTask for TxPoolSyncTask +where + P2P: TxPoolSyncPort + Send + Sync, +{ + async fn run(&mut self, watcher: &mut StateWatcher) -> anyhow::Result { + let should_continue; + + dbg!("Inside TxPoolSyncTask::run"); + + let mut stream = self.shared.p2p.new_connection(); + + tokio::select! { + biased; + + _ = watcher.while_started() => { + should_continue = false; + } + + new_connection = stream.next() => { + should_continue = true; + if let Some(peer_id) = new_connection { + dbg!("Peer connected, inside TxPoolSyncTask: {:?}", peer_id); + } else { + dbg!(&new_connection); + } + } + } + + Ok(should_continue) + } + + async fn shutdown(self) -> anyhow::Result<()> { + // Nothing to shut down because we don't have any temporary state that should be dumped, + // and we don't spawn any sub-tasks that we need to finish or await. + // Maybe we will save and load the previous list of transactions in the future to + // avoid losing them. + + Ok(()) + } +} + pub struct Task { gossiped_tx_stream: BoxStream, committed_block_stream: BoxStream>, @@ -356,6 +414,26 @@ pub enum TxStatusMessage { FailedStatus, } +// wip +pub fn new_txpool_syncing_service( + config: Config, + p2p_config: fuel_core_p2p::config::Config, + p2p: PoolSync, +) -> TxPoolSyncService +where + PoolSync: TxPoolSyncPort + 'static, +{ + let p2p = Arc::new(p2p); + let mut ttl_timer = tokio::time::interval(config.transaction_ttl); + ttl_timer.set_missed_tick_behavior(MissedTickBehavior::Skip); + let tx_sync_task = TxPoolSyncTask { + shared: TxPoolSyncSharedState { p2p }, + ttl_timer, + }; + + TxPoolSyncService::new(tx_sync_task) +} + pub fn new_service( config: Config, db: DB, diff --git a/tests/tests/sync.rs b/tests/tests/sync.rs index 0c9ac5eb2c6..ada3671730c 100644 --- a/tests/tests/sync.rs +++ b/tests/tests/sync.rs @@ -1,23 +1,10 @@ use fuel_core::p2p_test_helpers::*; -use fuel_core_types::{ - fuel_crypto::SecretKey, - fuel_tx::Input, -}; +use fuel_core_types::{fuel_crypto::SecretKey, fuel_tx::Input}; use itertools::Itertools; -use rand::{ - rngs::StdRng, - SeedableRng, -}; +use rand::{rngs::StdRng, SeedableRng}; use std::{ - collections::{ - hash_map::DefaultHasher, - HashMap, - VecDeque, - }, - hash::{ - Hash, - Hasher, - }, + collections::{hash_map::DefaultHasher, HashMap, VecDeque}, + hash::{Hash, Hasher}, }; use test_case::test_case; From be9f6cecc73c13ceba26f9de4941e77a9a336fb9 Mon Sep 17 00:00:00 2001 From: Rodrigo Araujo Date: Fri, 23 Jun 2023 11:23:27 -0700 Subject: [PATCH 02/18] Checkpoint: pooled transactions request handling --- .../fuel-core/src/service/adapters/txpool.rs | 44 +++--- crates/fuel-core/src/service/sub_services.rs | 10 -- crates/services/p2p/src/codecs/postcard.rs | 53 ++++---- crates/services/p2p/src/lib.rs | 6 +- crates/services/p2p/src/p2p_service.rs | 8 +- .../p2p/src/request_response/messages.rs | 23 ++-- crates/services/p2p/src/service.rs | 45 ++++++- crates/services/src/lib.rs | 18 +-- crates/services/txpool/src/ports.rs | 15 ++- crates/services/txpool/src/service.rs | 120 ++++++++++------- .../txpool/src/service/test_helpers.rs | 34 +++-- .../services/txpool/src/service/tests_p2p.rs | 15 +-- crates/services/txpool/src/txpool.rs | 49 ++----- crates/types/src/services/block_importer.rs | 10 +- tests/tests/lib.rs | 2 + tests/tests/tx_gossip.rs | 22 +-- tests/tests/tx_request_response.rs | 126 ++++++++++++++++++ 17 files changed, 366 insertions(+), 234 deletions(-) create mode 100644 tests/tests/tx_request_response.rs diff --git a/crates/fuel-core/src/service/adapters/txpool.rs b/crates/fuel-core/src/service/adapters/txpool.rs index 4ab690db592..6d630211165 100644 --- a/crates/fuel-core/src/service/adapters/txpool.rs +++ b/crates/fuel-core/src/service/adapters/txpool.rs @@ -12,7 +12,7 @@ use fuel_core_storage::{ tables::{Coins, ContractsRawCode, Messages, SpentMessages}, Result as StorageResult, StorageAsRef, }; -use fuel_core_txpool::ports::{BlockImporter, TxPoolSyncPort}; +use fuel_core_txpool::ports::BlockImporter; use fuel_core_types::{ entities::{coins::coin::CompressedCoin, message::Message}, fuel_tx::{Transaction, UtxoId}, @@ -35,21 +35,7 @@ impl BlockImporter for BlockImporterAdapter { } #[cfg(feature = "p2p")] -impl TxPoolSyncPort for P2PAdapter { - fn new_connection(&self) -> BoxStream { - use tokio_stream::{wrappers::BroadcastStream, StreamExt}; - if let Some(service) = &self.service { - Box::pin( - BroadcastStream::new(service.subscribe_to_connections()) - .filter_map(|result| result.ok()), - ) - } else { - fuel_core_services::stream::IntoBoxStream::into_boxed(tokio_stream::pending()) - } - } -} - -#[cfg(feature = "p2p")] +#[async_trait::async_trait] impl fuel_core_txpool::ports::PeerToPeer for P2PAdapter { type GossipedTransaction = TransactionGossipData; @@ -61,6 +47,20 @@ impl fuel_core_txpool::ports::PeerToPeer for P2PAdapter { } } + async fn request_pooled_transactions( + &self, + peer_id: PeerId, + ) -> anyhow::Result>> { + // temp string + if let Some(service) = &self.service { + service + .get_pooled_transactions_from_peer(peer_id.to_bytes()) + .await + } else { + Ok(None) + } + } + fn gossiped_transaction_events(&self) -> BoxStream { use tokio_stream::{wrappers::BroadcastStream, StreamExt}; if let Some(service) = &self.service { @@ -84,6 +84,18 @@ impl fuel_core_txpool::ports::PeerToPeer for P2PAdapter { Ok(()) } } + + fn new_connection(&self) -> BoxStream { + use tokio_stream::{wrappers::BroadcastStream, StreamExt}; + if let Some(service) = &self.service { + Box::pin( + BroadcastStream::new(service.subscribe_to_connections()) + .filter_map(|result| result.ok()), + ) + } else { + fuel_core_services::stream::IntoBoxStream::into_boxed(tokio_stream::pending()) + } + } } #[cfg(not(feature = "p2p"))] diff --git a/crates/fuel-core/src/service/sub_services.rs b/crates/fuel-core/src/service/sub_services.rs index 6537a17ca0d..e475ea26210 100644 --- a/crates/fuel-core/src/service/sub_services.rs +++ b/crates/fuel-core/src/service/sub_services.rs @@ -106,15 +106,6 @@ pub fn init_sub_services( ); let tx_pool_adapter = TxPoolAdapter::new(txpool.shared.clone()); - let genesis = database.get_genesis()?; - let p2p_config = config.p2p.clone().expect("p2p config is required"); - let p2p_config = p2p_config.init(genesis)?; - let tx_pool_sync = new_txpool_syncing_service( - config.txpool.clone(), - p2p_config, - p2p_adapter.clone(), - ); - let block_producer = fuel_core_producer::Producer { config: config.block_producer.clone(), db: database.clone(), @@ -221,7 +212,6 @@ pub fn init_sub_services( if let Some(network) = network.take() { services.push(Box::new(network)); services.push(Box::new(sync)); - services.push(Box::new(tx_pool_sync)); } } diff --git a/crates/services/p2p/src/codecs/postcard.rs b/crates/services/p2p/src/codecs/postcard.rs index feb016c7c2f..67c85736fcf 100644 --- a/crates/services/p2p/src/codecs/postcard.rs +++ b/crates/services/p2p/src/codecs/postcard.rs @@ -1,42 +1,21 @@ -use super::{ - GossipsubCodec, - NetworkCodec, - RequestResponseConverter, -}; +use super::{GossipsubCodec, NetworkCodec, RequestResponseConverter}; use crate::{ - gossipsub::messages::{ - GossipTopicTag, - GossipsubBroadcastRequest, - GossipsubMessage, - }, + gossipsub::messages::{GossipTopicTag, GossipsubBroadcastRequest, GossipsubMessage}, request_response::messages::{ - NetworkResponse, - OutboundResponse, - RequestMessage, - ResponseMessage, - MAX_REQUEST_SIZE, - REQUEST_RESPONSE_PROTOCOL_ID, + NetworkResponse, OutboundResponse, RequestMessage, ResponseMessage, + MAX_REQUEST_SIZE, REQUEST_RESPONSE_PROTOCOL_ID, }, }; use async_trait::async_trait; -use futures::{ - AsyncRead, - AsyncWriteExt, -}; +use futures::{AsyncRead, AsyncWriteExt}; use libp2p::{ core::{ - upgrade::{ - read_length_prefixed, - write_length_prefixed, - }, + upgrade::{read_length_prefixed, write_length_prefixed}, ProtocolName, }, request_response::RequestResponseCodec, }; -use serde::{ - Deserialize, - Serialize, -}; +use serde::{Deserialize, Serialize}; use std::io; #[derive(Debug, Clone)] @@ -222,6 +201,15 @@ impl RequestResponseConverter for PostcardCodec { Ok(ResponseMessage::Transactions(response)) } + NetworkResponse::PooledTransactions(tx_bytes) => { + let response = if let Some(tx_bytes) = tx_bytes { + Some(self.deserialize(tx_bytes)?) + } else { + None + }; + + Ok(ResponseMessage::PooledTransactions(response)) + } } } @@ -257,6 +245,15 @@ impl RequestResponseConverter for PostcardCodec { Ok(NetworkResponse::Transactions(response)) } + OutboundResponse::PooledTransactions(transactions) => { + let response = if let Some(transactions) = transactions { + Some(self.serialize(transactions.as_ref())?) + } else { + None + }; + + Ok(NetworkResponse::PooledTransactions(response)) + } } } } diff --git a/crates/services/p2p/src/lib.rs b/crates/services/p2p/src/lib.rs index 83ad440dfdf..5dd785ebad9 100644 --- a/crates/services/p2p/src/lib.rs +++ b/crates/services/p2p/src/lib.rs @@ -14,11 +14,7 @@ pub mod service; pub use gossipsub::config as gossipsub_config; pub use heartbeat::HeartbeatConfig; -pub use libp2p::{ - multiaddr::Protocol, - Multiaddr, - PeerId, -}; +pub use libp2p::{multiaddr::Protocol, Multiaddr, PeerId}; #[cfg(feature = "test-helpers")] pub mod network_service { diff --git a/crates/services/p2p/src/p2p_service.rs b/crates/services/p2p/src/p2p_service.rs index 4a7873ac2dc..34674528982 100644 --- a/crates/services/p2p/src/p2p_service.rs +++ b/crates/services/p2p/src/p2p_service.rs @@ -1390,7 +1390,7 @@ mod tests { tokio::select! { message_sent = rx_test_end.recv() => { // we received a signal to end the test - assert_eq!(message_sent, Some(true), "Receuved incorrect or missing missing messsage"); + assert_eq!(message_sent, Some(true), "Received incorrect or missing missing message"); break; } node_a_event = node_a.next_event() => { @@ -1401,6 +1401,9 @@ mod tests { request_sent = true; match request_msg { + RequestMessage::PooledTransactions => { + todo!("PooledTransactions not implemented yet") + } RequestMessage::Block(_) => { let (tx_orchestrator, rx_orchestrator) = oneshot::channel(); assert!(node_a.send_request_msg(None, request_msg, ResponseChannelItem::Block(tx_orchestrator)).is_ok()); @@ -1461,6 +1464,9 @@ mod tests { // 2. Node B receives the RequestMessage from Node A initiated by the NetworkOrchestrator if let Some(FuelP2PEvent::RequestMessage{ request_id, request_message: received_request_message }) = node_b_event { match received_request_message { + RequestMessage::PooledTransactions => { + todo!("PooledTransactions not implemented yet") + } RequestMessage::Block(_) => { let block = Block::new(PartialBlockHeader::default(), (0..5).map(|_| Transaction::default_test_tx()).collect(), &[]); diff --git a/crates/services/p2p/src/request_response/messages.rs b/crates/services/p2p/src/request_response/messages.rs index a996a2e76b3..62293d90bd9 100644 --- a/crates/services/p2p/src/request_response/messages.rs +++ b/crates/services/p2p/src/request_response/messages.rs @@ -1,23 +1,13 @@ use std::sync::Arc; use fuel_core_types::{ - blockchain::{ - primitives::BlockId, - SealedBlock, - SealedBlockHeader, - }, + blockchain::{primitives::BlockId, SealedBlock, SealedBlockHeader}, fuel_tx::Transaction, fuel_types::BlockHeight, }; use libp2p::PeerId; -use serde::{ - Deserialize, - Serialize, -}; -use serde_with::{ - serde_as, - FromInto, -}; +use serde::{Deserialize, Serialize}; +use serde_with::{serde_as, FromInto}; use tokio::sync::oneshot; pub(crate) const REQUEST_RESPONSE_PROTOCOL_ID: &[u8] = b"/fuel/req_res/0.0.1"; @@ -32,7 +22,7 @@ pub type ChannelItem = oneshot::Sender>; // This `OutboundResponse` gets prepared to be sent over the wire in `NetworkResponse` format. // The Peer that requested the message receives the response over the wire in `NetworkResponse` format. // It then unpacks it into `ResponseMessage`. -// `ResponseChannelItem` is used to forward the data within `ResponseMessage` to the receving channel. +// `ResponseChannelItem` is used to forward the data within `ResponseMessage` to the receiving channel. // Client Peer: `RequestMessage` (send request) // Server Peer: `RequestMessage` (receive request) -> `OutboundResponse` -> `NetworkResponse` (send response) // Client Peer: `NetworkResponse` (receive response) -> `ResponseMessage(data)` -> `ResponseChannelItem(channel, data)` (handle response) @@ -43,6 +33,7 @@ pub enum RequestMessage { Block(BlockHeight), SealedHeader(BlockHeight), Transactions(#[serde_as(as = "FromInto<[u8; 32]>")] BlockId), + PooledTransactions, } /// Final Response Message that p2p service sends to the Orchestrator @@ -51,6 +42,7 @@ pub enum ResponseMessage { SealedBlock(Option), SealedHeader(Option), Transactions(Option>), + PooledTransactions(Option>), // temp as string } /// Holds oneshot channels for specific responses @@ -59,6 +51,7 @@ pub enum ResponseChannelItem { Block(ChannelItem), SealedHeader(ChannelItem<(PeerId, SealedBlockHeader)>), Transactions(ChannelItem>), + PooledTransactions(ChannelItem>), // temp as string } /// Response that is sent over the wire @@ -68,6 +61,7 @@ pub enum NetworkResponse { Block(Option>), Header(Option>), Transactions(Option>), + PooledTransactions(Option>), } /// Initial state of the `ResponseMessage` prior to having its inner value serialized @@ -77,6 +71,7 @@ pub enum OutboundResponse { Block(Option>), SealedHeader(Option>), Transactions(Option>>), + PooledTransactions(Option>>), // temp as string } #[derive(Debug)] diff --git a/crates/services/p2p/src/service.rs b/crates/services/p2p/src/service.rs index 7f1d240c182..842a10b8a6e 100644 --- a/crates/services/p2p/src/service.rs +++ b/crates/services/p2p/src/service.rs @@ -46,6 +46,11 @@ enum TaskRequest { height: BlockHeight, channel: oneshot::Sender>, }, + // Request get current pooled transactions + GetPooledTransactions { + from_peer: PeerId, + channel: oneshot::Sender>>, // temp string + }, GetTransactions { block_id: BlockId, from_peer: PeerId, @@ -86,6 +91,7 @@ impl Task { let (request_sender, request_receiver) = mpsc::channel(100); let (tx_broadcast, _) = broadcast::channel(100); let (block_height_broadcast, _) = broadcast::channel(100); + let (connection_broadcast, _) = broadcast::channel(100); let next_block_height = block_importer.next_block_height(); let max_block_size = config.max_block_size; @@ -104,6 +110,7 @@ impl Task { tx_broadcast, reserved_peers_broadcast, block_height_broadcast, + connection_broadcast, }, } } @@ -172,6 +179,12 @@ where tracing::error!("Got an error during vote broadcasting {}", e); } } + Some(TaskRequest::GetPooledTransactions{ channel, from_peer }) => { + dbg!("### Task::RequestGetPooledTransactions ###"); + let request_msg = RequestMessage::PooledTransactions; + let channel_item = ResponseChannelItem::PooledTransactions(channel); + let _ = self.p2p_service.send_request_msg(Some(from_peer), request_msg, channel_item); + } Some(TaskRequest::GetPeerIds(channel)) => { let peer_ids = self.p2p_service.get_peers_ids().copied().collect(); let _ = channel.send(peer_ids); @@ -207,6 +220,13 @@ where p2p_event = self.p2p_service.next_event() => { should_continue = true; match p2p_event { + Some(FuelP2PEvent::PeerConnected (peer_id) ) => { + // Send the peer ID that connected to this node to a + // channel that the TxPoolSync service is listening to. + // This is the first step of the protocol for the initial + // pool sync between two nodes. + let _ = self.shared.connection_broadcast.send(peer_id); + } Some(FuelP2PEvent::PeerInfoUpdated { peer_id, block_height }) => { let peer_id: Vec = peer_id.into(); let block_height_data = BlockHeightHeartbeatData { @@ -248,6 +268,9 @@ where let _ = self.p2p_service.send_response_msg(request_id, OutboundResponse::Transactions(transactions_response)); } + RequestMessage::PooledTransactions => { + todo!("Implement `RequestMessage::PooledTransactions`"); + } RequestMessage::SealedHeader(block_height) => { let response = self.db.get_sealed_header(&block_height)? .map(Arc::new); @@ -294,6 +317,8 @@ pub struct SharedState { request_sender: mpsc::Sender, /// Sender of p2p blopck height data block_height_broadcast: broadcast::Sender, + /// Sender of new incoming connections + connection_broadcast: broadcast::Sender, } impl SharedState { @@ -364,6 +389,24 @@ impl SharedState { receiver.await.map_err(|e| anyhow!("{}", e)) } + pub async fn get_pooled_transactions_from_peer( + &self, + peer_id: Vec, + ) -> anyhow::Result>> { + // temp string + let (sender, receiver) = oneshot::channel(); + let from_peer = PeerId::from_bytes(&peer_id).expect("Valid PeerId"); + + self.request_sender + .send(TaskRequest::GetPooledTransactions { + from_peer, + channel: sender, + }) + .await?; + + receiver.await.map_err(|e| anyhow!("{}", e)) + } + pub fn broadcast_vote(&self, vote: Arc) -> anyhow::Result<()> { self.request_sender .try_send(TaskRequest::BroadcastVote(vote))?; @@ -412,7 +455,7 @@ impl SharedState { } pub fn subscribe_to_connections(&self) -> broadcast::Receiver { - todo!("Implement subscribe_to_connections") + self.connection_broadcast.subscribe() } pub fn report_peer( diff --git a/crates/services/src/lib.rs b/crates/services/src/lib.rs index d1077e65a64..1671919437f 100644 --- a/crates/services/src/lib.rs +++ b/crates/services/src/lib.rs @@ -8,11 +8,7 @@ mod state; /// Re-exports for streaming utilities pub mod stream { #[doc(no_inline)] - pub use futures::stream::{ - pending, - unfold, - Stream, - }; + pub use futures::stream::{empty, pending, unfold, Stream}; /// A Send + Sync BoxStream pub type BoxStream = @@ -37,15 +33,7 @@ pub mod stream { } pub use service::{ - EmptyShared, - RunnableService, - RunnableTask, - Service, - ServiceRunner, - Shared, + EmptyShared, RunnableService, RunnableTask, Service, ServiceRunner, Shared, SharedMutex, }; -pub use state::{ - State, - StateWatcher, -}; +pub use state::{State, StateWatcher}; diff --git a/crates/services/txpool/src/ports.rs b/crates/services/txpool/src/ports.rs index 48d0dbe4557..4b8eb759ad7 100644 --- a/crates/services/txpool/src/ports.rs +++ b/crates/services/txpool/src/ports.rs @@ -13,6 +13,7 @@ use fuel_core_types::{ }; use std::sync::Arc; +#[async_trait::async_trait] pub trait PeerToPeer: Send + Sync { type GossipedTransaction: NetworkData; @@ -28,6 +29,15 @@ pub trait PeerToPeer: Send + Sync { message_info: GossipsubMessageInfo, validity: GossipsubMessageAcceptance, ) -> anyhow::Result<()>; + + /// Streams new connections to the node. + fn new_connection(&self) -> BoxStream; + + /// Request pooled transactions from a peer. + async fn request_pooled_transactions( + &self, + peer_id: PeerId, + ) -> anyhow::Result>>; } pub trait BlockImporter: Send + Sync { @@ -48,8 +58,3 @@ pub trait TxPoolDb: Send + Sync { fn transaction_status(&self, tx_id: &Bytes32) -> StorageResult; } - -pub trait TxPoolSyncPort: Send + Sync { - /// Streams new connections to the node. - fn new_connection(&self) -> BoxStream; -} diff --git a/crates/services/txpool/src/service.rs b/crates/services/txpool/src/service.rs index 003cb509503..c18f13d9764 100644 --- a/crates/services/txpool/src/service.rs +++ b/crates/services/txpool/src/service.rs @@ -1,15 +1,15 @@ use crate::{ - ports::{BlockImporter, PeerToPeer, TxPoolDb, TxPoolSyncPort}, + ports::{BlockImporter, PeerToPeer, TxPoolDb}, transaction_selector::select_transactions, - Config, Error as TxPoolError, TxInfo, TxPool, + txpool, Config, Error as TxPoolError, TxInfo, TxPool, }; use fuel_core_p2p::{ - codecs::postcard::PostcardCodec, network_service::FuelP2PEvent, - network_service::FuelP2PService, PeerId, + codecs::postcard::PostcardCodec, network_service::FuelP2PService, PeerId, }; use fuel_core_services::{ - stream::BoxStream, RunnableService, RunnableTask, ServiceRunner, StateWatcher, + stream::BoxStream, RunnableService, RunnableTask, Service as _, ServiceRunner, + StateWatcher, }; use fuel_core_types::{ fuel_tx::{ConsensusParameters, Transaction, TxId, UniqueIdentifier}, @@ -25,17 +25,24 @@ use fuel_core_types::{ tai64::Tai64, }; use parking_lot::Mutex as ParkingMutex; -use std::sync::Arc; +use std::{pin::Pin, sync::Arc}; use tokio::{sync::broadcast, time::MissedTickBehavior}; -use tokio_stream::StreamExt; +use tokio_stream::{Stream, StreamExt}; use update_sender::UpdateSender; use self::update_sender::{MpscChannel, TxStatusStream}; mod update_sender; +// Main transaction pool service. It is mainly responsible for +// listening for new transactions being gossiped through the network and +// adding those to its own pool. pub type Service = ServiceRunner>; -pub type TxPoolSyncService = ServiceRunner>; +// Sidecar service that is responsible for a request-response style of syncing +// pending transactions with other peers upon connection. In other words, +// when a new connection between peers happen, they will compare their +// pending transactions and sync them with each other. +pub type TxPoolSyncService = ServiceRunner>; #[derive(Clone)] pub struct TxStatusChange { @@ -101,37 +108,26 @@ impl Clone for SharedState { } } -pub struct TxPoolSyncSharedState { - p2p: Arc, -} - -impl Clone for TxPoolSyncSharedState { - fn clone(&self) -> Self { - Self { - p2p: self.p2p.clone(), - } - } -} - -pub struct TxPoolSyncTask { - shared: TxPoolSyncSharedState, +pub struct TxPoolSyncTask { ttl_timer: tokio::time::Interval, + peer_connections: BoxStream, + txpool: Arc>>, + p2p: Arc, } #[async_trait::async_trait] -impl RunnableService for TxPoolSyncTask +impl RunnableService for TxPoolSyncTask where - P2P: TxPoolSyncPort + Send + Sync, + DB: TxPoolDb, + P2P: PeerToPeer + Send + Sync, { const NAME: &'static str = "TxPoolSync"; - type SharedData = TxPoolSyncSharedState; - type Task = TxPoolSyncTask; + type SharedData = (); + type Task = TxPoolSyncTask; type TaskParams = (); - fn shared_data(&self) -> Self::SharedData { - self.shared.clone() - } + fn shared_data(&self) -> Self::SharedData {} async fn into_task( mut self, @@ -144,17 +140,14 @@ where } #[async_trait::async_trait] -impl RunnableTask for TxPoolSyncTask +impl RunnableTask for TxPoolSyncTask where - P2P: TxPoolSyncPort + Send + Sync, + DB: TxPoolDb, + P2P: PeerToPeer + Send + Sync, { async fn run(&mut self, watcher: &mut StateWatcher) -> anyhow::Result { let should_continue; - dbg!("Inside TxPoolSyncTask::run"); - - let mut stream = self.shared.p2p.new_connection(); - tokio::select! { biased; @@ -162,11 +155,26 @@ where should_continue = false; } - new_connection = stream.next() => { - should_continue = true; + new_connection = self.peer_connections.next() => { + dbg!("New connection: {:?}", new_connection); if let Some(peer_id) = new_connection { + should_continue = true; dbg!("Peer connected, inside TxPoolSyncTask: {:?}", peer_id); + + // New connection just happened, request the list of + // pooled transactions from the peer. + let peer_txs = self.p2p.request_pooled_transactions(peer_id).await; + dbg!(&peer_txs); + + // TODO: Once the above is implemented, we compare this node's list of + // transactions with the peer's list of transactions, and request the + // transactions that are missing from this node's list. + + for tx in self.txpool.lock().txs() { + dbg!(&tx.0); + } } else { + should_continue = false; dbg!(&new_connection); } } @@ -185,10 +193,15 @@ where } } -pub struct Task { +pub struct Task +where + DB: TxPoolDb + 'static, + P2P: PeerToPeer + Send + Sync + 'static, +{ gossiped_tx_stream: BoxStream, committed_block_stream: BoxStream>, shared: SharedState, + txpool_sync_task: ServiceRunner>, ttl_timer: tokio::time::Interval, } @@ -213,6 +226,9 @@ where _: &StateWatcher, _: Self::TaskParams, ) -> anyhow::Result { + // Transaction pool sync task work as a sub service to the transaction pool task. + // So we start it here and shut it down when this task is shut down. + self.txpool_sync_task.start()?; self.ttl_timer.reset(); Ok(self) } @@ -290,10 +306,12 @@ where } async fn shutdown(self) -> anyhow::Result<()> { - // Nothing to shut down because we don't have any temporary state that should be dumped, + // Nothing other than the txpool sync task to shut down + // because we don't have any temporary state that should be dumped, // and we don't spawn any sub-tasks that we need to finish or await. // Maybe we will save and load the previous list of transactions in the future to // avoid losing them. + self.txpool_sync_task.stop_and_await().await?; Ok(()) } } @@ -414,20 +432,21 @@ pub enum TxStatusMessage { FailedStatus, } -// wip -pub fn new_txpool_syncing_service( +pub fn new_txpool_syncing_service( config: Config, - p2p_config: fuel_core_p2p::config::Config, - p2p: PoolSync, -) -> TxPoolSyncService + txpool: Arc>>, + p2p: Arc, +) -> TxPoolSyncService where - PoolSync: TxPoolSyncPort + 'static, + DB: TxPoolDb + 'static, + P2P: PeerToPeer + 'static, { - let p2p = Arc::new(p2p); let mut ttl_timer = tokio::time::interval(config.transaction_ttl); ttl_timer.set_missed_tick_behavior(MissedTickBehavior::Skip); let tx_sync_task = TxPoolSyncTask { - shared: TxPoolSyncSharedState { p2p }, + peer_connections: p2p.new_connection(), + txpool, + p2p, ttl_timer, }; @@ -452,10 +471,15 @@ where ttl_timer.set_missed_tick_behavior(MissedTickBehavior::Skip); let consensus_params = config.chain_config.transaction_parameters; let number_of_active_subscription = config.number_of_active_subscription; - let txpool = Arc::new(ParkingMutex::new(TxPool::new(config, db))); + let txpool = Arc::new(ParkingMutex::new(TxPool::new(config.clone(), db))); + + let txpool_sync_task = + new_txpool_syncing_service(config, txpool.clone(), p2p.clone()); + let task = Task { gossiped_tx_stream, committed_block_stream, + txpool_sync_task, shared: SharedState { tx_status_sender: TxStatusChange::new(number_of_active_subscription), txpool, diff --git a/crates/services/txpool/src/service/test_helpers.rs b/crates/services/txpool/src/service/test_helpers.rs index 97c437119d0..26f65ff0244 100644 --- a/crates/services/txpool/src/service/test_helpers.rs +++ b/crates/services/txpool/src/service/test_helpers.rs @@ -1,26 +1,11 @@ use super::*; -use crate::{ - ports::BlockImporter, - MockDb, -}; -use fuel_core_services::{ - stream::BoxStream, - Service as ServiceTrait, -}; +use crate::{ports::BlockImporter, MockDb}; +use fuel_core_services::{stream::BoxStream, Service as ServiceTrait}; use fuel_core_types::{ blockchain::SealedBlock, entities::coins::coin::Coin, - fuel_crypto::rand::{ - rngs::StdRng, - SeedableRng, - }, - fuel_tx::{ - Cacheable, - Input, - Transaction, - TransactionBuilder, - Word, - }, + fuel_crypto::rand::{rngs::StdRng, SeedableRng}, + fuel_tx::{Cacheable, Input, Transaction, TransactionBuilder, Word}, services::p2p::GossipsubMessageAcceptance, }; use std::cell::RefCell; @@ -63,6 +48,7 @@ impl TestContext { mockall::mock! { pub P2P {} + #[async_trait::async_trait] impl PeerToPeer for P2P { type GossipedTransaction = GossipedTransaction; @@ -70,6 +56,10 @@ mockall::mock! { fn gossiped_transaction_events(&self) -> BoxStream; + fn new_connection(&self) -> BoxStream; + + async fn request_pooled_transactions(&self, peer_id: PeerId) -> anyhow::Result>>; + fn notify_gossip_transaction_validity( &self, message_info: GossipsubMessageInfo, @@ -95,6 +85,12 @@ impl MockP2P { }); p2p.expect_broadcast_transaction() .returning(move |_| Ok(())); + p2p.expect_notify_gossip_transaction_validity() + .returning(move |_, _| Ok(())); + p2p.expect_new_connection().returning(move || { + let stream = fuel_core_services::stream::empty::(); + Box::pin(stream) + }); p2p } } diff --git a/crates/services/txpool/src/service/tests_p2p.rs b/crates/services/txpool/src/service/tests_p2p.rs index c257fe7dd43..1f7ceb32dd0 100644 --- a/crates/services/txpool/src/service/tests_p2p.rs +++ b/crates/services/txpool/src/service/tests_p2p.rs @@ -1,17 +1,8 @@ use super::*; -use crate::service::test_helpers::{ - MockP2P, - TestContextBuilder, -}; +use crate::service::test_helpers::{MockP2P, TestContextBuilder}; use fuel_core_services::Service; -use fuel_core_types::fuel_tx::{ - Transaction, - UniqueIdentifier, -}; -use std::{ - ops::Deref, - time::Duration, -}; +use fuel_core_types::fuel_tx::{Transaction, UniqueIdentifier}; +use std::{ops::Deref, time::Duration}; #[tokio::test] async fn can_insert_from_p2p() { diff --git a/crates/services/txpool/src/txpool.rs b/crates/services/txpool/src/txpool.rs index 1bbf88ec746..fb749261bb9 100644 --- a/crates/services/txpool/src/txpool.rs +++ b/crates/services/txpool/src/txpool.rs @@ -1,40 +1,19 @@ use crate::{ - containers::{ - dependency::Dependency, - price_sort::PriceSort, - time_sort::TimeSort, - }, + containers::{dependency::Dependency, price_sort::PriceSort, time_sort::TimeSort}, ports::TxPoolDb, service::TxStatusChange, types::*, - Config, - Error, - TxInfo, + Config, Error, TxInfo, }; use fuel_core_metrics::txpool_metrics::TXPOOL_METRICS; use fuel_core_types::{ blockchain::SealedBlock, - fuel_tx::{ - Chargeable, - Transaction, - UniqueIdentifier, - }, - fuel_vm::checked_transaction::{ - CheckedTransaction, - IntoChecked, - }, - services::txpool::{ - ArcPoolTx, - InsertionResult, - }, + fuel_tx::{Chargeable, Transaction, UniqueIdentifier}, + fuel_vm::checked_transaction::{CheckedTransaction, IntoChecked}, + services::txpool::{ArcPoolTx, InsertionResult}, tai64::Tai64, }; -use std::{ - cmp::Reverse, - collections::HashMap, - ops::Deref, - sync::Arc, -}; +use std::{cmp::Reverse, collections::HashMap, ops::Deref, sync::Arc}; #[derive(Debug, Clone)] pub struct TxPool { @@ -81,7 +60,7 @@ where let current_height = self.database.current_block_height()?; if tx.is_mint() { - return Err(Error::NotSupportedTransactionType.into()) + return Err(Error::NotSupportedTransactionType.into()); } // verify gas price is at least the minimum @@ -113,7 +92,7 @@ where }); if !tx.is_computed() { - return Err(Error::NoMetadata.into()) + return Err(Error::NoMetadata.into()); } // verify max gas is less than block limit @@ -122,11 +101,11 @@ where tx_gas: tx.max_gas(), block_limit: self.config.chain_config.block_gas_limit, } - .into()) + .into()); } if self.by_hash.contains_key(&tx.id()) { - return Err(Error::NotInsertedTxKnown.into()) + return Err(Error::NotInsertedTxKnown.into()); } let mut max_limit_hit = false; @@ -136,7 +115,7 @@ where // limit is hit, check if we can push out lowest priced tx let lowest_price = self.by_gas_price.lowest_value().unwrap_or_default(); if lowest_price >= tx.price() { - return Err(Error::NotInsertedLimitHit.into()) + return Err(Error::NotInsertedLimitHit.into()); } } if self.config.metrics { @@ -208,7 +187,7 @@ where for remove in removed.iter() { self.remove_tx(&remove.id()); } - return removed + return removed; } Vec::new() } @@ -245,7 +224,7 @@ where TXPOOL_METRICS.gas_price_histogram.observe(price as f64); } if price < self.config.min_gas_price { - return Err(Error::NotInsertedGasPriceTooLow) + return Err(Error::NotInsertedGasPriceTooLow); } Ok(()) } @@ -381,7 +360,7 @@ where let removed = self.remove_inner(&oldest_tx); result.extend(removed.into_iter()); } else { - break + break; } } diff --git a/crates/types/src/services/block_importer.rs b/crates/types/src/services/block_importer.rs index 494abb8b572..953f32e23be 100644 --- a/crates/types/src/services/block_importer.rs +++ b/crates/types/src/services/block_importer.rs @@ -1,14 +1,8 @@ //! Types related to block importer service. use crate::{ - blockchain::{ - header::BlockHeader, - SealedBlock, - }, - services::{ - executor::TransactionExecutionStatus, - Uncommitted, - }, + blockchain::{header::BlockHeader, SealedBlock}, + services::{executor::TransactionExecutionStatus, Uncommitted}, }; /// The uncommitted result of the block importing. diff --git a/tests/tests/lib.rs b/tests/tests/lib.rs index 6469c8627ec..da97fd78f97 100644 --- a/tests/tests/lib.rs +++ b/tests/tests/lib.rs @@ -22,5 +22,7 @@ mod trigger_integration; mod tx; #[cfg(feature = "p2p")] mod tx_gossip; +#[cfg(feature = "p2p")] +mod tx_request_response; fuel_core_trace::enable_tracing!(); diff --git a/tests/tests/tx_gossip.rs b/tests/tests/tx_gossip.rs index 5441c07c750..3f50751b10c 100644 --- a/tests/tests/tx_gossip.rs +++ b/tests/tests/tx_gossip.rs @@ -1,30 +1,18 @@ use fuel_core::{ - chain_config::{ - CoinConfig, - StateConfig, - }, - service::{ - Config, - FuelService, - }, + chain_config::{CoinConfig, StateConfig}, + service::{Config, FuelService}, }; use fuel_core_client::client::FuelClient; +use fuel_core_poa::Trigger; use fuel_core_types::{ fuel_tx::{ field::*, - input::coin::{ - CoinPredicate, - CoinSigned, - }, + input::coin::{CoinPredicate, CoinSigned}, *, }, fuel_vm::*, }; -use rand::{ - rngs::StdRng, - Rng, - SeedableRng, -}; +use rand::{rngs::StdRng, Rng, SeedableRng}; use std::time::Duration; fn create_node_config_from_inputs(inputs: &[Input]) -> Config { diff --git a/tests/tests/tx_request_response.rs b/tests/tests/tx_request_response.rs new file mode 100644 index 00000000000..7a30ef02e47 --- /dev/null +++ b/tests/tests/tx_request_response.rs @@ -0,0 +1,126 @@ +use fuel_core::{ + chain_config::{CoinConfig, StateConfig}, + service::{Config, FuelService}, +}; +use fuel_core_client::client::FuelClient; +use fuel_core_poa::Trigger; +use fuel_core_types::{ + fuel_tx::{ + field::*, + input::coin::{CoinPredicate, CoinSigned}, + *, + }, + fuel_vm::*, +}; +use rand::{rngs::StdRng, Rng, SeedableRng}; +use std::time::Duration; + +// Creates a simple node config with no block production. +// This is used for these tests because we don't want the transactions to be +// included in a block. We want them to be included in the mempool and check +// that new connected nodes sync the mempool. +fn create_node_config_from_inputs_no_block_production(inputs: &[Input]) -> Config { + let mut node_config = Config::local_node(); + let mut initial_state = StateConfig::default(); + let mut coin_configs = vec![]; + + for input in inputs { + if let Input::CoinSigned(CoinSigned { + amount, + owner, + asset_id, + utxo_id, + .. + }) + | Input::CoinPredicate(CoinPredicate { + amount, + owner, + asset_id, + utxo_id, + .. + }) = input + { + let coin_config = CoinConfig { + tx_id: Some(*utxo_id.tx_id()), + output_index: Some(utxo_id.output_index()), + tx_pointer_block_height: None, + tx_pointer_tx_idx: None, + maturity: None, + owner: *owner, + amount: *amount, + asset_id: *asset_id, + }; + coin_configs.push(coin_config); + }; + } + + initial_state.coins = Some(coin_configs); + node_config.chain_conf.initial_state = Some(initial_state); + node_config.utxo_validation = true; + node_config.block_production = Trigger::Never; + node_config.p2p.as_mut().unwrap().enable_mdns = true; + node_config +} + +// This test is set up in such a way that the transaction is not committed +// as we've disabled the block production. This is to test that the peer +// will request this transaction from the other peer upon connection. +#[tokio::test(flavor = "multi_thread")] +async fn test_tx_request() { + use futures::StreamExt; + let mut rng = StdRng::seed_from_u64(2322); + + let tx = TransactionBuilder::script(vec![], vec![]) + .gas_limit(100) + .gas_price(1) + .add_unsigned_coin_input( + SecretKey::random(&mut rng), + rng.gen(), + 1000, + Default::default(), + Default::default(), + Default::default(), + ) + .add_output(Output::Change { + amount: 0, + asset_id: Default::default(), + to: rng.gen(), + }) + .finalize(); + + let node_config = create_node_config_from_inputs_no_block_production(tx.inputs()); + let params = node_config.chain_conf.transaction_parameters; + let node_one = FuelService::new_node(node_config).await.unwrap(); + let client_one = FuelClient::from(node_one.bound_address); + + let node_config = create_node_config_from_inputs_no_block_production(tx.inputs()); + let node_two = FuelService::new_node(node_config).await.unwrap(); + let client_two = FuelClient::from(node_two.bound_address); + + let wait_time = Duration::from_secs(10); + + let tx_id = tx.id(¶ms.chain_id); + let tx = tx.into(); + let res = client_one.submit(&tx).await.unwrap(); + dbg!(&res); + + tokio::time::sleep(wait_time).await; + + // At this point, the transaction should be in the mempool of node_one + // and node_two should request it from node_one. + + // Below code not necessary at this moment. + // let response = client_one.transaction(&tx_id).await.unwrap(); + // assert!(response.is_some()); + + // let mut client_two_subscription = client_two + // .subscribe_transaction_status(&tx_id) + // .await + // .expect("Should be able to subscribe for events"); + // tokio::time::timeout(wait_time, client_two_subscription.next()) + // .await + // .expect("Should await transaction notification in time"); + + // let response = client_two.transaction(&tx_id).await.unwrap(); + // assert!(response.is_some()); +} From f1e2c91c415d1e544bc2f77d09199554b25ac53a Mon Sep 17 00:00:00 2001 From: Rodrigo Araujo Date: Tue, 18 Jul 2023 12:22:57 -0700 Subject: [PATCH 03/18] Format with nightly --- crates/fuel-core/src/service.rs | 35 ++++- .../fuel-core/src/service/adapters/txpool.rs | 52 +++++-- crates/services/p2p/src/behavior.rs | 46 ++++-- crates/services/p2p/src/codecs/postcard.rs | 35 ++++- crates/services/p2p/src/lib.rs | 6 +- crates/services/p2p/src/p2p_service.rs | 142 +++++++++++++----- .../p2p/src/request_response/messages.rs | 16 +- crates/services/p2p/src/service.rs | 71 +++++++-- crates/services/src/lib.rs | 19 ++- crates/services/src/service.rs | 15 +- crates/services/txpool/src/ports.rs | 23 ++- crates/services/txpool/src/service.rs | 50 ++++-- .../txpool/src/service/test_helpers.rs | 23 ++- .../services/txpool/src/service/tests_p2p.rs | 15 +- crates/services/txpool/src/txpool.rs | 49 ++++-- crates/types/src/services/block_importer.rs | 10 +- tests/tests/sync.rs | 21 ++- 17 files changed, 492 insertions(+), 136 deletions(-) diff --git a/crates/fuel-core/src/service.rs b/crates/fuel-core/src/service.rs index 71b859bd957..fd2ad0a5041 100644 --- a/crates/fuel-core/src/service.rs +++ b/crates/fuel-core/src/service.rs @@ -1,11 +1,22 @@ -use crate::{database::Database, service::adapters::P2PAdapter}; +use crate::{ + database::Database, + service::adapters::P2PAdapter, +}; use fuel_core_services::{ - RunnableService, RunnableTask, ServiceRunner, State, StateWatcher, + RunnableService, + RunnableTask, + ServiceRunner, + State, + StateWatcher, }; use std::net::SocketAddr; use tracing::warn; -pub use config::{Config, DbType, VMConfig}; +pub use config::{ + Config, + DbType, + VMConfig, +}; pub use fuel_core_services::Service as ServiceTrait; pub use fuel_core_consensus_module::RelayerVerifierConfig; @@ -262,9 +273,19 @@ impl RunnableTask for Task { #[cfg(test)] mod tests { - use crate::service::{Config, Task}; - use fuel_core_services::{RunnableService, RunnableTask, State}; - use std::{thread::sleep, time::Duration}; + use crate::service::{ + Config, + Task, + }; + use fuel_core_services::{ + RunnableService, + RunnableTask, + State, + }; + use std::{ + thread::sleep, + time::Duration, + }; #[tokio::test] async fn run_start_and_stop() { @@ -284,7 +305,7 @@ mod tests { task.sub_services()[i].stop_and_await().await.unwrap(); assert!(!task.run(&mut watcher).await.unwrap()); } else { - break; + break } i += 1; } diff --git a/crates/fuel-core/src/service/adapters/txpool.rs b/crates/fuel-core/src/service/adapters/txpool.rs index 249d5425230..37daf29e8a4 100644 --- a/crates/fuel-core/src/service/adapters/txpool.rs +++ b/crates/fuel-core/src/service/adapters/txpool.rs @@ -1,29 +1,55 @@ use crate::{ database::Database, - service::adapters::{BlockImporterAdapter, P2PAdapter}, + service::adapters::{ + BlockImporterAdapter, + P2PAdapter, + }, }; use fuel_core_p2p::PeerId; use fuel_core_services::stream::BoxStream; use fuel_core_storage::{ not_found, - tables::{Coins, ContractsRawCode, Messages, SpentMessages}, - Result as StorageResult, StorageAsRef, + tables::{ + Coins, + ContractsRawCode, + Messages, + SpentMessages, + }, + Result as StorageResult, + StorageAsRef, }; use fuel_core_txpool::ports::BlockImporter; use fuel_core_types::{ - entities::{coins::coin::CompressedCoin, message::Message}, - fuel_tx::{Transaction, UtxoId}, - fuel_types::{BlockHeight, ContractId, Nonce}, + entities::{ + coins::coin::CompressedCoin, + message::Message, + }, + fuel_tx::{ + Transaction, + UtxoId, + }, + fuel_types::{ + BlockHeight, + ContractId, + Nonce, + }, services::{ block_importer::ImportResult, - p2p::{GossipsubMessageAcceptance, GossipsubMessageInfo, TransactionGossipData}, + p2p::{ + GossipsubMessageAcceptance, + GossipsubMessageInfo, + TransactionGossipData, + }, }, }; use std::sync::Arc; impl BlockImporter for BlockImporterAdapter { fn block_events(&self) -> BoxStream> { - use tokio_stream::{wrappers::BroadcastStream, StreamExt}; + use tokio_stream::{ + wrappers::BroadcastStream, + StreamExt, + }; Box::pin( BroadcastStream::new(self.block_importer.subscribe()) .filter_map(|result| result.ok()), @@ -59,7 +85,10 @@ impl fuel_core_txpool::ports::PeerToPeer for P2PAdapter { } fn gossiped_transaction_events(&self) -> BoxStream { - use tokio_stream::{wrappers::BroadcastStream, StreamExt}; + use tokio_stream::{ + wrappers::BroadcastStream, + StreamExt, + }; if let Some(service) = &self.service { Box::pin( BroadcastStream::new(service.subscribe_tx()) @@ -83,7 +112,10 @@ impl fuel_core_txpool::ports::PeerToPeer for P2PAdapter { } fn new_connection(&self) -> BoxStream { - use tokio_stream::{wrappers::BroadcastStream, StreamExt}; + use tokio_stream::{ + wrappers::BroadcastStream, + StreamExt, + }; if let Some(service) = &self.service { Box::pin( BroadcastStream::new(service.subscribe_to_connections()) diff --git a/crates/services/p2p/src/behavior.rs b/crates/services/p2p/src/behavior.rs index 2e48b6ac60a..7a596f39cb2 100644 --- a/crates/services/p2p/src/behavior.rs +++ b/crates/services/p2p/src/behavior.rs @@ -1,24 +1,50 @@ use crate::{ codecs::NetworkCodec, config::Config, - discovery::{DiscoveryBehaviour, DiscoveryConfig, DiscoveryEvent}, - gossipsub::{config::build_gossipsub_behaviour, topics::GossipTopic}, - peer_report::{PeerReportBehaviour, PeerReportEvent}, - request_response::messages::{NetworkResponse, RequestMessage}, + discovery::{ + DiscoveryBehaviour, + DiscoveryConfig, + DiscoveryEvent, + }, + gossipsub::{ + config::build_gossipsub_behaviour, + topics::GossipTopic, + }, + peer_report::{ + PeerReportBehaviour, + PeerReportEvent, + }, + request_response::messages::{ + NetworkResponse, + RequestMessage, + }, }; use fuel_core_types::fuel_types::BlockHeight; use libp2p::{ gossipsub::{ - error::PublishError, Gossipsub, GossipsubEvent, MessageAcceptance, MessageId, + error::PublishError, + Gossipsub, + GossipsubEvent, + MessageAcceptance, + MessageId, }, request_response::{ - ProtocolSupport, RequestId, RequestResponse, RequestResponseConfig, - RequestResponseEvent, ResponseChannel, + ProtocolSupport, + RequestId, + RequestResponse, + RequestResponseConfig, + RequestResponseEvent, + ResponseChannel, }, swarm::NetworkBehaviour, - Multiaddr, PeerId, + Multiaddr, + PeerId, +}; +use tracing::{ + debug, + error, + log::warn, }; -use tracing::{debug, error, log::warn}; #[derive(Debug)] pub enum FuelBehaviourEvent { @@ -145,7 +171,7 @@ impl FuelBehaviour { Ok(true) => { debug!(target: "fuel-p2p", "Sent a report for MessageId: {} from PeerId: {}", msg_id, propagation_source); if should_check_score { - return self.gossipsub.peer_score(propagation_source); + return self.gossipsub.peer_score(propagation_source) } } Ok(false) => { diff --git a/crates/services/p2p/src/codecs/postcard.rs b/crates/services/p2p/src/codecs/postcard.rs index 67c85736fcf..5849eb41d4e 100644 --- a/crates/services/p2p/src/codecs/postcard.rs +++ b/crates/services/p2p/src/codecs/postcard.rs @@ -1,21 +1,42 @@ -use super::{GossipsubCodec, NetworkCodec, RequestResponseConverter}; +use super::{ + GossipsubCodec, + NetworkCodec, + RequestResponseConverter, +}; use crate::{ - gossipsub::messages::{GossipTopicTag, GossipsubBroadcastRequest, GossipsubMessage}, + gossipsub::messages::{ + GossipTopicTag, + GossipsubBroadcastRequest, + GossipsubMessage, + }, request_response::messages::{ - NetworkResponse, OutboundResponse, RequestMessage, ResponseMessage, - MAX_REQUEST_SIZE, REQUEST_RESPONSE_PROTOCOL_ID, + NetworkResponse, + OutboundResponse, + RequestMessage, + ResponseMessage, + MAX_REQUEST_SIZE, + REQUEST_RESPONSE_PROTOCOL_ID, }, }; use async_trait::async_trait; -use futures::{AsyncRead, AsyncWriteExt}; +use futures::{ + AsyncRead, + AsyncWriteExt, +}; use libp2p::{ core::{ - upgrade::{read_length_prefixed, write_length_prefixed}, + upgrade::{ + read_length_prefixed, + write_length_prefixed, + }, ProtocolName, }, request_response::RequestResponseCodec, }; -use serde::{Deserialize, Serialize}; +use serde::{ + Deserialize, + Serialize, +}; use std::io; #[derive(Debug, Clone)] diff --git a/crates/services/p2p/src/lib.rs b/crates/services/p2p/src/lib.rs index 5dd785ebad9..83ad440dfdf 100644 --- a/crates/services/p2p/src/lib.rs +++ b/crates/services/p2p/src/lib.rs @@ -14,7 +14,11 @@ pub mod service; pub use gossipsub::config as gossipsub_config; pub use heartbeat::HeartbeatConfig; -pub use libp2p::{multiaddr::Protocol, Multiaddr, PeerId}; +pub use libp2p::{ + multiaddr::Protocol, + Multiaddr, + PeerId, +}; #[cfg(feature = "test-helpers")] pub mod network_service { diff --git a/crates/services/p2p/src/p2p_service.rs b/crates/services/p2p/src/p2p_service.rs index f160e25fd4e..4e8f4ded428 100644 --- a/crates/services/p2p/src/p2p_service.rs +++ b/crates/services/p2p/src/p2p_service.rs @@ -1,37 +1,75 @@ use crate::{ - behavior::{FuelBehaviour, FuelBehaviourEvent}, + behavior::{ + FuelBehaviour, + FuelBehaviourEvent, + }, codecs::NetworkCodec, - config::{build_transport, Config}, + config::{ + build_transport, + Config, + }, gossipsub::{ - messages::{GossipsubBroadcastRequest, GossipsubMessage as FuelGossipsubMessage}, + messages::{ + GossipsubBroadcastRequest, + GossipsubMessage as FuelGossipsubMessage, + }, topics::GossipsubTopics, }, - peer_manager::{PeerManager, Punisher}, + peer_manager::{ + PeerManager, + Punisher, + }, peer_report::PeerReportEvent, request_response::messages::{ - NetworkResponse, OutboundResponse, RequestError, RequestMessage, - ResponseChannelItem, ResponseError, ResponseMessage, + NetworkResponse, + OutboundResponse, + RequestError, + RequestMessage, + ResponseChannelItem, + ResponseError, + ResponseMessage, }, }; use fuel_core_metrics::p2p_metrics::P2P_METRICS; use fuel_core_types::{ - fuel_types::BlockHeight, services::p2p::peer_reputation::AppScore, + fuel_types::BlockHeight, + services::p2p::peer_reputation::AppScore, }; use futures::prelude::*; use libp2p::{ gossipsub::{ - error::PublishError, GossipsubEvent, MessageAcceptance, MessageId, TopicHash, + error::PublishError, + GossipsubEvent, + MessageAcceptance, + MessageId, + TopicHash, }, multiaddr::Protocol, request_response::{ - RequestId, RequestResponseEvent, RequestResponseMessage, ResponseChannel, + RequestId, + RequestResponseEvent, + RequestResponseMessage, + ResponseChannel, + }, + swarm::{ + AddressScore, + ConnectionLimits, + SwarmBuilder, + SwarmEvent, }, - swarm::{AddressScore, ConnectionLimits, SwarmBuilder, SwarmEvent}, - Multiaddr, PeerId, Swarm, + Multiaddr, + PeerId, + Swarm, }; use rand::seq::IteratorRandom; -use std::{collections::HashMap, time::Duration}; -use tracing::{debug, warn}; +use std::{ + collections::HashMap, + time::Duration, +}; +use tracing::{ + debug, + warn, +}; impl Punisher for Swarm> { fn ban_peer(&mut self, peer_id: PeerId) { @@ -219,7 +257,7 @@ impl FuelP2PService { loop { if let SwarmEvent::NewListenAddr { .. } = self.swarm.select_next_some().await { - break; + break } } } @@ -275,7 +313,7 @@ impl FuelP2PService { let peers_count = self.peer_manager.total_peers_connected(); if peers_count == 0 { - return Err(RequestError::NoPeersConnected); + return Err(RequestError::NoPeersConnected) } let mut range = rand::thread_rng(); @@ -312,16 +350,16 @@ impl FuelP2PService { .is_err() { debug!("Failed to send ResponseMessage for {:?}", request_id); - return Err(ResponseError::SendingResponseFailed); + return Err(ResponseError::SendingResponseFailed) } } (Ok(_), None) => { debug!("ResponseChannel for {:?} does not exist!", request_id); - return Err(ResponseError::ResponseChannelDoesNotExist); + return Err(ResponseError::ResponseChannelDoesNotExist) } (Err(e), _) => { debug!("Failed to convert to IntermediateResponse with {:?}", e); - return Err(ResponseError::ConversionToIntermediateFailed); + return Err(ResponseError::ConversionToIntermediateFailed) } } @@ -502,7 +540,7 @@ impl FuelP2PService { return Some(FuelP2PEvent::PeerInfoUpdated { peer_id, block_height, - }); + }) } PeerReportEvent::PeerConnected { peer_id, @@ -516,14 +554,14 @@ impl FuelP2PService { ) { let _ = self.swarm.disconnect_peer_id(peer_id); } else if initial_connection { - return Some(FuelP2PEvent::PeerConnected(peer_id)); + return Some(FuelP2PEvent::PeerConnected(peer_id)) } } PeerReportEvent::PeerDisconnected { peer_id } => { if self.peer_manager.handle_peer_disconnect(peer_id) { let _ = self.swarm.dial(peer_id); } - return Some(FuelP2PEvent::PeerDisconnected(peer_id)); + return Some(FuelP2PEvent::PeerDisconnected(peer_id)) } } } @@ -539,7 +577,7 @@ impl FuelP2PService { return Some(FuelP2PEvent::RequestMessage { request_id, request_message: request, - }); + }) } RequestResponseMessage::Response { request_id, @@ -626,46 +664,77 @@ mod tests { codecs::postcard::PostcardCodec, config::Config, gossipsub::{ - messages::{GossipsubBroadcastRequest, GossipsubMessage}, + messages::{ + GossipsubBroadcastRequest, + GossipsubMessage, + }, topics::{ - GossipTopic, CON_VOTE_GOSSIP_TOPIC, NEW_BLOCK_GOSSIP_TOPIC, + GossipTopic, + CON_VOTE_GOSSIP_TOPIC, + NEW_BLOCK_GOSSIP_TOPIC, NEW_TX_GOSSIP_TOPIC, }, }, p2p_service::FuelP2PEvent, peer_manager::PeerInfo, request_response::messages::{ - OutboundResponse, RequestMessage, ResponseChannelItem, + OutboundResponse, + RequestMessage, + ResponseChannelItem, }, service::to_message_acceptance, }; use fuel_core_types::{ blockchain::{ block::Block, - consensus::{poa::PoAConsensus, Consensus, ConsensusVote}, + consensus::{ + poa::PoAConsensus, + Consensus, + ConsensusVote, + }, header::PartialBlockHeader, primitives::BlockId, - SealedBlock, SealedBlockHeader, + SealedBlock, + SealedBlockHeader, + }, + fuel_tx::{ + Transaction, + TransactionBuilder, }, - fuel_tx::{Transaction, TransactionBuilder}, services::p2p::GossipsubMessageAcceptance, }; - use futures::{future::join_all, StreamExt}; + use futures::{ + future::join_all, + StreamExt, + }; use libp2p::{ - gossipsub::{error::PublishError, Topic}, + gossipsub::{ + error::PublishError, + Topic, + }, identity::Keypair, swarm::SwarmEvent, - Multiaddr, PeerId, + Multiaddr, + PeerId, }; use libp2p_swarm::PendingInboundConnectionError; use rand::Rng; use std::{ collections::HashSet, - net::{IpAddr, Ipv4Addr, SocketAddrV4, TcpListener}, + net::{ + IpAddr, + Ipv4Addr, + SocketAddrV4, + TcpListener, + }, sync::Arc, time::Duration, }; - use tokio::sync::{mpsc, oneshot, watch}; + use tokio::sync::{ + mpsc, + oneshot, + watch, + }; use tracing_attributes::instrument; type P2PService = FuelP2PService; @@ -721,7 +790,7 @@ mod tests { match fuel_p2p_service.swarm.select_next_some().await { SwarmEvent::NewListenAddr { .. } => { // listener address registered, we are good to go - break; + break } SwarmEvent::Behaviour(_) => {} other_event => { @@ -1026,7 +1095,10 @@ mod tests { #[tokio::test] #[instrument] async fn nodes_cannot_connect_due_to_different_checksum() { - use libp2p::{swarm::DialError, TransportError}; + use libp2p::{ + swarm::DialError, + TransportError, + }; // Node A let mut p2p_config = Config::default_initialized("nodes_cannot_connect_due_to_different_checksum"); diff --git a/crates/services/p2p/src/request_response/messages.rs b/crates/services/p2p/src/request_response/messages.rs index 62293d90bd9..3b32d0f78e7 100644 --- a/crates/services/p2p/src/request_response/messages.rs +++ b/crates/services/p2p/src/request_response/messages.rs @@ -1,13 +1,23 @@ use std::sync::Arc; use fuel_core_types::{ - blockchain::{primitives::BlockId, SealedBlock, SealedBlockHeader}, + blockchain::{ + primitives::BlockId, + SealedBlock, + SealedBlockHeader, + }, fuel_tx::Transaction, fuel_types::BlockHeight, }; use libp2p::PeerId; -use serde::{Deserialize, Serialize}; -use serde_with::{serde_as, FromInto}; +use serde::{ + Deserialize, + Serialize, +}; +use serde_with::{ + serde_as, + FromInto, +}; use tokio::sync::oneshot; pub(crate) const REQUEST_RESPONSE_PROTOCOL_ID: &[u8] = b"/fuel/req_res/0.0.1"; diff --git a/crates/services/p2p/src/service.rs b/crates/services/p2p/src/service.rs index 6315a907bfe..4dd755ecb17 100644 --- a/crates/services/p2p/src/service.rs +++ b/crates/services/p2p/src/service.rs @@ -1,32 +1,72 @@ use crate::{ - codecs::{postcard::PostcardCodec, NetworkCodec}, + codecs::{ + postcard::PostcardCodec, + NetworkCodec, + }, config::Config, - gossipsub::messages::{GossipsubBroadcastRequest, GossipsubMessage}, - p2p_service::{FuelP2PEvent, FuelP2PService}, - ports::{BlockHeightImporter, P2pDb}, - request_response::messages::{OutboundResponse, RequestMessage, ResponseChannelItem}, + gossipsub::messages::{ + GossipsubBroadcastRequest, + GossipsubMessage, + }, + p2p_service::{ + FuelP2PEvent, + FuelP2PService, + }, + ports::{ + BlockHeightImporter, + P2pDb, + }, + request_response::messages::{ + OutboundResponse, + RequestMessage, + ResponseChannelItem, + }, }; use anyhow::anyhow; use fuel_core_services::{ - stream::BoxStream, RunnableService, RunnableTask, ServiceRunner, StateWatcher, + stream::BoxStream, + RunnableService, + RunnableTask, + ServiceRunner, + StateWatcher, }; use fuel_core_types::{ blockchain::{ - block::Block, consensus::ConsensusVote, primitives::BlockId, SealedBlock, + block::Block, + consensus::ConsensusVote, + primitives::BlockId, + SealedBlock, SealedBlockHeader, }, fuel_tx::Transaction, fuel_types::BlockHeight, services::p2p::{ - peer_reputation::{AppScore, PeerReport}, - BlockHeightHeartbeatData, GossipData, GossipsubMessageAcceptance, - GossipsubMessageInfo, PeerId as FuelPeerId, TransactionGossipData, + peer_reputation::{ + AppScore, + PeerReport, + }, + BlockHeightHeartbeatData, + GossipData, + GossipsubMessageAcceptance, + GossipsubMessageInfo, + PeerId as FuelPeerId, + TransactionGossipData, }, }; use futures::StreamExt; -use libp2p::{gossipsub::MessageAcceptance, PeerId}; -use std::{fmt::Debug, sync::Arc}; -use tokio::sync::{broadcast, mpsc, oneshot}; +use libp2p::{ + gossipsub::MessageAcceptance, + PeerId, +}; +use std::{ + fmt::Debug, + sync::Arc, +}; +use tokio::sync::{ + broadcast, + mpsc, + oneshot, +}; use tracing::warn; pub type Service = ServiceRunner>; @@ -539,7 +579,10 @@ pub mod tests { use fuel_core_types::{ blockchain::{ block::Block, - consensus::{poa::PoAConsensus, Consensus}, + consensus::{ + poa::PoAConsensus, + Consensus, + }, }, fuel_types::BlockHeight, }; diff --git a/crates/services/src/lib.rs b/crates/services/src/lib.rs index c93eaa9e51d..63a18e6a5f6 100644 --- a/crates/services/src/lib.rs +++ b/crates/services/src/lib.rs @@ -9,7 +9,12 @@ mod state; /// Re-exports for streaming utilities pub mod stream { #[doc(no_inline)] - pub use futures::stream::{empty, pending, unfold, Stream}; + pub use futures::stream::{ + empty, + pending, + unfold, + Stream, + }; /// A Send + Sync BoxStream pub type BoxStream = @@ -34,7 +39,15 @@ pub mod stream { } pub use service::{ - EmptyShared, RunnableService, RunnableTask, Service, ServiceRunner, Shared, + EmptyShared, + RunnableService, + RunnableTask, + Service, + ServiceRunner, + Shared, SharedMutex, }; -pub use state::{State, StateWatcher}; +pub use state::{ + State, + StateWatcher, +}; diff --git a/crates/services/src/service.rs b/crates/services/src/service.rs index 865db6e2dc7..4c2e6d49261 100644 --- a/crates/services/src/service.rs +++ b/crates/services/src/service.rs @@ -1,4 +1,7 @@ -use crate::state::{State, StateWatcher}; +use crate::state::{ + State, + StateWatcher, +}; use anyhow::anyhow; use futures::FutureExt; use tokio::sync::watch; @@ -156,7 +159,7 @@ where loop { let state = start.borrow().clone(); if !state.starting() { - return Ok(state); + return Ok(state) } start.changed().await?; } @@ -166,7 +169,7 @@ where loop { let state = stop.borrow().clone(); if state.stopped() { - return Ok(state); + return Ok(state) } stop.changed().await?; } @@ -300,7 +303,7 @@ where // If the state after update is not `Starting` then return to stop the service. if !state.borrow().starting() { - return; + return } // We can panic here, because it is inside of the task. @@ -328,7 +331,7 @@ where Ok(Ok(should_continue)) => { if !should_continue { tracing::debug!("stopping"); - break; + break } tracing::debug!("run loop"); } @@ -339,7 +342,7 @@ where Err(panic) => { tracing::debug!("got a panic"); got_panic = Some(panic); - break; + break } } } diff --git a/crates/services/txpool/src/ports.rs b/crates/services/txpool/src/ports.rs index 4b8eb759ad7..29a2cc54516 100644 --- a/crates/services/txpool/src/ports.rs +++ b/crates/services/txpool/src/ports.rs @@ -2,12 +2,27 @@ use fuel_core_p2p::PeerId; use fuel_core_services::stream::BoxStream; use fuel_core_storage::Result as StorageResult; use fuel_core_types::{ - entities::{coins::coin::CompressedCoin, message::Message}, - fuel_tx::{Transaction, UtxoId}, - fuel_types::{BlockHeight, Bytes32, ContractId, Nonce}, + entities::{ + coins::coin::CompressedCoin, + message::Message, + }, + fuel_tx::{ + Transaction, + UtxoId, + }, + fuel_types::{ + BlockHeight, + Bytes32, + ContractId, + Nonce, + }, services::{ block_importer::ImportResult, - p2p::{GossipsubMessageAcceptance, GossipsubMessageInfo, NetworkData}, + p2p::{ + GossipsubMessageAcceptance, + GossipsubMessageInfo, + NetworkData, + }, txpool::TransactionStatus, }, }; diff --git a/crates/services/txpool/src/service.rs b/crates/services/txpool/src/service.rs index 80bdc6648f5..537f9663417 100644 --- a/crates/services/txpool/src/service.rs +++ b/crates/services/txpool/src/service.rs @@ -1,34 +1,66 @@ use crate::{ - ports::{BlockImporter, PeerToPeer, TxPoolDb}, + ports::{ + BlockImporter, + PeerToPeer, + TxPoolDb, + }, transaction_selector::select_transactions, - Config, Error as TxPoolError, TxInfo, TxPool, + Config, + Error as TxPoolError, + TxInfo, + TxPool, }; use fuel_core_p2p::PeerId; use fuel_core_services::{ - stream::BoxStream, RunnableService, RunnableTask, Service as _, ServiceRunner, + stream::BoxStream, + RunnableService, + RunnableTask, + Service as _, + ServiceRunner, StateWatcher, }; use fuel_core_types::{ - fuel_tx::{ConsensusParameters, Transaction, TxId, UniqueIdentifier}, - fuel_types::{BlockHeight, Bytes32}, + fuel_tx::{ + ConsensusParameters, + Transaction, + TxId, + UniqueIdentifier, + }, + fuel_types::{ + BlockHeight, + Bytes32, + }, services::{ block_importer::ImportResult, p2p::{ - GossipData, GossipsubMessageAcceptance, GossipsubMessageInfo, + GossipData, + GossipsubMessageAcceptance, + GossipsubMessageInfo, TransactionGossipData, }, - txpool::{ArcPoolTx, Error, InsertionResult, TransactionStatus}, + txpool::{ + ArcPoolTx, + Error, + InsertionResult, + TransactionStatus, + }, }, tai64::Tai64, }; use parking_lot::Mutex as ParkingMutex; use std::sync::Arc; -use tokio::{sync::broadcast, time::MissedTickBehavior}; +use tokio::{ + sync::broadcast, + time::MissedTickBehavior, +}; use tokio_stream::StreamExt; use update_sender::UpdateSender; -use self::update_sender::{MpscChannel, TxStatusStream}; +use self::update_sender::{ + MpscChannel, + TxStatusStream, +}; mod update_sender; diff --git a/crates/services/txpool/src/service/test_helpers.rs b/crates/services/txpool/src/service/test_helpers.rs index 26f65ff0244..bf88a25eb1e 100644 --- a/crates/services/txpool/src/service/test_helpers.rs +++ b/crates/services/txpool/src/service/test_helpers.rs @@ -1,11 +1,26 @@ use super::*; -use crate::{ports::BlockImporter, MockDb}; -use fuel_core_services::{stream::BoxStream, Service as ServiceTrait}; +use crate::{ + ports::BlockImporter, + MockDb, +}; +use fuel_core_services::{ + stream::BoxStream, + Service as ServiceTrait, +}; use fuel_core_types::{ blockchain::SealedBlock, entities::coins::coin::Coin, - fuel_crypto::rand::{rngs::StdRng, SeedableRng}, - fuel_tx::{Cacheable, Input, Transaction, TransactionBuilder, Word}, + fuel_crypto::rand::{ + rngs::StdRng, + SeedableRng, + }, + fuel_tx::{ + Cacheable, + Input, + Transaction, + TransactionBuilder, + Word, + }, services::p2p::GossipsubMessageAcceptance, }; use std::cell::RefCell; diff --git a/crates/services/txpool/src/service/tests_p2p.rs b/crates/services/txpool/src/service/tests_p2p.rs index 1f7ceb32dd0..c257fe7dd43 100644 --- a/crates/services/txpool/src/service/tests_p2p.rs +++ b/crates/services/txpool/src/service/tests_p2p.rs @@ -1,8 +1,17 @@ use super::*; -use crate::service::test_helpers::{MockP2P, TestContextBuilder}; +use crate::service::test_helpers::{ + MockP2P, + TestContextBuilder, +}; use fuel_core_services::Service; -use fuel_core_types::fuel_tx::{Transaction, UniqueIdentifier}; -use std::{ops::Deref, time::Duration}; +use fuel_core_types::fuel_tx::{ + Transaction, + UniqueIdentifier, +}; +use std::{ + ops::Deref, + time::Duration, +}; #[tokio::test] async fn can_insert_from_p2p() { diff --git a/crates/services/txpool/src/txpool.rs b/crates/services/txpool/src/txpool.rs index 474e2919542..9718beb032d 100644 --- a/crates/services/txpool/src/txpool.rs +++ b/crates/services/txpool/src/txpool.rs @@ -1,19 +1,40 @@ use crate::{ - containers::{dependency::Dependency, price_sort::PriceSort, time_sort::TimeSort}, + containers::{ + dependency::Dependency, + price_sort::PriceSort, + time_sort::TimeSort, + }, ports::TxPoolDb, service::TxStatusChange, types::*, - Config, Error, TxInfo, + Config, + Error, + TxInfo, }; use fuel_core_metrics::txpool_metrics::TXPOOL_METRICS; use fuel_core_types::{ - fuel_tx::{Chargeable, Transaction, UniqueIdentifier}, + fuel_tx::{ + Chargeable, + Transaction, + UniqueIdentifier, + }, fuel_types::BlockHeight, - fuel_vm::checked_transaction::{CheckedTransaction, IntoChecked}, - services::txpool::{ArcPoolTx, InsertionResult}, + fuel_vm::checked_transaction::{ + CheckedTransaction, + IntoChecked, + }, + services::txpool::{ + ArcPoolTx, + InsertionResult, + }, tai64::Tai64, }; -use std::{cmp::Reverse, collections::HashMap, ops::Deref, sync::Arc}; +use std::{ + cmp::Reverse, + collections::HashMap, + ops::Deref, + sync::Arc, +}; #[derive(Debug, Clone)] pub struct TxPool { @@ -60,7 +81,7 @@ where let current_height = self.database.current_block_height()?; if tx.is_mint() { - return Err(Error::NotSupportedTransactionType.into()); + return Err(Error::NotSupportedTransactionType.into()) } // verify gas price is at least the minimum @@ -94,7 +115,7 @@ where }); if !tx.is_computed() { - return Err(Error::NoMetadata.into()); + return Err(Error::NoMetadata.into()) } // verify max gas is less than block limit @@ -103,11 +124,11 @@ where tx_gas: tx.max_gas(), block_limit: self.config.chain_config.block_gas_limit, } - .into()); + .into()) } if self.by_hash.contains_key(&tx.id()) { - return Err(Error::NotInsertedTxKnown.into()); + return Err(Error::NotInsertedTxKnown.into()) } let mut max_limit_hit = false; @@ -117,7 +138,7 @@ where // limit is hit, check if we can push out lowest priced tx let lowest_price = self.by_gas_price.lowest_value().unwrap_or_default(); if lowest_price >= tx.price() { - return Err(Error::NotInsertedLimitHit.into()); + return Err(Error::NotInsertedLimitHit.into()) } } if self.config.metrics { @@ -188,7 +209,7 @@ where for remove in removed.iter() { self.remove_tx(&remove.id()); } - return removed; + return removed } Vec::new() } @@ -225,7 +246,7 @@ where TXPOOL_METRICS.gas_price_histogram.observe(price as f64); } if price < self.config.min_gas_price { - return Err(Error::NotInsertedGasPriceTooLow); + return Err(Error::NotInsertedGasPriceTooLow) } Ok(()) } @@ -362,7 +383,7 @@ where let removed = self.remove_inner(&oldest_tx); result.extend(removed.into_iter()); } else { - break; + break } } diff --git a/crates/types/src/services/block_importer.rs b/crates/types/src/services/block_importer.rs index 953f32e23be..494abb8b572 100644 --- a/crates/types/src/services/block_importer.rs +++ b/crates/types/src/services/block_importer.rs @@ -1,8 +1,14 @@ //! Types related to block importer service. use crate::{ - blockchain::{header::BlockHeader, SealedBlock}, - services::{executor::TransactionExecutionStatus, Uncommitted}, + blockchain::{ + header::BlockHeader, + SealedBlock, + }, + services::{ + executor::TransactionExecutionStatus, + Uncommitted, + }, }; /// The uncommitted result of the block importing. diff --git a/tests/tests/sync.rs b/tests/tests/sync.rs index ada3671730c..0c9ac5eb2c6 100644 --- a/tests/tests/sync.rs +++ b/tests/tests/sync.rs @@ -1,10 +1,23 @@ use fuel_core::p2p_test_helpers::*; -use fuel_core_types::{fuel_crypto::SecretKey, fuel_tx::Input}; +use fuel_core_types::{ + fuel_crypto::SecretKey, + fuel_tx::Input, +}; use itertools::Itertools; -use rand::{rngs::StdRng, SeedableRng}; +use rand::{ + rngs::StdRng, + SeedableRng, +}; use std::{ - collections::{hash_map::DefaultHasher, HashMap, VecDeque}, - hash::{Hash, Hasher}, + collections::{ + hash_map::DefaultHasher, + HashMap, + VecDeque, + }, + hash::{ + Hash, + Hasher, + }, }; use test_case::test_case; From 791433de6eca868ec68910b5ef230ef799813dd9 Mon Sep 17 00:00:00 2001 From: Rodrigo Araujo Date: Tue, 18 Jul 2023 12:23:35 -0700 Subject: [PATCH 04/18] Format with nightly --- crates/fuel-core/src/p2p_test_helpers.rs | 52 +++++++++++++++++++----- 1 file changed, 42 insertions(+), 10 deletions(-) diff --git a/crates/fuel-core/src/p2p_test_helpers.rs b/crates/fuel-core/src/p2p_test_helpers.rs index 1675330552b..0ba6d6d759b 100644 --- a/crates/fuel-core/src/p2p_test_helpers.rs +++ b/crates/fuel-core/src/p2p_test_helpers.rs @@ -4,27 +4,59 @@ use crate::{ chain_config::ChainConfig, database::Database, p2p::Multiaddr, - service::{genesis::maybe_initialize_state, Config, FuelService, ServiceTrait}, + service::{ + genesis::maybe_initialize_state, + Config, + FuelService, + ServiceTrait, + }, +}; +use fuel_core_p2p::{ + codecs::postcard::PostcardCodec, + network_service::FuelP2PService, +}; +use fuel_core_poa::{ + ports::BlockImporter, + Trigger, +}; +use fuel_core_storage::{ + tables::Transactions, + StorageAsRef, }; -use fuel_core_p2p::{codecs::postcard::PostcardCodec, network_service::FuelP2PService}; -use fuel_core_poa::{ports::BlockImporter, Trigger}; -use fuel_core_storage::{tables::Transactions, StorageAsRef}; use fuel_core_types::{ - fuel_asm::{op, RegId}, + fuel_asm::{ + op, + RegId, + }, fuel_crypto::SecretKey, fuel_tx::{ - ConsensusParameters, Input, Transaction, TransactionBuilder, TxId, - UniqueIdentifier, UtxoId, + ConsensusParameters, + Input, + Transaction, + TransactionBuilder, + TxId, + UniqueIdentifier, + UtxoId, + }, + fuel_types::{ + Address, + Bytes32, }, - fuel_types::{Address, Bytes32}, secrecy::Secret, }; use futures::StreamExt; use itertools::Itertools; -use rand::{rngs::StdRng, Rng, SeedableRng}; +use rand::{ + rngs::StdRng, + Rng, + SeedableRng, +}; use std::{ collections::HashMap, - ops::{Index, IndexMut}, + ops::{ + Index, + IndexMut, + }, sync::Arc, time::Duration, }; From b4eb7ca8a07a02d08653fbcc5f726d5b193a76dc Mon Sep 17 00:00:00 2001 From: Rodrigo Araujo Date: Sat, 16 Sep 2023 16:25:36 -0700 Subject: [PATCH 05/18] Move from requesting txs to sending txs upon connection --- crates/fuel-core/src/p2p_test_helpers.rs | 9 +- .../fuel-core/src/service/adapters/txpool.rs | 62 ++---- crates/services/p2p/src/p2p_service.rs | 169 ++++++---------- .../p2p/src/request_response/messages.rs | 2 +- crates/services/p2p/src/service.rs | 109 ++++------ crates/services/txpool/src/ports.rs | 30 +-- crates/services/txpool/src/service.rs | 82 +++----- crates/services/txpool/src/txpool.rs | 49 ++--- tests/tests/poa.rs | 5 +- tests/tests/sync.rs | 10 + tests/tests/tx_gossip.rs | 35 ++-- tests/tests/tx_request_response.rs | 186 +++++++++--------- 12 files changed, 278 insertions(+), 470 deletions(-) diff --git a/crates/fuel-core/src/p2p_test_helpers.rs b/crates/fuel-core/src/p2p_test_helpers.rs index c72e86e07f4..8db1daecfad 100644 --- a/crates/fuel-core/src/p2p_test_helpers.rs +++ b/crates/fuel-core/src/p2p_test_helpers.rs @@ -148,6 +148,7 @@ pub async fn make_nodes( bootstrap_setup: impl IntoIterator>, producers_setup: impl IntoIterator>, validators_setup: impl IntoIterator>, + disable_block_production: bool, ) -> Nodes { let producers: Vec<_> = producers_setup.into_iter().collect(); @@ -227,6 +228,7 @@ pub async fn make_nodes( .then_some(name) .unwrap_or_else(|| format!("b:{i}")), chain_config.clone(), + disable_block_production ); if let Some(BootstrapSetup { pub_key, .. }) = boot { match &mut node_config.chain_conf.consensus { @@ -252,6 +254,7 @@ pub async fn make_nodes( .then_some(name) .unwrap_or_else(|| format!("p:{i}")), chain_config.clone(), + disable_block_production ); let mut test_txs = Vec::with_capacity(0); @@ -284,6 +287,7 @@ pub async fn make_nodes( .then_some(name) .unwrap_or_else(|| format!("v:{i}")), chain_config.clone(), + disable_block_production ); node_config.block_production = Trigger::Never; node_config.p2p.as_mut().unwrap().bootstrap_nodes = boots.clone(); @@ -305,10 +309,13 @@ pub async fn make_nodes( } } -pub fn make_config(name: String, chain_config: ChainConfig) -> Config { +pub fn make_config(name: String, chain_config: ChainConfig, disable_block_production: bool) -> Config { let mut node_config = Config::local_node(); node_config.chain_conf = chain_config; node_config.utxo_validation = true; + if disable_block_production { + node_config.block_production = Trigger::Never; + } node_config.name = name; node_config } diff --git a/crates/fuel-core/src/service/adapters/txpool.rs b/crates/fuel-core/src/service/adapters/txpool.rs index 37daf29e8a4..2a8bbbbc2cb 100644 --- a/crates/fuel-core/src/service/adapters/txpool.rs +++ b/crates/fuel-core/src/service/adapters/txpool.rs @@ -1,55 +1,29 @@ use crate::{ database::Database, - service::adapters::{ - BlockImporterAdapter, - P2PAdapter, - }, + service::adapters::{BlockImporterAdapter, P2PAdapter}, }; use fuel_core_p2p::PeerId; use fuel_core_services::stream::BoxStream; use fuel_core_storage::{ not_found, - tables::{ - Coins, - ContractsRawCode, - Messages, - SpentMessages, - }, - Result as StorageResult, - StorageAsRef, + tables::{Coins, ContractsRawCode, Messages, SpentMessages}, + Result as StorageResult, StorageAsRef, }; use fuel_core_txpool::ports::BlockImporter; use fuel_core_types::{ - entities::{ - coins::coin::CompressedCoin, - message::Message, - }, - fuel_tx::{ - Transaction, - UtxoId, - }, - fuel_types::{ - BlockHeight, - ContractId, - Nonce, - }, + entities::{coins::coin::CompressedCoin, message::Message}, + fuel_tx::{Transaction, UtxoId}, + fuel_types::{BlockHeight, ContractId, Nonce}, services::{ block_importer::ImportResult, - p2p::{ - GossipsubMessageAcceptance, - GossipsubMessageInfo, - TransactionGossipData, - }, + p2p::{GossipsubMessageAcceptance, GossipsubMessageInfo, TransactionGossipData}, }, }; use std::sync::Arc; impl BlockImporter for BlockImporterAdapter { fn block_events(&self) -> BoxStream> { - use tokio_stream::{ - wrappers::BroadcastStream, - StreamExt, - }; + use tokio_stream::{wrappers::BroadcastStream, StreamExt}; Box::pin( BroadcastStream::new(self.block_importer.subscribe()) .filter_map(|result| result.ok()), @@ -70,25 +44,22 @@ impl fuel_core_txpool::ports::PeerToPeer for P2PAdapter { } } - async fn request_pooled_transactions( + async fn send_pooled_transactions( &self, peer_id: PeerId, - ) -> anyhow::Result>> { - // temp string + transactions: Vec, + ) -> anyhow::Result<()> { if let Some(service) = &self.service { service - .get_pooled_transactions_from_peer(peer_id.to_bytes()) + .send_pooled_transactions_to_peer(peer_id.to_bytes(), transactions) .await } else { - Ok(None) + Ok(()) } } fn gossiped_transaction_events(&self) -> BoxStream { - use tokio_stream::{ - wrappers::BroadcastStream, - StreamExt, - }; + use tokio_stream::{wrappers::BroadcastStream, StreamExt}; if let Some(service) = &self.service { Box::pin( BroadcastStream::new(service.subscribe_tx()) @@ -112,10 +83,7 @@ impl fuel_core_txpool::ports::PeerToPeer for P2PAdapter { } fn new_connection(&self) -> BoxStream { - use tokio_stream::{ - wrappers::BroadcastStream, - StreamExt, - }; + use tokio_stream::{wrappers::BroadcastStream, StreamExt}; if let Some(service) = &self.service { Box::pin( BroadcastStream::new(service.subscribe_to_connections()) diff --git a/crates/services/p2p/src/p2p_service.rs b/crates/services/p2p/src/p2p_service.rs index 4b277c71540..8423ea1ebed 100644 --- a/crates/services/p2p/src/p2p_service.rs +++ b/crates/services/p2p/src/p2p_service.rs @@ -1,75 +1,37 @@ use crate::{ - behavior::{ - FuelBehaviour, - FuelBehaviourEvent, - }, + behavior::{FuelBehaviour, FuelBehaviourEvent}, codecs::NetworkCodec, - config::{ - build_transport, - Config, - }, + config::{build_transport, Config}, gossipsub::{ - messages::{ - GossipsubBroadcastRequest, - GossipsubMessage as FuelGossipsubMessage, - }, + messages::{GossipsubBroadcastRequest, GossipsubMessage as FuelGossipsubMessage}, topics::GossipsubTopics, }, - peer_manager::{ - PeerManager, - Punisher, - }, + peer_manager::{PeerManager, Punisher}, peer_report::PeerReportEvent, request_response::messages::{ - NetworkResponse, - OutboundResponse, - RequestError, - RequestMessage, - ResponseChannelItem, - ResponseError, - ResponseMessage, + NetworkResponse, OutboundResponse, RequestError, RequestMessage, + ResponseChannelItem, ResponseError, ResponseMessage, }, }; use fuel_core_metrics::p2p_metrics::P2P_METRICS; use fuel_core_types::{ - fuel_types::BlockHeight, - services::p2p::peer_reputation::AppScore, + fuel_types::BlockHeight, services::p2p::peer_reputation::AppScore, }; use futures::prelude::*; use libp2p::{ gossipsub::{ - error::PublishError, - GossipsubEvent, - MessageAcceptance, - MessageId, - TopicHash, + error::PublishError, GossipsubEvent, MessageAcceptance, MessageId, TopicHash, }, multiaddr::Protocol, request_response::{ - RequestId, - RequestResponseEvent, - RequestResponseMessage, - ResponseChannel, + RequestId, RequestResponseEvent, RequestResponseMessage, ResponseChannel, }, - swarm::{ - AddressScore, - ConnectionLimits, - SwarmBuilder, - SwarmEvent, - }, - Multiaddr, - PeerId, - Swarm, + swarm::{AddressScore, ConnectionLimits, SwarmBuilder, SwarmEvent}, + Multiaddr, PeerId, Swarm, }; use rand::seq::IteratorRandom; -use std::{ - collections::HashMap, - time::Duration, -}; -use tracing::{ - debug, - warn, -}; +use std::{collections::HashMap, time::Duration}; +use tracing::{debug, warn}; impl Punisher for Swarm> { fn ban_peer(&mut self, peer_id: PeerId) { @@ -257,7 +219,7 @@ impl FuelP2PService { loop { if let SwarmEvent::NewListenAddr { .. } = self.swarm.select_next_some().await { - break + break; } } } @@ -313,7 +275,7 @@ impl FuelP2PService { let peers_count = self.peer_manager.total_peers_connected(); if peers_count == 0 { - return Err(RequestError::NoPeersConnected) + return Err(RequestError::NoPeersConnected); } let mut range = rand::thread_rng(); @@ -321,6 +283,11 @@ impl FuelP2PService { } }; + println!( + "I'm {:?} and I'm sending request to peer: {:?} with message: {:?}", + self.local_peer_id, peer_id, message_request + ); + let request_id = self .swarm .behaviour_mut() @@ -350,16 +317,16 @@ impl FuelP2PService { .is_err() { debug!("Failed to send ResponseMessage for {:?}", request_id); - return Err(ResponseError::SendingResponseFailed) + return Err(ResponseError::SendingResponseFailed); } } (Ok(_), None) => { debug!("ResponseChannel for {:?} does not exist!", request_id); - return Err(ResponseError::ResponseChannelDoesNotExist) + return Err(ResponseError::ResponseChannelDoesNotExist); } (Err(e), _) => { debug!("Failed to convert to IntermediateResponse with {:?}", e); - return Err(ResponseError::ConversionToIntermediateFailed) + return Err(ResponseError::ConversionToIntermediateFailed); } } @@ -540,7 +507,7 @@ impl FuelP2PService { return Some(FuelP2PEvent::PeerInfoUpdated { peer_id, block_height, - }) + }); } PeerReportEvent::PeerConnected { peer_id, @@ -554,14 +521,17 @@ impl FuelP2PService { ) { let _ = self.swarm.disconnect_peer_id(peer_id); } else if initial_connection { - return Some(FuelP2PEvent::PeerConnected(peer_id)) + return Some(FuelP2PEvent::PeerConnected(peer_id)); } } PeerReportEvent::PeerDisconnected { peer_id } => { + // So... it seems like the problem is that the peer is + // disconnecting... why? + println!("Peer {:?} disconnected", peer_id); if self.peer_manager.handle_peer_disconnect(peer_id) { let _ = self.swarm.dial(peer_id); } - return Some(FuelP2PEvent::PeerDisconnected(peer_id)) + return Some(FuelP2PEvent::PeerDisconnected(peer_id)); } } } @@ -577,7 +547,7 @@ impl FuelP2PService { return Some(FuelP2PEvent::RequestMessage { request_id, request_message: request, - }) + }); } RequestResponseMessage::Response { request_id, @@ -598,6 +568,17 @@ impl FuelP2PService { ); } } + ( + Some(ResponseChannelItem::PooledTransactions(channel)), + Ok(ResponseMessage::PooledTransactions(transactions)), + ) => { + if channel.send(transactions).is_err() { + debug!( + "Failed to send through the channel for {:?}", + request_id + ); + } + } ( Some(ResponseChannelItem::Transactions(channel)), Ok(ResponseMessage::Transactions(transactions)), @@ -664,81 +645,47 @@ mod tests { codecs::postcard::PostcardCodec, config::Config, gossipsub::{ - messages::{ - GossipsubBroadcastRequest, - GossipsubMessage, - }, + messages::{GossipsubBroadcastRequest, GossipsubMessage}, topics::{ - GossipTopic, - CON_VOTE_GOSSIP_TOPIC, - NEW_BLOCK_GOSSIP_TOPIC, + GossipTopic, CON_VOTE_GOSSIP_TOPIC, NEW_BLOCK_GOSSIP_TOPIC, NEW_TX_GOSSIP_TOPIC, }, }, p2p_service::FuelP2PEvent, peer_manager::PeerInfo, request_response::messages::{ - OutboundResponse, - RequestMessage, - ResponseChannelItem, + OutboundResponse, RequestMessage, ResponseChannelItem, }, service::to_message_acceptance, }; use fuel_core_types::{ blockchain::{ block::Block, - consensus::{ - poa::PoAConsensus, - Consensus, - ConsensusVote, - }, - header::{ - BlockHeader, - PartialBlockHeader, - }, + consensus::{poa::PoAConsensus, Consensus, ConsensusVote}, + header::{BlockHeader, PartialBlockHeader}, primitives::BlockId, - SealedBlock, - SealedBlockHeader, - }, - fuel_tx::{ - Transaction, - TransactionBuilder, + SealedBlock, SealedBlockHeader, }, + fuel_tx::{Transaction, TransactionBuilder}, services::p2p::GossipsubMessageAcceptance, }; - use futures::{ - future::join_all, - StreamExt, - }; + use futures::{future::join_all, StreamExt}; use libp2p::{ - gossipsub::{ - error::PublishError, - Topic, - }, + gossipsub::{error::PublishError, Topic}, identity::Keypair, swarm::SwarmEvent, - Multiaddr, - PeerId, + Multiaddr, PeerId, }; use libp2p_swarm::PendingInboundConnectionError; use rand::Rng; use std::{ collections::HashSet, - net::{ - IpAddr, - Ipv4Addr, - SocketAddrV4, - TcpListener, - }, + net::{IpAddr, Ipv4Addr, SocketAddrV4, TcpListener}, ops::Range, sync::Arc, time::Duration, }; - use tokio::sync::{ - mpsc, - oneshot, - watch, - }; + use tokio::sync::{mpsc, oneshot, watch}; use tracing_attributes::instrument; type P2PService = FuelP2PService; @@ -788,13 +735,14 @@ mod tests { #[instrument] async fn p2p_service_works() { let mut fuel_p2p_service = - build_service_from_config(Config::default_initialized("p2p_service_works")); + build_service_from_config(Config::default_initialized("p2p_service_works")) + .await; loop { match fuel_p2p_service.swarm.select_next_some().await { SwarmEvent::NewListenAddr { .. } => { // listener address registered, we are good to go - break + break; } SwarmEvent::Behaviour(_) => {} other_event => { @@ -1099,10 +1047,7 @@ mod tests { #[tokio::test] #[instrument] async fn nodes_cannot_connect_due_to_different_checksum() { - use libp2p::{ - swarm::DialError, - TransportError, - }; + use libp2p::{swarm::DialError, TransportError}; // Node A let mut p2p_config = Config::default_initialized("nodes_cannot_connect_due_to_different_checksum"); diff --git a/crates/services/p2p/src/request_response/messages.rs b/crates/services/p2p/src/request_response/messages.rs index aaf82dc3aa2..51f020c42e3 100644 --- a/crates/services/p2p/src/request_response/messages.rs +++ b/crates/services/p2p/src/request_response/messages.rs @@ -46,7 +46,7 @@ pub enum RequestMessage { Block(BlockHeight), SealedHeaders(Range), Transactions(#[serde_as(as = "FromInto<[u8; 32]>")] BlockId), - PooledTransactions, + PooledTransactions(Vec), // temp as string } /// Final Response Message that p2p service sends to the Orchestrator diff --git a/crates/services/p2p/src/service.rs b/crates/services/p2p/src/service.rs index 5df2d3b9dfb..9d8c9aa8787 100644 --- a/crates/services/p2p/src/service.rs +++ b/crates/services/p2p/src/service.rs @@ -1,73 +1,32 @@ use crate::{ - codecs::{ - postcard::PostcardCodec, - NetworkCodec, - }, + codecs::{postcard::PostcardCodec, NetworkCodec}, config::Config, - gossipsub::messages::{ - GossipsubBroadcastRequest, - GossipsubMessage, - }, - p2p_service::{ - FuelP2PEvent, - FuelP2PService, - }, - ports::{ - BlockHeightImporter, - P2pDb, - }, - request_response::messages::{ - OutboundResponse, - RequestMessage, - ResponseChannelItem, - }, + gossipsub::messages::{GossipsubBroadcastRequest, GossipsubMessage}, + p2p_service::{FuelP2PEvent, FuelP2PService}, + ports::{BlockHeightImporter, P2pDb}, + request_response::messages::{OutboundResponse, RequestMessage, ResponseChannelItem}, }; use anyhow::anyhow; use fuel_core_services::{ - stream::BoxStream, - RunnableService, - RunnableTask, - ServiceRunner, - StateWatcher, + stream::BoxStream, RunnableService, RunnableTask, ServiceRunner, StateWatcher, }; use fuel_core_types::{ blockchain::{ - block::Block, - consensus::ConsensusVote, - primitives::BlockId, - SealedBlock, + block::Block, consensus::ConsensusVote, primitives::BlockId, SealedBlock, SealedBlockHeader, }, fuel_tx::Transaction, fuel_types::BlockHeight, services::p2p::{ - peer_reputation::{ - AppScore, - PeerReport, - }, - BlockHeightHeartbeatData, - GossipData, - GossipsubMessageAcceptance, - GossipsubMessageInfo, - PeerId as FuelPeerId, - TransactionGossipData, + peer_reputation::{AppScore, PeerReport}, + BlockHeightHeartbeatData, GossipData, GossipsubMessageAcceptance, + GossipsubMessageInfo, PeerId as FuelPeerId, TransactionGossipData, }, }; use futures::StreamExt; -use libp2p::{ - gossipsub::MessageAcceptance, - PeerId, -}; -use std::{ - fmt::Debug, - ops::Range, - sync::Arc, -}; -use tokio::sync::{ - broadcast, - mpsc, - oneshot, -}; +use libp2p::{gossipsub::MessageAcceptance, PeerId}; +use std::{fmt::Debug, ops::Range, sync::Arc}; +use tokio::sync::{broadcast, mpsc, oneshot}; use tracing::warn; pub type Service = ServiceRunner>; @@ -87,10 +46,9 @@ enum TaskRequest { block_height_range: Range, channel: oneshot::Sender>)>>, }, - // Request get current pooled transactions - GetPooledTransactions { - from_peer: PeerId, - channel: oneshot::Sender>>, // temp string + SendPooledTransactions { + to_peer: PeerId, + transactions: Vec, // temp string }, GetTransactions { block_id: BlockId, @@ -203,6 +161,11 @@ where next_service_request = self.request_receiver.recv() => { should_continue = true; match next_service_request { + Some(TaskRequest::SendPooledTransactions { to_peer, transactions }) => { + let request_msg = RequestMessage::PooledTransactions(transactions); + let channel_item = ResponseChannelItem::PooledTransactions(oneshot::channel().0); + let _ = self.p2p_service.send_request_msg(Some(to_peer), request_msg, channel_item); + } Some(TaskRequest::BroadcastTransaction(transaction)) => { let broadcast = GossipsubBroadcastRequest::NewTx(transaction); let result = self.p2p_service.publish_message(broadcast); @@ -224,12 +187,6 @@ where tracing::error!("Got an error during vote broadcasting {}", e); } } - Some(TaskRequest::GetPooledTransactions{ channel, from_peer }) => { - dbg!("### Task::RequestGetPooledTransactions ###"); - let request_msg = RequestMessage::PooledTransactions; - let channel_item = ResponseChannelItem::PooledTransactions(channel); - let _ = self.p2p_service.send_request_msg(Some(from_peer), request_msg, channel_item); - } Some(TaskRequest::GetPeerIds(channel)) => { let peer_ids = self.p2p_service.get_peers_ids().copied().collect(); let _ = channel.send(peer_ids); @@ -318,8 +275,10 @@ where let _ = self.p2p_service.send_response_msg(request_id, OutboundResponse::Transactions(transactions_response)); } - RequestMessage::PooledTransactions => { - todo!("Implement `RequestMessage::PooledTransactions`"); + RequestMessage::PooledTransactions(transactions) => { + // Temp `todo!` because with this new implementation we're + // never getting to this point, as the node was already disconnected. + todo!("Pooled transactions are not yet implemented"); } RequestMessage::SealedHeaders(range) => { let max_len = self.max_headers_per_request.try_into().expect("u32 should always fit into usize"); @@ -415,7 +374,7 @@ impl SharedState { if block_height_range.is_empty() { return Err(anyhow!( "Cannot retrieve headers for an empty range of block heights" - )) + )); } self.request_sender @@ -450,22 +409,22 @@ impl SharedState { receiver.await.map_err(|e| anyhow!("{}", e)) } - pub async fn get_pooled_transactions_from_peer( + pub async fn send_pooled_transactions_to_peer( &self, peer_id: Vec, - ) -> anyhow::Result>> { + transactions: Vec, + ) -> anyhow::Result<()> { // temp string - let (sender, receiver) = oneshot::channel(); - let from_peer = PeerId::from_bytes(&peer_id).expect("Valid PeerId"); + let to_peer = PeerId::from_bytes(&peer_id).expect("Valid PeerId"); self.request_sender - .send(TaskRequest::GetPooledTransactions { - from_peer, - channel: sender, + .send(TaskRequest::SendPooledTransactions { + to_peer, + transactions, }) .await?; - receiver.await.map_err(|e| anyhow!("{}", e)) + Ok(()) } pub fn broadcast_vote(&self, vote: Arc) -> anyhow::Result<()> { diff --git a/crates/services/txpool/src/ports.rs b/crates/services/txpool/src/ports.rs index 29a2cc54516..44d929a1e0c 100644 --- a/crates/services/txpool/src/ports.rs +++ b/crates/services/txpool/src/ports.rs @@ -2,27 +2,12 @@ use fuel_core_p2p::PeerId; use fuel_core_services::stream::BoxStream; use fuel_core_storage::Result as StorageResult; use fuel_core_types::{ - entities::{ - coins::coin::CompressedCoin, - message::Message, - }, - fuel_tx::{ - Transaction, - UtxoId, - }, - fuel_types::{ - BlockHeight, - Bytes32, - ContractId, - Nonce, - }, + entities::{coins::coin::CompressedCoin, message::Message}, + fuel_tx::{Transaction, UtxoId}, + fuel_types::{BlockHeight, Bytes32, ContractId, Nonce}, services::{ block_importer::ImportResult, - p2p::{ - GossipsubMessageAcceptance, - GossipsubMessageInfo, - NetworkData, - }, + p2p::{GossipsubMessageAcceptance, GossipsubMessageInfo, NetworkData}, txpool::TransactionStatus, }, }; @@ -48,11 +33,12 @@ pub trait PeerToPeer: Send + Sync { /// Streams new connections to the node. fn new_connection(&self) -> BoxStream; - /// Request pooled transactions from a peer. - async fn request_pooled_transactions( + /// Send pooled transactions to a peer. + async fn send_pooled_transactions( &self, peer_id: PeerId, - ) -> anyhow::Result>>; + transactions: Vec, + ) -> anyhow::Result<()>; } pub trait BlockImporter: Send + Sync { diff --git a/crates/services/txpool/src/service.rs b/crates/services/txpool/src/service.rs index c0a7fbb0ffe..6c8db810f39 100644 --- a/crates/services/txpool/src/service.rs +++ b/crates/services/txpool/src/service.rs @@ -1,72 +1,37 @@ use crate::{ - ports::{ - BlockImporter, - PeerToPeer, - TxPoolDb, - }, + ports::{BlockImporter, PeerToPeer, TxPoolDb}, transaction_selector::select_transactions, - txpool::{ - check_single_tx, - check_transactions, - }, - Config, - Error as TxPoolError, - TxInfo, - TxPool, + txpool::{check_single_tx, check_transactions}, + Config, Error as TxPoolError, TxInfo, TxPool, }; use fuel_core_p2p::PeerId; use fuel_core_services::{ - stream::BoxStream, - RunnableService, - RunnableTask, - Service as _, - ServiceRunner, + stream::BoxStream, RunnableService, RunnableTask, Service as _, ServiceRunner, StateWatcher, }; use fuel_core_types::{ - fuel_tx::{ - ConsensusParameters, - Transaction, - TxId, - UniqueIdentifier, - }, - fuel_types::{ - BlockHeight, - Bytes32, - }, + fuel_tx::{ConsensusParameters, Transaction, TxId, UniqueIdentifier}, + fuel_types::{BlockHeight, Bytes32}, services::{ block_importer::ImportResult, p2p::{ - GossipData, - GossipsubMessageAcceptance, - GossipsubMessageInfo, + GossipData, GossipsubMessageAcceptance, GossipsubMessageInfo, TransactionGossipData, }, - txpool::{ - ArcPoolTx, - Error, - InsertionResult, - TransactionStatus, - }, + txpool::{ArcPoolTx, Error, InsertionResult, TransactionStatus}, }, tai64::Tai64, }; use parking_lot::Mutex as ParkingMutex; use std::sync::Arc; -use tokio::{ - sync::broadcast, - time::MissedTickBehavior, -}; +use tokio::{sync::broadcast, time::MissedTickBehavior}; use tokio_stream::StreamExt; use update_sender::UpdateSender; -use self::update_sender::{ - MpscChannel, - TxStatusStream, -}; +use self::update_sender::{MpscChannel, TxStatusStream}; mod update_sender; @@ -196,26 +161,29 @@ where } new_connection = self.peer_connections.next() => { - dbg!("New connection: {:?}", new_connection); if let Some(peer_id) = new_connection { should_continue = true; - dbg!("Peer connected, inside TxPoolSyncTask: {:?}", peer_id); + println!("Peer {:?} connected to me", peer_id); - // New connection just happened, request the list of - // pooled transactions from the peer. - let peer_txs = self.p2p.request_pooled_transactions(peer_id).await; - dbg!(&peer_txs); + let mut txs = vec![]; + for tx in self.txpool.lock().txs() { + txs.push(tx.0.to_string()) + } - // TODO: Once the above is implemented, we compare this node's list of - // transactions with the peer's list of transactions, and request the - // transactions that are missing from this node's list. + // let _ = self.p2p.clone(); - for tx in self.txpool.lock().txs() { - dbg!(&tx.0); + // Current issue comes from this part: once we send + // the pooled transactions here, which happens when a + // node connects to this one, the node will be disconnected + // due to it being penalized for sharing a tx. + if !txs.is_empty() { + // let txs = self.txpool.lock().txs(); + println!("Sending my pooled transactions: {:?}", txs); + let _ = self.p2p.send_pooled_transactions(peer_id, txs).await; } + } else { should_continue = false; - dbg!(&new_connection); } } } diff --git a/crates/services/txpool/src/txpool.rs b/crates/services/txpool/src/txpool.rs index e196addaed2..05d4a6c0566 100644 --- a/crates/services/txpool/src/txpool.rs +++ b/crates/services/txpool/src/txpool.rs @@ -1,49 +1,28 @@ use crate::{ - containers::{ - dependency::Dependency, - price_sort::PriceSort, - time_sort::TimeSort, - }, + containers::{dependency::Dependency, price_sort::PriceSort, time_sort::TimeSort}, ports::TxPoolDb, service::TxStatusChange, types::*, - Config, - Error, - TxInfo, + Config, Error, TxInfo, }; use fuel_core_metrics::txpool_metrics::TXPOOL_METRICS; use fuel_core_types::{ - fuel_tx::{ - Chargeable, - Transaction, - }, + fuel_tx::{Chargeable, Transaction}, fuel_types::BlockHeight, fuel_vm::{ checked_transaction::{ - CheckPredicates, - Checked, - CheckedTransaction, - Checks, - IntoChecked, + CheckPredicates, Checked, CheckedTransaction, Checks, IntoChecked, ParallelExecutor, }, PredicateVerificationFailed, }, - services::txpool::{ - ArcPoolTx, - InsertionResult, - }, + services::txpool::{ArcPoolTx, InsertionResult}, tai64::Tai64, }; use fuel_core_types::fuel_vm::checked_transaction::CheckPredicateParams; -use std::{ - cmp::Reverse, - collections::HashMap, - ops::Deref, - sync::Arc, -}; +use std::{cmp::Reverse, collections::HashMap, ops::Deref, sync::Arc}; use tokio_rayon::AsyncRayonHandle; #[derive(Debug, Clone)] @@ -103,7 +82,7 @@ where }); if !tx.is_computed() { - return Err(Error::NoMetadata.into()) + return Err(Error::NoMetadata.into()); } // verify max gas is less than block limit @@ -112,11 +91,11 @@ where tx_gas: tx.max_gas(), block_limit: self.config.chain_config.block_gas_limit, } - .into()) + .into()); } if self.by_hash.contains_key(&tx.id()) { - return Err(Error::NotInsertedTxKnown.into()) + return Err(Error::NotInsertedTxKnown.into()); } let mut max_limit_hit = false; @@ -126,7 +105,7 @@ where // limit is hit, check if we can push out lowest priced tx let lowest_price = self.by_gas_price.lowest_value().unwrap_or_default(); if lowest_price >= tx.price() { - return Err(Error::NotInsertedLimitHit.into()) + return Err(Error::NotInsertedLimitHit.into()); } } if self.config.metrics { @@ -197,7 +176,7 @@ where for remove in removed.iter() { self.remove_tx(&remove.id()); } - return removed + return removed; } Vec::new() } @@ -355,7 +334,7 @@ where let removed = self.remove_inner(&oldest_tx); result.extend(removed.into_iter()); } else { - break + break; } } @@ -384,7 +363,7 @@ pub async fn check_single_tx( config: &Config, ) -> anyhow::Result> { if tx.is_mint() { - return Err(Error::NotSupportedTransactionType.into()) + return Err(Error::NotSupportedTransactionType.into()); } verify_tx_min_gas_price(&tx, config)?; @@ -425,7 +404,7 @@ fn verify_tx_min_gas_price(tx: &Transaction, config: &Config) -> Result<(), Erro TXPOOL_METRICS.gas_price_histogram.observe(price as f64); } if price < config.min_gas_price { - return Err(Error::NotInsertedGasPriceTooLow) + return Err(Error::NotInsertedGasPriceTooLow); } Ok(()) } diff --git a/tests/tests/poa.rs b/tests/tests/poa.rs index 40bde7ce5ae..34cea946f89 100644 --- a/tests/tests/poa.rs +++ b/tests/tests/poa.rs @@ -124,11 +124,12 @@ mod p2p { signing_key: pub_key, }; - let bootstrap_config = make_config("Bootstrap".to_string(), chain_config.clone()); + let disable_block_production = false; + let bootstrap_config = make_config("Bootstrap".to_string(), chain_config.clone(), disable_block_production); let bootstrap = Bootstrap::new(&bootstrap_config).await; let make_node_config = |name: &str| { - let mut config = make_config(name.to_string(), chain_config.clone()); + let mut config = make_config(name.to_string(), chain_config.clone(), disable_block_production); config.block_production = Trigger::Interval { block_time: Duration::from_secs(INTERVAL), }; diff --git a/tests/tests/sync.rs b/tests/tests/sync.rs index eb806c77a89..046cbdb2792 100644 --- a/tests/tests/sync.rs +++ b/tests/tests/sync.rs @@ -28,6 +28,7 @@ async fn test_producer_getting_own_blocks_back() { // Create a producer and a validator that share the same key pair. let secret = SecretKey::random(&mut rng); let pub_key = Input::owner(&secret.public_key()); + let disable_block_production = false; let Nodes { mut producers, mut validators, @@ -38,6 +39,7 @@ async fn test_producer_getting_own_blocks_back() { ProducerSetup::new(secret).with_txs(1).with_name("Alice"), )], [Some(ValidatorSetup::new(pub_key).with_name("Bob"))], + disable_block_production, ) .await; @@ -73,6 +75,7 @@ async fn test_partition_single(num_txs: usize) { // Create a producer and two validators that share the same key pair. let secret = SecretKey::random(&mut rng); let pub_key = Input::owner(&secret.public_key()); + let disable_block_production = false; let Nodes { mut producers, validators, @@ -88,6 +91,7 @@ async fn test_partition_single(num_txs: usize) { Some(ValidatorSetup::new(pub_key).with_name("Bob")), Some(ValidatorSetup::new(pub_key).with_name("Carol")), ], + disable_block_production ) .await; @@ -138,6 +142,7 @@ async fn test_partitions_larger_groups( // Create a producer and a set of validators that share the same key pair. let secret = SecretKey::random(&mut rng); let pub_key = Input::owner(&secret.public_key()); + let disable_block_production = false; let Nodes { mut producers, mut validators, @@ -152,6 +157,7 @@ async fn test_partitions_larger_groups( (0..num_validators).map(|i| { Some(ValidatorSetup::new(pub_key).with_name(format!("{pub_key}:{i}"))) }), + disable_block_production ) .await; @@ -232,6 +238,7 @@ async fn test_multiple_producers_different_keys() { // Create a producer for each key pair and a set of validators that share // the same key pair. + let disable_block_production = false; let Nodes { mut producers, validators, @@ -252,6 +259,7 @@ async fn test_multiple_producers_different_keys() { Some(ValidatorSetup::new(*pub_key).with_name(format!("{pub_key}:{i}"))) }) }), + disable_block_production ) .await; @@ -300,6 +308,7 @@ async fn test_multiple_producers_same_key() { let secret = SecretKey::random(&mut rng); let pub_key = Input::owner(&secret.public_key()); + let disable_block_production = false; let Nodes { mut producers, mut validators, @@ -308,6 +317,7 @@ async fn test_multiple_producers_same_key() { std::iter::repeat(Some(BootstrapSetup::new(pub_key))).take(num_producers), std::iter::repeat(Some(ProducerSetup::new(secret))).take(num_producers), std::iter::repeat(Some(ValidatorSetup::new(pub_key))).take(num_validators), + disable_block_production ) .await; diff --git a/tests/tests/tx_gossip.rs b/tests/tests/tx_gossip.rs index f4fa5f1c103..7fe79572bb8 100644 --- a/tests/tests/tx_gossip.rs +++ b/tests/tests/tx_gossip.rs @@ -1,12 +1,9 @@ use fuel_core::{ chain_config::{CoinConfig, StateConfig}, + p2p_test_helpers::{ + make_nodes, BootstrapSetup, Nodes, ProducerSetup, ValidatorSetup, + }, service::{Config, FuelService}, -use fuel_core::p2p_test_helpers::{ - make_nodes, - BootstrapSetup, - Nodes, - ProducerSetup, - ValidatorSetup, }; use fuel_core_client::client::FuelClient; use fuel_core_poa::Trigger; @@ -18,8 +15,13 @@ use fuel_core_types::{ }, fuel_vm::*, }; -use rand::{rngs::StdRng, Rng, SeedableRng}; -use std::time::Duration; + +use rand::{rngs::StdRng, SeedableRng}; +use std::{ + collections::hash_map::DefaultHasher, + hash::{Hash, Hasher}, + time::Duration, +}; fn create_node_config_from_inputs(inputs: &[Input]) -> Config { let mut node_config = Config::local_node(); @@ -62,21 +64,6 @@ fn create_node_config_from_inputs(inputs: &[Input]) -> Config { node_config.p2p.as_mut().unwrap().enable_mdns = true; node_config } - fuel_tx::*, - fuel_vm::*, -}; -use rand::{ - rngs::StdRng, - SeedableRng, -}; -use std::{ - collections::hash_map::DefaultHasher, - hash::{ - Hash, - Hasher, - }, - time::Duration, -}; #[tokio::test(flavor = "multi_thread")] async fn test_tx_gossiping() { @@ -97,6 +84,7 @@ async fn test_tx_gossiping() { .map(|secret| Input::owner(&secret.public_key())) .collect(); + let disable_block_production = false; // Create a producer for each key pair and a set of validators that share // the same key pair. let Nodes { @@ -119,6 +107,7 @@ async fn test_tx_gossiping() { Some(ValidatorSetup::new(*pub_key).with_name(format!("{pub_key}:{i}"))) }) }), + disable_block_production ) .await; diff --git a/tests/tests/tx_request_response.rs b/tests/tests/tx_request_response.rs index 7a30ef02e47..a05586b5ed7 100644 --- a/tests/tests/tx_request_response.rs +++ b/tests/tests/tx_request_response.rs @@ -1,66 +1,42 @@ use fuel_core::{ - chain_config::{CoinConfig, StateConfig}, - service::{Config, FuelService}, + chain_config::{ + CoinConfig, + StateConfig, + }, + service::{ + Config, + FuelService, + }, +}; +use fuel_core::{ + p2p_test_helpers::{ + make_nodes, BootstrapSetup, Nodes, ProducerSetup, ValidatorSetup, + }, +}; +use std::{ + collections::hash_map::DefaultHasher, + hash::{Hash, Hasher}, }; use fuel_core_client::client::FuelClient; use fuel_core_poa::Trigger; use fuel_core_types::{ fuel_tx::{ field::*, - input::coin::{CoinPredicate, CoinSigned}, + input::coin::{ + CoinPredicate, + CoinSigned, + }, *, }, fuel_vm::*, }; -use rand::{rngs::StdRng, Rng, SeedableRng}; +use rand::{ + rngs::StdRng, + Rng, + SeedableRng, +}; use std::time::Duration; -// Creates a simple node config with no block production. -// This is used for these tests because we don't want the transactions to be -// included in a block. We want them to be included in the mempool and check -// that new connected nodes sync the mempool. -fn create_node_config_from_inputs_no_block_production(inputs: &[Input]) -> Config { - let mut node_config = Config::local_node(); - let mut initial_state = StateConfig::default(); - let mut coin_configs = vec![]; - - for input in inputs { - if let Input::CoinSigned(CoinSigned { - amount, - owner, - asset_id, - utxo_id, - .. - }) - | Input::CoinPredicate(CoinPredicate { - amount, - owner, - asset_id, - utxo_id, - .. - }) = input - { - let coin_config = CoinConfig { - tx_id: Some(*utxo_id.tx_id()), - output_index: Some(utxo_id.output_index()), - tx_pointer_block_height: None, - tx_pointer_tx_idx: None, - maturity: None, - owner: *owner, - amount: *amount, - asset_id: *asset_id, - }; - coin_configs.push(coin_config); - }; - } - - initial_state.coins = Some(coin_configs); - node_config.chain_conf.initial_state = Some(initial_state); - node_config.utxo_validation = true; - node_config.block_production = Trigger::Never; - node_config.p2p.as_mut().unwrap().enable_mdns = true; - node_config -} // This test is set up in such a way that the transaction is not committed // as we've disabled the block production. This is to test that the peer @@ -68,59 +44,79 @@ fn create_node_config_from_inputs_no_block_production(inputs: &[Input]) -> Confi #[tokio::test(flavor = "multi_thread")] async fn test_tx_request() { use futures::StreamExt; - let mut rng = StdRng::seed_from_u64(2322); + // Create a random seed based on the test parameters. + let mut hasher = DefaultHasher::new(); + let num_txs = 1; + let num_validators = 1; + let num_partitions = 1; + (num_txs, num_validators, num_partitions, line!()).hash(&mut hasher); + let mut rng = StdRng::seed_from_u64(hasher.finish()); - let tx = TransactionBuilder::script(vec![], vec![]) - .gas_limit(100) - .gas_price(1) - .add_unsigned_coin_input( - SecretKey::random(&mut rng), - rng.gen(), - 1000, - Default::default(), - Default::default(), - Default::default(), - ) - .add_output(Output::Change { - amount: 0, - asset_id: Default::default(), - to: rng.gen(), - }) - .finalize(); + // Create a set of key pairs. + let secrets: Vec<_> = (0..1).map(|_| SecretKey::random(&mut rng)).collect(); + let pub_keys: Vec<_> = secrets + .clone() + .into_iter() + .map(|secret| Input::owner(&secret.public_key())) + .collect(); - let node_config = create_node_config_from_inputs_no_block_production(tx.inputs()); - let params = node_config.chain_conf.transaction_parameters; - let node_one = FuelService::new_node(node_config).await.unwrap(); - let client_one = FuelClient::from(node_one.bound_address); + // Creates a simple node config with no block production. + // This is used for these tests because we don't want the transactions to be + // included in a block. We want them to be included in the mempool and check + // that new connected nodes sync the mempool. + let disable_block_production = true; + // Create a producer for each key pair and a set of validators that share + // the same key pair. + let Nodes { + mut producers, + mut validators, + bootstrap_nodes: _dont_drop, + } = make_nodes( + pub_keys + .iter() + .map(|pub_key| Some(BootstrapSetup::new(*pub_key))), + secrets.clone().into_iter().enumerate().map(|(i, secret)| { + Some( + ProducerSetup::new(secret) + .with_txs(num_txs) + .with_name(format!("{}:producer", pub_keys[i])), + ) + }), + pub_keys.iter().flat_map(|pub_key| { + (0..num_validators).map(move |i| { + Some(ValidatorSetup::new(*pub_key).with_name(format!("{pub_key}:{i}"))) + }) + }), + disable_block_production, + ) + .await; - let node_config = create_node_config_from_inputs_no_block_production(tx.inputs()); - let node_two = FuelService::new_node(node_config).await.unwrap(); - let client_two = FuelClient::from(node_two.bound_address); + let _client_one = FuelClient::from(validators[0].node.bound_address); - let wait_time = Duration::from_secs(10); + // Temporarily shut down the second node. + producers[0].shutdown().await; - let tx_id = tx.id(¶ms.chain_id); - let tx = tx.into(); - let res = client_one.submit(&tx).await.unwrap(); - dbg!(&res); - tokio::time::sleep(wait_time).await; - // At this point, the transaction should be in the mempool of node_one - // and node_two should request it from node_one. + // Insert transactions into the mempool of the first node. + validators[0].test_txs = producers[0].test_txs.clone(); + let (tx_id, _) = validators[0] + .insert_txs() + .await + .into_iter() + .next() + .expect("Validator is initialized with one transaction"); - // Below code not necessary at this moment. - // let response = client_one.transaction(&tx_id).await.unwrap(); - // assert!(response.is_some()); + println!("tx_id = {:?}", tx_id); - // let mut client_two_subscription = client_two - // .subscribe_transaction_status(&tx_id) - // .await - // .expect("Should be able to subscribe for events"); - // tokio::time::timeout(wait_time, client_two_subscription.next()) - // .await - // .expect("Should await transaction notification in time"); + // Sleep for 2 seconds + tokio::time::sleep(Duration::from_secs(2)).await; - // let response = client_two.transaction(&tx_id).await.unwrap(); - // assert!(response.is_some()); + // Start second node + producers[0].start().await; + + let client_two = FuelClient::from(producers[0].node.bound_address); + + let wait_time = Duration::from_secs(10); + tokio::time::sleep(wait_time).await; } From 27ba1785cc309770f1ca8dfd8e96404af9ba176e Mon Sep 17 00:00:00 2001 From: Rodrigo Araujo Date: Thu, 5 Oct 2023 20:18:34 -0700 Subject: [PATCH 06/18] Working draft. Some cleanups and TODOs left. --- crates/fuel-core/src/p2p_test_helpers.rs | 12 +- .../fuel-core/src/service/adapters/txpool.rs | 70 +++++++-- crates/services/p2p/src/behavior.rs | 40 +++-- crates/services/p2p/src/p2p_service.rs | 147 +++++++++++++----- .../p2p/src/request_response/messages.rs | 25 ++- crates/services/p2p/src/service.rs | 34 +++- crates/services/txpool/src/ports.rs | 28 +++- crates/services/txpool/src/service.rs | 130 ++++++++++++---- crates/services/txpool/src/txpool.rs | 51 ++++-- tests/tests/poa.rs | 12 +- tests/tests/sync.rs | 8 +- tests/tests/tx_gossip.rs | 30 +++- tests/tests/tx_request_response.rs | 94 ++++++----- 13 files changed, 504 insertions(+), 177 deletions(-) diff --git a/crates/fuel-core/src/p2p_test_helpers.rs b/crates/fuel-core/src/p2p_test_helpers.rs index 4f98f24aae0..0d8379d49c7 100644 --- a/crates/fuel-core/src/p2p_test_helpers.rs +++ b/crates/fuel-core/src/p2p_test_helpers.rs @@ -228,7 +228,7 @@ pub async fn make_nodes( .then_some(name) .unwrap_or_else(|| format!("b:{i}")), chain_config.clone(), - disable_block_production + disable_block_production, ); if let Some(BootstrapSetup { pub_key, .. }) = boot { match &mut node_config.chain_conf.consensus { @@ -254,7 +254,7 @@ pub async fn make_nodes( .then_some(name) .unwrap_or_else(|| format!("p:{i}")), chain_config.clone(), - disable_block_production + disable_block_production, ); let mut test_txs = Vec::with_capacity(0); @@ -287,7 +287,7 @@ pub async fn make_nodes( .then_some(name) .unwrap_or_else(|| format!("v:{i}")), chain_config.clone(), - disable_block_production + disable_block_production, ); node_config.block_production = Trigger::Never; node_config.p2p.as_mut().unwrap().bootstrap_nodes = boots.clone(); @@ -309,7 +309,11 @@ pub async fn make_nodes( } } -pub fn make_config(name: String, chain_config: ChainConfig, disable_block_production: bool) -> Config { +pub fn make_config( + name: String, + chain_config: ChainConfig, + disable_block_production: bool, +) -> Config { let mut node_config = Config::local_node(); node_config.chain_conf = chain_config; node_config.utxo_validation = true; diff --git a/crates/fuel-core/src/service/adapters/txpool.rs b/crates/fuel-core/src/service/adapters/txpool.rs index 2a8bbbbc2cb..b16d0fc91c1 100644 --- a/crates/fuel-core/src/service/adapters/txpool.rs +++ b/crates/fuel-core/src/service/adapters/txpool.rs @@ -1,29 +1,55 @@ use crate::{ database::Database, - service::adapters::{BlockImporterAdapter, P2PAdapter}, + service::adapters::{ + BlockImporterAdapter, + P2PAdapter, + }, }; use fuel_core_p2p::PeerId; use fuel_core_services::stream::BoxStream; use fuel_core_storage::{ not_found, - tables::{Coins, ContractsRawCode, Messages, SpentMessages}, - Result as StorageResult, StorageAsRef, + tables::{ + Coins, + ContractsRawCode, + Messages, + SpentMessages, + }, + Result as StorageResult, + StorageAsRef, }; use fuel_core_txpool::ports::BlockImporter; use fuel_core_types::{ - entities::{coins::coin::CompressedCoin, message::Message}, - fuel_tx::{Transaction, UtxoId}, - fuel_types::{BlockHeight, ContractId, Nonce}, + entities::{ + coins::coin::CompressedCoin, + message::Message, + }, + fuel_tx::{ + Transaction, + UtxoId, + }, + fuel_types::{ + BlockHeight, + ContractId, + Nonce, + }, services::{ block_importer::ImportResult, - p2p::{GossipsubMessageAcceptance, GossipsubMessageInfo, TransactionGossipData}, + p2p::{ + GossipsubMessageAcceptance, + GossipsubMessageInfo, + TransactionGossipData, + }, }, }; use std::sync::Arc; impl BlockImporter for BlockImporterAdapter { fn block_events(&self) -> BoxStream> { - use tokio_stream::{wrappers::BroadcastStream, StreamExt}; + use tokio_stream::{ + wrappers::BroadcastStream, + StreamExt, + }; Box::pin( BroadcastStream::new(self.block_importer.subscribe()) .filter_map(|result| result.ok()), @@ -47,7 +73,7 @@ impl fuel_core_txpool::ports::PeerToPeer for P2PAdapter { async fn send_pooled_transactions( &self, peer_id: PeerId, - transactions: Vec, + transactions: Vec, ) -> anyhow::Result<()> { if let Some(service) = &self.service { service @@ -59,7 +85,10 @@ impl fuel_core_txpool::ports::PeerToPeer for P2PAdapter { } fn gossiped_transaction_events(&self) -> BoxStream { - use tokio_stream::{wrappers::BroadcastStream, StreamExt}; + use tokio_stream::{ + wrappers::BroadcastStream, + StreamExt, + }; if let Some(service) = &self.service { Box::pin( BroadcastStream::new(service.subscribe_tx()) @@ -82,8 +111,26 @@ impl fuel_core_txpool::ports::PeerToPeer for P2PAdapter { } } + fn incoming_pooled_transactions(&self) -> BoxStream> { + use tokio_stream::{ + wrappers::BroadcastStream, + StreamExt, + }; + if let Some(service) = &self.service { + Box::pin( + BroadcastStream::new(service.subscribe_to_incoming_pooled_transactions()) + .filter_map(|result| result.ok()), + ) + } else { + fuel_core_services::stream::IntoBoxStream::into_boxed(tokio_stream::pending()) + } + } + fn new_connection(&self) -> BoxStream { - use tokio_stream::{wrappers::BroadcastStream, StreamExt}; + use tokio_stream::{ + wrappers::BroadcastStream, + StreamExt, + }; if let Some(service) = &self.service { Box::pin( BroadcastStream::new(service.subscribe_to_connections()) @@ -117,6 +164,7 @@ impl fuel_core_txpool::ports::PeerToPeer for P2PAdapter { ) -> anyhow::Result<()> { Ok(()) } + // TODO: implement missing methods for this adapter. } impl fuel_core_txpool::ports::TxPoolDb for Database { diff --git a/crates/services/p2p/src/behavior.rs b/crates/services/p2p/src/behavior.rs index 65a9e301c0a..6749bf7c5e1 100644 --- a/crates/services/p2p/src/behavior.rs +++ b/crates/services/p2p/src/behavior.rs @@ -1,25 +1,45 @@ use crate::{ codecs::NetworkCodec, config::Config, - discovery::{DiscoveryBehaviour, DiscoveryConfig}, - gossipsub::{config::build_gossipsub_behaviour, topics::GossipTopic}, - peer_report::{PeerReportBehaviour, PeerReportEvent}, - request_response::messages::{NetworkResponse, RequestMessage}, + discovery::{ + DiscoveryBehaviour, + DiscoveryConfig, + }, + gossipsub::{ + config::build_gossipsub_behaviour, + topics::GossipTopic, + }, + peer_report::{ + PeerReportBehaviour, + PeerReportEvent, + }, + request_response::messages::{ + NetworkResponse, + RequestMessage, + }, }; use fuel_core_types::fuel_types::BlockHeight; use libp2p::{ gossipsub::{ - error::PublishError, Gossipsub, GossipsubEvent, MessageAcceptance, MessageId, + error::PublishError, + Gossipsub, + GossipsubEvent, + MessageAcceptance, + MessageId, }, request_response::{ - ProtocolSupport, RequestId, RequestResponse, RequestResponseConfig, - RequestResponseEvent, ResponseChannel, + ProtocolSupport, + RequestId, + RequestResponse, + RequestResponseConfig, + RequestResponseEvent, + ResponseChannel, }, swarm::NetworkBehaviour, - Multiaddr, PeerId, + Multiaddr, + PeerId, }; use libp2p_kad::KademliaEvent; -use tracing::{debug, error, log::warn}; #[derive(Debug)] pub enum FuelBehaviourEvent { @@ -146,7 +166,7 @@ impl FuelBehaviour { Ok(true) => { tracing::debug!(target: "fuel-p2p", "Sent a report for MessageId: {} from PeerId: {}", msg_id, propagation_source); if should_check_score { - return self.gossipsub.peer_score(propagation_source); + return self.gossipsub.peer_score(propagation_source) } } Ok(false) => { diff --git a/crates/services/p2p/src/p2p_service.rs b/crates/services/p2p/src/p2p_service.rs index 109a88361c7..572ab9ef85f 100644 --- a/crates/services/p2p/src/p2p_service.rs +++ b/crates/services/p2p/src/p2p_service.rs @@ -1,37 +1,75 @@ use crate::{ - behavior::{FuelBehaviour, FuelBehaviourEvent}, + behavior::{ + FuelBehaviour, + FuelBehaviourEvent, + }, codecs::NetworkCodec, - config::{build_transport, Config}, + config::{ + build_transport, + Config, + }, gossipsub::{ - messages::{GossipsubBroadcastRequest, GossipsubMessage as FuelGossipsubMessage}, + messages::{ + GossipsubBroadcastRequest, + GossipsubMessage as FuelGossipsubMessage, + }, topics::GossipsubTopics, }, - peer_manager::{PeerManager, Punisher}, + peer_manager::{ + PeerManager, + Punisher, + }, peer_report::PeerReportEvent, request_response::messages::{ - NetworkResponse, OutboundResponse, RequestError, RequestMessage, - ResponseChannelItem, ResponseError, ResponseMessage, + NetworkResponse, + OutboundResponse, + RequestError, + RequestMessage, + ResponseChannelItem, + ResponseError, + ResponseMessage, }, }; use fuel_core_metrics::p2p_metrics::p2p_metrics; use fuel_core_types::{ - fuel_types::BlockHeight, services::p2p::peer_reputation::AppScore, + fuel_types::BlockHeight, + services::p2p::peer_reputation::AppScore, }; use futures::prelude::*; use libp2p::{ gossipsub::{ - error::PublishError, GossipsubEvent, MessageAcceptance, MessageId, TopicHash, + error::PublishError, + GossipsubEvent, + MessageAcceptance, + MessageId, + TopicHash, }, multiaddr::Protocol, request_response::{ - RequestId, RequestResponseEvent, RequestResponseMessage, ResponseChannel, + RequestId, + RequestResponseEvent, + RequestResponseMessage, + ResponseChannel, + }, + swarm::{ + AddressScore, + ConnectionLimits, + SwarmBuilder, + SwarmEvent, }, - swarm::{AddressScore, ConnectionLimits, SwarmBuilder, SwarmEvent}, - Multiaddr, PeerId, Swarm, + Multiaddr, + PeerId, + Swarm, }; use rand::seq::IteratorRandom; -use std::{collections::HashMap, time::Duration}; -use tracing::{debug, warn}; +use std::{ + collections::HashMap, + time::Duration, +}; +use tracing::{ + debug, + warn, +}; impl Punisher for Swarm> { fn ban_peer(&mut self, peer_id: PeerId) { @@ -219,7 +257,7 @@ impl FuelP2PService { loop { if let SwarmEvent::NewListenAddr { .. } = self.swarm.select_next_some().await { - break; + break } } } @@ -275,7 +313,7 @@ impl FuelP2PService { let peers_count = self.peer_manager.total_peers_connected(); if peers_count == 0 { - return Err(RequestError::NoPeersConnected); + return Err(RequestError::NoPeersConnected) } let mut range = rand::thread_rng(); @@ -317,16 +355,16 @@ impl FuelP2PService { .is_err() { debug!("Failed to send ResponseMessage for {:?}", request_id); - return Err(ResponseError::SendingResponseFailed); + return Err(ResponseError::SendingResponseFailed) } } (Ok(_), None) => { debug!("ResponseChannel for {:?} does not exist!", request_id); - return Err(ResponseError::ResponseChannelDoesNotExist); + return Err(ResponseError::ResponseChannelDoesNotExist) } (Err(e), _) => { debug!("Failed to convert to IntermediateResponse with {:?}", e); - return Err(ResponseError::ConversionToIntermediateFailed); + return Err(ResponseError::ConversionToIntermediateFailed) } } @@ -507,7 +545,7 @@ impl FuelP2PService { return Some(FuelP2PEvent::PeerInfoUpdated { peer_id, block_height, - }); + }) } PeerReportEvent::PeerConnected { peer_id, @@ -521,7 +559,7 @@ impl FuelP2PService { ) { let _ = self.swarm.disconnect_peer_id(peer_id); } else if initial_connection { - return Some(FuelP2PEvent::PeerConnected(peer_id)); + return Some(FuelP2PEvent::PeerConnected(peer_id)) } } PeerReportEvent::PeerDisconnected { peer_id } => { @@ -531,7 +569,7 @@ impl FuelP2PService { if self.peer_manager.handle_peer_disconnect(peer_id) { let _ = self.swarm.dial(peer_id); } - return Some(FuelP2PEvent::PeerDisconnected(peer_id)); + return Some(FuelP2PEvent::PeerDisconnected(peer_id)) } } } @@ -547,7 +585,7 @@ impl FuelP2PService { return Some(FuelP2PEvent::RequestMessage { request_id, request_message: request, - }); + }) } RequestResponseMessage::Response { request_id, @@ -645,47 +683,81 @@ mod tests { codecs::postcard::PostcardCodec, config::Config, gossipsub::{ - messages::{GossipsubBroadcastRequest, GossipsubMessage}, + messages::{ + GossipsubBroadcastRequest, + GossipsubMessage, + }, topics::{ - GossipTopic, CON_VOTE_GOSSIP_TOPIC, NEW_BLOCK_GOSSIP_TOPIC, + GossipTopic, + CON_VOTE_GOSSIP_TOPIC, + NEW_BLOCK_GOSSIP_TOPIC, NEW_TX_GOSSIP_TOPIC, }, }, p2p_service::FuelP2PEvent, peer_manager::PeerInfo, request_response::messages::{ - OutboundResponse, RequestMessage, ResponseChannelItem, + OutboundResponse, + RequestMessage, + ResponseChannelItem, }, service::to_message_acceptance, }; use fuel_core_types::{ blockchain::{ block::Block, - consensus::{poa::PoAConsensus, Consensus, ConsensusVote}, - header::{BlockHeader, PartialBlockHeader}, + consensus::{ + poa::PoAConsensus, + Consensus, + ConsensusVote, + }, + header::{ + BlockHeader, + PartialBlockHeader, + }, primitives::BlockId, - SealedBlock, SealedBlockHeader, + SealedBlock, + SealedBlockHeader, + }, + fuel_tx::{ + Transaction, + TransactionBuilder, }, - fuel_tx::{Transaction, TransactionBuilder}, services::p2p::GossipsubMessageAcceptance, }; - use futures::{future::join_all, StreamExt}; + use futures::{ + future::join_all, + StreamExt, + }; use libp2p::{ - gossipsub::{error::PublishError, Topic}, + gossipsub::{ + error::PublishError, + Topic, + }, identity::Keypair, swarm::SwarmEvent, - Multiaddr, PeerId, + Multiaddr, + PeerId, }; use libp2p_swarm::PendingInboundConnectionError; use rand::Rng; use std::{ collections::HashSet, - net::{IpAddr, Ipv4Addr, SocketAddrV4, TcpListener}, + net::{ + IpAddr, + Ipv4Addr, + SocketAddrV4, + TcpListener, + }, ops::Range, sync::Arc, time::Duration, }; - use tokio::sync::{mpsc, oneshot, watch}; + use tokio::sync::{ + mpsc, + oneshot, + watch, + }; use tracing_attributes::instrument; type P2PService = FuelP2PService; @@ -742,7 +814,7 @@ mod tests { match fuel_p2p_service.swarm.select_next_some().await { SwarmEvent::NewListenAddr { .. } => { // listener address registered, we are good to go - break; + break } SwarmEvent::Behaviour(_) => {} other_event => { @@ -1047,7 +1119,10 @@ mod tests { #[tokio::test] #[instrument] async fn nodes_cannot_connect_due_to_different_checksum() { - use libp2p::{swarm::DialError, TransportError}; + use libp2p::{ + swarm::DialError, + TransportError, + }; // Node A let mut p2p_config = Config::default_initialized("nodes_cannot_connect_due_to_different_checksum"); diff --git a/crates/services/p2p/src/request_response/messages.rs b/crates/services/p2p/src/request_response/messages.rs index 6c23fe616d6..3025c23f4d8 100644 --- a/crates/services/p2p/src/request_response/messages.rs +++ b/crates/services/p2p/src/request_response/messages.rs @@ -1,20 +1,33 @@ -use std::{ops::Range, sync::Arc}; +use std::{ + ops::Range, + sync::Arc, +}; use fuel_core_types::{ - blockchain::{primitives::BlockId, SealedBlock, SealedBlockHeader}, + blockchain::{ + primitives::BlockId, + SealedBlock, + SealedBlockHeader, + }, fuel_tx::Transaction, fuel_types::BlockHeight, }; use libp2p::PeerId; -use serde::{Deserialize, Serialize}; -use serde_with::{serde_as, FromInto}; +use serde::{ + Deserialize, + Serialize, +}; +use serde_with::{ + serde_as, + FromInto, +}; use thiserror::Error; use tokio::sync::oneshot; pub(crate) const REQUEST_RESPONSE_PROTOCOL_ID: &[u8] = b"/fuel/req_res/0.0.1"; /// Max Size in Bytes of the Request Message -pub(crate) const MAX_REQUEST_SIZE: usize = core::mem::size_of::(); +pub(crate) const MAX_REQUEST_SIZE: usize = 32 * 1024; // Peer receives a `RequestMessage`. // It prepares a response in form of `OutboundResponse` @@ -32,7 +45,7 @@ pub enum RequestMessage { Block(BlockHeight), SealedHeaders(Range), Transactions(#[serde_as(as = "FromInto<[u8; 32]>")] BlockId), - PooledTransactions(Vec), // temp as string + PooledTransactions(Vec), } /// Final Response Message that p2p service sends to the Orchestrator diff --git a/crates/services/p2p/src/service.rs b/crates/services/p2p/src/service.rs index 4af23d27ad1..3fb2eaf4c05 100644 --- a/crates/services/p2p/src/service.rs +++ b/crates/services/p2p/src/service.rs @@ -100,7 +100,7 @@ enum TaskRequest { }, SendPooledTransactions { to_peer: PeerId, - transactions: Vec, // temp string + transactions: Vec, }, GetTransactions { block_id: BlockId, @@ -252,6 +252,11 @@ pub trait Broadcast: Send { fn tx_broadcast(&self, transaction: TransactionGossipData) -> anyhow::Result<()>; fn connection_broadcast(&self, peer_id: PeerId) -> anyhow::Result<()>; + + fn incoming_pooled_transactions( + &self, + transactions: Vec, + ) -> anyhow::Result<()>; } impl Broadcast for SharedState { @@ -281,6 +286,14 @@ impl Broadcast for SharedState { self.connection_broadcast.send(peer_id)?; Ok(()) } + + fn incoming_pooled_transactions( + &self, + transactions: Vec, + ) -> anyhow::Result<()> { + self.incoming_pooled_transactions.send(transactions)?; + Ok(()) + } } /// Orchestrates various p2p-related events between the inner `P2pService` @@ -325,6 +338,7 @@ impl Task, D, SharedState> { let (tx_broadcast, _) = broadcast::channel(100); let (block_height_broadcast, _) = broadcast::channel(100); let (connection_broadcast, _) = broadcast::channel(100); + let (incoming_pooled_transactions, _) = broadcast::channel(100); // Hardcoded for now, but left here to be configurable in the future. // TODO: https://github.com/FuelLabs/fuel-core/issues/1340 @@ -352,6 +366,7 @@ impl Task, D, SharedState> { reserved_peers_broadcast, block_height_broadcast, connection_broadcast, + incoming_pooled_transactions, }, max_headers_per_request, heartbeat_check_interval, @@ -589,9 +604,9 @@ where } } RequestMessage::PooledTransactions(transactions) => { - // Temp `todo!` because with this new implementation we're - // never getting to this point, as the node was already disconnected. - todo!("Pooled transactions are not yet implemented"); + // Received pooled transactions from a peer. Send those + // transactions to the txpool service. + self.broadcast.incoming_pooled_transactions(transactions)?; } RequestMessage::SealedHeaders(range) => { let max_len = self.max_headers_per_request.try_into().expect("u32 should always fit into usize"); @@ -669,6 +684,8 @@ pub struct SharedState { block_height_broadcast: broadcast::Sender, /// Sender of new incoming connections connection_broadcast: broadcast::Sender, + /// Sender of incoming pooled Transactions + incoming_pooled_transactions: broadcast::Sender>, } impl SharedState { @@ -748,9 +765,8 @@ impl SharedState { pub async fn send_pooled_transactions_to_peer( &self, peer_id: Vec, - transactions: Vec, + transactions: Vec, ) -> anyhow::Result<()> { - // temp string let to_peer = PeerId::from_bytes(&peer_id).expect("Valid PeerId"); self.request_sender @@ -814,6 +830,12 @@ impl SharedState { self.connection_broadcast.subscribe() } + pub fn subscribe_to_incoming_pooled_transactions( + &self, + ) -> broadcast::Receiver> { + self.incoming_pooled_transactions.subscribe() + } + pub fn report_peer( &self, peer_id: FuelPeerId, diff --git a/crates/services/txpool/src/ports.rs b/crates/services/txpool/src/ports.rs index 44d929a1e0c..1608e7a9908 100644 --- a/crates/services/txpool/src/ports.rs +++ b/crates/services/txpool/src/ports.rs @@ -2,12 +2,27 @@ use fuel_core_p2p::PeerId; use fuel_core_services::stream::BoxStream; use fuel_core_storage::Result as StorageResult; use fuel_core_types::{ - entities::{coins::coin::CompressedCoin, message::Message}, - fuel_tx::{Transaction, UtxoId}, - fuel_types::{BlockHeight, Bytes32, ContractId, Nonce}, + entities::{ + coins::coin::CompressedCoin, + message::Message, + }, + fuel_tx::{ + Transaction, + UtxoId, + }, + fuel_types::{ + BlockHeight, + Bytes32, + ContractId, + Nonce, + }, services::{ block_importer::ImportResult, - p2p::{GossipsubMessageAcceptance, GossipsubMessageInfo, NetworkData}, + p2p::{ + GossipsubMessageAcceptance, + GossipsubMessageInfo, + NetworkData, + }, txpool::TransactionStatus, }, }; @@ -33,11 +48,14 @@ pub trait PeerToPeer: Send + Sync { /// Streams new connections to the node. fn new_connection(&self) -> BoxStream; + /// Streams incoming pooled transactions. + fn incoming_pooled_transactions(&self) -> BoxStream>; + /// Send pooled transactions to a peer. async fn send_pooled_transactions( &self, peer_id: PeerId, - transactions: Vec, + transactions: Vec, ) -> anyhow::Result<()>; } diff --git a/crates/services/txpool/src/service.rs b/crates/services/txpool/src/service.rs index 6c8db810f39..94e5a1584c7 100644 --- a/crates/services/txpool/src/service.rs +++ b/crates/services/txpool/src/service.rs @@ -1,37 +1,72 @@ use crate::{ - ports::{BlockImporter, PeerToPeer, TxPoolDb}, + ports::{ + BlockImporter, + PeerToPeer, + TxPoolDb, + }, transaction_selector::select_transactions, - txpool::{check_single_tx, check_transactions}, - Config, Error as TxPoolError, TxInfo, TxPool, + txpool::{ + check_single_tx, + check_transactions, + }, + Config, + Error as TxPoolError, + TxInfo, + TxPool, }; use fuel_core_p2p::PeerId; use fuel_core_services::{ - stream::BoxStream, RunnableService, RunnableTask, Service as _, ServiceRunner, + stream::BoxStream, + RunnableService, + RunnableTask, + Service as _, + ServiceRunner, StateWatcher, }; use fuel_core_types::{ - fuel_tx::{ConsensusParameters, Transaction, TxId, UniqueIdentifier}, - fuel_types::{BlockHeight, Bytes32}, + fuel_tx::{ + ConsensusParameters, + Transaction, + TxId, + UniqueIdentifier, + }, + fuel_types::{ + BlockHeight, + Bytes32, + }, services::{ block_importer::ImportResult, p2p::{ - GossipData, GossipsubMessageAcceptance, GossipsubMessageInfo, + GossipData, + GossipsubMessageAcceptance, + GossipsubMessageInfo, TransactionGossipData, }, - txpool::{ArcPoolTx, Error, InsertionResult, TransactionStatus}, + txpool::{ + ArcPoolTx, + Error, + InsertionResult, + TransactionStatus, + }, }, tai64::Tai64, }; use parking_lot::Mutex as ParkingMutex; use std::sync::Arc; -use tokio::{sync::broadcast, time::MissedTickBehavior}; +use tokio::{ + sync::broadcast, + time::MissedTickBehavior, +}; use tokio_stream::StreamExt; use update_sender::UpdateSender; -use self::update_sender::{MpscChannel, TxStatusStream}; +use self::update_sender::{ + MpscChannel, + TxStatusStream, +}; mod update_sender; @@ -116,23 +151,25 @@ impl Clone for SharedState { pub struct TxPoolSyncTask { ttl_timer: tokio::time::Interval, peer_connections: BoxStream, - txpool: Arc>>, - p2p: Arc, + incoming_pooled_transactions: BoxStream>, + shared: SharedState, } #[async_trait::async_trait] impl RunnableService for TxPoolSyncTask where - DB: TxPoolDb, + DB: TxPoolDb + Clone, P2P: PeerToPeer + Send + Sync, { const NAME: &'static str = "TxPoolSync"; - type SharedData = (); + type SharedData = SharedState; type Task = TxPoolSyncTask; type TaskParams = (); - fn shared_data(&self) -> Self::SharedData {} + fn shared_data(&self) -> Self::SharedData { + self.shared.clone() + } async fn into_task( mut self, @@ -160,26 +197,39 @@ where should_continue = false; } + incoming_pooled_transactions = self.incoming_pooled_transactions.next() => { + if let Some(incoming_pooled_transactions) = incoming_pooled_transactions { + let current_height = self.shared.db.current_block_height()?; + + let mut res = Vec::new(); + for tx in incoming_pooled_transactions.into_iter() { + // verify tx + let checked_tx = check_single_tx(tx, current_height, &self.shared.config).await.unwrap(); // TODO: remove temp unwrap + + res.push(self.shared.txpool.lock().insert_inner(checked_tx)); + } + + should_continue = true; + + } else { + should_continue = false; + } + } + new_connection = self.peer_connections.next() => { if let Some(peer_id) = new_connection { should_continue = true; - println!("Peer {:?} connected to me", peer_id); let mut txs = vec![]; - for tx in self.txpool.lock().txs() { - txs.push(tx.0.to_string()) + for tx in self.shared.txpool.lock().txs() { + txs.push(Transaction::from(&*tx.1.tx)); } - // let _ = self.p2p.clone(); - - // Current issue comes from this part: once we send - // the pooled transactions here, which happens when a - // node connects to this one, the node will be disconnected - // due to it being penalized for sharing a tx. if !txs.is_empty() { - // let txs = self.txpool.lock().txs(); println!("Sending my pooled transactions: {:?}", txs); - let _ = self.p2p.send_pooled_transactions(peer_id, txs).await; + // TODO: we still have to make sure we're splitting the txs into + // MAX_REQUEST_SIZE. + let _ = self.shared.p2p.send_pooled_transactions(peer_id, txs).await; } } else { @@ -203,7 +253,7 @@ where pub struct Task where - DB: TxPoolDb + 'static, + DB: TxPoolDb + 'static + Clone, P2P: PeerToPeer + Send + Sync + 'static, { gossiped_tx_stream: BoxStream, @@ -246,7 +296,7 @@ where impl RunnableTask for Task where P2P: PeerToPeer + Send + Sync, - DB: TxPoolDb, + DB: TxPoolDb + Clone, { async fn run(&mut self, watcher: &mut StateWatcher) -> anyhow::Result { let should_continue; @@ -505,18 +555,28 @@ pub fn new_txpool_syncing_service( config: Config, txpool: Arc>>, p2p: Arc, + db: DB, ) -> TxPoolSyncService where - DB: TxPoolDb + 'static, + DB: TxPoolDb + 'static + Clone, P2P: PeerToPeer + 'static, { let mut ttl_timer = tokio::time::interval(config.transaction_ttl); ttl_timer.set_missed_tick_behavior(MissedTickBehavior::Skip); + let number_of_active_subscription = config.number_of_active_subscription; + let consensus_params = config.chain_config.consensus_parameters.clone(); let tx_sync_task = TxPoolSyncTask { peer_connections: p2p.new_connection(), - txpool, - p2p, + incoming_pooled_transactions: p2p.incoming_pooled_transactions(), ttl_timer, + shared: SharedState { + db, + config, + txpool, + p2p, + tx_status_sender: TxStatusChange::new(number_of_active_subscription), + consensus_params, + }, }; TxPoolSyncService::new(tx_sync_task) @@ -542,8 +602,12 @@ where let number_of_active_subscription = config.number_of_active_subscription; let txpool = Arc::new(ParkingMutex::new(TxPool::new(config.clone(), db.clone()))); - let txpool_sync_task = - new_txpool_syncing_service(config.clone(), txpool.clone(), p2p.clone()); + let txpool_sync_task = new_txpool_syncing_service( + config.clone(), + txpool.clone(), + p2p.clone(), + db.clone(), + ); let task = Task { gossiped_tx_stream, diff --git a/crates/services/txpool/src/txpool.rs b/crates/services/txpool/src/txpool.rs index f1379a46b44..58230ab8e7b 100644 --- a/crates/services/txpool/src/txpool.rs +++ b/crates/services/txpool/src/txpool.rs @@ -1,27 +1,48 @@ use crate::{ - containers::{dependency::Dependency, price_sort::PriceSort, time_sort::TimeSort}, + containers::{ + dependency::Dependency, + price_sort::PriceSort, + time_sort::TimeSort, + }, ports::TxPoolDb, service::TxStatusChange, types::*, - Config, Error, TxInfo, + Config, + Error, + TxInfo, }; use fuel_core_types::{ - fuel_tx::{Chargeable, Transaction}, + fuel_tx::{ + Chargeable, + Transaction, + }, fuel_types::BlockHeight, fuel_vm::{ checked_transaction::{ - CheckPredicates, Checked, CheckedTransaction, Checks, IntoChecked, + CheckPredicates, + Checked, + CheckedTransaction, + Checks, + IntoChecked, ParallelExecutor, }, PredicateVerificationFailed, }, - services::txpool::{ArcPoolTx, InsertionResult}, + services::txpool::{ + ArcPoolTx, + InsertionResult, + }, tai64::Tai64, }; use fuel_core_metrics::txpool_metrics::txpool_metrics; use fuel_core_types::fuel_vm::checked_transaction::CheckPredicateParams; -use std::{cmp::Reverse, collections::HashMap, ops::Deref, sync::Arc}; +use std::{ + cmp::Reverse, + collections::HashMap, + ops::Deref, + sync::Arc, +}; use tokio_rayon::AsyncRayonHandle; #[derive(Debug, Clone)] @@ -66,7 +87,7 @@ where #[tracing::instrument(level = "info", skip_all, fields(tx_id = %tx.id()), ret, err)] // this is atomic operation. Return removed(pushed out/replaced) transactions - fn insert_inner( + pub fn insert_inner( &mut self, tx: Checked, ) -> anyhow::Result { @@ -81,7 +102,7 @@ where }); if !tx.is_computed() { - return Err(Error::NoMetadata.into()); + return Err(Error::NoMetadata.into()) } // verify max gas is less than block limit @@ -90,11 +111,11 @@ where tx_gas: tx.max_gas(), block_limit: self.config.chain_config.block_gas_limit, } - .into()); + .into()) } if self.by_hash.contains_key(&tx.id()) { - return Err(Error::NotInsertedTxKnown.into()); + return Err(Error::NotInsertedTxKnown.into()) } let mut max_limit_hit = false; @@ -104,7 +125,7 @@ where // limit is hit, check if we can push out lowest priced tx let lowest_price = self.by_gas_price.lowest_value().unwrap_or_default(); if lowest_price >= tx.price() { - return Err(Error::NotInsertedLimitHit.into()); + return Err(Error::NotInsertedLimitHit.into()) } } if self.config.metrics { @@ -175,7 +196,7 @@ where for remove in removed.iter() { self.remove_tx(&remove.id()); } - return removed; + return removed } Vec::new() } @@ -333,7 +354,7 @@ where let removed = self.remove_inner(&oldest_tx); result.extend(removed.into_iter()); } else { - break; + break } } @@ -362,7 +383,7 @@ pub async fn check_single_tx( config: &Config, ) -> anyhow::Result> { if tx.is_mint() { - return Err(Error::NotSupportedTransactionType.into()); + return Err(Error::NotSupportedTransactionType.into()) } verify_tx_min_gas_price(&tx, config)?; @@ -407,7 +428,7 @@ fn verify_tx_min_gas_price(tx: &Transaction, config: &Config) -> Result<(), Erro txpool_metrics().gas_price_histogram.observe(price as f64); } if price < config.min_gas_price { - return Err(Error::NotInsertedGasPriceTooLow); + return Err(Error::NotInsertedGasPriceTooLow) } Ok(()) } diff --git a/tests/tests/poa.rs b/tests/tests/poa.rs index 34cea946f89..a47e13e72f6 100644 --- a/tests/tests/poa.rs +++ b/tests/tests/poa.rs @@ -125,11 +125,19 @@ mod p2p { }; let disable_block_production = false; - let bootstrap_config = make_config("Bootstrap".to_string(), chain_config.clone(), disable_block_production); + let bootstrap_config = make_config( + "Bootstrap".to_string(), + chain_config.clone(), + disable_block_production, + ); let bootstrap = Bootstrap::new(&bootstrap_config).await; let make_node_config = |name: &str| { - let mut config = make_config(name.to_string(), chain_config.clone(), disable_block_production); + let mut config = make_config( + name.to_string(), + chain_config.clone(), + disable_block_production, + ); config.block_production = Trigger::Interval { block_time: Duration::from_secs(INTERVAL), }; diff --git a/tests/tests/sync.rs b/tests/tests/sync.rs index 046cbdb2792..d6bbc300162 100644 --- a/tests/tests/sync.rs +++ b/tests/tests/sync.rs @@ -91,7 +91,7 @@ async fn test_partition_single(num_txs: usize) { Some(ValidatorSetup::new(pub_key).with_name("Bob")), Some(ValidatorSetup::new(pub_key).with_name("Carol")), ], - disable_block_production + disable_block_production, ) .await; @@ -157,7 +157,7 @@ async fn test_partitions_larger_groups( (0..num_validators).map(|i| { Some(ValidatorSetup::new(pub_key).with_name(format!("{pub_key}:{i}"))) }), - disable_block_production + disable_block_production, ) .await; @@ -259,7 +259,7 @@ async fn test_multiple_producers_different_keys() { Some(ValidatorSetup::new(*pub_key).with_name(format!("{pub_key}:{i}"))) }) }), - disable_block_production + disable_block_production, ) .await; @@ -317,7 +317,7 @@ async fn test_multiple_producers_same_key() { std::iter::repeat(Some(BootstrapSetup::new(pub_key))).take(num_producers), std::iter::repeat(Some(ProducerSetup::new(secret))).take(num_producers), std::iter::repeat(Some(ValidatorSetup::new(pub_key))).take(num_validators), - disable_block_production + disable_block_production, ) .await; diff --git a/tests/tests/tx_gossip.rs b/tests/tests/tx_gossip.rs index 7fe79572bb8..5fa13bc49d1 100644 --- a/tests/tests/tx_gossip.rs +++ b/tests/tests/tx_gossip.rs @@ -1,25 +1,41 @@ use fuel_core::{ - chain_config::{CoinConfig, StateConfig}, + chain_config::{ + CoinConfig, + StateConfig, + }, p2p_test_helpers::{ - make_nodes, BootstrapSetup, Nodes, ProducerSetup, ValidatorSetup, + make_nodes, + BootstrapSetup, + Nodes, + ProducerSetup, + ValidatorSetup, }, - service::{Config, FuelService}, + service::Config, }; use fuel_core_client::client::FuelClient; use fuel_core_poa::Trigger; use fuel_core_types::{ fuel_tx::{ field::*, - input::coin::{CoinPredicate, CoinSigned}, + input::coin::{ + CoinPredicate, + CoinSigned, + }, *, }, fuel_vm::*, }; -use rand::{rngs::StdRng, SeedableRng}; +use rand::{ + rngs::StdRng, + SeedableRng, +}; use std::{ collections::hash_map::DefaultHasher, - hash::{Hash, Hasher}, + hash::{ + Hash, + Hasher, + }, time::Duration, }; @@ -107,7 +123,7 @@ async fn test_tx_gossiping() { Some(ValidatorSetup::new(*pub_key).with_name(format!("{pub_key}:{i}"))) }) }), - disable_block_production + disable_block_production, ) .await; diff --git a/tests/tests/tx_request_response.rs b/tests/tests/tx_request_response.rs index a05586b5ed7..96c566632ca 100644 --- a/tests/tests/tx_request_response.rs +++ b/tests/tests/tx_request_response.rs @@ -1,21 +1,9 @@ -use fuel_core::{ - chain_config::{ - CoinConfig, - StateConfig, - }, - service::{ - Config, - FuelService, - }, -}; -use fuel_core::{ - p2p_test_helpers::{ - make_nodes, BootstrapSetup, Nodes, ProducerSetup, ValidatorSetup, - }, -}; -use std::{ - collections::hash_map::DefaultHasher, - hash::{Hash, Hasher}, +use fuel_core::p2p_test_helpers::{ + make_nodes, + BootstrapSetup, + Nodes, + ProducerSetup, + ValidatorSetup, }; use fuel_core_client::client::FuelClient; use fuel_core_poa::Trigger; @@ -32,18 +20,22 @@ use fuel_core_types::{ }; use rand::{ rngs::StdRng, - Rng, SeedableRng, }; -use std::time::Duration; - +use std::{ + collections::hash_map::DefaultHasher, + hash::{ + Hash, + Hasher, + }, + time::Duration, +}; // This test is set up in such a way that the transaction is not committed // as we've disabled the block production. This is to test that the peer // will request this transaction from the other peer upon connection. #[tokio::test(flavor = "multi_thread")] async fn test_tx_request() { - use futures::StreamExt; // Create a random seed based on the test parameters. let mut hasher = DefaultHasher::new(); let num_txs = 1; @@ -89,34 +81,60 @@ async fn test_tx_request() { }), disable_block_production, ) - .await; + .await; - let _client_one = FuelClient::from(validators[0].node.bound_address); + let client_one = FuelClient::from(validators[0].node.bound_address); // Temporarily shut down the second node. - producers[0].shutdown().await; - - + producers[0].shutdown().await; // Insert transactions into the mempool of the first node. validators[0].test_txs = producers[0].test_txs.clone(); - let (tx_id, _) = validators[0] - .insert_txs() - .await - .into_iter() - .next() - .expect("Validator is initialized with one transaction"); - - println!("tx_id = {:?}", tx_id); + let _tx = validators[0].insert_txs().await; // Sleep for 2 seconds tokio::time::sleep(Duration::from_secs(2)).await; // Start second node - producers[0].start().await; + producers[0].start().await; let client_two = FuelClient::from(producers[0].node.bound_address); - let wait_time = Duration::from_secs(10); - tokio::time::sleep(wait_time).await; + // Produce a new block with the participating nodes. + // It's expected that they will: + // 1. Connect to each other; + // 2. Share the transaction; + // 3. Produce a block with the transaction; + let _ = client_one.produce_blocks(1, None).await.unwrap(); + let _ = client_two.produce_blocks(1, None).await.unwrap(); + + let chain_id = validators[0] + .config + .chain_conf + .consensus_parameters + .chain_id(); + + let request = fuel_core_client::client::pagination::PaginationRequest { + cursor: None, + results: 10, + direction: fuel_core_client::client::pagination::PageDirection::Forward, + }; + + let first_node_txs = client_two.transactions(request.clone()).await.unwrap(); + let second_node_txs = client_one.transactions(request).await.unwrap(); + + let first_node_tx = first_node_txs + .results + .first() + .expect("txs should have at least one result") + .transaction + .id(&chain_id); + let second_node_tx = second_node_txs + .results + .first() + .expect("txs should have at least one result") + .transaction + .id(&chain_id); + + assert_eq!(first_node_tx, second_node_tx); } From ff93950fd62fd2e384df79f0e54a832042b35b6f Mon Sep 17 00:00:00 2001 From: Rodrigo Araujo Date: Fri, 6 Oct 2023 11:27:11 -0700 Subject: [PATCH 07/18] Remove the need for a response channel when sending a one way request msg --- crates/services/p2p/src/codecs/postcard.rs | 18 --------------- crates/services/p2p/src/p2p_service.rs | 22 +++++++++---------- .../p2p/src/request_response/messages.rs | 4 ---- crates/services/p2p/src/service.rs | 20 +++++++++++++++-- 4 files changed, 29 insertions(+), 35 deletions(-) diff --git a/crates/services/p2p/src/codecs/postcard.rs b/crates/services/p2p/src/codecs/postcard.rs index 315efd64d84..930a8c876a9 100644 --- a/crates/services/p2p/src/codecs/postcard.rs +++ b/crates/services/p2p/src/codecs/postcard.rs @@ -220,15 +220,6 @@ impl RequestResponseConverter for PostcardCodec { .transpose()?; Ok(ResponseMessage::SealedHeaders(response)) } - NetworkResponse::PooledTransactions(tx_bytes) => { - let response = if let Some(tx_bytes) = tx_bytes { - Some(self.deserialize(tx_bytes)?) - } else { - None - }; - - Ok(ResponseMessage::PooledTransactions(response)) - } } } @@ -262,15 +253,6 @@ impl RequestResponseConverter for PostcardCodec { .transpose()?; Ok(NetworkResponse::Headers(response)) } - OutboundResponse::PooledTransactions(transactions) => { - let response = if let Some(transactions) = transactions { - Some(self.serialize(transactions.as_ref())?) - } else { - None - }; - - Ok(NetworkResponse::PooledTransactions(response)) - } } } } diff --git a/crates/services/p2p/src/p2p_service.rs b/crates/services/p2p/src/p2p_service.rs index c9b683445c8..b1185812391 100644 --- a/crates/services/p2p/src/p2p_service.rs +++ b/crates/services/p2p/src/p2p_service.rs @@ -298,6 +298,17 @@ impl FuelP2PService { } } + /// Sends a one way message to a peer. + /// It leverages the `RequestMessage` type to send the message. + /// But, unlike `send_request_msg`, it does not expect a + /// response through a response channel. + pub fn send_msg(&mut self, peer_id: PeerId, message_request: RequestMessage) -> RequestId { + self + .swarm + .behaviour_mut() + .send_request_msg(message_request, &peer_id) + } + /// Sends RequestMessage to a peer /// If the peer is not defined it will pick one at random pub fn send_request_msg( @@ -598,17 +609,6 @@ impl FuelP2PService { ); } } - ( - Some(ResponseChannelItem::PooledTransactions(channel)), - Ok(ResponseMessage::PooledTransactions(transactions)), - ) => { - if channel.send(transactions).is_err() { - debug!( - "Failed to send through the channel for {:?}", - request_id - ); - } - } ( Some(ResponseChannelItem::Transactions(channel)), Ok(ResponseMessage::Transactions(transactions)), diff --git a/crates/services/p2p/src/request_response/messages.rs b/crates/services/p2p/src/request_response/messages.rs index 2cf3fc7bc64..1e9f162eca9 100644 --- a/crates/services/p2p/src/request_response/messages.rs +++ b/crates/services/p2p/src/request_response/messages.rs @@ -48,7 +48,6 @@ pub enum RequestMessage { pub enum ResponseMessage { SealedBlock(Box>), SealedHeaders(Option>), - PooledTransactions(Option>), // temp as string Transactions(Option>), } @@ -57,7 +56,6 @@ pub enum ResponseMessage { pub enum ResponseChannelItem { Block(oneshot::Sender>), SealedHeaders(oneshot::Sender<(PeerId, Option>)>), - PooledTransactions(oneshot::Sender>>), // temp as string Transactions(oneshot::Sender>>), } @@ -68,7 +66,6 @@ pub enum NetworkResponse { Block(Option>), Headers(Option>), Transactions(Option>), - PooledTransactions(Option>), } /// Initial state of the `ResponseMessage` prior to having its inner value serialized @@ -77,7 +74,6 @@ pub enum NetworkResponse { pub enum OutboundResponse { Block(Option>), SealedHeaders(Option>), - PooledTransactions(Option>>), // temp as string Transactions(Option>>), } diff --git a/crates/services/p2p/src/service.rs b/crates/services/p2p/src/service.rs index bc68c9e6d8d..d5e32312e97 100644 --- a/crates/services/p2p/src/service.rs +++ b/crates/services/p2p/src/service.rs @@ -139,6 +139,13 @@ pub trait TaskP2PService: Send { &mut self, message: GossipsubBroadcastRequest, ) -> anyhow::Result<()>; + + fn send_msg( + &mut self, + peer_id: PeerId, + request_msg: RequestMessage, + ) -> anyhow::Result<()>; + fn send_request_msg( &mut self, peer_id: Option, @@ -192,6 +199,16 @@ impl TaskP2PService for FuelP2PService { Ok(()) } + /// Sends a one way request message to a peer. + fn send_msg( + &mut self, + peer_id: PeerId, + request_msg: RequestMessage, + ) -> anyhow::Result<()> { + self.send_msg(peer_id, request_msg); + Ok(()) + } + fn send_request_msg( &mut self, peer_id: Option, @@ -474,8 +491,7 @@ where match next_service_request { Some(TaskRequest::SendPooledTransactions { to_peer, transactions }) => { let request_msg = RequestMessage::PooledTransactions(transactions); - let channel_item = ResponseChannelItem::PooledTransactions(oneshot::channel().0); - let _ = self.p2p_service.send_request_msg(Some(to_peer), request_msg, channel_item); + let _ = self.p2p_service.send_msg(to_peer, request_msg); } Some(TaskRequest::BroadcastTransaction(transaction)) => { let broadcast = GossipsubBroadcastRequest::NewTx(transaction); From 1e1aea936773cb37fccd24b10b2b109be6276223 Mon Sep 17 00:00:00 2001 From: Rodrigo Araujo Date: Fri, 13 Oct 2023 14:45:17 -0700 Subject: [PATCH 08/18] Split transactions into batches --- Cargo.lock | 1 + crates/services/p2p/src/lib.rs | 2 +- crates/services/p2p/src/p2p_service.rs | 15 ++- .../p2p/src/request_response/messages.rs | 2 +- crates/services/txpool/Cargo.toml | 1 + crates/services/txpool/src/service.rs | 36 +++++- tests/tests/tx_request_response.rs | 115 ++++++++++++++++-- 7 files changed, 153 insertions(+), 19 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 90741a33821..036ca4b6822 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -3183,6 +3183,7 @@ dependencies = [ "itertools 0.10.5", "mockall", "parking_lot 0.12.1", + "postcard", "proptest", "rstest", "test-strategy", diff --git a/crates/services/p2p/src/lib.rs b/crates/services/p2p/src/lib.rs index 83ad440dfdf..af8c5451821 100644 --- a/crates/services/p2p/src/lib.rs +++ b/crates/services/p2p/src/lib.rs @@ -8,7 +8,7 @@ mod p2p_service; mod peer_manager; mod peer_report; pub mod ports; -mod request_response; +pub mod request_response; pub mod service; pub use gossipsub::config as gossipsub_config; diff --git a/crates/services/p2p/src/p2p_service.rs b/crates/services/p2p/src/p2p_service.rs index b1185812391..fc54886849e 100644 --- a/crates/services/p2p/src/p2p_service.rs +++ b/crates/services/p2p/src/p2p_service.rs @@ -298,13 +298,16 @@ impl FuelP2PService { } } - /// Sends a one way message to a peer. - /// It leverages the `RequestMessage` type to send the message. - /// But, unlike `send_request_msg`, it does not expect a + /// Sends a one way message to a peer. + /// It leverages the `RequestMessage` type to send the message. + /// But, unlike `send_request_msg`, it does not expect a /// response through a response channel. - pub fn send_msg(&mut self, peer_id: PeerId, message_request: RequestMessage) -> RequestId { - self - .swarm + pub fn send_msg( + &mut self, + peer_id: PeerId, + message_request: RequestMessage, + ) -> RequestId { + self.swarm .behaviour_mut() .send_request_msg(message_request, &peer_id) } diff --git a/crates/services/p2p/src/request_response/messages.rs b/crates/services/p2p/src/request_response/messages.rs index 1e9f162eca9..695c77496c1 100644 --- a/crates/services/p2p/src/request_response/messages.rs +++ b/crates/services/p2p/src/request_response/messages.rs @@ -23,7 +23,7 @@ use tokio::sync::oneshot; pub(crate) const REQUEST_RESPONSE_PROTOCOL_ID: &[u8] = b"/fuel/req_res/0.0.1"; /// Max Size in Bytes of the Request Message -pub(crate) const MAX_REQUEST_SIZE: usize = 32 * 1024; +pub const MAX_REQUEST_SIZE: usize = 32 * 1024; // Peer receives a `RequestMessage`. // It prepares a response in form of `OutboundResponse` diff --git a/crates/services/txpool/Cargo.toml b/crates/services/txpool/Cargo.toml index adb20bfdd9f..0491fff4cf0 100644 --- a/crates/services/txpool/Cargo.toml +++ b/crates/services/txpool/Cargo.toml @@ -21,6 +21,7 @@ fuel-core-p2p = { workspace = true } fuel-core-types = { workspace = true } futures = { workspace = true } parking_lot = { workspace = true } +postcard = { workspace = true, features = ["use-std"] } tokio = { workspace = true, default-features = false, features = ["sync"] } tokio-rayon = { workspace = true } tokio-stream = { workspace = true } diff --git a/crates/services/txpool/src/service.rs b/crates/services/txpool/src/service.rs index f4646b7ee33..887d8f3def6 100644 --- a/crates/services/txpool/src/service.rs +++ b/crates/services/txpool/src/service.rs @@ -15,6 +15,11 @@ use crate::{ TxPool, }; +use fuel_core_p2p::request_response::messages::{ + RequestMessage, + MAX_REQUEST_SIZE, +}; + use fuel_core_p2p::PeerId; use fuel_core_services::{ @@ -225,10 +230,14 @@ where txs.push(Transaction::from(&*tx.1.tx)); } + if !txs.is_empty() { - // TODO: we still have to make sure we're splitting the txs into - // MAX_REQUEST_SIZE. - let _ = self.shared.p2p.send_pooled_transactions(peer_id, txs).await; + for txs in split_into_batches(txs) { + let result = self.shared.p2p.send_pooled_transactions(peer_id, txs).await; + if let Err(e) = result { + tracing::error!("Unable to send pooled transactions, got an {} error", e); + } + } } } else { @@ -250,6 +259,27 @@ where } } +// Split transactions into batches of size less than MAX_REQUEST_SIZE. +fn split_into_batches(txs: Vec) -> Vec> { + let mut batches = Vec::new(); + let mut batch = Vec::new(); + let mut size = 0; + for tx in txs.into_iter() { + let m = RequestMessage::PooledTransactions(vec![tx.clone()]); + let tx_size = postcard::to_stdvec(&m).unwrap().len(); + if size + tx_size < MAX_REQUEST_SIZE { + batch.push(tx); + size += tx_size; + } else { + batches.push(batch); + batch = vec![tx]; + size = tx_size; + } + } + batches.push(batch); + batches +} + pub struct Task where DB: TxPoolDb + 'static + Clone, diff --git a/tests/tests/tx_request_response.rs b/tests/tests/tx_request_response.rs index 96c566632ca..403a2d3b84c 100644 --- a/tests/tests/tx_request_response.rs +++ b/tests/tests/tx_request_response.rs @@ -83,22 +83,27 @@ async fn test_tx_request() { ) .await; - let client_one = FuelClient::from(validators[0].node.bound_address); + let first_node = &mut validators[0]; + let second_node = &mut producers[0]; + + let client_one = FuelClient::from(first_node.node.bound_address); // Temporarily shut down the second node. - producers[0].shutdown().await; + second_node.shutdown().await; // Insert transactions into the mempool of the first node. - validators[0].test_txs = producers[0].test_txs.clone(); - let _tx = validators[0].insert_txs().await; + first_node.test_txs = second_node.test_txs.clone(); + let _tx = first_node.insert_txs().await; // Sleep for 2 seconds tokio::time::sleep(Duration::from_secs(2)).await; - // Start second node - producers[0].start().await; + second_node.start().await; - let client_two = FuelClient::from(producers[0].node.bound_address); + // Wait for nodes to share their transaction pool. + tokio::time::sleep(Duration::from_secs(2)).await; + + let client_two = FuelClient::from(second_node.node.bound_address); // Produce a new block with the participating nodes. // It's expected that they will: @@ -108,7 +113,7 @@ async fn test_tx_request() { let _ = client_one.produce_blocks(1, None).await.unwrap(); let _ = client_two.produce_blocks(1, None).await.unwrap(); - let chain_id = validators[0] + let chain_id = first_node .config .chain_conf .consensus_parameters @@ -138,3 +143,97 @@ async fn test_tx_request() { assert_eq!(first_node_tx, second_node_tx); } + +// Similar to the previous test, but this time we're testing +// sending enough requests that it won't fit into a single batch. +#[tokio::test(flavor = "multi_thread")] +async fn test_tx_request_multiple_batches() { + // Create a random seed based on the test parameters. + let mut hasher = DefaultHasher::new(); + let num_txs = 200; + let num_validators = 1; + let num_partitions = 1; + (num_txs, num_validators, num_partitions, line!()).hash(&mut hasher); + let mut rng = StdRng::seed_from_u64(hasher.finish()); + + // Create a set of key pairs. + let secrets: Vec<_> = (0..1).map(|_| SecretKey::random(&mut rng)).collect(); + let pub_keys: Vec<_> = secrets + .clone() + .into_iter() + .map(|secret| Input::owner(&secret.public_key())) + .collect(); + + // Creates a simple node config with no block production. + // This is used for these tests because we don't want the transactions to be + // included in a block. We want them to be included in the mempool and check + // that new connected nodes sync the mempool. + let disable_block_production = true; + // Create a producer for each key pair and a set of validators that share + // the same key pair. + let Nodes { + mut producers, + mut validators, + bootstrap_nodes: _dont_drop, + } = make_nodes( + pub_keys + .iter() + .map(|pub_key| Some(BootstrapSetup::new(*pub_key))), + secrets.clone().into_iter().enumerate().map(|(i, secret)| { + Some( + ProducerSetup::new(secret) + .with_txs(num_txs) + .with_name(format!("{}:producer", pub_keys[i])), + ) + }), + pub_keys.iter().flat_map(|pub_key| { + (0..num_validators).map(move |i| { + Some(ValidatorSetup::new(*pub_key).with_name(format!("{pub_key}:{i}"))) + }) + }), + disable_block_production, + ) + .await; + + let first_node = &mut validators[0]; + let second_node = &mut producers[0]; + + let client_one = FuelClient::from(first_node.node.bound_address); + + // Temporarily shut down the second node. + second_node.shutdown().await; + + // Insert transactions into the mempool of the first node. + first_node.test_txs = second_node.test_txs.clone(); + let _tx = first_node.insert_txs().await; + + // Sleep for 2 seconds + tokio::time::sleep(Duration::from_secs(2)).await; + + // Start second node + second_node.start().await; + + // Wait for nodes to share their transaction pool. + tokio::time::sleep(Duration::from_secs(5)).await; + + let client_two = FuelClient::from(second_node.node.bound_address); + + // Produce a new block with the participating nodes. + // It's expected that they will: + // 1. Connect to each other; + // 2. Share the transaction; + // 3. Produce a block with the transaction; + let _ = client_one.produce_blocks(1, None).await.unwrap(); + let _ = client_two.produce_blocks(1, None).await.unwrap(); + + let request = fuel_core_client::client::pagination::PaginationRequest { + cursor: None, + results: 1000, + direction: fuel_core_client::client::pagination::PageDirection::Forward, + }; + + let first_node_txs = client_one.transactions(request.clone()).await.unwrap(); + let second_node_txs = client_two.transactions(request).await.unwrap(); + + assert_eq!(first_node_txs.results.len(), second_node_txs.results.len()); +} From 1b5b6fb6ab870ba820d5a03899726a5f15f273ca Mon Sep 17 00:00:00 2001 From: Rodrigo Araujo Date: Fri, 20 Oct 2023 11:02:45 -0700 Subject: [PATCH 09/18] Handle error and other nits --- crates/services/p2p/src/p2p_service.rs | 24 +++++++++---------- .../p2p/src/request_response/messages.rs | 2 +- crates/services/txpool/Cargo.toml | 2 +- crates/services/txpool/src/service.rs | 11 ++++++--- tests/tests/tx_request_response.rs | 8 ++----- 5 files changed, 24 insertions(+), 23 deletions(-) diff --git a/crates/services/p2p/src/p2p_service.rs b/crates/services/p2p/src/p2p_service.rs index fc54886849e..ebeca471224 100644 --- a/crates/services/p2p/src/p2p_service.rs +++ b/crates/services/p2p/src/p2p_service.rs @@ -740,12 +740,6 @@ mod tests { use rand::Rng; use std::{ collections::HashSet, - net::{ - IpAddr, - Ipv4Addr, - SocketAddrV4, - TcpListener, - }, ops::Range, sync::Arc, time::Duration, @@ -1609,9 +1603,6 @@ mod tests { request_sent = true; match request_msg.clone() { - RequestMessage::PooledTransactions => { - todo!("PooledTransactions not implemented yet") - } RequestMessage::Block(_) => { let (tx_orchestrator, rx_orchestrator) = oneshot::channel(); assert!(node_a.send_request_msg(None, request_msg.clone(), ResponseChannelItem::Block(tx_orchestrator)).is_ok()); @@ -1665,6 +1656,12 @@ mod tests { } }); } + RequestMessage::PooledTransactions(_) => { + // This test case isn't applicable here because + // we send a one way message containing the pooled + // transactions. The node doesn't respond with anything + // so we can't test the response. + } } } } @@ -1676,9 +1673,6 @@ mod tests { // 2. Node B receives the RequestMessage from Node A initiated by the NetworkOrchestrator if let Some(FuelP2PEvent::RequestMessage{ request_id, request_message: received_request_message }) = &node_b_event { match received_request_message { - RequestMessage::PooledTransactions => { - todo!("PooledTransactions not implemented yet") - } RequestMessage::Block(_) => { let block = Block::new(PartialBlockHeader::default(), (0..5).map(|_| Transaction::default_test_tx()).collect(), &[]); @@ -1699,6 +1693,12 @@ mod tests { let transactions = vec![Transactions(txs)]; let _ = node_b.send_response_msg(*request_id, OutboundResponse::Transactions(Some(Arc::new(transactions)))); } + RequestMessage::PooledTransactions(_) => { + // This test case isn't applicable here because + // we send a one way message containing the pooled + // transactions. The node doesn't respond with anything + // so we can't test the response. + } } } diff --git a/crates/services/p2p/src/request_response/messages.rs b/crates/services/p2p/src/request_response/messages.rs index 695c77496c1..35c0dddc179 100644 --- a/crates/services/p2p/src/request_response/messages.rs +++ b/crates/services/p2p/src/request_response/messages.rs @@ -8,9 +8,9 @@ use fuel_core_types::{ SealedBlock, SealedBlockHeader, }, + fuel_tx::Transaction, fuel_types::BlockHeight, services::p2p::Transactions, - fuel_tx::Transaction, }; use libp2p::PeerId; use serde::{ diff --git a/crates/services/txpool/Cargo.toml b/crates/services/txpool/Cargo.toml index 0491fff4cf0..600efe24175 100644 --- a/crates/services/txpool/Cargo.toml +++ b/crates/services/txpool/Cargo.toml @@ -15,9 +15,9 @@ anyhow = { workspace = true } async-trait = { workspace = true } fuel-core-chain-config = { workspace = true } fuel-core-metrics = { workspace = true } +fuel-core-p2p = { workspace = true } fuel-core-services = { workspace = true } fuel-core-storage = { workspace = true } -fuel-core-p2p = { workspace = true } fuel-core-types = { workspace = true } futures = { workspace = true } parking_lot = { workspace = true } diff --git a/crates/services/txpool/src/service.rs b/crates/services/txpool/src/service.rs index 887d8f3def6..ed11a27bf40 100644 --- a/crates/services/txpool/src/service.rs +++ b/crates/services/txpool/src/service.rs @@ -207,10 +207,15 @@ where let current_height = self.shared.db.current_block_height()?; let mut res = Vec::new(); - for tx in incoming_pooled_transactions.into_iter() { - // verify tx - let checked_tx = check_single_tx(tx, current_height, &self.shared.config).await.unwrap(); // TODO: remove temp unwrap + for tx in incoming_pooled_transactions.into_iter() { + let checked_tx = match check_single_tx(tx, current_height, &self.shared.config).await { + Ok(tx) => tx, + Err(e) => { + tracing::error!("Unable to insert pooled transaction coming from a newly connected peer, got an {} error", e); + continue; + } + }; res.push(self.shared.txpool.lock().insert_inner(checked_tx)); } diff --git a/tests/tests/tx_request_response.rs b/tests/tests/tx_request_response.rs index 403a2d3b84c..2672e4c041a 100644 --- a/tests/tests/tx_request_response.rs +++ b/tests/tests/tx_request_response.rs @@ -113,11 +113,7 @@ async fn test_tx_request() { let _ = client_one.produce_blocks(1, None).await.unwrap(); let _ = client_two.produce_blocks(1, None).await.unwrap(); - let chain_id = first_node - .config - .chain_conf - .consensus_parameters - .chain_id(); + let chain_id = first_node.config.chain_conf.consensus_parameters.chain_id(); let request = fuel_core_client::client::pagination::PaginationRequest { cursor: None, @@ -225,7 +221,7 @@ async fn test_tx_request_multiple_batches() { // 3. Produce a block with the transaction; let _ = client_one.produce_blocks(1, None).await.unwrap(); let _ = client_two.produce_blocks(1, None).await.unwrap(); - + let request = fuel_core_client::client::pagination::PaginationRequest { cursor: None, results: 1000, From 47f0983a3be86adcde3f8087532935ccb75d2fa9 Mon Sep 17 00:00:00 2001 From: Rodrigo Araujo Date: Fri, 20 Oct 2023 11:18:28 -0700 Subject: [PATCH 10/18] Some more nits --- crates/services/txpool/src/service.rs | 2 +- tests/tests/tx_gossip.rs | 67 ++------------------------- tests/tests/tx_request_response.rs | 9 +--- 3 files changed, 7 insertions(+), 71 deletions(-) diff --git a/crates/services/txpool/src/service.rs b/crates/services/txpool/src/service.rs index 21a32365ef2..cd424936472 100644 --- a/crates/services/txpool/src/service.rs +++ b/crates/services/txpool/src/service.rs @@ -609,10 +609,10 @@ where ttl_timer, shared: SharedState { db, + tx_status_sender: TxStatusChange::new(number_of_active_subscription, 2 * config.transaction_ttl), config, txpool, p2p, - tx_status_sender: TxStatusChange::new(number_of_active_subscription), consensus_params, }, }; diff --git a/tests/tests/tx_gossip.rs b/tests/tests/tx_gossip.rs index 5fa13bc49d1..abd951f491a 100644 --- a/tests/tests/tx_gossip.rs +++ b/tests/tests/tx_gossip.rs @@ -1,30 +1,13 @@ -use fuel_core::{ - chain_config::{ - CoinConfig, - StateConfig, - }, - p2p_test_helpers::{ +use fuel_core::p2p_test_helpers::{ make_nodes, BootstrapSetup, Nodes, ProducerSetup, ValidatorSetup, - }, - service::Config, -}; + }; use fuel_core_client::client::FuelClient; -use fuel_core_poa::Trigger; -use fuel_core_types::{ - fuel_tx::{ - field::*, - input::coin::{ - CoinPredicate, - CoinSigned, - }, - *, - }, - fuel_vm::*, -}; +use fuel_core_types::fuel_vm::*; +use fuel_core_types::fuel_tx::input::Input; use rand::{ rngs::StdRng, @@ -39,48 +22,6 @@ use std::{ time::Duration, }; -fn create_node_config_from_inputs(inputs: &[Input]) -> Config { - let mut node_config = Config::local_node(); - let mut initial_state = StateConfig::default(); - let mut coin_configs = vec![]; - - for input in inputs { - if let Input::CoinSigned(CoinSigned { - amount, - owner, - asset_id, - utxo_id, - .. - }) - | Input::CoinPredicate(CoinPredicate { - amount, - owner, - asset_id, - utxo_id, - .. - }) = input - { - let coin_config = CoinConfig { - tx_id: Some(*utxo_id.tx_id()), - output_index: Some(utxo_id.output_index()), - tx_pointer_block_height: None, - tx_pointer_tx_idx: None, - maturity: None, - owner: *owner, - amount: *amount, - asset_id: *asset_id, - }; - coin_configs.push(coin_config); - }; - } - - initial_state.coins = Some(coin_configs); - node_config.chain_conf.initial_state = Some(initial_state); - node_config.utxo_validation = true; - node_config.p2p.as_mut().unwrap().enable_mdns = true; - node_config -} - #[tokio::test(flavor = "multi_thread")] async fn test_tx_gossiping() { use futures::StreamExt; diff --git a/tests/tests/tx_request_response.rs b/tests/tests/tx_request_response.rs index 2672e4c041a..25574a56a96 100644 --- a/tests/tests/tx_request_response.rs +++ b/tests/tests/tx_request_response.rs @@ -6,15 +6,10 @@ use fuel_core::p2p_test_helpers::{ ValidatorSetup, }; use fuel_core_client::client::FuelClient; -use fuel_core_poa::Trigger; use fuel_core_types::{ fuel_tx::{ - field::*, - input::coin::{ - CoinPredicate, - CoinSigned, - }, - *, + input::Input, + UniqueIdentifier, }, fuel_vm::*, }; From 1871b268fc1bc55212d7bf3132f60185f3f483eb Mon Sep 17 00:00:00 2001 From: Rodrigo Araujo Date: Fri, 20 Oct 2023 11:20:06 -0700 Subject: [PATCH 11/18] fmt --- crates/services/txpool/src/service.rs | 5 ++++- tests/tests/tx_gossip.rs | 18 ++++++++++-------- 2 files changed, 14 insertions(+), 9 deletions(-) diff --git a/crates/services/txpool/src/service.rs b/crates/services/txpool/src/service.rs index cd424936472..0eb1ec8d966 100644 --- a/crates/services/txpool/src/service.rs +++ b/crates/services/txpool/src/service.rs @@ -609,7 +609,10 @@ where ttl_timer, shared: SharedState { db, - tx_status_sender: TxStatusChange::new(number_of_active_subscription, 2 * config.transaction_ttl), + tx_status_sender: TxStatusChange::new( + number_of_active_subscription, + 2 * config.transaction_ttl, + ), config, txpool, p2p, diff --git a/tests/tests/tx_gossip.rs b/tests/tests/tx_gossip.rs index abd951f491a..22387da9628 100644 --- a/tests/tests/tx_gossip.rs +++ b/tests/tests/tx_gossip.rs @@ -1,13 +1,15 @@ use fuel_core::p2p_test_helpers::{ - make_nodes, - BootstrapSetup, - Nodes, - ProducerSetup, - ValidatorSetup, - }; + make_nodes, + BootstrapSetup, + Nodes, + ProducerSetup, + ValidatorSetup, +}; use fuel_core_client::client::FuelClient; -use fuel_core_types::fuel_vm::*; -use fuel_core_types::fuel_tx::input::Input; +use fuel_core_types::{ + fuel_tx::input::Input, + fuel_vm::*, +}; use rand::{ rngs::StdRng, From e3fd9d000838acdaaa9155328fa00e9fecc5a783 Mon Sep 17 00:00:00 2001 From: Rodrigo Araujo Date: Fri, 20 Oct 2023 11:25:31 -0700 Subject: [PATCH 12/18] Update changelog --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 85a31d08256..2ab4b1b71f4 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -10,6 +10,7 @@ Description of the upcoming release here. ### Added +- [#1228](https://github.com/FuelLabs/fuel-core/pull/1228): Add a new capability of sharing pooled transactions when two nodes connect. - [#1432](https://github.com/FuelLabs/fuel-core/pull/1432): Add a new `--api-request-timeout` argument to control TTL for GraphQL requests. - [#1419](https://github.com/FuelLabs/fuel-core/pull/1419): Add additional "sanity" benchmarks for arithmetic op code instructions. - [#1411](https://github.com/FuelLabs/fuel-core/pull/1411): Added WASM and `no_std` compatibility From adce48261055461d94182ba0d9088040b634bebd Mon Sep 17 00:00:00 2001 From: Rodrigo Araujo Date: Fri, 20 Oct 2023 12:18:34 -0700 Subject: [PATCH 13/18] Implement missing trait fns --- Cargo.lock | 1 + .../fuel-core/src/service/adapters/txpool.rs | 22 ++++++++++++++++++- crates/types/Cargo.toml | 1 + crates/types/src/services/p2p.rs | 2 ++ 4 files changed, 25 insertions(+), 1 deletion(-) diff --git a/Cargo.lock b/Cargo.lock index 8bb558933b1..8cab7a6c30a 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -3189,6 +3189,7 @@ dependencies = [ "anyhow", "derive_more", "fuel-vm", + "libp2p-core 0.38.0", "secrecy", "serde", "tai64", diff --git a/crates/fuel-core/src/service/adapters/txpool.rs b/crates/fuel-core/src/service/adapters/txpool.rs index b16d0fc91c1..6924b2c1d6b 100644 --- a/crates/fuel-core/src/service/adapters/txpool.rs +++ b/crates/fuel-core/src/service/adapters/txpool.rs @@ -5,7 +5,11 @@ use crate::{ P2PAdapter, }, }; +#[cfg(feature = "p2p")] use fuel_core_p2p::PeerId; +#[cfg(not(feature = "p2p"))] +use fuel_core_types::services::p2p::Libp2pPeerId; + use fuel_core_services::stream::BoxStream; use fuel_core_storage::{ not_found, @@ -143,6 +147,7 @@ impl fuel_core_txpool::ports::PeerToPeer for P2PAdapter { } #[cfg(not(feature = "p2p"))] +#[async_trait::async_trait] impl fuel_core_txpool::ports::PeerToPeer for P2PAdapter { type GossipedTransaction = TransactionGossipData; @@ -164,7 +169,22 @@ impl fuel_core_txpool::ports::PeerToPeer for P2PAdapter { ) -> anyhow::Result<()> { Ok(()) } - // TODO: implement missing methods for this adapter. + + fn new_connection(&self) -> BoxStream { + Box::pin(fuel_core_services::stream::pending()) + } + + fn incoming_pooled_transactions(&self) -> BoxStream> { + Box::pin(fuel_core_services::stream::pending()) + } + + async fn send_pooled_transactions( + &self, + _peer_id: Libp2pPeerId, + _transactions: Vec, + ) -> anyhow::Result<()> { + Ok(()) + } } impl fuel_core_txpool::ports::TxPoolDb for Database { diff --git a/crates/types/Cargo.toml b/crates/types/Cargo.toml index d3cde9bb4cf..362cb5213dd 100644 --- a/crates/types/Cargo.toml +++ b/crates/types/Cargo.toml @@ -20,6 +20,7 @@ version = { workspace = true } anyhow = { workspace = true } derive_more = { version = "0.99" } fuel-vm-private = { workspace = true, default-features = false, features = ["alloc"] } +libp2p-core = "=0.38" secrecy = "0.8" serde = { workspace = true, features = ["derive"], optional = true } tai64 = { version = "4.0", features = ["serde"] } diff --git a/crates/types/src/services/p2p.rs b/crates/types/src/services/p2p.rs index 758ca83df2d..cc5ea574fc6 100644 --- a/crates/types/src/services/p2p.rs +++ b/crates/types/src/services/p2p.rs @@ -9,6 +9,8 @@ use std::fmt::Debug; /// Contains types and logic for Peer Reputation pub mod peer_reputation; +pub use libp2p_core::PeerId as Libp2pPeerId; + /// List of transactions #[derive(Debug, Clone, Default)] #[cfg_attr(feature = "serde", derive(serde::Serialize, serde::Deserialize))] From 7c8223852ab0bed70b53761b3d7d827bcb793f63 Mon Sep 17 00:00:00 2001 From: Rodrigo Araujo Date: Fri, 20 Oct 2023 14:47:24 -0700 Subject: [PATCH 14/18] More CI-related fixes/improvements --- Cargo.lock | 1 - .../tests/integration_tests.rs | 2 ++ .../fuel-core/src/service/adapters/txpool.rs | 11 +++---- crates/services/p2p/src/service.rs | 29 +++++++++++++++---- crates/services/txpool/src/ports.rs | 2 +- crates/services/txpool/src/service.rs | 5 ++-- .../txpool/src/service/test_helpers.rs | 8 ++++- crates/types/Cargo.toml | 1 - crates/types/src/services/p2p.rs | 2 -- 9 files changed, 40 insertions(+), 21 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 8cab7a6c30a..8bb558933b1 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -3189,7 +3189,6 @@ dependencies = [ "anyhow", "derive_more", "fuel-vm", - "libp2p-core 0.38.0", "secrecy", "serde", "tai64", diff --git a/bin/e2e-test-client/tests/integration_tests.rs b/bin/e2e-test-client/tests/integration_tests.rs index 530df1a66da..b311abd804d 100644 --- a/bin/e2e-test-client/tests/integration_tests.rs +++ b/bin/e2e-test-client/tests/integration_tests.rs @@ -41,6 +41,7 @@ async fn works_in_multinode_local_env() { let mut rng = StdRng::seed_from_u64(line!() as u64); let secret = SecretKey::random(&mut rng); let pub_key = Input::owner(&secret.public_key()); + let disable_block_production = false; let Nodes { mut producers, mut validators, @@ -51,6 +52,7 @@ async fn works_in_multinode_local_env() { ProducerSetup::new(secret).with_txs(1).with_name("Alice"), )], [Some(ValidatorSetup::new(pub_key).with_name("Bob"))], + disable_block_production, ) .await; diff --git a/crates/fuel-core/src/service/adapters/txpool.rs b/crates/fuel-core/src/service/adapters/txpool.rs index 6924b2c1d6b..304303f71e8 100644 --- a/crates/fuel-core/src/service/adapters/txpool.rs +++ b/crates/fuel-core/src/service/adapters/txpool.rs @@ -5,10 +5,6 @@ use crate::{ P2PAdapter, }, }; -#[cfg(feature = "p2p")] -use fuel_core_p2p::PeerId; -#[cfg(not(feature = "p2p"))] -use fuel_core_types::services::p2p::Libp2pPeerId; use fuel_core_services::stream::BoxStream; use fuel_core_storage::{ @@ -40,6 +36,7 @@ use fuel_core_types::{ services::{ block_importer::ImportResult, p2p::{ + PeerId, GossipsubMessageAcceptance, GossipsubMessageInfo, TransactionGossipData, @@ -81,7 +78,7 @@ impl fuel_core_txpool::ports::PeerToPeer for P2PAdapter { ) -> anyhow::Result<()> { if let Some(service) = &self.service { service - .send_pooled_transactions_to_peer(peer_id.to_bytes(), transactions) + .send_pooled_transactions_to_peer(peer_id.into(), transactions) .await } else { Ok(()) @@ -170,7 +167,7 @@ impl fuel_core_txpool::ports::PeerToPeer for P2PAdapter { Ok(()) } - fn new_connection(&self) -> BoxStream { + fn new_connection(&self) -> BoxStream { Box::pin(fuel_core_services::stream::pending()) } @@ -180,7 +177,7 @@ impl fuel_core_txpool::ports::PeerToPeer for P2PAdapter { async fn send_pooled_transactions( &self, - _peer_id: Libp2pPeerId, + _peer_id: PeerId, _transactions: Vec, ) -> anyhow::Result<()> { Ok(()) diff --git a/crates/services/p2p/src/service.rs b/crates/services/p2p/src/service.rs index d98e4096926..f3ba62b10d3 100644 --- a/crates/services/p2p/src/service.rs +++ b/crates/services/p2p/src/service.rs @@ -274,7 +274,7 @@ pub trait Broadcast: Send { fn tx_broadcast(&self, transaction: TransactionGossipData) -> anyhow::Result<()>; - fn connection_broadcast(&self, peer_id: PeerId) -> anyhow::Result<()>; + fn connection_broadcast(&self, peer_id: FuelPeerId) -> anyhow::Result<()>; fn incoming_pooled_transactions( &self, @@ -305,7 +305,7 @@ impl Broadcast for SharedState { Ok(()) } - fn connection_broadcast(&self, peer_id: PeerId) -> anyhow::Result<()> { + fn connection_broadcast(&self, peer_id: FuelPeerId) -> anyhow::Result<()> { self.connection_broadcast.send(peer_id)?; Ok(()) } @@ -570,7 +570,7 @@ where // channel that the TxPoolSync service is listening to. // This is the first step of the protocol for the initial // pool sync between two nodes. - let _ = self.broadcast.connection_broadcast(peer_id); + let _ = self.broadcast.connection_broadcast(FuelPeerId::from(peer_id.to_bytes())); } Some(FuelP2PEvent::PeerInfoUpdated { peer_id, block_height }) => { let peer_id: Vec = peer_id.into(); @@ -709,7 +709,7 @@ pub struct SharedState { /// Sender of p2p blopck height data block_height_broadcast: broadcast::Sender, /// Sender of new incoming connections - connection_broadcast: broadcast::Sender, + connection_broadcast: broadcast::Sender, /// Sender of incoming pooled Transactions incoming_pooled_transactions: broadcast::Sender>, } @@ -851,7 +851,7 @@ impl SharedState { self.reserved_peers_broadcast.subscribe() } - pub fn subscribe_to_connections(&self) -> broadcast::Receiver { + pub fn subscribe_to_connections(&self) -> broadcast::Receiver { self.connection_broadcast.subscribe() } @@ -1032,6 +1032,14 @@ pub mod tests { std::future::pending().boxed() } + fn send_msg( + &mut self, + peer_id: PeerId, + request_msg: RequestMessage, + ) -> anyhow::Result<()> { + todo!() + } + fn publish_message( &mut self, _message: GossipsubBroadcastRequest, @@ -1136,6 +1144,17 @@ pub mod tests { todo!() } + fn incoming_pooled_transactions( + &self, + _transactions: Vec, + ) -> anyhow::Result<()> { + todo!() + } + + fn connection_broadcast(&self, _peer_id: FuelPeerId) -> anyhow::Result<()> { + todo!() + } + fn tx_broadcast( &self, _transaction: TransactionGossipData, diff --git a/crates/services/txpool/src/ports.rs b/crates/services/txpool/src/ports.rs index 1608e7a9908..6fb59280a54 100644 --- a/crates/services/txpool/src/ports.rs +++ b/crates/services/txpool/src/ports.rs @@ -1,4 +1,3 @@ -use fuel_core_p2p::PeerId; use fuel_core_services::stream::BoxStream; use fuel_core_storage::Result as StorageResult; use fuel_core_types::{ @@ -19,6 +18,7 @@ use fuel_core_types::{ services::{ block_importer::ImportResult, p2p::{ + PeerId, GossipsubMessageAcceptance, GossipsubMessageInfo, NetworkData, diff --git a/crates/services/txpool/src/service.rs b/crates/services/txpool/src/service.rs index 0eb1ec8d966..c11289f5f81 100644 --- a/crates/services/txpool/src/service.rs +++ b/crates/services/txpool/src/service.rs @@ -20,8 +20,6 @@ use fuel_core_p2p::request_response::messages::{ MAX_REQUEST_SIZE, }; -use fuel_core_p2p::PeerId; - use fuel_core_services::{ stream::BoxStream, RunnableService, @@ -44,6 +42,7 @@ use fuel_core_types::{ services::{ block_importer::ImportResult, p2p::{ + PeerId, GossipData, GossipsubMessageAcceptance, GossipsubMessageInfo, @@ -242,7 +241,7 @@ where if !txs.is_empty() { for txs in split_into_batches(txs) { - let result = self.shared.p2p.send_pooled_transactions(peer_id, txs).await; + let result = self.shared.p2p.send_pooled_transactions(peer_id.clone(), txs).await; if let Err(e) = result { tracing::error!("Unable to send pooled transactions, got an {} error", e); } diff --git a/crates/services/txpool/src/service/test_helpers.rs b/crates/services/txpool/src/service/test_helpers.rs index 411eb74c7c3..4ce884b8b0a 100644 --- a/crates/services/txpool/src/service/test_helpers.rs +++ b/crates/services/txpool/src/service/test_helpers.rs @@ -73,7 +73,13 @@ mockall::mock! { fn new_connection(&self) -> BoxStream; - async fn request_pooled_transactions(&self, peer_id: PeerId) -> anyhow::Result>>; + fn incoming_pooled_transactions(&self) -> BoxStream>; + + async fn send_pooled_transactions( + &self, + peer_id: PeerId, + transactions: Vec, + ) -> anyhow::Result<()>; fn notify_gossip_transaction_validity( &self, diff --git a/crates/types/Cargo.toml b/crates/types/Cargo.toml index 362cb5213dd..d3cde9bb4cf 100644 --- a/crates/types/Cargo.toml +++ b/crates/types/Cargo.toml @@ -20,7 +20,6 @@ version = { workspace = true } anyhow = { workspace = true } derive_more = { version = "0.99" } fuel-vm-private = { workspace = true, default-features = false, features = ["alloc"] } -libp2p-core = "=0.38" secrecy = "0.8" serde = { workspace = true, features = ["derive"], optional = true } tai64 = { version = "4.0", features = ["serde"] } diff --git a/crates/types/src/services/p2p.rs b/crates/types/src/services/p2p.rs index cc5ea574fc6..758ca83df2d 100644 --- a/crates/types/src/services/p2p.rs +++ b/crates/types/src/services/p2p.rs @@ -9,8 +9,6 @@ use std::fmt::Debug; /// Contains types and logic for Peer Reputation pub mod peer_reputation; -pub use libp2p_core::PeerId as Libp2pPeerId; - /// List of transactions #[derive(Debug, Clone, Default)] #[cfg_attr(feature = "serde", derive(serde::Serialize, serde::Deserialize))] From dbed77c8f6d1fc547665453274f7dfc34cc42730 Mon Sep 17 00:00:00 2001 From: Rodrigo Araujo Date: Fri, 20 Oct 2023 14:49:07 -0700 Subject: [PATCH 15/18] Rustfmt --- crates/fuel-core/src/service/adapters/txpool.rs | 2 +- crates/services/p2p/src/service.rs | 14 +++++++------- crates/services/txpool/src/ports.rs | 2 +- crates/services/txpool/src/service.rs | 2 +- 4 files changed, 10 insertions(+), 10 deletions(-) diff --git a/crates/fuel-core/src/service/adapters/txpool.rs b/crates/fuel-core/src/service/adapters/txpool.rs index 304303f71e8..22e8d063648 100644 --- a/crates/fuel-core/src/service/adapters/txpool.rs +++ b/crates/fuel-core/src/service/adapters/txpool.rs @@ -36,9 +36,9 @@ use fuel_core_types::{ services::{ block_importer::ImportResult, p2p::{ - PeerId, GossipsubMessageAcceptance, GossipsubMessageInfo, + PeerId, TransactionGossipData, }, }, diff --git a/crates/services/p2p/src/service.rs b/crates/services/p2p/src/service.rs index f3ba62b10d3..0a4d3339c8c 100644 --- a/crates/services/p2p/src/service.rs +++ b/crates/services/p2p/src/service.rs @@ -1033,10 +1033,10 @@ pub mod tests { } fn send_msg( - &mut self, - peer_id: PeerId, - request_msg: RequestMessage, - ) -> anyhow::Result<()> { + &mut self, + peer_id: PeerId, + request_msg: RequestMessage, + ) -> anyhow::Result<()> { todo!() } @@ -1145,9 +1145,9 @@ pub mod tests { } fn incoming_pooled_transactions( - &self, - _transactions: Vec, - ) -> anyhow::Result<()> { + &self, + _transactions: Vec, + ) -> anyhow::Result<()> { todo!() } diff --git a/crates/services/txpool/src/ports.rs b/crates/services/txpool/src/ports.rs index 6fb59280a54..3d040664c51 100644 --- a/crates/services/txpool/src/ports.rs +++ b/crates/services/txpool/src/ports.rs @@ -18,10 +18,10 @@ use fuel_core_types::{ services::{ block_importer::ImportResult, p2p::{ - PeerId, GossipsubMessageAcceptance, GossipsubMessageInfo, NetworkData, + PeerId, }, txpool::TransactionStatus, }, diff --git a/crates/services/txpool/src/service.rs b/crates/services/txpool/src/service.rs index c11289f5f81..fee5aba21ae 100644 --- a/crates/services/txpool/src/service.rs +++ b/crates/services/txpool/src/service.rs @@ -42,10 +42,10 @@ use fuel_core_types::{ services::{ block_importer::ImportResult, p2p::{ - PeerId, GossipData, GossipsubMessageAcceptance, GossipsubMessageInfo, + PeerId, TransactionGossipData, }, txpool::{ From 6a3d0339b7c4bce0c0808ac133bc71b42c7f8d69 Mon Sep 17 00:00:00 2001 From: Rodrigo Araujo Date: Fri, 20 Oct 2023 14:57:23 -0700 Subject: [PATCH 16/18] Remove unused dependencies --- crates/services/p2p/src/p2p_service.rs | 5 +---- crates/services/p2p/src/service.rs | 4 ++-- 2 files changed, 3 insertions(+), 6 deletions(-) diff --git a/crates/services/p2p/src/p2p_service.rs b/crates/services/p2p/src/p2p_service.rs index ebeca471224..8f34440d2c6 100644 --- a/crates/services/p2p/src/p2p_service.rs +++ b/crates/services/p2p/src/p2p_service.rs @@ -1110,10 +1110,7 @@ mod tests { #[tokio::test] #[instrument] async fn nodes_cannot_connect_due_to_different_checksum() { - use libp2p::{ - swarm::DialError, - TransportError, - }; + use libp2p::TransportError; // Node A let mut p2p_config = Config::default_initialized("nodes_cannot_connect_due_to_different_checksum"); diff --git a/crates/services/p2p/src/service.rs b/crates/services/p2p/src/service.rs index 0a4d3339c8c..be7f87f0b87 100644 --- a/crates/services/p2p/src/service.rs +++ b/crates/services/p2p/src/service.rs @@ -1034,8 +1034,8 @@ pub mod tests { fn send_msg( &mut self, - peer_id: PeerId, - request_msg: RequestMessage, + _peer_id: PeerId, + _request_msg: RequestMessage, ) -> anyhow::Result<()> { todo!() } From 6423138df77284f39d70a024a0db8e2fe436ffa6 Mon Sep 17 00:00:00 2001 From: Rodrigo Araujo Date: Fri, 20 Oct 2023 15:21:21 -0700 Subject: [PATCH 17/18] Fix test --- crates/services/txpool/src/service/test_helpers.rs | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/crates/services/txpool/src/service/test_helpers.rs b/crates/services/txpool/src/service/test_helpers.rs index 4ce884b8b0a..0a7fe324be2 100644 --- a/crates/services/txpool/src/service/test_helpers.rs +++ b/crates/services/txpool/src/service/test_helpers.rs @@ -112,6 +112,11 @@ impl MockP2P { let stream = fuel_core_services::stream::empty::(); Box::pin(stream) }); + p2p.expect_incoming_pooled_transactions() + .returning(move || { + let stream = fuel_core_services::stream::empty::>(); + Box::pin(stream) + }); p2p } } From 14c00b774ce5ea7e7428d1a29b77720c3fc3d65c Mon Sep 17 00:00:00 2001 From: Rodrigo Araujo Date: Fri, 20 Oct 2023 15:48:50 -0700 Subject: [PATCH 18/18] Update multiple batches test --- tests/tests/tx_request_response.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/tests/tx_request_response.rs b/tests/tests/tx_request_response.rs index 25574a56a96..35a097480ed 100644 --- a/tests/tests/tx_request_response.rs +++ b/tests/tests/tx_request_response.rs @@ -205,7 +205,7 @@ async fn test_tx_request_multiple_batches() { second_node.start().await; // Wait for nodes to share their transaction pool. - tokio::time::sleep(Duration::from_secs(5)).await; + tokio::time::sleep(Duration::from_secs(10)).await; let client_two = FuelClient::from(second_node.node.bound_address);