From 56763daa8cf3e3659d5b9acb0563e266c48d6327 Mon Sep 17 00:00:00 2001 From: leviathanbeak88 Date: Fri, 15 Jul 2022 09:08:21 +0200 Subject: [PATCH 01/29] add network service --- fuel-core-interfaces/src/p2p.rs | 7 ++ fuel-p2p/Cargo.toml | 3 +- fuel-p2p/src/config.rs | 30 ++++- fuel-p2p/src/orchestrator.rs | 189 ++++++++++++++++++++++++++------ fuel-p2p/src/service.rs | 4 +- 5 files changed, 195 insertions(+), 38 deletions(-) diff --git a/fuel-core-interfaces/src/p2p.rs b/fuel-core-interfaces/src/p2p.rs index 3d6af9cb73b..6f45a865d8c 100644 --- a/fuel-core-interfaces/src/p2p.rs +++ b/fuel-core-interfaces/src/p2p.rs @@ -1,5 +1,6 @@ use super::model::{BlockHeight, FuelBlock, SealedFuelBlock}; use crate::model::ConsensusVote; +use async_trait::async_trait; use fuel_tx::Transaction; use std::sync::Arc; use tokio::sync::oneshot; @@ -31,4 +32,10 @@ pub enum P2pRequestEvent { BroadcastConsensusVote { vote: Arc, }, + Stop, +} + +#[async_trait] +pub trait P2pDb: Send + Sync { + async fn get_sealed_block(&self, height: BlockHeight) -> Option>; } diff --git a/fuel-p2p/Cargo.toml b/fuel-p2p/Cargo.toml index 71a7ff69173..538fec62ed7 100644 --- a/fuel-p2p/Cargo.toml +++ b/fuel-p2p/Cargo.toml @@ -12,7 +12,8 @@ description = "Fuel client networking" # See more keys and their definitions at https://doc.rust-lang.org/cargo/reference/manifest.html [dependencies] -async-trait = "0.1.52" +anyhow = "1.0" +async-trait = "0.1" bincode = "1.3" fuel-core-interfaces = { path = "../fuel-core-interfaces", features = ["serde"], version = "0.9.4" } futures = "0.3" diff --git a/fuel-p2p/src/config.rs b/fuel-p2p/src/config.rs index faebe73257b..48bc93bbd6f 100644 --- a/fuel-p2p/src/config.rs +++ b/fuel-p2p/src/config.rs @@ -3,7 +3,10 @@ use libp2p::{ identity::Keypair, mplex, noise, yamux, Multiaddr, PeerId, Transport, }; -use std::{net::IpAddr, time::Duration}; +use std::{ + net::{IpAddr, Ipv4Addr}, + time::Duration, +}; pub const REQ_RES_TIMEOUT: Duration = Duration::from_secs(20); @@ -57,6 +60,31 @@ pub struct P2PConfig { pub set_connection_keep_alive: Option, } +impl P2PConfig { + pub fn default_with_network(network_name: &str) -> Self { + P2PConfig { + network_name: network_name.into(), + address: IpAddr::V4(Ipv4Addr::from([0, 0, 0, 0])), + tcp_port: 4000, + max_block_size: 100_000, + bootstrap_nodes: vec![], + enable_mdns: false, + max_peers_connected: 50, + allow_private_addresses: true, + enable_random_walk: true, + connection_idle_timeout: Some(Duration::from_secs(120)), + topics: vec![], + max_mesh_size: 12, + min_mesh_size: 4, + ideal_mesh_size: 6, + set_request_timeout: None, + set_connection_keep_alive: None, + info_interval: Some(Duration::from_secs(3)), + identify_interval: Some(Duration::from_secs(5)), + } + } +} + /// Transport for libp2p communication: /// TCP/IP, Websocket /// Noise as encryption layer diff --git a/fuel-p2p/src/orchestrator.rs b/fuel-p2p/src/orchestrator.rs index a1b382b97eb..da879400ef2 100644 --- a/fuel-p2p/src/orchestrator.rs +++ b/fuel-p2p/src/orchestrator.rs @@ -1,15 +1,15 @@ -use std::error::Error; use std::sync::Arc; -use std::{future::Future, pin::Pin}; -use fuel_core_interfaces::{ - p2p::{BlockBroadcast, ConsensusBroadcast, P2pRequestEvent, TransactionBroadcast}, - relayer::RelayerDb, +use anyhow::anyhow; +use fuel_core_interfaces::p2p::{ + BlockBroadcast, ConsensusBroadcast, P2pDb, P2pRequestEvent, TransactionBroadcast, }; -use futures::{stream::futures_unordered::FuturesUnordered, StreamExt}; + use libp2p::identity::Keypair; use libp2p::request_response::RequestId; use tokio::sync::mpsc::{Receiver, Sender}; +use tokio::sync::Mutex; +use tokio::task::JoinHandle; use tracing::warn; use crate::{ @@ -20,26 +20,25 @@ use crate::{ service::{FuelP2PEvent, FuelP2PService}, }; -type ResponseFuture = Pin>>>; - pub struct NetworkOrchestrator { - p2p_service: FuelP2PService, + local_keypair: Keypair, + p2p_config: P2PConfig, /// receives messages from different Fuel components rx_request_event: Receiver, + rx_outbound_responses: Receiver>, // senders tx_consensus: Sender, tx_transaction: Sender, tx_block: Sender, + tx_outbound_responses: Sender>, - db: Arc, - - outbound_responses: FuturesUnordered, + db: Arc>, } impl NetworkOrchestrator { - pub async fn new( + pub fn new( local_keypair: Keypair, p2p_config: P2PConfig, rx_request_event: Receiver, @@ -48,30 +47,35 @@ impl NetworkOrchestrator { tx_transaction: Sender, tx_block: Sender, - db: Arc, - ) -> Result> { - let p2p_service = FuelP2PService::new(local_keypair, p2p_config).await?; + db: Arc>, + ) -> Self { + let (tx_outbound_responses, rx_outbound_responses) = tokio::sync::mpsc::channel(100); - Ok(Self { - p2p_service, + Self { + local_keypair, + p2p_config, rx_request_event, + rx_outbound_responses, tx_block, tx_consensus, tx_transaction, + tx_outbound_responses, db, - outbound_responses: Default::default(), - }) + } } - pub async fn run(&mut self) { + pub async fn run(mut self) -> anyhow::Result { + let mut p2p_service = + FuelP2PService::new(self.local_keypair.clone(), self.p2p_config.clone()).await?; + loop { tokio::select! { - next_response = self.outbound_responses.next() => { + next_response = self.rx_outbound_responses.recv() => { if let Some(Some((response, request_id))) = next_response { - let _ = self.p2p_service.send_response_msg(request_id, response); + let _ = p2p_service.send_response_msg(request_id, response); } }, - p2p_event = self.p2p_service.next_event() => { + p2p_event = p2p_service.next_event() => { if let FuelP2PEvent::Behaviour(behaviour_event) = p2p_event { match behaviour_event { FuelBehaviourEvent::GossipsubMessage { message, .. } => { @@ -91,12 +95,12 @@ impl NetworkOrchestrator { match request_message { RequestMessage::RequestBlock(block_height) => { let db = self.db.clone(); + let tx_outbound_response = self.tx_outbound_responses.clone(); - self.outbound_responses.push( - Box::pin(async move { - db.get_sealed_block(block_height).await.map(|block| (OutboundResponse::ResponseBlock(block), request_id)) - }) - ); + tokio::spawn(async move { + let res = db.get_sealed_block(block_height).await.map(|block| (OutboundResponse::ResponseBlock(block), request_id)); + let _ = tx_outbound_response.send(res); + }); } } }, @@ -110,20 +114,21 @@ impl NetworkOrchestrator { P2pRequestEvent::RequestBlock { height, response } => { let request_msg = RequestMessage::RequestBlock(height); let channel_item = ResponseChannelItem::ResponseBlock(response); - let _ = self.p2p_service.send_request_msg(None, request_msg, channel_item); + let _ = p2p_service.send_request_msg(None, request_msg, channel_item); }, P2pRequestEvent::BroadcastNewBlock { block } => { let broadcast = GossipsubBroadcastRequest::NewBlock(block); - let _ = self.p2p_service.publish_message(broadcast); + let _ = p2p_service.publish_message(broadcast); }, P2pRequestEvent::BroadcastNewTransaction { transaction } => { let broadcast = GossipsubBroadcastRequest::NewTx(transaction); - let _ = self.p2p_service.publish_message(broadcast); + let _ = p2p_service.publish_message(broadcast); }, P2pRequestEvent::BroadcastConsensusVote { vote } => { let broadcast = GossipsubBroadcastRequest::ConsensusVote(vote); - let _ = self.p2p_service.publish_message(broadcast); - } + let _ = p2p_service.publish_message(broadcast); + }, + P2pRequestEvent::Stop => break, } } else { warn!(target: "fuel-libp2p", "Failed to receive P2PRequestEvent"); @@ -131,5 +136,121 @@ impl NetworkOrchestrator { } } } + + Ok(self) + } +} + +pub struct Service { + join: Mutex>>>, + network_orchestrator: Arc>>, + tx_request_event: Sender, +} + +impl Service { + pub fn new(local_keypair: Keypair, p2p_config: P2PConfig, db: Arc>) -> Self { + let (tx_request_event, rx_request_event) = tokio::sync::mpsc::channel(100); + let (tx_consensus, _) = tokio::sync::mpsc::channel(100); + let (tx_transaction, _) = tokio::sync::mpsc::channel(100); + let (tx_block, _) = tokio::sync::mpsc::channel(100); + + let network_orchestrator = NetworkOrchestrator::new( + local_keypair, + p2p_config, + rx_request_event, + tx_consensus, + tx_transaction, + tx_block, + db, + ); + + Self { + join: Mutex::new(None), + network_orchestrator: Arc::new(Mutex::new(Some(network_orchestrator))), + tx_request_event: tx_request_event.clone(), + } + } + + pub async fn start(&self) -> anyhow::Result<()> { + let mut join = self.join.lock().await; + + if join.is_none() { + if let Some(network_orchestrator) = self.network_orchestrator.lock().await.take() { + *join = Some(tokio::spawn(async { network_orchestrator.run().await })); + + Ok(()) + } else { + Err(anyhow!("Starting Network Orchestrator that is stopping")) + } + } else { + Err(anyhow!("Network Orchestrator already started")) + } + } + + pub async fn stop(&self) -> Option> { + let join_handle = self.join.lock().await.take(); + + if let Some(join_handle) = join_handle { + let network_orchestrator = self.network_orchestrator.clone(); + let _ = self.tx_request_event.send(P2pRequestEvent::Stop).await; + Some(tokio::spawn(async move { + if let Ok(res) = join_handle.await { + *network_orchestrator.lock().await = res.ok(); + } + })) + } else { + None + } + } +} + +#[cfg(test)] +pub mod tests { + use super::*; + use async_trait::async_trait; + use fuel_core_interfaces::model::{ + BlockHeight, FuelBlock, FuelBlockConsensus, SealedFuelBlock, + }; + use tokio::time::{sleep, Duration}; + + #[derive(Clone, Debug)] + struct FakeDb; + + #[async_trait] + impl P2pDb for FakeDb { + async fn get_sealed_block(&self, _height: BlockHeight) -> Option> { + let block = FuelBlock { + header: Default::default(), + transactions: vec![], + }; + + Some(Arc::new(SealedFuelBlock { + block, + consensus: FuelBlockConsensus { + required_stake: 100_000, + validators: Default::default(), + }, + })) + } + } + + #[tokio::test] + async fn start_stop_works() { + let mut p2p_config = P2PConfig::default_with_network("start_stop_works"); + p2p_config.tcp_port = 4018; + let local_keypair = Keypair::generate_secp256k1(); + let db: Arc> = Arc::new(Box::new(FakeDb)); + + let service = Service::new(local_keypair, p2p_config, db.clone()); + + // Node with p2p service started + assert!(service.start().await.is_ok()); + sleep(Duration::from_secs(1)).await; + // Node with p2p service stopped + assert!(service.stop().await.is_some()); + sleep(Duration::from_secs(1)).await; + + // Node with p2p service successfully restarted + assert!(service.start().await.is_ok()); } } diff --git a/fuel-p2p/src/service.rs b/fuel-p2p/src/service.rs index e0117873832..324e65565f2 100644 --- a/fuel-p2p/src/service.rs +++ b/fuel-p2p/src/service.rs @@ -19,7 +19,7 @@ use libp2p::{ Multiaddr, PeerId, Swarm, }; use rand::Rng; -use std::{collections::HashMap, error::Error}; +use std::collections::HashMap; /// Listens to the events on the p2p network /// And forwards them to the Orchestrator @@ -38,7 +38,7 @@ pub enum FuelP2PEvent { } impl FuelP2PService { - pub async fn new(local_keypair: Keypair, config: P2PConfig) -> Result> { + pub async fn new(local_keypair: Keypair, config: P2PConfig) -> anyhow::Result { let local_peer_id = PeerId::from(local_keypair.public()); // configure and build P2P Serivce From f04981494fefa6db8f6600f925d4de50b07b5849 Mon Sep 17 00:00:00 2001 From: leviathanbeak88 Date: Fri, 15 Jul 2022 09:58:01 +0200 Subject: [PATCH 02/29] reuse config method --- fuel-p2p/src/service.rs | 44 ++++++++--------------------------------- 1 file changed, 8 insertions(+), 36 deletions(-) diff --git a/fuel-p2p/src/service.rs b/fuel-p2p/src/service.rs index 324e65565f2..75da29cd218 100644 --- a/fuel-p2p/src/service.rs +++ b/fuel-p2p/src/service.rs @@ -151,39 +151,11 @@ mod tests { use fuel_core_interfaces::model::{ConsensusVote, FuelBlock}; use libp2p::{gossipsub::Topic, identity::Keypair}; use std::collections::HashMap; - use std::{ - net::{IpAddr, Ipv4Addr}, - sync::Arc, - time::Duration, - }; + use std::{sync::Arc, time::Duration}; use tokio::sync::{mpsc, oneshot}; use tracing_attributes::instrument; use tracing_subscriber::{fmt, layer::SubscriberExt, EnvFilter}; - /// helper function for building default testing config - fn build_p2p_config(network_name: &str) -> P2PConfig { - P2PConfig { - network_name: network_name.into(), - address: IpAddr::V4(Ipv4Addr::from([0, 0, 0, 0])), - tcp_port: 4000, - max_block_size: 100_000, - bootstrap_nodes: vec![], - enable_mdns: false, - max_peers_connected: 50, - allow_private_addresses: true, - enable_random_walk: true, - connection_idle_timeout: Some(Duration::from_secs(120)), - topics: vec![], - max_mesh_size: 12, - min_mesh_size: 4, - ideal_mesh_size: 6, - set_request_timeout: None, - set_connection_keep_alive: None, - info_interval: Some(Duration::from_secs(3)), - identify_interval: Some(Duration::from_secs(5)), - } - } - /// Conditionally initializes tracing, depending if RUST_LOG env variable is set /// Logs to stderr & to a file #[ctor] @@ -219,7 +191,7 @@ mod tests { #[instrument] async fn p2p_service_works() { let mut fuel_p2p_service = - build_fuel_p2p_service(build_p2p_config("p2p_service_works")).await; + build_fuel_p2p_service(P2PConfig::default_with_network("p2p_service_works")).await; loop { match fuel_p2p_service.next_event().await { @@ -241,7 +213,7 @@ mod tests { #[instrument] async fn nodes_connected_via_mdns() { // Node A - let mut p2p_config = build_p2p_config("nodes_connected_via_mdns"); + let mut p2p_config = P2PConfig::default_with_network("nodes_connected_via_mdns"); p2p_config.tcp_port = 4001; p2p_config.enable_mdns = true; let mut node_a = build_fuel_p2p_service(p2p_config.clone()).await; @@ -272,7 +244,7 @@ mod tests { #[instrument] async fn nodes_connected_via_identify() { // Node A - let mut p2p_config = build_p2p_config("nodes_connected_via_identify"); + let mut p2p_config = P2PConfig::default_with_network("nodes_connected_via_identify"); p2p_config.tcp_port = 4003; let mut node_a = build_fuel_p2p_service(p2p_config.clone()).await; @@ -318,7 +290,7 @@ mod tests { #[instrument] async fn peer_info_updates_work() { // Node A - let mut p2p_config = build_p2p_config("peer_info_updates_work"); + let mut p2p_config = P2PConfig::default_with_network("peer_info_updates_work"); p2p_config.tcp_port = 4006; let mut node_a = build_fuel_p2p_service(p2p_config.clone()).await; @@ -395,7 +367,7 @@ mod tests { port_a: u16, port_b: u16, ) { - let mut p2p_config = build_p2p_config("gossipsub_exchanges_messages"); + let mut p2p_config = P2PConfig::default_with_network("gossipsub_exchanges_messages"); let topics = vec![ NEW_TX_GOSSIP_TOPIC.into(), NEW_BLOCK_GOSSIP_TOPIC.into(), @@ -491,7 +463,7 @@ mod tests { FuelBlock, FuelBlockConsensus, FuelBlockHeader, SealedFuelBlock, }; - let mut p2p_config = build_p2p_config("request_response_works"); + let mut p2p_config = P2PConfig::default_with_network("request_response_works"); // Node A p2p_config.tcp_port = 4014; @@ -578,7 +550,7 @@ mod tests { #[tokio::test] #[instrument] async fn req_res_outbound_timeout_works() { - let mut p2p_config = build_p2p_config("req_res_outbound_timeout_works"); + let mut p2p_config = P2PConfig::default_with_network("req_res_outbound_timeout_works"); // Node A p2p_config.tcp_port = 4016; From 7e5ecbfb9780779efe79c3402aebd042e59daa37 Mon Sep 17 00:00:00 2001 From: leviathanbeak88 Date: Fri, 15 Jul 2022 11:15:09 +0200 Subject: [PATCH 03/29] add local_keypair to p2p_config --- fuel-p2p/src/behavior.rs | 7 +++---- fuel-p2p/src/config.rs | 5 +++++ fuel-p2p/src/orchestrator.rs | 17 +++++------------ fuel-p2p/src/service.rs | 22 +++++++++------------- 4 files changed, 22 insertions(+), 29 deletions(-) diff --git a/fuel-p2p/src/behavior.rs b/fuel-p2p/src/behavior.rs index f0e27a3416d..603aa1bab5b 100644 --- a/fuel-p2p/src/behavior.rs +++ b/fuel-p2p/src/behavior.rs @@ -18,7 +18,6 @@ use libp2p::{ error::{PublishError, SubscriptionError}, Gossipsub, GossipsubEvent, MessageId, TopicHash, }, - identity::Keypair, request_response::{ ProtocolSupport, RequestId, RequestResponse, RequestResponseConfig, RequestResponseEvent, RequestResponseMessage, ResponseChannel, @@ -104,8 +103,8 @@ pub struct FuelBehaviour { } impl FuelBehaviour { - pub fn new(local_keypair: Keypair, p2p_config: &P2PConfig, codec: Codec) -> Self { - let local_public_key = local_keypair.public(); + pub fn new(p2p_config: &P2PConfig, codec: Codec) -> Self { + let local_public_key = p2p_config.local_keypair.public(); let local_peer_id = PeerId::from_public_key(&local_public_key); let discovery_config = { @@ -148,7 +147,7 @@ impl FuelBehaviour { Self { discovery: discovery_config.finish(), - gossipsub: gossipsub::build_gossipsub(&local_keypair, p2p_config), + gossipsub: gossipsub::build_gossipsub(&p2p_config.local_keypair, p2p_config), peer_info, request_response, diff --git a/fuel-p2p/src/config.rs b/fuel-p2p/src/config.rs index 48bc93bbd6f..15f518a0269 100644 --- a/fuel-p2p/src/config.rs +++ b/fuel-p2p/src/config.rs @@ -19,6 +19,8 @@ const TRANSPORT_TIMEOUT: Duration = Duration::from_secs(20); #[derive(Clone, Debug)] pub struct P2PConfig { + pub local_keypair: Keypair, + /// Name of the Network pub network_name: String, @@ -62,7 +64,10 @@ pub struct P2PConfig { impl P2PConfig { pub fn default_with_network(network_name: &str) -> Self { + let local_keypair = Keypair::generate_secp256k1(); + P2PConfig { + local_keypair, network_name: network_name.into(), address: IpAddr::V4(Ipv4Addr::from([0, 0, 0, 0])), tcp_port: 4000, diff --git a/fuel-p2p/src/orchestrator.rs b/fuel-p2p/src/orchestrator.rs index da879400ef2..543e768012c 100644 --- a/fuel-p2p/src/orchestrator.rs +++ b/fuel-p2p/src/orchestrator.rs @@ -5,7 +5,6 @@ use fuel_core_interfaces::p2p::{ BlockBroadcast, ConsensusBroadcast, P2pDb, P2pRequestEvent, TransactionBroadcast, }; -use libp2p::identity::Keypair; use libp2p::request_response::RequestId; use tokio::sync::mpsc::{Receiver, Sender}; use tokio::sync::Mutex; @@ -21,7 +20,6 @@ use crate::{ }; pub struct NetworkOrchestrator { - local_keypair: Keypair, p2p_config: P2PConfig, /// receives messages from different Fuel components @@ -39,7 +37,6 @@ pub struct NetworkOrchestrator { impl NetworkOrchestrator { pub fn new( - local_keypair: Keypair, p2p_config: P2PConfig, rx_request_event: Receiver, @@ -52,7 +49,6 @@ impl NetworkOrchestrator { let (tx_outbound_responses, rx_outbound_responses) = tokio::sync::mpsc::channel(100); Self { - local_keypair, p2p_config, rx_request_event, rx_outbound_responses, @@ -65,8 +61,7 @@ impl NetworkOrchestrator { } pub async fn run(mut self) -> anyhow::Result { - let mut p2p_service = - FuelP2PService::new(self.local_keypair.clone(), self.p2p_config.clone()).await?; + let mut p2p_service = FuelP2PService::new(self.p2p_config.clone()).await?; loop { tokio::select! { @@ -148,14 +143,13 @@ pub struct Service { } impl Service { - pub fn new(local_keypair: Keypair, p2p_config: P2PConfig, db: Arc>) -> Self { + pub fn new(p2p_config: P2PConfig, db: Arc>) -> Self { let (tx_request_event, rx_request_event) = tokio::sync::mpsc::channel(100); let (tx_consensus, _) = tokio::sync::mpsc::channel(100); let (tx_transaction, _) = tokio::sync::mpsc::channel(100); let (tx_block, _) = tokio::sync::mpsc::channel(100); let network_orchestrator = NetworkOrchestrator::new( - local_keypair, p2p_config, rx_request_event, tx_consensus, @@ -167,7 +161,7 @@ impl Service { Self { join: Mutex::new(None), network_orchestrator: Arc::new(Mutex::new(Some(network_orchestrator))), - tx_request_event: tx_request_event.clone(), + tx_request_event, } } @@ -237,11 +231,10 @@ pub mod tests { #[tokio::test] async fn start_stop_works() { let mut p2p_config = P2PConfig::default_with_network("start_stop_works"); - p2p_config.tcp_port = 4018; - let local_keypair = Keypair::generate_secp256k1(); + p2p_config.tcp_port = 4018; // an unused port let db: Arc> = Arc::new(Box::new(FakeDb)); - let service = Service::new(local_keypair, p2p_config, db.clone()); + let service = Service::new(p2p_config, db.clone()); // Node with p2p service started assert!(service.start().await.is_ok()); diff --git a/fuel-p2p/src/service.rs b/fuel-p2p/src/service.rs index 75da29cd218..46be9158681 100644 --- a/fuel-p2p/src/service.rs +++ b/fuel-p2p/src/service.rs @@ -12,7 +12,6 @@ use crate::{ use futures::prelude::*; use libp2p::{ gossipsub::{error::PublishError, MessageId, Topic}, - identity::Keypair, multiaddr::Protocol, request_response::RequestId, swarm::SwarmEvent, @@ -38,16 +37,12 @@ pub enum FuelP2PEvent { } impl FuelP2PService { - pub async fn new(local_keypair: Keypair, config: P2PConfig) -> anyhow::Result { - let local_peer_id = PeerId::from(local_keypair.public()); + pub async fn new(config: P2PConfig) -> anyhow::Result { + let local_peer_id = PeerId::from(config.local_keypair.public()); // configure and build P2P Serivce - let transport = build_transport(local_keypair.clone()).await; - let behaviour = FuelBehaviour::new( - local_keypair, - &config, - BincodeCodec::new(config.max_block_size), - ); + let transport = build_transport(config.local_keypair.clone()).await; + let behaviour = FuelBehaviour::new(&config, BincodeCodec::new(config.max_block_size)); let mut swarm = Swarm::new(transport, behaviour, local_peer_id); // set up node's address to listen on @@ -149,7 +144,8 @@ mod tests { use ctor::ctor; use fuel_core_interfaces::common::fuel_tx::Transaction; use fuel_core_interfaces::model::{ConsensusVote, FuelBlock}; - use libp2p::{gossipsub::Topic, identity::Keypair}; + use libp2p::gossipsub::Topic; + use libp2p::identity::Keypair; use std::collections::HashMap; use std::{sync::Arc, time::Duration}; use tokio::sync::{mpsc, oneshot}; @@ -180,9 +176,9 @@ mod tests { } /// helper function for building FuelP2PService - async fn build_fuel_p2p_service(p2p_config: P2PConfig) -> FuelP2PService { - let keypair = Keypair::generate_secp256k1(); - let fuel_p2p_service = FuelP2PService::new(keypair, p2p_config).await.unwrap(); + async fn build_fuel_p2p_service(mut p2p_config: P2PConfig) -> FuelP2PService { + p2p_config.local_keypair = Keypair::generate_secp256k1(); // change keypair for each Node + let fuel_p2p_service = FuelP2PService::new(p2p_config).await.unwrap(); fuel_p2p_service } From 5f47f0c35032bd8abdf8c2d30c76f1553db6f42a Mon Sep 17 00:00:00 2001 From: leviathanbeak88 Date: Fri, 15 Jul 2022 11:30:56 +0200 Subject: [PATCH 04/29] add network service to fuel-core --- Cargo.lock | 2 ++ fuel-core/Cargo.toml | 1 + fuel-core/src/config.rs | 2 ++ fuel-core/src/database.rs | 7 +++++++ fuel-core/src/service/modules.rs | 8 ++++++++ fuel-p2p/src/lib.rs | 3 ++- 6 files changed, 22 insertions(+), 1 deletion(-) diff --git a/Cargo.lock b/Cargo.lock index b6ec04d9aff..63672d38859 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -2118,6 +2118,7 @@ dependencies = [ "fuel-block-producer", "fuel-core-bft", "fuel-core-interfaces", + "fuel-p2p", "fuel-relayer", "fuel-sync", "fuel-txpool", @@ -2264,6 +2265,7 @@ dependencies = [ name = "fuel-p2p" version = "0.9.4" dependencies = [ + "anyhow", "async-trait", "bincode", "ctor", diff --git a/fuel-core/Cargo.toml b/fuel-core/Cargo.toml index 7e5bbd7d9cf..581716f711b 100644 --- a/fuel-core/Cargo.toml +++ b/fuel-core/Cargo.toml @@ -43,6 +43,7 @@ fuel-core-interfaces = { path = "../fuel-core-interfaces", version = "0.9.4", fe fuel-relayer = { path = "../fuel-relayer", version = "0.9.4" } fuel-sync = { path = "../fuel-sync", version = "0.9.4" } fuel-txpool = { path = "../fuel-txpool", version = "0.9.4" } +fuel-p2p = { path = "../fuel-p2p", version = "0.9.4" } futures = "0.3" graphql-parser = "0.3.0" hex = { version = "0.4", features = ["serde"] } diff --git a/fuel-core/src/config.rs b/fuel-core/src/config.rs index 426e68929ea..c77769cd3f6 100644 --- a/fuel-core/src/config.rs +++ b/fuel-core/src/config.rs @@ -26,6 +26,7 @@ pub struct Config { pub bft: fuel_core_bft::Config, pub sync: fuel_sync::Config, pub relayer: fuel_relayer::Config, + pub p2p: fuel_p2p::P2PConfig, } impl Config { @@ -46,6 +47,7 @@ impl Config { bft: Default::default(), sync: Default::default(), relayer: Default::default(), + p2p: fuel_p2p::P2PConfig::default_with_network("test_network"), } } } diff --git a/fuel-core/src/database.rs b/fuel-core/src/database.rs index 813fdbf4d00..da4565259eb 100644 --- a/fuel-core/src/database.rs +++ b/fuel-core/src/database.rs @@ -17,6 +17,7 @@ use fuel_core_interfaces::{ model::{ BlockHeight, ConsensusId, DaBlockHeight, SealedFuelBlock, ValidatorId, ValidatorStake, }, + p2p::P2pDb, relayer::{RelayerDb, StakingDiff}, txpool::TxPoolDb, }; @@ -128,6 +129,12 @@ unsafe impl Send for Database {} unsafe impl Sync for Database {} impl TxPoolDb for Database {} +#[async_trait] +impl P2pDb for Database { + async fn get_sealed_block(&self, height: BlockHeight) -> Option> { + ::get_sealed_block(&self, height).await + } +} impl Database { #[cfg(feature = "rocksdb")] diff --git a/fuel-core/src/service/modules.rs b/fuel-core/src/service/modules.rs index 653bea1d8f8..595692986f9 100644 --- a/fuel-core/src/service/modules.rs +++ b/fuel-core/src/service/modules.rs @@ -2,6 +2,7 @@ use crate::config::Config; use crate::database::Database; use anyhow::Result; +use fuel_core_interfaces::p2p::P2pDb; use fuel_core_interfaces::relayer::RelayerDb; use fuel_core_interfaces::txpool::TxPoolDb; use futures::future::join_all; @@ -15,6 +16,7 @@ pub struct Modules { pub bft: Arc, pub sync: Arc, pub relayer: Arc, + pub network_service: Arc, } impl Modules { @@ -25,6 +27,7 @@ impl Modules { self.block_producer.stop().await, self.bft.stop().await, self.sync.stop().await, + self.network_service.stop().await, ] .into_iter() .flatten() @@ -96,6 +99,10 @@ pub async fn start_modules(config: &Config, database: &Database) -> Result> = Arc::new(Box::new(database.clone())); + let network_service = fuel_p2p::orchestrator::Service::new(config.p2p.clone(), p2p_db); + network_service.start().await?; + Ok(Modules { txpool: Arc::new(txpool), block_importer: Arc::new(block_importer), @@ -103,5 +110,6 @@ pub async fn start_modules(config: &Config, database: &Database) -> Result Date: Fri, 15 Jul 2022 14:09:26 +0200 Subject: [PATCH 05/29] add p2p args --- fuel-core/src/cli.rs | 1 + fuel-core/src/cli/run.rs | 14 ++++ fuel-core/src/cli/run/p2p.rs | 150 +++++++++++++++++++++++++++++++++++ fuel-p2p/src/lib.rs | 2 +- 4 files changed, 166 insertions(+), 1 deletion(-) create mode 100644 fuel-core/src/cli/run/p2p.rs diff --git a/fuel-core/src/cli.rs b/fuel-core/src/cli.rs index 8efcd6abd7f..be91e93947d 100644 --- a/fuel-core/src/cli.rs +++ b/fuel-core/src/cli.rs @@ -23,6 +23,7 @@ pub struct Opt { command: Fuel, } +#[allow(clippy::large_enum_variant)] #[derive(Debug, Parser)] pub enum Fuel { Run(run::Command), diff --git a/fuel-core/src/cli/run.rs b/fuel-core/src/cli/run.rs index 90c4f89a232..f0a444c63c5 100644 --- a/fuel-core/src/cli/run.rs +++ b/fuel-core/src/cli/run.rs @@ -6,6 +6,7 @@ use std::{env, io, net, path::PathBuf}; use strum::VariantNames; use tracing::{info, trace}; +mod p2p; mod relayer; #[derive(Debug, Clone, Parser)] @@ -59,6 +60,9 @@ pub struct Command { #[clap(flatten)] pub relayer_args: relayer::RelayerArgs, + + #[clap(flatten)] + pub p2p_args: p2p::P2pArgs, } impl Command { @@ -76,9 +80,18 @@ impl Command { min_byte_price, predicates, relayer_args, + p2p_args, } = self; let addr = net::SocketAddr::new(ip, port); + + let p2p = { + match p2p_args.into() { + Ok(value) => value, + Err(e) => return Err(io::Error::new(io::ErrorKind::Other, e)), + } + }; + Ok(Config { addr, database_path, @@ -101,6 +114,7 @@ impl Command { relayer: relayer_args.into(), bft: Default::default(), sync: Default::default(), + p2p, }) } } diff --git a/fuel-core/src/cli/run/p2p.rs b/fuel-core/src/cli/run/p2p.rs new file mode 100644 index 00000000000..edb2d718cc8 --- /dev/null +++ b/fuel-core/src/cli/run/p2p.rs @@ -0,0 +1,150 @@ +use std::{ + net::{IpAddr, Ipv4Addr}, + path::PathBuf, + str::FromStr, + time::Duration, +}; + +use clap::Args; + +use fuel_p2p::{Keypair, Multiaddr, P2PConfig, PeerId}; + +#[derive(Debug, Clone, Args)] +pub struct P2pArgs { + /// Path to the location of DER-encoded Secp256k1 Keypair + #[clap(long = "keypair")] + pub keypair: Option, + + /// The name of the p2p Network + #[clap(long = "network")] + pub network: String, + + /// p2p network's IP Address + #[clap(long = "address")] + pub address: Option, + + /// p2p network's TCP Port + #[clap(long = "port", default_value = "4000")] + pub port: u16, + + /// Max Block size + #[clap(long = "max_block_size", default_value = "100_000")] + pub max_block_size: usize, + + /// Path to the location of where bootstrap nodes are listed + #[clap(long = "bootstrap_nodes")] + pub bootstrap_nodes: Option, + + /// Allow nodes to be discoverable on the local network + #[clap(long = "enable_mdns")] + pub enable_mdns: bool, + + /// Maximum amount of allowed connected peers + #[clap(long = "max_peers_connected", default_value = "50")] + pub max_peers_connected: usize, + + /// Enable random walk for p2p node discovery + #[clap(long = "enable_random_walk")] + pub enable_random_walk: bool, + + /// Choose to include private IPv4/IPv6 addresses as discoverable + /// xcept for the ones stored in `bootstrap_nodes` + #[clap(long = "allow_private_addresses")] + pub allow_private_addresses: bool, + + /// Choose how long will connection keep alive if idle + #[clap(long = "allow_private_addresses", default_value = "120")] + pub connection_idle_timeout: u64, + + /// Choose how often to recieve PeerInfo from other nodes + #[clap(long = "info_interval", default_value = "3")] + pub info_interval: u64, + + /// Choose the interval at which identification requests are sent to + /// the remote on established connections after the first request + #[clap(long = "identify_interval", default_value = "5")] + pub identify_interval: u64, + + /// Choose which topics to subscribe to via gossipsub protocol + #[clap(long = "topics", default_value = "new_tx, new_block, consensus_vote")] + pub topics: Vec, + + /// Choose max mesh size for gossipsub protocol + #[clap(long = "max_mesh_size")] + pub max_mesh_size: usize, + + /// Choose min mesh size for gossipsub protocol + #[clap(long = "min_mesh_size")] + pub min_mesh_size: usize, + + /// Choose ideal mesh size for gossipsub protocol + #[clap(long = "ideal_mesh_size")] + pub ideal_mesh_size: usize, + + /// Choose timeout for sent requests in RequestResponse protocol + #[clap(long = "request_timeout")] + pub request_timeout: Option, + + /// Choose how long RequestResponse protocol connections will live if idle + #[clap(long = "connection_keep_alive")] + pub connection_keep_alive: Option, +} + +impl From for anyhow::Result { + fn from(args: P2pArgs) -> Self { + let local_keypair = { + match args.keypair { + Some(path) => { + let mut bytes = std::fs::read(path)?; + Keypair::secp256k1_from_der(&mut bytes)? + } + _ => Keypair::generate_secp256k1(), + } + }; + + let bootstrap_nodes = { + match args.bootstrap_nodes { + Some(path) => std::fs::read_to_string(path)? + .lines() + .map(|line| { + let mut item = line.split_whitespace(); + + match (item.next(), item.next()) { + (Some(peer_id), Some(multiaddr)) => { + Ok((PeerId::from_str(peer_id)?, Multiaddr::from_str(multiaddr)?)) + } + _ => Err(anyhow::anyhow!("Incorrect format of bootstrap nodes")), + } + }) + .filter_map(|v| v.ok()) + .collect(), + + _ => vec![], + } + }; + + Ok(P2PConfig { + local_keypair, + network_name: args.network, + address: args + .address + .unwrap_or_else(|| IpAddr::V4(Ipv4Addr::from([0, 0, 0, 0]))), + tcp_port: args.port, + max_block_size: args.max_block_size, + bootstrap_nodes, + enable_mdns: args.enable_mdns, + max_peers_connected: args.max_peers_connected, + allow_private_addresses: args.allow_private_addresses, + enable_random_walk: args.enable_random_walk, + connection_idle_timeout: Some(Duration::from_secs(args.connection_idle_timeout)), + topics: args.topics, + max_mesh_size: args.max_mesh_size, + min_mesh_size: args.min_mesh_size, + ideal_mesh_size: args.ideal_mesh_size, + set_request_timeout: args.request_timeout.map(Duration::from_secs), + set_connection_keep_alive: args.connection_keep_alive.map(Duration::from_secs), + info_interval: Some(Duration::from_secs(args.info_interval)), + identify_interval: Some(Duration::from_secs(args.identify_interval)), + }) + } +} diff --git a/fuel-p2p/src/lib.rs b/fuel-p2p/src/lib.rs index 88918fba852..01a280270b9 100644 --- a/fuel-p2p/src/lib.rs +++ b/fuel-p2p/src/lib.rs @@ -8,4 +8,4 @@ mod peer_info; mod request_response; mod service; pub use config::P2PConfig; -pub use libp2p::identity::Keypair; +pub use libp2p::{identity::Keypair, Multiaddr, PeerId}; From 1d1f0fda1233bf0652043aa368edb78f5c2907d9 Mon Sep 17 00:00:00 2001 From: leviathanbeak88 Date: Fri, 15 Jul 2022 14:45:43 +0200 Subject: [PATCH 06/29] move channels out of service initialization --- fuel-core/src/service/modules.rs | 17 ++++++++++++++++- fuel-p2p/src/orchestrator.rs | 30 +++++++++++++++++++++++------- 2 files changed, 39 insertions(+), 8 deletions(-) diff --git a/fuel-core/src/service/modules.rs b/fuel-core/src/service/modules.rs index 595692986f9..c2a91ae4405 100644 --- a/fuel-core/src/service/modules.rs +++ b/fuel-core/src/service/modules.rs @@ -7,6 +7,7 @@ use fuel_core_interfaces::relayer::RelayerDb; use fuel_core_interfaces::txpool::TxPoolDb; use futures::future::join_all; use std::sync::Arc; +use tokio::sync::mpsc; use tokio::task::JoinHandle; pub struct Modules { @@ -100,7 +101,21 @@ pub async fn start_modules(config: &Config, database: &Database) -> Result> = Arc::new(Box::new(database.clone())); - let network_service = fuel_p2p::orchestrator::Service::new(config.p2p.clone(), p2p_db); + + let (tx_request_event, rx_request_event) = mpsc::channel(100); + let (tx_consensus, _) = mpsc::channel(100); + let (tx_transaction, _) = mpsc::channel(100); + let (tx_block, _) = mpsc::channel(100); + + let network_service = fuel_p2p::orchestrator::Service::new( + config.p2p.clone(), + p2p_db, + tx_request_event, + rx_request_event, + tx_consensus, + tx_transaction, + tx_block, + ); network_service.start().await?; Ok(Modules { diff --git a/fuel-p2p/src/orchestrator.rs b/fuel-p2p/src/orchestrator.rs index 543e768012c..643e4ee736f 100644 --- a/fuel-p2p/src/orchestrator.rs +++ b/fuel-p2p/src/orchestrator.rs @@ -143,12 +143,15 @@ pub struct Service { } impl Service { - pub fn new(p2p_config: P2PConfig, db: Arc>) -> Self { - let (tx_request_event, rx_request_event) = tokio::sync::mpsc::channel(100); - let (tx_consensus, _) = tokio::sync::mpsc::channel(100); - let (tx_transaction, _) = tokio::sync::mpsc::channel(100); - let (tx_block, _) = tokio::sync::mpsc::channel(100); - + pub fn new( + p2p_config: P2PConfig, + db: Arc>, + tx_request_event: Sender, + rx_request_event: Receiver, + tx_consensus: Sender, + tx_transaction: Sender, + tx_block: Sender, + ) -> Self { let network_orchestrator = NetworkOrchestrator::new( p2p_config, rx_request_event, @@ -234,7 +237,20 @@ pub mod tests { p2p_config.tcp_port = 4018; // an unused port let db: Arc> = Arc::new(Box::new(FakeDb)); - let service = Service::new(p2p_config, db.clone()); + let (tx_request_event, rx_request_event) = tokio::sync::mpsc::channel(100); + let (tx_consensus, _) = tokio::sync::mpsc::channel(100); + let (tx_transaction, _) = tokio::sync::mpsc::channel(100); + let (tx_block, _) = tokio::sync::mpsc::channel(100); + + let service = Service::new( + p2p_config, + db.clone(), + tx_request_event, + rx_request_event, + tx_consensus, + tx_transaction, + tx_block, + ); // Node with p2p service started assert!(service.start().await.is_ok()); From f9f1d17969ca4e99a7780e13f6f038d152690d46 Mon Sep 17 00:00:00 2001 From: leviathanbeak88 Date: Fri, 15 Jul 2022 17:13:06 +0200 Subject: [PATCH 07/29] add p2p channels to other modules/services --- fuel-core-bft/src/service.rs | 3 ++- fuel-core/src/service/modules.rs | 13 +++++-------- fuel-sync/src/service.rs | 10 +++++++--- 3 files changed, 14 insertions(+), 12 deletions(-) diff --git a/fuel-core-bft/src/service.rs b/fuel-core-bft/src/service.rs index bb94eecaed2..7dd488759ac 100644 --- a/fuel-core-bft/src/service.rs +++ b/fuel-core-bft/src/service.rs @@ -3,6 +3,7 @@ use fuel_core_interfaces::{ bft::BftMpsc, block_importer::{ImportBlockBroadcast, ImportBlockMpsc}, block_producer::BlockProducerMpsc, + p2p::P2pRequestEvent, relayer, }; use parking_lot::Mutex; @@ -28,7 +29,7 @@ impl Service { pub async fn start( &self, _relayer: relayer::Sender, - _p2p_consensus: (), + _p2p_consensus: mpsc::Sender, _block_producer: mpsc::Sender, _block_importer_sender: mpsc::Sender, _block_importer_broadcast: broadcast::Receiver, diff --git a/fuel-core/src/service/modules.rs b/fuel-core/src/service/modules.rs index c2a91ae4405..937bf7383ab 100644 --- a/fuel-core/src/service/modules.rs +++ b/fuel-core/src/service/modules.rs @@ -65,16 +65,15 @@ pub async fn start_modules(config: &Config, database: &Database) -> Result) .import_block_event(block_importer.subscribe()); - let p2p_mpsc = (); - let p2p_broadcast_consensus = (); - let p2p_broadcast_block = (); + let (tx_request_event, rx_request_event) = mpsc::channel(100); + let (tx_block, rx_block) = mpsc::channel(100); block_importer.start().await; block_producer.start(txpool_builder.sender().clone()).await; bft.start( relayer_builder.sender().clone(), - p2p_broadcast_consensus, + tx_request_event.clone(), block_producer.sender().clone(), block_importer.sender().clone(), block_importer.subscribe(), @@ -82,8 +81,8 @@ pub async fn start_modules(config: &Config, database: &Database) -> Result Result> = Arc::new(Box::new(database.clone())); - let (tx_request_event, rx_request_event) = mpsc::channel(100); let (tx_consensus, _) = mpsc::channel(100); let (tx_transaction, _) = mpsc::channel(100); - let (tx_block, _) = mpsc::channel(100); let network_service = fuel_p2p::orchestrator::Service::new( config.p2p.clone(), diff --git a/fuel-sync/src/service.rs b/fuel-sync/src/service.rs index 06a70ff7112..4b0fc16e11b 100644 --- a/fuel-sync/src/service.rs +++ b/fuel-sync/src/service.rs @@ -1,6 +1,10 @@ use crate::Config; use fuel_core_interfaces::{ - bft::BftMpsc, block_importer::ImportBlockMpsc, relayer, sync::SyncMpsc, + bft::BftMpsc, + block_importer::ImportBlockMpsc, + p2p::{BlockBroadcast, P2pRequestEvent}, + relayer, + sync::SyncMpsc, }; use parking_lot::Mutex; use tokio::{sync::mpsc, task::JoinHandle}; @@ -21,8 +25,8 @@ impl Service { pub async fn start( &self, - _p2p_block: (), // broadcast::Receiver, - _p2p_request: (), // mpsc::Sender, + _p2p_block: mpsc::Receiver, + _p2p_request: mpsc::Sender, _relayer: relayer::Sender, _bft: mpsc::Sender, _block_importer: mpsc::Sender, From 49677f96be7a992e8f938b3c4052f560bf29bae5 Mon Sep 17 00:00:00 2001 From: ControlC ControlV Date: Thu, 28 Jul 2022 15:38:57 -0600 Subject: [PATCH 08/29] fmt is hard in the online editor --- fuel-p2p/src/service.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/fuel-p2p/src/service.rs b/fuel-p2p/src/service.rs index a2e59ecea5b..315dae0d224 100644 --- a/fuel-p2p/src/service.rs +++ b/fuel-p2p/src/service.rs @@ -39,7 +39,7 @@ pub enum FuelP2PEvent { impl FuelP2PService { pub async fn new(config: P2PConfig) -> anyhow::Result { let local_peer_id = PeerId::from(config.local_keypair.public()); - + // configure and build P2P Service let transport = build_transport(local_keypair.clone()).await; let behaviour = FuelBehaviour::new( From 9cbc3472f6451174dde73434323595595d3ee6aa Mon Sep 17 00:00:00 2001 From: ControlC ControlV Date: Thu, 28 Jul 2022 15:43:28 -0600 Subject: [PATCH 09/29] ooops --- fuel-core/src/database.rs | 2 +- fuel-p2p/src/service.rs | 3 +-- 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/fuel-core/src/database.rs b/fuel-core/src/database.rs index 1950eaddc28..3cae943ead7 100644 --- a/fuel-core/src/database.rs +++ b/fuel-core/src/database.rs @@ -132,7 +132,7 @@ impl TxPoolDb for Database {} #[async_trait] impl P2pDb for Database { async fn get_sealed_block(&self, height: BlockHeight) -> Option> { - ::get_sealed_block(&self, height).await + ::get_sealed_block(self, height).await } } diff --git a/fuel-p2p/src/service.rs b/fuel-p2p/src/service.rs index 315dae0d224..c299bcb832f 100644 --- a/fuel-p2p/src/service.rs +++ b/fuel-p2p/src/service.rs @@ -41,9 +41,8 @@ impl FuelP2PService { let local_peer_id = PeerId::from(config.local_keypair.public()); // configure and build P2P Service - let transport = build_transport(local_keypair.clone()).await; + let transport = build_transport(config.local_keypair.clone()).await; let behaviour = FuelBehaviour::new( - local_keypair, &config, BincodeCodec::new(config.max_block_size), ); From f29f0b5af940cdaae6908a999f8296992838edc4 Mon Sep 17 00:00:00 2001 From: ControlC ControlV Date: Thu, 28 Jul 2022 15:45:30 -0600 Subject: [PATCH 10/29] Sorry for ruining commit history --- fuel-p2p/src/service.rs | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/fuel-p2p/src/service.rs b/fuel-p2p/src/service.rs index c299bcb832f..c1b43f17cda 100644 --- a/fuel-p2p/src/service.rs +++ b/fuel-p2p/src/service.rs @@ -42,10 +42,7 @@ impl FuelP2PService { // configure and build P2P Service let transport = build_transport(config.local_keypair.clone()).await; - let behaviour = FuelBehaviour::new( - &config, - BincodeCodec::new(config.max_block_size), - ); + let behaviour = FuelBehaviour::new(&config, BincodeCodec::new(config.max_block_size)); let mut swarm = Swarm::new(transport, behaviour, local_peer_id); // set up node's address to listen on From d88d81af19613ad9c74dd1ed8db13bb6330a592f Mon Sep 17 00:00:00 2001 From: ControlC ControlV Date: Thu, 28 Jul 2022 15:48:41 -0600 Subject: [PATCH 11/29] oops --- fuel-core/Cargo.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/fuel-core/Cargo.toml b/fuel-core/Cargo.toml index 3bbe33b157b..673b61da074 100644 --- a/fuel-core/Cargo.toml +++ b/fuel-core/Cargo.toml @@ -40,10 +40,10 @@ fuel-core-bft = { path = "../fuel-core-bft", version = "0.9.5" } fuel-core-interfaces = { path = "../fuel-core-interfaces", version = "0.9.5", features = [ "serde", ] } +fuel-p2p = { path = "../fuel-p2p", version = "0.9.5" } fuel-relayer = { path = "../fuel-relayer", version = "0.9.5" } fuel-sync = { path = "../fuel-sync", version = "0.9.5" } fuel-txpool = { path = "../fuel-txpool", version = "0.9.5" } -fuel-p2p = { path = "../fuel-p2p", version = "0.9.5" } futures = "0.3" graphql-parser = "0.3.0" hex = { version = "0.4", features = ["serde"] } From fe7d5fb614891927bb2c6fb33a69b57da89568fb Mon Sep 17 00:00:00 2001 From: leviathanbeak88 Date: Tue, 2 Aug 2022 17:41:32 +0200 Subject: [PATCH 12/29] pass bootstrap nodes in a vector instead --- fuel-core/src/cli/run/p2p.rs | 33 ++++------------------ fuel-p2p/src/config.rs | 2 +- fuel-p2p/src/discovery.rs | 12 ++++++-- fuel-p2p/src/discovery/discovery_config.rs | 20 ++++++++++--- fuel-p2p/src/service.rs | 31 ++++++++++++++++---- 5 files changed, 59 insertions(+), 39 deletions(-) diff --git a/fuel-core/src/cli/run/p2p.rs b/fuel-core/src/cli/run/p2p.rs index edb2d718cc8..47382b4dc5a 100644 --- a/fuel-core/src/cli/run/p2p.rs +++ b/fuel-core/src/cli/run/p2p.rs @@ -1,13 +1,12 @@ use std::{ net::{IpAddr, Ipv4Addr}, path::PathBuf, - str::FromStr, time::Duration, }; use clap::Args; -use fuel_p2p::{Keypair, Multiaddr, P2PConfig, PeerId}; +use fuel_p2p::{Keypair, Multiaddr, P2PConfig}; #[derive(Debug, Clone, Args)] pub struct P2pArgs { @@ -31,9 +30,10 @@ pub struct P2pArgs { #[clap(long = "max_block_size", default_value = "100_000")] pub max_block_size: usize, - /// Path to the location of where bootstrap nodes are listed + /// Addresses of the bootstrap nodes + /// They should contain PeerId at the end of the specified Multiaddr #[clap(long = "bootstrap_nodes")] - pub bootstrap_nodes: Option, + pub bootstrap_nodes: Option>, /// Allow nodes to be discoverable on the local network #[clap(long = "enable_mdns")] @@ -48,7 +48,7 @@ pub struct P2pArgs { pub enable_random_walk: bool, /// Choose to include private IPv4/IPv6 addresses as discoverable - /// xcept for the ones stored in `bootstrap_nodes` + /// except for the ones stored in `bootstrap_nodes` #[clap(long = "allow_private_addresses")] pub allow_private_addresses: bool, @@ -102,27 +102,6 @@ impl From for anyhow::Result { } }; - let bootstrap_nodes = { - match args.bootstrap_nodes { - Some(path) => std::fs::read_to_string(path)? - .lines() - .map(|line| { - let mut item = line.split_whitespace(); - - match (item.next(), item.next()) { - (Some(peer_id), Some(multiaddr)) => { - Ok((PeerId::from_str(peer_id)?, Multiaddr::from_str(multiaddr)?)) - } - _ => Err(anyhow::anyhow!("Incorrect format of bootstrap nodes")), - } - }) - .filter_map(|v| v.ok()) - .collect(), - - _ => vec![], - } - }; - Ok(P2PConfig { local_keypair, network_name: args.network, @@ -131,7 +110,7 @@ impl From for anyhow::Result { .unwrap_or_else(|| IpAddr::V4(Ipv4Addr::from([0, 0, 0, 0]))), tcp_port: args.port, max_block_size: args.max_block_size, - bootstrap_nodes, + bootstrap_nodes: args.bootstrap_nodes.unwrap_or(vec![]), enable_mdns: args.enable_mdns, max_peers_connected: args.max_peers_connected, allow_private_addresses: args.allow_private_addresses, diff --git a/fuel-p2p/src/config.rs b/fuel-p2p/src/config.rs index 15f518a0269..358ebca3a9f 100644 --- a/fuel-p2p/src/config.rs +++ b/fuel-p2p/src/config.rs @@ -34,7 +34,7 @@ pub struct P2PConfig { pub max_block_size: usize, // `DiscoveryBehaviour` related fields - pub bootstrap_nodes: Vec<(PeerId, Multiaddr)>, + pub bootstrap_nodes: Vec, pub enable_mdns: bool, pub max_peers_connected: usize, pub allow_private_addresses: bool, diff --git a/fuel-p2p/src/discovery.rs b/fuel-p2p/src/discovery.rs index e47372553a9..4fb86311b1d 100644 --- a/fuel-p2p/src/discovery.rs +++ b/fuel-p2p/src/discovery.rs @@ -342,7 +342,7 @@ mod tests { /// helper function for building Discovery Behaviour for testing fn build_fuel_discovery( - bootstrap_nodes: Vec<(PeerId, Multiaddr)>, + bootstrap_nodes: Vec, ) -> (Swarm, Multiaddr, PeerId) { let keypair = Keypair::generate_secp256k1(); let public_key = keypair.public(); @@ -390,7 +390,15 @@ mod tests { let (first_swarm, first_peer_addr, first_peer_id) = build_fuel_discovery(vec![]); let mut discovery_swarms = (0..num_of_swarms - 1) - .map(|_| build_fuel_discovery(vec![(first_peer_id, first_peer_addr.clone())])) + .map(|_| { + build_fuel_discovery(vec![format!( + "{}/p2p/{}", + first_peer_addr.clone(), + first_peer_id + ) + .parse() + .unwrap()]) + }) .collect::>(); discovery_swarms.push_front((first_swarm, first_peer_addr, first_peer_id)); diff --git a/fuel-p2p/src/discovery/discovery_config.rs b/fuel-p2p/src/discovery/discovery_config.rs index 6dd6b168a51..88d65bfeaef 100644 --- a/fuel-p2p/src/discovery/discovery_config.rs +++ b/fuel-p2p/src/discovery/discovery_config.rs @@ -13,7 +13,7 @@ use tracing::warn; #[derive(Clone, Debug)] pub struct DiscoveryConfig { local_peer_id: PeerId, - bootstrap_nodes: Vec<(PeerId, Multiaddr)>, + bootstrap_nodes: Vec, with_mdns: bool, with_random_walk: bool, allow_private_addresses: bool, @@ -57,7 +57,7 @@ impl DiscoveryConfig { // List of bootstrap nodes to bootstrap the network pub fn with_bootstrap_nodes(&mut self, bootstrap_nodes: I) -> &mut Self where - I: IntoIterator, + I: IntoIterator, { self.bootstrap_nodes.extend(bootstrap_nodes); self @@ -92,8 +92,20 @@ impl DiscoveryConfig { kademlia_config.set_connection_idle_timeout(connection_idle_timeout); let mut kademlia = Kademlia::with_config(local_peer_id, memory_store, kademlia_config); - for (peer_id, addr) in &bootstrap_nodes { - kademlia.add_address(peer_id, addr.clone()); + // bootstrap nodes need to have their peer_id defined in the Multiaddr + let bootstrap_nodes = bootstrap_nodes + .into_iter() + .filter_map(|node| { + if let Some(peer_id) = PeerId::try_from_multiaddr(&node) { + Some((peer_id, node)) + } else { + None + } + }) + .collect::>(); + + for (peer_id, address) in &bootstrap_nodes { + kademlia.add_address(&peer_id, address.clone()); } if let Err(e) = kademlia.bootstrap() { diff --git a/fuel-p2p/src/service.rs b/fuel-p2p/src/service.rs index 46be9158681..b861487963d 100644 --- a/fuel-p2p/src/service.rs +++ b/fuel-p2p/src/service.rs @@ -146,6 +146,7 @@ mod tests { use fuel_core_interfaces::model::{ConsensusVote, FuelBlock}; use libp2p::gossipsub::Topic; use libp2p::identity::Keypair; + use libp2p::{Multiaddr, PeerId}; use std::collections::HashMap; use std::{sync::Arc, time::Duration}; use tokio::sync::{mpsc, oneshot}; @@ -183,6 +184,11 @@ mod tests { fuel_p2p_service } + /// attaches PeerId to the Multiaddr + fn build_bootstrap_node(peer_id: PeerId, address: Multiaddr) -> Multiaddr { + format!("{}/p2p/{}", address, peer_id).parse().unwrap() + } + #[tokio::test] #[instrument] async fn p2p_service_works() { @@ -251,7 +257,10 @@ mod tests { // Node B p2p_config.tcp_port = 4004; - p2p_config.bootstrap_nodes = vec![(node_a.local_peer_id, node_a_address.clone().unwrap())]; + p2p_config.bootstrap_nodes = vec![build_bootstrap_node( + node_a.local_peer_id, + node_a_address.clone().unwrap(), + )]; let mut node_b = build_fuel_p2p_service(p2p_config.clone()).await; // Node C @@ -297,7 +306,10 @@ mod tests { // Node B p2p_config.tcp_port = 4007; - p2p_config.bootstrap_nodes = vec![(node_a.local_peer_id, node_a_address.clone().unwrap())]; + p2p_config.bootstrap_nodes = vec![build_bootstrap_node( + node_a.local_peer_id, + node_a_address.clone().unwrap(), + )]; let mut node_b = build_fuel_p2p_service(p2p_config).await; loop { @@ -394,7 +406,10 @@ mod tests { // Node B p2p_config.tcp_port = port_b; - p2p_config.bootstrap_nodes = vec![(node_a.local_peer_id, node_a_address.clone().unwrap())]; + p2p_config.bootstrap_nodes = vec![build_bootstrap_node( + node_a.local_peer_id, + node_a_address.clone().unwrap(), + )]; let mut node_b = build_fuel_p2p_service(p2p_config.clone()).await; loop { @@ -472,7 +487,10 @@ mod tests { // Node B p2p_config.tcp_port = 4015; - p2p_config.bootstrap_nodes = vec![(node_a.local_peer_id, node_a_address.clone().unwrap())]; + p2p_config.bootstrap_nodes = vec![build_bootstrap_node( + node_a.local_peer_id, + node_a_address.clone().unwrap(), + )]; let mut node_b = build_fuel_p2p_service(p2p_config.clone()).await; let (tx_test_end, mut rx_test_end) = mpsc::channel(1); @@ -561,7 +579,10 @@ mod tests { // Node B p2p_config.tcp_port = 4017; - p2p_config.bootstrap_nodes = vec![(node_a.local_peer_id, node_a_address.clone().unwrap())]; + p2p_config.bootstrap_nodes = vec![build_bootstrap_node( + node_a.local_peer_id, + node_a_address.clone().unwrap(), + )]; let mut node_b = build_fuel_p2p_service(p2p_config.clone()).await; let (tx_test_end, mut rx_test_end) = tokio::sync::mpsc::channel(1); From 156e2d6cd95c5d247ce915f3d96afc5ee8cca44a Mon Sep 17 00:00:00 2001 From: leviathanbeak88 Date: Tue, 2 Aug 2022 17:52:50 +0200 Subject: [PATCH 13/29] remove box from db trait object --- fuel-core/src/service/modules.rs | 2 +- fuel-p2p/src/orchestrator.rs | 8 ++++---- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/fuel-core/src/service/modules.rs b/fuel-core/src/service/modules.rs index 937bf7383ab..257c447ec35 100644 --- a/fuel-core/src/service/modules.rs +++ b/fuel-core/src/service/modules.rs @@ -99,7 +99,7 @@ pub async fn start_modules(config: &Config, database: &Database) -> Result> = Arc::new(Box::new(database.clone())); + let p2p_db: Arc = Arc::new(database.clone()); let (tx_consensus, _) = mpsc::channel(100); let (tx_transaction, _) = mpsc::channel(100); diff --git a/fuel-p2p/src/orchestrator.rs b/fuel-p2p/src/orchestrator.rs index 643e4ee736f..0cc65a285e0 100644 --- a/fuel-p2p/src/orchestrator.rs +++ b/fuel-p2p/src/orchestrator.rs @@ -32,7 +32,7 @@ pub struct NetworkOrchestrator { tx_block: Sender, tx_outbound_responses: Sender>, - db: Arc>, + db: Arc, } impl NetworkOrchestrator { @@ -44,7 +44,7 @@ impl NetworkOrchestrator { tx_transaction: Sender, tx_block: Sender, - db: Arc>, + db: Arc, ) -> Self { let (tx_outbound_responses, rx_outbound_responses) = tokio::sync::mpsc::channel(100); @@ -145,7 +145,7 @@ pub struct Service { impl Service { pub fn new( p2p_config: P2PConfig, - db: Arc>, + db: Arc, tx_request_event: Sender, rx_request_event: Receiver, tx_consensus: Sender, @@ -235,7 +235,7 @@ pub mod tests { async fn start_stop_works() { let mut p2p_config = P2PConfig::default_with_network("start_stop_works"); p2p_config.tcp_port = 4018; // an unused port - let db: Arc> = Arc::new(Box::new(FakeDb)); + let db: Arc = Arc::new(FakeDb); let (tx_request_event, rx_request_event) = tokio::sync::mpsc::channel(100); let (tx_consensus, _) = tokio::sync::mpsc::channel(100); From b8d08daa8781cb69848e43b881b64da5500af44c Mon Sep 17 00:00:00 2001 From: leviathanbeak88 Date: Tue, 2 Aug 2022 19:25:43 +0200 Subject: [PATCH 14/29] update peering port --- fuel-core/src/cli/run/p2p.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/fuel-core/src/cli/run/p2p.rs b/fuel-core/src/cli/run/p2p.rs index 47382b4dc5a..a12de134004 100644 --- a/fuel-core/src/cli/run/p2p.rs +++ b/fuel-core/src/cli/run/p2p.rs @@ -23,7 +23,7 @@ pub struct P2pArgs { pub address: Option, /// p2p network's TCP Port - #[clap(long = "port", default_value = "4000")] + #[clap(long = "peering-port", default_value = "4001")] pub port: u16, /// Max Block size From 566318384a9afd2ad508cd2a3e528ba94e7a4b07 Mon Sep 17 00:00:00 2001 From: leviathanbeak88 Date: Wed, 3 Aug 2022 09:37:23 +0200 Subject: [PATCH 15/29] cleanup filter map --- fuel-p2p/src/discovery/discovery_config.rs | 10 ++-------- 1 file changed, 2 insertions(+), 8 deletions(-) diff --git a/fuel-p2p/src/discovery/discovery_config.rs b/fuel-p2p/src/discovery/discovery_config.rs index 88d65bfeaef..b45823d381d 100644 --- a/fuel-p2p/src/discovery/discovery_config.rs +++ b/fuel-p2p/src/discovery/discovery_config.rs @@ -95,17 +95,11 @@ impl DiscoveryConfig { // bootstrap nodes need to have their peer_id defined in the Multiaddr let bootstrap_nodes = bootstrap_nodes .into_iter() - .filter_map(|node| { - if let Some(peer_id) = PeerId::try_from_multiaddr(&node) { - Some((peer_id, node)) - } else { - None - } - }) + .filter_map(|node| PeerId::try_from_multiaddr(&node).map(|peer_id| (peer_id, node))) .collect::>(); for (peer_id, address) in &bootstrap_nodes { - kademlia.add_address(&peer_id, address.clone()); + kademlia.add_address(peer_id, address.clone()); } if let Err(e) = kademlia.bootstrap() { From 46e952895ee32521f69262c5992af137ddba40a5 Mon Sep 17 00:00:00 2001 From: leviathanbeak88 Date: Wed, 3 Aug 2022 09:49:46 +0200 Subject: [PATCH 16/29] unwrap to default --- fuel-core/src/cli/run/p2p.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/fuel-core/src/cli/run/p2p.rs b/fuel-core/src/cli/run/p2p.rs index a12de134004..31e5404de37 100644 --- a/fuel-core/src/cli/run/p2p.rs +++ b/fuel-core/src/cli/run/p2p.rs @@ -110,7 +110,7 @@ impl From for anyhow::Result { .unwrap_or_else(|| IpAddr::V4(Ipv4Addr::from([0, 0, 0, 0]))), tcp_port: args.port, max_block_size: args.max_block_size, - bootstrap_nodes: args.bootstrap_nodes.unwrap_or(vec![]), + bootstrap_nodes: args.bootstrap_nodes.unwrap_or_default(), enable_mdns: args.enable_mdns, max_peers_connected: args.max_peers_connected, allow_private_addresses: args.allow_private_addresses, From 31d1c8606c33836d9c1190895404244fc5f5bf54 Mon Sep 17 00:00:00 2001 From: leviathanbeak88 Date: Wed, 3 Aug 2022 21:01:58 +0200 Subject: [PATCH 17/29] add default values --- fuel-core/src/cli/run/p2p.rs | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/fuel-core/src/cli/run/p2p.rs b/fuel-core/src/cli/run/p2p.rs index 31e5404de37..a421374277a 100644 --- a/fuel-core/src/cli/run/p2p.rs +++ b/fuel-core/src/cli/run/p2p.rs @@ -15,7 +15,8 @@ pub struct P2pArgs { pub keypair: Option, /// The name of the p2p Network - #[clap(long = "network")] + /// If this value is not provided the p2p network won't start + #[clap(long = "network", default_value = "")] pub network: String, /// p2p network's IP Address @@ -70,15 +71,15 @@ pub struct P2pArgs { pub topics: Vec, /// Choose max mesh size for gossipsub protocol - #[clap(long = "max_mesh_size")] + #[clap(long = "max_mesh_size", default_value = "12")] pub max_mesh_size: usize, /// Choose min mesh size for gossipsub protocol - #[clap(long = "min_mesh_size")] + #[clap(long = "min_mesh_size", default_value = "4")] pub min_mesh_size: usize, /// Choose ideal mesh size for gossipsub protocol - #[clap(long = "ideal_mesh_size")] + #[clap(long = "ideal_mesh_size", default_value = "6")] pub ideal_mesh_size: usize, /// Choose timeout for sent requests in RequestResponse protocol From 03fae453de598624e0979c8107ab8dd048198ec6 Mon Sep 17 00:00:00 2001 From: leviathanbeak88 Date: Wed, 3 Aug 2022 21:02:17 +0200 Subject: [PATCH 18/29] start p2p network service only if name is provided --- fuel-core/src/service/modules.rs | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/fuel-core/src/service/modules.rs b/fuel-core/src/service/modules.rs index 257c447ec35..bb663780464 100644 --- a/fuel-core/src/service/modules.rs +++ b/fuel-core/src/service/modules.rs @@ -113,7 +113,10 @@ pub async fn start_modules(config: &Config, database: &Database) -> Result Date: Wed, 3 Aug 2022 22:09:16 +0200 Subject: [PATCH 19/29] update cli values --- fuel-core/src/cli/run/p2p.rs | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/fuel-core/src/cli/run/p2p.rs b/fuel-core/src/cli/run/p2p.rs index a421374277a..7028fe56c74 100644 --- a/fuel-core/src/cli/run/p2p.rs +++ b/fuel-core/src/cli/run/p2p.rs @@ -25,10 +25,10 @@ pub struct P2pArgs { /// p2p network's TCP Port #[clap(long = "peering-port", default_value = "4001")] - pub port: u16, + pub peering_port: u16, /// Max Block size - #[clap(long = "max_block_size", default_value = "100_000")] + #[clap(long = "max_block_size", default_value = "100000")] pub max_block_size: usize, /// Addresses of the bootstrap nodes @@ -54,7 +54,7 @@ pub struct P2pArgs { pub allow_private_addresses: bool, /// Choose how long will connection keep alive if idle - #[clap(long = "allow_private_addresses", default_value = "120")] + #[clap(long = "connection_idle_timeout ", default_value = "120")] pub connection_idle_timeout: u64, /// Choose how often to recieve PeerInfo from other nodes @@ -67,7 +67,7 @@ pub struct P2pArgs { pub identify_interval: u64, /// Choose which topics to subscribe to via gossipsub protocol - #[clap(long = "topics", default_value = "new_tx, new_block, consensus_vote")] + #[clap(long = "topics")] pub topics: Vec, /// Choose max mesh size for gossipsub protocol @@ -109,7 +109,7 @@ impl From for anyhow::Result { address: args .address .unwrap_or_else(|| IpAddr::V4(Ipv4Addr::from([0, 0, 0, 0]))), - tcp_port: args.port, + tcp_port: args.peering_port, max_block_size: args.max_block_size, bootstrap_nodes: args.bootstrap_nodes.unwrap_or_default(), enable_mdns: args.enable_mdns, From d5a2b17602e5071ff081288b56c33f1770b35163 Mon Sep 17 00:00:00 2001 From: leviathanbeak88 Date: Wed, 3 Aug 2022 22:14:59 +0200 Subject: [PATCH 20/29] add correct default values for topics --- fuel-core/src/cli/run/p2p.rs | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/fuel-core/src/cli/run/p2p.rs b/fuel-core/src/cli/run/p2p.rs index 7028fe56c74..9ae56d009f2 100644 --- a/fuel-core/src/cli/run/p2p.rs +++ b/fuel-core/src/cli/run/p2p.rs @@ -34,7 +34,7 @@ pub struct P2pArgs { /// Addresses of the bootstrap nodes /// They should contain PeerId at the end of the specified Multiaddr #[clap(long = "bootstrap_nodes")] - pub bootstrap_nodes: Option>, + pub bootstrap_nodes: Vec, /// Allow nodes to be discoverable on the local network #[clap(long = "enable_mdns")] @@ -67,7 +67,7 @@ pub struct P2pArgs { pub identify_interval: u64, /// Choose which topics to subscribe to via gossipsub protocol - #[clap(long = "topics")] + #[clap(long = "topics", default_values = &["new_tx", "new_block", "consensus_vote"])] pub topics: Vec, /// Choose max mesh size for gossipsub protocol @@ -111,7 +111,7 @@ impl From for anyhow::Result { .unwrap_or_else(|| IpAddr::V4(Ipv4Addr::from([0, 0, 0, 0]))), tcp_port: args.peering_port, max_block_size: args.max_block_size, - bootstrap_nodes: args.bootstrap_nodes.unwrap_or_default(), + bootstrap_nodes: args.bootstrap_nodes, enable_mdns: args.enable_mdns, max_peers_connected: args.max_peers_connected, allow_private_addresses: args.allow_private_addresses, From d66ac427d8b464c2e7949a8a40d184970531b5a2 Mon Sep 17 00:00:00 2001 From: leviathanbeak88 Date: Tue, 9 Aug 2022 14:10:35 +0200 Subject: [PATCH 21/29] set tcp port to be chosen automatically --- fuel-p2p/src/config.rs | 2 +- fuel-p2p/src/orchestrator.rs | 3 +-- fuel-p2p/src/service.rs | 43 +++++++++--------------------------- 3 files changed, 12 insertions(+), 36 deletions(-) diff --git a/fuel-p2p/src/config.rs b/fuel-p2p/src/config.rs index 358ebca3a9f..d4bff32c667 100644 --- a/fuel-p2p/src/config.rs +++ b/fuel-p2p/src/config.rs @@ -70,7 +70,7 @@ impl P2PConfig { local_keypair, network_name: network_name.into(), address: IpAddr::V4(Ipv4Addr::from([0, 0, 0, 0])), - tcp_port: 4000, + tcp_port: 0, max_block_size: 100_000, bootstrap_nodes: vec![], enable_mdns: false, diff --git a/fuel-p2p/src/orchestrator.rs b/fuel-p2p/src/orchestrator.rs index 0cc65a285e0..eb4b4c1c0c4 100644 --- a/fuel-p2p/src/orchestrator.rs +++ b/fuel-p2p/src/orchestrator.rs @@ -233,8 +233,7 @@ pub mod tests { #[tokio::test] async fn start_stop_works() { - let mut p2p_config = P2PConfig::default_with_network("start_stop_works"); - p2p_config.tcp_port = 4018; // an unused port + let p2p_config = P2PConfig::default_with_network("start_stop_works"); let db: Arc = Arc::new(FakeDb); let (tx_request_event, rx_request_event) = tokio::sync::mpsc::channel(100); diff --git a/fuel-p2p/src/service.rs b/fuel-p2p/src/service.rs index b4e7cbd077f..6e8b711b1e6 100644 --- a/fuel-p2p/src/service.rs +++ b/fuel-p2p/src/service.rs @@ -216,12 +216,10 @@ mod tests { async fn nodes_connected_via_mdns() { // Node A let mut p2p_config = P2PConfig::default_with_network("nodes_connected_via_mdns"); - p2p_config.tcp_port = 4001; p2p_config.enable_mdns = true; let mut node_a = build_fuel_p2p_service(p2p_config.clone()).await; // Node B - p2p_config.tcp_port = 4002; let mut node_b = build_fuel_p2p_service(p2p_config).await; loop { @@ -247,7 +245,6 @@ mod tests { async fn nodes_connected_via_identify() { // Node A let mut p2p_config = P2PConfig::default_with_network("nodes_connected_via_identify"); - p2p_config.tcp_port = 4003; let mut node_a = build_fuel_p2p_service(p2p_config.clone()).await; let node_a_address = match node_a.next_event().await { @@ -256,7 +253,6 @@ mod tests { }; // Node B - p2p_config.tcp_port = 4004; p2p_config.bootstrap_nodes = vec![build_bootstrap_node( node_a.local_peer_id, node_a_address.clone().unwrap(), @@ -264,7 +260,6 @@ mod tests { let mut node_b = build_fuel_p2p_service(p2p_config.clone()).await; // Node C - p2p_config.tcp_port = 4005; let mut node_c = build_fuel_p2p_service(p2p_config).await; loop { @@ -296,7 +291,6 @@ mod tests { async fn peer_info_updates_work() { // Node A let mut p2p_config = P2PConfig::default_with_network("peer_info_updates_work"); - p2p_config.tcp_port = 4006; let mut node_a = build_fuel_p2p_service(p2p_config.clone()).await; let node_a_address = match node_a.next_event().await { @@ -305,7 +299,6 @@ mod tests { }; // Node B - p2p_config.tcp_port = 4007; p2p_config.bootstrap_nodes = vec![build_bootstrap_node( node_a.local_peer_id, node_a_address.clone().unwrap(), @@ -339,42 +332,32 @@ mod tests { #[tokio::test] #[instrument] async fn gossipsub_broadcast_tx() { - gossipsub_broadcast( - GossipsubBroadcastRequest::NewTx(Arc::new(Transaction::default())), - 4008, - 4009, - ) + gossipsub_broadcast(GossipsubBroadcastRequest::NewTx(Arc::new( + Transaction::default(), + ))) .await; } #[tokio::test] #[instrument] async fn gossipsub_broadcast_vote() { - gossipsub_broadcast( - GossipsubBroadcastRequest::ConsensusVote(Arc::new(ConsensusVote::default())), - 4010, - 4011, - ) + gossipsub_broadcast(GossipsubBroadcastRequest::ConsensusVote(Arc::new( + ConsensusVote::default(), + ))) .await; } #[tokio::test] #[instrument] async fn gossipsub_broadcast_block() { - gossipsub_broadcast( - GossipsubBroadcastRequest::NewBlock(Arc::new(FuelBlock::default())), - 4012, - 4013, - ) + gossipsub_broadcast(GossipsubBroadcastRequest::NewBlock(Arc::new( + FuelBlock::default(), + ))) .await; } /// Reusable helper function for Broadcasting Gossipsub requests - async fn gossipsub_broadcast( - broadcast_request: GossipsubBroadcastRequest, - port_a: u16, - port_b: u16, - ) { + async fn gossipsub_broadcast(broadcast_request: GossipsubBroadcastRequest) { let mut p2p_config = P2PConfig::default_with_network("gossipsub_exchanges_messages"); let topics = vec![ NEW_TX_GOSSIP_TOPIC.into(), @@ -395,7 +378,6 @@ mod tests { let mut message_sent = false; // Node A - p2p_config.tcp_port = port_a; p2p_config.topics = topics.clone(); let mut node_a = build_fuel_p2p_service(p2p_config.clone()).await; @@ -405,7 +387,6 @@ mod tests { }; // Node B - p2p_config.tcp_port = port_b; p2p_config.bootstrap_nodes = vec![build_bootstrap_node( node_a.local_peer_id, node_a_address.clone().unwrap(), @@ -477,7 +458,6 @@ mod tests { let mut p2p_config = P2PConfig::default_with_network("request_response_works"); // Node A - p2p_config.tcp_port = 4014; let mut node_a = build_fuel_p2p_service(p2p_config.clone()).await; let node_a_address = match node_a.next_event().await { @@ -486,7 +466,6 @@ mod tests { }; // Node B - p2p_config.tcp_port = 4015; p2p_config.bootstrap_nodes = vec![build_bootstrap_node( node_a.local_peer_id, node_a_address.clone().unwrap(), @@ -567,7 +546,6 @@ mod tests { let mut p2p_config = P2PConfig::default_with_network("req_res_outbound_timeout_works"); // Node A - p2p_config.tcp_port = 4016; // setup request timeout to 0 in order for the Request to fail p2p_config.set_request_timeout = Some(Duration::from_secs(0)); let mut node_a = build_fuel_p2p_service(p2p_config.clone()).await; @@ -578,7 +556,6 @@ mod tests { }; // Node B - p2p_config.tcp_port = 4017; p2p_config.bootstrap_nodes = vec![build_bootstrap_node( node_a.local_peer_id, node_a_address.clone().unwrap(), From 5fc9b6c2ac6d5a66022b7dcc7e9b3a91a1b1c4f0 Mon Sep 17 00:00:00 2001 From: leviathanbeak88 Date: Thu, 11 Aug 2022 13:31:50 +0200 Subject: [PATCH 22/29] merge master --- .github/workflows/ci.yml | 36 +- .github/workflows/scripts/verify_openssl.sh | 21 + Cargo.lock | 1169 +++++++---------- README.md | 13 +- ci/Dockerfile.x86_64-unknown-linux-gnu-clang | 12 +- ci/centos-entrypoint | 3 - deployment/charts/Chart.yaml | 2 +- .../charts/templates/fuel-core-deploy.yaml | 2 - deployment/charts/values.yaml | 1 - deployment/scripts/.env | 1 - fuel-block-executor/Cargo.toml | 4 +- fuel-block-importer/Cargo.toml | 4 +- fuel-block-producer/Cargo.toml | 4 +- fuel-client/Cargo.toml | 6 +- fuel-client/assets/schema.sdl | 118 +- fuel-client/src/client.rs | 13 + fuel-client/src/client/schema.rs | 3 +- fuel-client/src/client/schema/chain.rs | 6 +- fuel-client/src/client/schema/message.rs | 107 ++ fuel-client/src/client/schema/node_info.rs | 1 - fuel-client/src/client/schema/primitives.rs | 25 + ..._chain__tests__chain_gql_query_output.snap | 5 +- ...tests__owned_message_query_gql_output.snap | 29 + ...fo__tests__node_info_query_gql_output.snap | 2 +- ...nt_transaction_by_id_query_gql_output.snap | 29 +- .../schema/tx/tests/transparent_receipt.rs | 41 +- .../client/schema/tx/tests/transparent_tx.rs | 75 +- fuel-core-bft/Cargo.toml | 4 +- fuel-core-interfaces/Cargo.toml | 12 +- fuel-core-interfaces/src/db.rs | 81 +- fuel-core-interfaces/src/model.rs | 6 +- fuel-core-interfaces/src/model/block.rs | 2 +- .../src/model/deposit_coin.rs | 20 - fuel-core-interfaces/src/model/messages.rs | 70 + fuel-core-interfaces/src/relayer.rs | 20 +- fuel-core-interfaces/src/txpool.rs | 54 +- fuel-core/Cargo.toml | 22 +- fuel-core/src/{config => }/chain_config.rs | 88 +- .../{config => chain_config}/serialization.rs | 0 fuel-core/src/cli/run.rs | 8 +- fuel-core/src/cli/run/relayer.rs | 2 +- fuel-core/src/cli/snapshot.rs | 4 +- fuel-core/src/database.rs | 32 +- fuel-core/src/database/coin.rs | 2 +- fuel-core/src/database/contracts.rs | 2 +- fuel-core/src/database/deposit_coin.rs | 31 - fuel-core/src/database/message.rs | 158 +++ fuel-core/src/database/metadata.rs | 2 +- fuel-core/src/executor.rs | 771 ++++++++--- fuel-core/src/lib.rs | 2 +- fuel-core/src/schema.rs | 4 +- fuel-core/src/schema/balance.rs | 10 +- fuel-core/src/schema/block.rs | 11 +- fuel-core/src/schema/chain.rs | 16 +- fuel-core/src/schema/coin.rs | 12 +- fuel-core/src/schema/dap.rs | 30 +- fuel-core/src/schema/message.rs | 151 +++ fuel-core/src/schema/node_info.rs | 8 +- fuel-core/src/schema/scalars.rs | 67 +- fuel-core/src/schema/scalars/tx_pointer.rs | 67 + fuel-core/src/schema/scalars/utxo_id.rs | 67 + fuel-core/src/schema/tx.rs | 21 +- fuel-core/src/schema/tx/input.rs | 137 +- fuel-core/src/schema/tx/output.rs | 30 +- fuel-core/src/schema/tx/receipt.rs | 15 + fuel-core/src/schema/tx/types.rs | 13 - fuel-core/src/service.rs | 8 +- fuel-core/src/{ => service}/config.rs | 6 +- fuel-core/src/service/genesis.rs | 75 +- fuel-core/src/service/graph_api.rs | 2 +- fuel-core/src/service/modules.rs | 2 +- ...__snapshot_configurable_block_height.snap} | 8 +- ...sts__snapshot_contract_with_balances.snap} | 8 +- ..._tests__snapshot_contract_with_state.snap} | 8 +- ...tests__snapshot_local_testnet_config.snap} | 8 +- ...g__tests__snapshot_simple_coin_state.snap} | 8 +- ...fig__tests__snapshot_simple_contract.snap} | 8 +- ..._tests__snapshot_simple_message_state.snap | 37 + fuel-core/src/state.rs | 4 +- fuel-core/src/state/in_memory/memory_store.rs | 7 +- fuel-core/src/state/in_memory/transaction.rs | 67 +- fuel-core/src/state/rocks_db.rs | 36 +- fuel-p2p/Cargo.toml | 4 +- fuel-relayer/Cargo.toml | 14 +- fuel-relayer/README.md | 4 +- fuel-relayer/abi/IFuelMessageOutbox.json | 108 ++ fuel-relayer/src/abi.rs | 5 + fuel-relayer/src/config.rs | 21 +- fuel-relayer/src/finalization_queue.rs | 113 +- fuel-relayer/src/log.rs | 214 ++- fuel-relayer/src/relayer.rs | 12 +- fuel-relayer/src/service.rs | 2 +- fuel-sync/Cargo.toml | 4 +- fuel-tests/Cargo.toml | 4 +- fuel-tests/tests/balances.rs | 10 +- fuel-tests/tests/blocks.rs | 3 +- fuel-tests/tests/chain.rs | 2 +- fuel-tests/tests/coin.rs | 8 +- fuel-tests/tests/dap.rs | 2 +- fuel-tests/tests/debugger.rs | 2 +- fuel-tests/tests/example_tx.json | 1 - fuel-tests/tests/health.rs | 2 +- fuel-tests/tests/helpers.rs | 10 +- fuel-tests/tests/lib.rs | 1 + fuel-tests/tests/messages.rs | 186 +++ fuel-tests/tests/metrics.rs | 6 +- fuel-tests/tests/node_info.rs | 4 +- fuel-tests/tests/snapshot.rs | 22 +- fuel-tests/tests/tx.rs | 10 +- fuel-tests/tests/tx/predicates.rs | 3 + fuel-tests/tests/tx/utxo_validation.rs | 21 +- fuel-txpool/Cargo.toml | 6 +- fuel-txpool/src/config.rs | 3 - fuel-txpool/src/containers/dependency.rs | 131 +- fuel-txpool/src/service.rs | 7 +- fuel-txpool/src/txpool.rs | 407 +++++- 116 files changed, 3749 insertions(+), 1679 deletions(-) create mode 100755 .github/workflows/scripts/verify_openssl.sh delete mode 100755 ci/centos-entrypoint create mode 100644 fuel-client/src/client/schema/message.rs create mode 100644 fuel-client/src/client/schema/snapshots/fuel_gql_client__client__schema__message__tests__owned_message_query_gql_output.snap delete mode 100644 fuel-core-interfaces/src/model/deposit_coin.rs create mode 100644 fuel-core-interfaces/src/model/messages.rs rename fuel-core/src/{config => }/chain_config.rs (82%) rename fuel-core/src/{config => chain_config}/serialization.rs (100%) delete mode 100644 fuel-core/src/database/deposit_coin.rs create mode 100644 fuel-core/src/database/message.rs create mode 100644 fuel-core/src/schema/message.rs create mode 100644 fuel-core/src/schema/scalars/tx_pointer.rs create mode 100644 fuel-core/src/schema/scalars/utxo_id.rs rename fuel-core/src/{ => service}/config.rs (96%) rename fuel-core/src/{config/snapshots/fuel_core__config__chain_config__tests__snapshot_configurable_block_height.snap => snapshots/fuel_core__chain_config__tests__snapshot_configurable_block_height.snap} (77%) rename fuel-core/src/{config/snapshots/fuel_core__config__chain_config__tests__snapshot_contract_with_balances.snap => snapshots/fuel_core__chain_config__tests__snapshot_contract_with_balances.snap} (84%) rename fuel-core/src/{config/snapshots/fuel_core__config__chain_config__tests__snapshot_contract_with_state.snap => snapshots/fuel_core__chain_config__tests__snapshot_contract_with_state.snap} (84%) rename fuel-core/src/{config/snapshots/fuel_core__config__chain_config__tests__snapshot_local_testnet_config.snap => snapshots/fuel_core__chain_config__tests__snapshot_local_testnet_config.snap} (91%) rename fuel-core/src/{config/snapshots/fuel_core__config__chain_config__tests__snapshot_simple_coin_state.snap => snapshots/fuel_core__chain_config__tests__snapshot_simple_coin_state.snap} (86%) rename fuel-core/src/{config/snapshots/fuel_core__config__chain_config__tests__snapshot_simple_contract.snap => snapshots/fuel_core__chain_config__tests__snapshot_simple_contract.snap} (80%) create mode 100644 fuel-core/src/snapshots/fuel_core__chain_config__tests__snapshot_simple_message_state.snap create mode 100644 fuel-relayer/abi/IFuelMessageOutbox.json create mode 100644 fuel-tests/tests/messages.rs diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml index 1acba95fc19..a16df2fa713 100644 --- a/.github/workflows/ci.yml +++ b/.github/workflows/ci.yml @@ -46,7 +46,31 @@ jobs: env: SLACK_WEBHOOK_URL: ${{ secrets.SLACK_WEBHOOK_NOTIFY_BUILD }} + prevent-openssl: + runs-on: ubuntu-latest + steps: + - uses: actions/checkout@v3 + # ensure openssl hasn't crept into the dep tree + - name: Check if openssl is included + run: ./.github/workflows/scripts/verify_openssl.sh + + - name: Notify if Job Fails + uses: ravsamhq/notify-slack-action@master + if: always() && github.ref == 'refs/heads/master' + with: + status: ${{ job.status }} + token: ${{ secrets.GITHUB_TOKEN }} + notification_title: '{workflow} has {status_message}' + message_format: '{emoji} *{workflow}* {status_message} in <{repo_url}|{repo}> : <{run_url}|View Run Results>' + footer: '' + notify_when: 'failure' + env: + SLACK_WEBHOOK_URL: ${{ secrets.SLACK_WEBHOOK_NOTIFY_BUILD }} + cargo-verifications: + needs: + - lint-toml-files + - prevent-openssl runs-on: ubuntu-latest strategy: matrix: @@ -98,7 +122,6 @@ jobs: publish-crates: # Only do this job if publishing a release needs: - - lint-toml-files - cargo-verifications if: github.event_name == 'release' && github.event.action == 'published' runs-on: ubuntu-latest @@ -154,21 +177,12 @@ jobs: publish-docker-image: needs: - - lint-toml-files - cargo-verifications - - if: always() runs-on: ubuntu-latest permissions: contents: read packages: write steps: - # This is a way to make this job run after publish-crates even if it's skipped on master or pr branches - # https://stackoverflow.com/a/69252812/680811 - - name: fail if any dependent jobs failed - if: ${{ contains(needs.*.result, 'failure') }} - run: exit 1 - - name: Checkout repository uses: actions/checkout@v3 @@ -228,7 +242,9 @@ jobs: - lint-toml-files - cargo-verifications if: github.event_name == 'release' && github.event.action == 'published' + continue-on-error: true strategy: + fail-fast: false matrix: job: - os: ubuntu-latest diff --git a/.github/workflows/scripts/verify_openssl.sh b/.github/workflows/scripts/verify_openssl.sh new file mode 100755 index 00000000000..b4ece347295 --- /dev/null +++ b/.github/workflows/scripts/verify_openssl.sh @@ -0,0 +1,21 @@ +#!/usr/bin/env bash + +err() { + echo -e "\e[31m\e[1merror:\e[0m $@" 1>&2; +} + +status() { + WIDTH=12 + printf "\e[32m\e[1m%${WIDTH}s\e[0m %s\n" "$1" "$2" +} + +grep "openssl-sys" Cargo.lock &> /dev/null +test $? -ne 0 +openssl=$? + +if [ $openssl != 0 ]; then + err "detected openssl" + exit 1 +else + status "no openssl dependencies detected" +fi diff --git a/Cargo.lock b/Cargo.lock index 0c275ed1cdf..0ac8e226808 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -247,9 +247,9 @@ dependencies = [ [[package]] name = "async-graphql" -version = "4.0.1" +version = "4.0.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "19bc50aa64e3bc176fe1530a270d5efccd79bde469d6957629c590fe47558689" +checksum = "9d7bef6458f7fb35c16443c6c8af7d219c54e676f47c5a47fd834c8bb8f3bd16" dependencies = [ "async-graphql-derive", "async-graphql-parser", @@ -281,13 +281,13 @@ dependencies = [ [[package]] name = "async-graphql-derive" -version = "4.0.1" +version = "4.0.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "77c3836e72f564e4d502f9ed70ef67d9ef29163901027774fec7970f2785ef27" +checksum = "8c50c6f67f2cc6a0f22e8efdf1e92b459151feefd4c2ae8adc8c1a223e0ebd89" dependencies = [ "Inflector", "async-graphql-parser", - "darling", + "darling 0.14.1", "proc-macro-crate", "proc-macro2", "quote", @@ -297,9 +297,9 @@ dependencies = [ [[package]] name = "async-graphql-parser" -version = "4.0.1" +version = "4.0.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "09c540cdcb24755edf53571ac4228b8288b0cdc1eb6c7f9af333ed185bb8e0ed" +checksum = "a73c90d34c7456bf28a764bfa201c41ac1929b3c376af1815d5f6b176bf74fe4" dependencies = [ "async-graphql-value", "pest", @@ -309,9 +309,9 @@ dependencies = [ [[package]] name = "async-graphql-value" -version = "4.0.1" +version = "4.0.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f7484b68c290bbf97e961003ffc2e781c0210e342c849bac43b5a5636101926e" +checksum = "b79f082786fccd251a8e9615df7f8d45dc8c5305f89062e79a9be570bcd08585" dependencies = [ "bytes", "indexmap", @@ -332,7 +332,7 @@ dependencies = [ "http-types", "httparse", "log", - "pin-project 1.0.10", + "pin-project 1.0.11", ] [[package]] @@ -446,9 +446,9 @@ dependencies = [ [[package]] name = "async-task" -version = "4.2.0" +version = "4.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "30696a84d817107fc028e049980e09d5e140e8da8f1caeb17e8e950658a3cea9" +checksum = "7a40729d2133846d9ed0ea60a8b9541bccddab49cd30f0715a1da672fe9a2524" [[package]] name = "async-tls" @@ -504,7 +504,7 @@ version = "0.5.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "b88d82667eca772c4aa12f0f1348b3ae643424c8876448f3f7bd5787032e234c" dependencies = [ - "autocfg 1.1.0", + "autocfg", ] [[package]] @@ -536,15 +536,6 @@ dependencies = [ "syn", ] -[[package]] -name = "autocfg" -version = "0.1.8" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0dde43e75fd43e8a1bf86103336bc699aa8d17ad1be60c76c0bdfd4828e19b78" -dependencies = [ - "autocfg 1.1.0", -] - [[package]] name = "autocfg" version = "1.1.0" @@ -597,9 +588,9 @@ dependencies = [ [[package]] name = "base-x" -version = "0.2.10" +version = "0.2.11" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "dc19a4937b4fbd3fe3379793130e42060d10627a360f2127802b10b87e7baf74" +checksum = "4cbbc9d0964165b47557570cce6c952866c2678457aca742aafc9fb771d30270" [[package]] name = "base16ct" @@ -658,9 +649,9 @@ dependencies = [ [[package]] name = "bindgen" -version = "0.59.2" +version = "0.60.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2bd2a9a458e8f4304c52c43ebb0cfbd520289f8379a52e329a38afda99bf8eb8" +checksum = "062dddbc1ba4aca46de6338e2bf87771414c335f7b2f2036e8f3e9befebf88e6" dependencies = [ "bitflags", "cexpr", @@ -693,9 +684,9 @@ dependencies = [ [[package]] name = "bitvec" -version = "1.0.0" +version = "1.0.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1489fcb93a5bb47da0462ca93ad252ad6af2145cce58d10d46a83931ba9f016b" +checksum = "1bc2832c24239b0141d5674bb9174f9d68a8b5b3f2753311927c172ca46f7e9c" dependencies = [ "funty", "radium 0.7.0", @@ -703,17 +694,6 @@ dependencies = [ "wyz", ] -[[package]] -name = "blake2" -version = "0.9.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0a4e37d16930f5459780f5621038b6382b9bb37c19016f39fb6b5808d831f174" -dependencies = [ - "crypto-mac 0.8.0", - "digest 0.9.0", - "opaque-debug 0.3.0", -] - [[package]] name = "blake2" version = "0.10.4" @@ -729,7 +709,7 @@ version = "0.7.3" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "c0940dc441f31689269e10ac70eb1002a3a1d3ad1390e030043662eb7fe4688b" dependencies = [ - "block-padding 0.1.5", + "block-padding", "byte-tools", "byteorder", "generic-array 0.12.4", @@ -741,7 +721,6 @@ version = "0.9.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "4152116fd6e9dadb291ae18fc1ec3575ed6d84c29642d97890f4b4a3417297e4" dependencies = [ - "block-padding 0.2.1", "generic-array 0.14.5", ] @@ -763,12 +742,6 @@ dependencies = [ "byte-tools", ] -[[package]] -name = "block-padding" -version = "0.2.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8d696c370c750c948ada61c69a0ee2cbbb9c50b1019ddb86d9317157a99c2cae" - [[package]] name = "blocking" version = "1.2.0" @@ -821,9 +794,9 @@ checksum = "14c189c53d098945499cdfa7ecc63567cf3886b3332b312a5b4585d8d3a6a610" [[package]] name = "bytes" -version = "1.1.0" +version = "1.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c4872d67bab6358e59559027aa3b9157c53d9358c51423c17554809a8858e0f8" +checksum = "f0b3de4a0c5e67e16066a0715723abd91edc2f9001d09c46e1dca929351e130e" dependencies = [ "serde", ] @@ -865,13 +838,13 @@ dependencies = [ [[package]] name = "cargo_metadata" -version = "0.14.2" +version = "0.15.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4acbb09d9ee8e23699b9634375c72795d095bf268439da88562cf9b501f181fa" +checksum = "3abb7553d5b9b8421c6de7cb02606ff15e0c6eea7d8eadd75ef013fd636bec36" dependencies = [ "camino", "cargo-platform", - "semver 1.0.10", + "semver 1.0.12", "serde", "serde_json", ] @@ -902,9 +875,9 @@ checksum = "baf1de4339761588bc0619e3cbc0120ee582ebb74b53b4efbf79117bd2da40fd" [[package]] name = "chacha20" -version = "0.8.1" +version = "0.8.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "01b72a433d0cf2aef113ba70f62634c56fddb0f244e6377185c56a7cadbd8f91" +checksum = "5c80e5460aa66fe3b91d40bcbdab953a597b60053e34d684ac6903f863b680a6" dependencies = [ "cfg-if", "cipher 0.3.0", @@ -914,9 +887,9 @@ dependencies = [ [[package]] name = "chacha20poly1305" -version = "0.9.0" +version = "0.9.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3b84ed6d1d5f7aa9bdde921a5090e0ca4d934d250ea3b402a5fab3a994e28a2a" +checksum = "a18446b09be63d457bbec447509e85f662f32952b035ce892290396bc0b0cff5" dependencies = [ "aead 0.4.3", "chacha20", @@ -935,15 +908,15 @@ dependencies = [ "num-integer", "num-traits", "serde", - "time 0.1.43", + "time 0.1.44", "winapi", ] [[package]] name = "chrono-tz" -version = "0.6.1" +version = "0.6.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "58549f1842da3080ce63002102d5bc954c7bc843d4f47818e642abdc36253552" +checksum = "29c39203181991a7dd4343b8005bd804e7a9a37afb8ac070e43771e8c820bbde" dependencies = [ "chrono", "chrono-tz-build", @@ -952,9 +925,9 @@ dependencies = [ [[package]] name = "chrono-tz-build" -version = "0.0.2" +version = "0.0.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "db058d493fb2f65f41861bfed7e3fe6335264a9f0f92710cab5bdf01fef09069" +checksum = "6f509c3a87b33437b05e2458750a0700e5bdd6956176773e6c7d6dd15a283a0c" dependencies = [ "parse-zoneinfo", "phf", @@ -992,9 +965,9 @@ dependencies = [ [[package]] name = "clap" -version = "3.2.12" +version = "3.2.15" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ab8b79fe3946ceb4a0b1c080b4018992b8d27e9ff363644c1c9b6387c854614d" +checksum = "44bbe24bbd31a185bc2c4f7c2abe80bea13a20d57ee4e55be70ac512bdc76417" dependencies = [ "atty", "bitflags", @@ -1009,9 +982,9 @@ dependencies = [ [[package]] name = "clap_derive" -version = "3.2.7" +version = "3.2.15" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "759bf187376e1afa7b85b959e6a664a3e7a95203415dba952ad19139e798f902" +checksum = "9ba52acd3b0a5c33aeada5cdaa3267cdc7c594a98731d4268cdc1532f4264cb4" dependencies = [ "heck 0.4.0", "proc-macro-error", @@ -1029,69 +1002,60 @@ dependencies = [ "os_str_bytes", ] -[[package]] -name = "cloudabi" -version = "0.0.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ddfc5b9aa5d4507acaf872de71051dfd0e309860e88966e1051e462a077aac4f" -dependencies = [ - "bitflags", -] - [[package]] name = "coins-bip32" -version = "0.6.0" +version = "0.7.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "471b39eadc9323de375dce5eff149a5a1ebd21c67f1da34a56f87ee62191d4ea" +checksum = "634c509653de24b439672164bbf56f5f582a2ab0e313d3b0f6af0b7345cf2560" dependencies = [ "bincode", "bs58", "coins-core", - "digest 0.9.0", + "digest 0.10.3", "getrandom 0.2.7", - "hmac 0.11.0", + "hmac 0.12.1", "k256", "lazy_static", "serde", - "sha2 0.9.9", + "sha2 0.10.2", "thiserror", ] [[package]] name = "coins-bip39" -version = "0.6.0" +version = "0.7.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8f473ea37dfc9d2cb94fdde50c3d41f28c3f384b367573d66386fea38d76d466" +checksum = "2a11892bcac83b4c6e95ab84b5b06c76d9d70ad73548dd07418269c5c7977171" dependencies = [ "bitvec 0.17.4", "coins-bip32", "getrandom 0.2.7", "hex", - "hmac 0.11.0", - "pbkdf2 0.8.0", + "hmac 0.12.1", + "pbkdf2 0.11.0", "rand 0.8.5", - "sha2 0.9.9", + "sha2 0.10.2", "thiserror", ] [[package]] name = "coins-core" -version = "0.2.2" +version = "0.7.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d257d975731955ee86fa7f348000c3fea09c262e84c70c11e994a85aa4f467a7" +checksum = "c94090a6663f224feae66ab01e41a2555a8296ee07b5f20dab8888bdefc9f617" dependencies = [ "base58check", "base64 0.12.3", "bech32", - "blake2 0.9.2", - "digest 0.9.0", + "blake2", + "digest 0.10.3", "generic-array 0.14.5", "hex", - "ripemd160", + "ripemd", "serde", "serde_derive", - "sha2 0.9.9", - "sha3 0.9.1", + "sha2 0.10.2", + "sha3", "thiserror", ] @@ -1110,9 +1074,9 @@ dependencies = [ [[package]] name = "concurrent-queue" -version = "1.2.2" +version = "1.2.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "30ed07550be01594c6026cff2a1d7fe9c8f683caa798e12b68694ac9e88286a3" +checksum = "af4780a44ab5696ea9e28294517f1fffb421a83a25af521333c838635509db9c" dependencies = [ "cache-padded", ] @@ -1130,9 +1094,9 @@ dependencies = [ [[package]] name = "console" -version = "0.15.0" +version = "0.15.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a28b32d32ca44b70c3e4acd7db1babf555fa026e385fb95f18028f88848b3c31" +checksum = "89eab4d20ce20cea182308bca13088fecea9c05f6776cf287205d41a0ed3c847" dependencies = [ "encode_unicode", "libc", @@ -1143,9 +1107,9 @@ dependencies = [ [[package]] name = "const-oid" -version = "0.7.1" +version = "0.9.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e4c78c047431fee22c1a7bb92e00ad095a02a983affe4d8a72e2a2c62c1b94f3" +checksum = "722e23542a15cea1f65d4a1419c4cfd7a26706c70871a13a04238ca3f40f1661" [[package]] name = "const_fn" @@ -1233,9 +1197,9 @@ dependencies = [ [[package]] name = "crossbeam-channel" -version = "0.5.4" +version = "0.5.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5aaa7bd5fb665c6864b5f963dd9097905c54125909c7aa94c9e18507cdbe6c53" +checksum = "c2dd04ddaf88237dc3b8d8f9a3c1004b506b54b3313403944054d23c0870c521" dependencies = [ "cfg-if", "crossbeam-utils", @@ -1243,9 +1207,9 @@ dependencies = [ [[package]] name = "crossbeam-queue" -version = "0.3.5" +version = "0.3.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1f25d8400f4a7a5778f0e4e52384a48cbd9b5c495d110786187fc750075277a2" +checksum = "1cd42583b04998a5363558e5f9291ee5a5ff6b49944332103f251e7479a82aa7" dependencies = [ "cfg-if", "crossbeam-utils", @@ -1253,9 +1217,9 @@ dependencies = [ [[package]] name = "crossbeam-utils" -version = "0.8.9" +version = "0.8.11" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8ff1f980957787286a554052d03c7aee98d99cc32e09f6d45f0a814133c87978" +checksum = "51887d4adc7b564537b15adcfb307936f8075dfcd5f00dde9a9f1d29383682bc" dependencies = [ "cfg-if", "once_cell", @@ -1269,9 +1233,9 @@ checksum = "7a81dae078cea95a014a339291cec439d2f232ebe854a9d672b796c6afafa9b7" [[package]] name = "crypto-bigint" -version = "0.3.2" +version = "0.4.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "03c6a1d5fa1de37e071642dfa44ec552ca5b299adb128fab16138e24b548fd21" +checksum = "9f2b443d17d49dad5ef0ede301c3179cc923b8822f3393b4d2c28c269dd4a122" dependencies = [ "generic-array 0.14.5", "rand_core 0.6.3", @@ -1281,9 +1245,9 @@ dependencies = [ [[package]] name = "crypto-common" -version = "0.1.3" +version = "0.1.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "57952ca27b5e3606ff4dd79b0020231aaf9d6aa76dc05fd30137538c50bd3ce8" +checksum = "1bfb12502f3fc46cca1bb51ac28df9d618d813cdc3d2f25b9fe775a34af26bb3" dependencies = [ "generic-array 0.14.5", "typenum", @@ -1309,16 +1273,6 @@ dependencies = [ "subtle", ] -[[package]] -name = "crypto-mac" -version = "0.11.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b1d1a86f49236c215f271d40892d5fc950490551400b02ef360692c29815c714" -dependencies = [ - "generic-array 0.14.5", - "subtle", -] - [[package]] name = "ctor" version = "0.1.22" @@ -1349,9 +1303,9 @@ dependencies = [ [[package]] name = "curve25519-dalek" -version = "3.2.1" +version = "3.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "90f9d052967f590a76e62eb387bd0bbb1b000182c3cefe5364db6b7211651bc0" +checksum = "0b9fdf9972b2bd6af2d913799d9ebc165ea4d2e65878e329d9c6b372c4491b61" dependencies = [ "byteorder", "digest 0.9.0", @@ -1362,12 +1316,12 @@ dependencies = [ [[package]] name = "curve25519-dalek" -version = "4.0.0-pre.2" +version = "4.0.0-pre.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "12dc3116fe595d7847c701796ac1b189bd86b81f4f593c6f775f9d80fb2e29f4" +checksum = "4033478fbf70d6acf2655ac70da91ee65852d69daf7a67bf7a2f518fb47aafcf" dependencies = [ "byteorder", - "digest 0.10.3", + "digest 0.9.0", "rand_core 0.6.3", "subtle", "zeroize", @@ -1394,7 +1348,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "dd9f0852fe3d3637d4dccb4b69e5a8f881214fc38907a528385ff71cd7b15c3e" dependencies = [ "Inflector", - "darling", + "darling 0.13.4", "graphql-parser", "lazy_static", "proc-macro2", @@ -1419,8 +1373,18 @@ version = "0.13.4" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "a01d95850c592940db9b8194bc39f4bc0e89dee5c4265e4b1807c34a9aba453c" dependencies = [ - "darling_core", - "darling_macro", + "darling_core 0.13.4", + "darling_macro 0.13.4", +] + +[[package]] +name = "darling" +version = "0.14.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4529658bdda7fd6769b8614be250cdcfc3aeb0ee72fe66f9e41e5e5eb73eac02" +dependencies = [ + "darling_core 0.14.1", + "darling_macro 0.14.1", ] [[package]] @@ -1437,13 +1401,38 @@ dependencies = [ "syn", ] +[[package]] +name = "darling_core" +version = "0.14.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "649c91bc01e8b1eac09fb91e8dbc7d517684ca6be8ebc75bb9cafc894f9fdb6f" +dependencies = [ + "fnv", + "ident_case", + "proc-macro2", + "quote", + "strsim", + "syn", +] + [[package]] name = "darling_macro" version = "0.13.4" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "9c972679f83bdf9c42bd905396b6c3588a843a17f0f16dfcfa3e2c5d57441835" dependencies = [ - "darling_core", + "darling_core 0.13.4", + "quote", + "syn", +] + +[[package]] +name = "darling_macro" +version = "0.14.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ddfc69c5bfcbd2fc09a0f38451d2daf0e372e367986a83906d1b0dbc88134fb5" +dependencies = [ + "darling_core 0.14.1", "quote", "syn", ] @@ -1455,7 +1444,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "3495912c9c1ccf2e18976439f4443f3fee0fd61f424ff99fde6a66b15ecb448f" dependencies = [ "cfg-if", - "hashbrown 0.12.1", + "hashbrown", "lock_api", "parking_lot_core 0.9.3", ] @@ -1482,11 +1471,12 @@ dependencies = [ [[package]] name = "der" -version = "0.5.1" +version = "0.6.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6919815d73839e7ad218de758883aae3a257ba6759ce7a9992501efbb53d705c" +checksum = "13dd2ae565c0a381dde7fade45fce95984c568bdcb4700a4fdbe3175e0380b2f" dependencies = [ "const-oid", + "zeroize", ] [[package]] @@ -1581,9 +1571,9 @@ checksum = "453440c271cf5577fd2a40e4942540cb7d0d2f85e27c8d07dd0023c925a67541" [[package]] name = "ecdsa" -version = "0.13.4" +version = "0.14.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d0d69ae62e0ce582d56380743515fefaf1a8c70cec685d9677636d7e30ae9dc9" +checksum = "3bd46e0c364655e5baf2f5e99b603e7a09905da9966d7928d7470af393b28670" dependencies = [ "der", "elliptic-curve", @@ -1606,7 +1596,7 @@ version = "1.0.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "c762bae6dcaf24c4c84667b8579785430908723d5c889f469d76a41d59cc7a9d" dependencies = [ - "curve25519-dalek 3.2.1", + "curve25519-dalek 3.2.0", "ed25519", "rand 0.7.3", "serde", @@ -1616,22 +1606,24 @@ dependencies = [ [[package]] name = "either" -version = "1.6.1" +version = "1.7.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e78d4f1cc4ae33bbfc157ed5d5a5ef3bc29227303d595861deb238fcec4e9457" +checksum = "3f107b87b6afc2a64fd13cac55fe06d6c8859f12d4b14cbcdd2c67d0976781be" [[package]] name = "elliptic-curve" -version = "0.11.12" +version = "0.12.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "25b477563c2bfed38a3b7a60964c49e058b2510ad3f12ba3483fd8f62c2306d6" +checksum = "e7bb888ab5300a19b8e5bceef25ac745ad065f3c9f7efc6de1b91958110891d3" dependencies = [ "base16ct", "crypto-bigint", "der", + "digest 0.10.3", "ff", "generic-array 0.14.5", "group", + "pkcs8", "rand_core 0.6.3", "sec1", "subtle", @@ -1695,16 +1687,16 @@ dependencies = [ "serde", "serde_json", "sha2 0.10.2", - "sha3 0.10.1", + "sha3", "thiserror", "uuid", ] [[package]] name = "ethabi" -version = "17.0.0" +version = "17.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b69517146dfab88e9238c00c724fd8e277951c3cc6f22b016d72f422a832213e" +checksum = "f186de076b3e77b8e6d73c99d1b52edc2a229e604f4b5eb6992c06c11d79d537" dependencies = [ "ethereum-types", "hex", @@ -1712,7 +1704,7 @@ dependencies = [ "regex", "serde", "serde_json", - "sha3 0.10.1", + "sha3", "thiserror", "uint", ] @@ -1746,9 +1738,9 @@ dependencies = [ [[package]] name = "ethers-contract" -version = "0.13.0" +version = "0.17.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "67ee47ba557b2dc845a90d59ac0ec49ecdd3e229978b8a471ec20374634359e9" +checksum = "2e0010fffc97c5abcf75a30fd75676b1ed917b2b82beb8270391333618e2847d" dependencies = [ "ethers-contract-abigen", "ethers-contract-derive", @@ -1757,7 +1749,7 @@ dependencies = [ "futures-util", "hex", "once_cell", - "pin-project 1.0.10", + "pin-project 1.0.11", "serde", "serde_json", "thiserror", @@ -1765,9 +1757,9 @@ dependencies = [ [[package]] name = "ethers-contract-abigen" -version = "0.13.0" +version = "0.17.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "46bf293e1edfe24b6a2285edfef194f378dbde7cca095ebb0e8b59bdd81f869d" +checksum = "bda76ce804d524f693a898dc5857d08f4db443f3da64d0c36237fa05c0ecef30" dependencies = [ "Inflector", "cfg-if", @@ -1788,9 +1780,9 @@ dependencies = [ [[package]] name = "ethers-contract-derive" -version = "0.13.0" +version = "0.17.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "37d852e4d22d8edd70cdc08e85ccd81bb6087c9064cbbf93e8e7fed1c67bf46a" +checksum = "41170ccb5950f559cba5a052158a28ec2d224af3a7d5b266b3278b929538ef55" dependencies = [ "ethers-contract-abigen", "ethers-core", @@ -1803,9 +1795,9 @@ dependencies = [ [[package]] name = "ethers-core" -version = "0.13.0" +version = "0.17.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4536e70feeae231b73ac2221f1339a9a1969dea9cf1d679be26b73a36f14f4e7" +checksum = "0ebdd63c828f58aa067f40f9adcbea5e114fb1f90144b3a1e2858e0c9b1ff4e8" dependencies = [ "arrayvec 0.7.2", "bytes", @@ -1814,6 +1806,7 @@ dependencies = [ "convert_case 0.5.0", "elliptic-curve", "ethabi", + "fastrlp", "generic-array 0.14.5", "hex", "k256", @@ -1825,7 +1818,7 @@ dependencies = [ "rust_decimal", "serde", "serde_json", - "strum 0.24.0", + "strum 0.24.1", "syn", "thiserror", "tiny-keccak", @@ -1834,13 +1827,14 @@ dependencies = [ [[package]] name = "ethers-etherscan" -version = "0.13.0" +version = "0.17.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "808898439b94c18e4f554881e337a02ccf18a6476a5ceb74a6b79a501996cb48" +checksum = "b279a3d00bd219caa2f9a34451b4accbfa9a1eaafc26dcda9d572591528435f0" dependencies = [ "ethers-core", + "getrandom 0.2.7", "reqwest", - "semver 1.0.10", + "semver 1.0.12", "serde", "serde-aux", "serde_json", @@ -1850,9 +1844,9 @@ dependencies = [ [[package]] name = "ethers-middleware" -version = "0.13.0" +version = "0.17.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "30f76a1a33a870060e8ace737218ac59a153614ccaa97100dd6e3a32cb4462b1" +checksum = "b1e7e8632d28175352b9454bbcb604643b6ca1de4d36dc99b3f86860d75c132b" dependencies = [ "async-trait", "ethers-contract", @@ -1875,9 +1869,9 @@ dependencies = [ [[package]] name = "ethers-providers" -version = "0.13.0" +version = "0.17.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "66c2c8dc8cd75ba70271aeabef0706248c58fc3b11886094e76a005f006957f6" +checksum = "e46482e4d1e79b20c338fd9db9e166184eb387f0a4e7c05c5b5c0aa2e8c8900c" dependencies = [ "async-trait", "auto_impl", @@ -1887,12 +1881,13 @@ dependencies = [ "futures-core", "futures-timer", "futures-util", + "getrandom 0.2.7", "hashers", "hex", "http", "once_cell", "parking_lot 0.11.2", - "pin-project 1.0.10", + "pin-project 1.0.11", "reqwest", "serde", "serde_json", @@ -1911,9 +1906,9 @@ dependencies = [ [[package]] name = "ethers-signers" -version = "0.13.0" +version = "0.17.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "338efd9a6a15986fb7ed13ac7a99fd540666a2d7f475ed624d4f2ffc5ce3ec27" +checksum = "73a72ecad124e8ccd18d6a43624208cab0199e59621b1f0fa6b776b2e0529107" dependencies = [ "async-trait", "coins-bip32", @@ -1923,15 +1918,15 @@ dependencies = [ "ethers-core", "hex", "rand 0.8.5", - "sha2 0.9.9", + "sha2 0.10.2", "thiserror", ] [[package]] name = "event-listener" -version = "2.5.2" +version = "2.5.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "77f3309417938f28bf8228fcff79a4a37103981e3e186d2ccd19c74b38f4eb71" +checksum = "0206175f82b8d6bf6652ff7d71a1e27fd2e4efde587fd368662814d6ec1d9ce0" [[package]] name = "eyre" @@ -1960,13 +1955,38 @@ dependencies = [ [[package]] name = "fastrand" -version = "1.7.0" +version = "1.8.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c3fcf0cee53519c866c09b5de1f6c56ff9d647101f81c1964fa632e148896cdf" +checksum = "a7a407cfaa3385c4ae6b23e84623d48c2798d06e3e6a1878f7f59f17b3f86499" dependencies = [ "instant", ] +[[package]] +name = "fastrlp" +version = "0.1.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "089263294bb1c38ac73649a6ad563dd9a5142c8dc0482be15b8b9acb22a1611e" +dependencies = [ + "arrayvec 0.7.2", + "auto_impl", + "bytes", + "ethereum-types", + "fastrlp-derive", +] + +[[package]] +name = "fastrlp-derive" +version = "0.1.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e1fa41ebc231af281098b11ad4a4f6182ec9096902afffe948034a20d4e1385a" +dependencies = [ + "bytes", + "proc-macro2", + "quote", + "syn", +] + [[package]] name = "features" version = "0.10.0" @@ -1978,9 +1998,9 @@ dependencies = [ [[package]] name = "ff" -version = "0.11.1" +version = "0.12.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "131655483be284720a17d74ff97592b8e76576dc25563148601df2d7c9080924" +checksum = "df689201f395c6b90dfe87127685f8dbfc083a5e779e613575d8bd7314300c3e" dependencies = [ "rand_core 0.6.3", "subtle", @@ -2000,9 +2020,9 @@ dependencies = [ [[package]] name = "fixedbitset" -version = "0.4.1" +version = "0.4.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "279fb028e20b3c4c320317955b77c5e0c9701f05a1d309905d6fc702cdc5053e" +checksum = "0ce7134b9999ecaf8bcd65542e436736ef32ddca1b3e06094cb6ec5755203b80" [[package]] name = "flate2" @@ -2021,21 +2041,6 @@ version = "1.0.7" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "3f9eec918d3f24069decb9af1554cad7c880e2da24a9afd88aca000531ab82c1" -[[package]] -name = "foreign-types" -version = "0.3.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f6f339eb8adc052cd2ca78910fda869aefa38d22d5cb648e6485e4d3fc06f3b1" -dependencies = [ - "foreign-types-shared", -] - -[[package]] -name = "foreign-types-shared" -version = "0.1.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "00b0228411908ca8685dba7fc2cdd70ec9990a6e753e89b6ac91a84c40fbaf4b" - [[package]] name = "form_urlencoded" version = "1.0.1" @@ -2052,17 +2057,11 @@ version = "1.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "2022715d62ab30faffd124d40b76f4134a550a87792276512b18d63272333394" -[[package]] -name = "fuchsia-cprng" -version = "0.1.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a06f77d526c1a601b7c4cdd98f54b5eaabffc14d5f2f0296febdc7f357c6d3ba" - [[package]] name = "fuel-asm" -version = "0.5.3" +version = "0.8.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "16628e172a06a411c57972c55a67404f1684118be3bbca79148bf5a00fff48f0" +checksum = "135ef9990debd1fa257988a0a8be5b9d787decaba911cc3b97d7712a4b70328d" dependencies = [ "fuel-types", "serde", @@ -2070,7 +2069,7 @@ dependencies = [ [[package]] name = "fuel-block-executor" -version = "0.9.5" +version = "0.10.0" dependencies = [ "anyhow", "fuel-core-interfaces", @@ -2079,7 +2078,7 @@ dependencies = [ [[package]] name = "fuel-block-importer" -version = "0.9.5" +version = "0.10.0" dependencies = [ "anyhow", "fuel-core-interfaces", @@ -2089,7 +2088,7 @@ dependencies = [ [[package]] name = "fuel-block-producer" -version = "0.9.5" +version = "0.10.0" dependencies = [ "anyhow", "fuel-core-interfaces", @@ -2099,7 +2098,7 @@ dependencies = [ [[package]] name = "fuel-core" -version = "0.9.5" +version = "0.10.0" dependencies = [ "anyhow", "assert_matches", @@ -2149,7 +2148,7 @@ dependencies = [ [[package]] name = "fuel-core-bft" -version = "0.9.5" +version = "0.10.0" dependencies = [ "anyhow", "fuel-core-interfaces", @@ -2159,7 +2158,7 @@ dependencies = [ [[package]] name = "fuel-core-interfaces" -version = "0.9.5" +version = "0.10.0" dependencies = [ "anyhow", "async-trait", @@ -2167,8 +2166,8 @@ dependencies = [ "derive_more", "fuel-asm", "fuel-crypto", - "fuel-merkle 0.3.0", - "fuel-storage 0.1.0", + "fuel-merkle", + "fuel-storage", "fuel-tx", "fuel-types", "fuel-vm", @@ -2182,22 +2181,24 @@ dependencies = [ [[package]] name = "fuel-crypto" -version = "0.5.0" +version = "0.6.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8af1477833b63cf956b71a63a67a7af0e7477bd2f774536556fe0338f909542e" +checksum = "c27380c9c0a97da57123eb981289431069371cc71439f8d0192b604c1414f705" dependencies = [ "borrown", + "coins-bip32", + "coins-bip39", "fuel-types", "lazy_static", "rand 0.8.5", "secp256k1", "serde", - "sha2 0.9.9", + "sha2 0.10.2", ] [[package]] name = "fuel-gql-client" -version = "0.9.5" +version = "0.10.0" dependencies = [ "anyhow", "chrono", @@ -2218,35 +2219,6 @@ dependencies = [ "thiserror", ] -[[package]] -name = "fuel-merkle" -version = "0.1.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8148b67b9ca99755b8c73fbc5bad7710590e1d7e245680ffd351289915258851" -dependencies = [ - "bytes", - "digest 0.9.0", - "fuel-storage 0.1.0", - "generic-array 0.14.5", - "hex", - "lazy_static", - "sha2 0.9.9", - "thiserror", -] - -[[package]] -name = "fuel-merkle" -version = "0.2.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fab10247f9eababf72a64120da05cfef57fbf811ebd2c93dbbbbeccddcce5f11" -dependencies = [ - "digest 0.10.3", - "fuel-storage 0.2.0", - "hex", - "sha2 0.10.2", - "thiserror", -] - [[package]] name = "fuel-merkle" version = "0.3.0" @@ -2254,8 +2226,8 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "1fa59f2a4e6cd6d83a51981c5ac706d58fc8ef53700003948c826827b88cfff1" dependencies = [ "digest 0.10.3", - "fuel-storage 0.2.0", - "hashbrown 0.12.1", + "fuel-storage", + "hashbrown", "hex", "sha2 0.10.2", "thiserror", @@ -2263,7 +2235,7 @@ dependencies = [ [[package]] name = "fuel-p2p" -version = "0.9.5" +version = "0.10.0" dependencies = [ "anyhow", "async-trait", @@ -2286,7 +2258,7 @@ dependencies = [ [[package]] name = "fuel-relayer" -version = "0.9.5" +version = "0.10.0" dependencies = [ "anyhow", "async-trait", @@ -2307,7 +2279,7 @@ dependencies = [ "rand 0.8.5", "serde", "serde_json", - "sha3 0.10.1", + "sha3", "thiserror", "tokio", "tracing", @@ -2316,12 +2288,6 @@ dependencies = [ "url", ] -[[package]] -name = "fuel-storage" -version = "0.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f97a6a68c3378e486d645a47026bcd7139500345ef214653811ea4f016f142ce" - [[package]] name = "fuel-storage" version = "0.2.0" @@ -2330,7 +2296,7 @@ checksum = "34b9161e86d434a93088409530a4f71f42e074b3bbcbb7a27bfe666583f92fd7" [[package]] name = "fuel-sync" -version = "0.9.5" +version = "0.10.0" dependencies = [ "anyhow", "fuel-core-interfaces", @@ -2362,22 +2328,23 @@ dependencies = [ [[package]] name = "fuel-tx" -version = "0.13.0" +version = "0.18.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "43dbfcd95f2b6fb19e6272832b5d6ecf14f8283c3329994baa38a4faed65d437" +checksum = "118b773c25dbf645457fbd9b0288ac923c044d03ce68f492e539c840c3595f5e" dependencies = [ "fuel-asm", "fuel-crypto", - "fuel-merkle 0.2.0", + "fuel-merkle", "fuel-types", "itertools", + "num-integer", "rand 0.8.5", "serde", ] [[package]] name = "fuel-txpool" -version = "0.9.5" +version = "0.10.0" dependencies = [ "anyhow", "async-trait", @@ -2402,22 +2369,23 @@ dependencies = [ [[package]] name = "fuel-vm" -version = "0.12.2" +version = "0.15.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "72d7835c37e4f13bc6eecdd946bd1e0485db30175757425fac689e889f8acc20" +checksum = "516d60fc504570c0ad79f6dffcbd31687b15b76ecf8753bccaeae201baac29d2" dependencies = [ "anyhow", "fuel-asm", "fuel-crypto", - "fuel-merkle 0.1.1", - "fuel-storage 0.1.0", + "fuel-merkle", + "fuel-storage", "fuel-tx", "fuel-types", "itertools", "rand 0.8.5", "secp256k1", "serde", - "sha3 0.9.1", + "sha3", + "tai64", "thiserror", "tracing", ] @@ -2663,9 +2631,9 @@ dependencies = [ [[package]] name = "group" -version = "0.11.0" +version = "0.12.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bc5ac374b108929de78460075f3dc439fa66df9d8fc77e8f12caa5165fcf0c89" +checksum = "7391856def869c1c81063a03457c676fbcd419709c3dfb33d8d319de484b154d" dependencies = [ "ff", "rand_core 0.6.3", @@ -2693,18 +2661,9 @@ dependencies = [ [[package]] name = "hashbrown" -version = "0.11.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ab5ef0d4909ef3724cc8cce6ccc8572c5c817592e9285f5464f8e86f8bd3726e" -dependencies = [ - "ahash", -] - -[[package]] -name = "hashbrown" -version = "0.12.1" +version = "0.12.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "db0d4cf898abf0081f964436dc980e96670a0f36863e4b83aaacdb65c9d7ccc3" +checksum = "8a9ee70c43aaf417c914396645a0fa852624801b24ebb7ae78fe8272889ac888" dependencies = [ "ahash", ] @@ -2787,16 +2746,6 @@ dependencies = [ "digest 0.9.0", ] -[[package]] -name = "hmac" -version = "0.11.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2a2a2320eb7ec0ebe8da8f744d7812d9fc4cb4d09344ac01898dbcb6a20ae69b" -dependencies = [ - "crypto-mac 0.11.1", - "digest 0.9.0", -] - [[package]] name = "hmac" version = "0.12.1" @@ -2830,9 +2779,9 @@ dependencies = [ [[package]] name = "http" -version = "0.2.7" +version = "0.2.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ff8670570af52249509a86f5e3e18a08c60b177071826898fde8997cf5f6bfbb" +checksum = "75f43d41e26995c17e71ee126451dd3941010b0514a81a9d11f3b341debc2399" dependencies = [ "bytes", "fnv", @@ -2917,9 +2866,9 @@ checksum = "9a3a5bfb195931eeb336b2a7b4d761daec841b97f947d34394601737a7bba5e4" [[package]] name = "hyper" -version = "0.14.19" +version = "0.14.20" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "42dc3c131584288d375f2d07f822b0cb012d8c6fb899a5b9fdb3cb7eb9b6004f" +checksum = "02c929dc5c39e335a03c405292728118860721b10190d98c2a0f0efd5baafbac" dependencies = [ "bytes", "futures-channel", @@ -2952,19 +2901,6 @@ dependencies = [ "tokio-rustls", ] -[[package]] -name = "hyper-tls" -version = "0.5.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d6183ddfa99b85da61a140bea0efc93fdf56ceaa041b37d553518030827f9905" -dependencies = [ - "bytes", - "hyper", - "native-tls", - "tokio", - "tokio-native-tls", -] - [[package]] name = "ident_case" version = "1.0.1" @@ -3056,12 +2992,12 @@ checksum = "ce23b50ad8242c51a442f3ff322d56b02f08852c77e4c0b4d3fd684abc89c683" [[package]] name = "indexmap" -version = "1.8.2" +version = "1.9.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e6012d540c5baa3589337a98ce73408de9b5a25ec9fc2c6fd6be8f0d39e0ca5a" +checksum = "10a35a97730320ffe8e2d410b5d3b69279b98d2c14bdb8b70ea89ecf7888d41e" dependencies = [ - "autocfg 1.1.0", - "hashbrown 0.11.2", + "autocfg", + "hashbrown", "serde", ] @@ -3073,9 +3009,9 @@ checksum = "64e9829a50b42bb782c1df523f78d332fe371b10c661e78b7a3c34b0198e9fac" [[package]] name = "insta" -version = "1.14.1" +version = "1.17.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bcc3e639bcba360d9237acabd22014c16f3df772db463b7446cd81b070714767" +checksum = "1e21173d5699a654cf191b0c05b0a937bb4f7cf07ee2e32da2af07a963e31577" dependencies = [ "console", "once_cell", @@ -3147,9 +3083,9 @@ dependencies = [ [[package]] name = "js-sys" -version = "0.3.58" +version = "0.3.59" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c3fac17f7123a73ca62df411b1bf727ccc805daa070338fda671c86dac1bdc27" +checksum = "258451ab10b34f8af53416d1fdab72c22e805f0c92a1136d59470ec0b11138b2" dependencies = [ "wasm-bindgen", ] @@ -3167,16 +3103,15 @@ dependencies = [ [[package]] name = "k256" -version = "0.10.4" +version = "0.11.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "19c3a5e0a0b8450278feda242592512e09f61c72e018b8cd5c859482802daf2d" +checksum = "2c8a5a96d92d849c4499d99461da81c9cdc1467418a8ed2aaeb407e8d85940ed" dependencies = [ "cfg-if", "ecdsa", "elliptic-curve", - "sec1", - "sha2 0.9.9", - "sha3 0.9.1", + "sha2 0.10.2", + "sha3", ] [[package]] @@ -3266,7 +3201,7 @@ dependencies = [ "libp2p-yamux", "multiaddr", "parking_lot 0.12.1", - "pin-project 1.0.10", + "pin-project 1.0.11", "rand 0.7.3", "smallvec", ] @@ -3292,7 +3227,7 @@ dependencies = [ "multihash", "multistream-select", "parking_lot 0.12.1", - "pin-project 1.0.10", + "pin-project 1.0.11", "prost", "prost-build", "rand 0.8.5", @@ -3454,7 +3389,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "9dd7e0c94051cda67123be68cf6b65211ba3dde7277be9068412de3e7ffd63ef" dependencies = [ "bytes", - "curve25519-dalek 3.2.1", + "curve25519-dalek 3.2.0", "futures", "lazy_static", "libp2p-core", @@ -3516,7 +3451,7 @@ dependencies = [ "instant", "libp2p-core", "log", - "pin-project 1.0.10", + "pin-project 1.0.11", "rand 0.7.3", "smallvec", "thiserror", @@ -3565,7 +3500,7 @@ dependencies = [ "rw-stream-sink", "soketto", "url", - "webpki-roots 0.22.3", + "webpki-roots 0.22.4", ] [[package]] @@ -3583,9 +3518,9 @@ dependencies = [ [[package]] name = "librocksdb-sys" -version = "0.6.1+6.28.2" +version = "0.8.0+7.4.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "81bc587013734dadb7cf23468e531aa120788b87243648be42e2d3a072186291" +checksum = "611804e4666a25136fcc5f8cf425ab4d26c7f74ea245ffe92ea23b85b6420b5d" dependencies = [ "bindgen", "bzip2-sys", @@ -3598,9 +3533,9 @@ dependencies = [ [[package]] name = "libsecp256k1" -version = "0.7.0" +version = "0.7.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b0452aac8bab02242429380e9b2f94ea20cea2b37e2c1777a1358799bbe97f37" +checksum = "95b09eff1b35ed3b33b877ced3a691fc7a481919c7e29c53c906226fcf55e2a1" dependencies = [ "arrayref", "base64 0.13.0", @@ -3657,9 +3592,9 @@ dependencies = [ [[package]] name = "linked-hash-map" -version = "0.5.4" +version = "0.5.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7fb9b38af92608140b86b693604b9ffcc5824240a484d1ecd4795bacb2fe88f3" +checksum = "0717cef1bc8b636c6e1c1bbdefc09e6322da8a9321966e8928ef80d20f7f770f" [[package]] name = "lock_api" @@ -3667,7 +3602,7 @@ version = "0.4.7" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "327fa5b6a6940e4699ec49a9beae1ea4845c6bab9314e4f84ac68742139d8c53" dependencies = [ - "autocfg 1.1.0", + "autocfg", "scopeguard", ] @@ -3683,11 +3618,11 @@ dependencies = [ [[package]] name = "lru" -version = "0.7.6" +version = "0.7.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8015d95cb7b2ddd3c0d32ca38283ceb1eea09b4713ee380bceb942d85a244228" +checksum = "e999beba7b6e8345721bd280141ed958096a2e4abdf74f67ff4ce49b4b54e47a" dependencies = [ - "hashbrown 0.11.2", + "hashbrown", ] [[package]] @@ -3777,9 +3712,9 @@ dependencies = [ [[package]] name = "multer" -version = "2.0.2" +version = "2.0.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5f8f35e687561d5c1667590911e6698a8cb714a134a7505718a182e7bc9d3836" +checksum = "a30ba6d97eb198c5e8a35d67d5779d6680cca35652a60ee90fc23dc431d4fde8" dependencies = [ "bytes", "encoding_rs", @@ -3789,7 +3724,7 @@ dependencies = [ "log", "memchr", "mime", - "spin 0.9.3", + "spin 0.9.4", "version_check", ] @@ -3853,29 +3788,11 @@ dependencies = [ "bytes", "futures", "log", - "pin-project 1.0.10", + "pin-project 1.0.11", "smallvec", "unsigned-varint", ] -[[package]] -name = "native-tls" -version = "0.2.10" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fd7e2f3618557f980e0b17e8856252eee3c97fa12c54dff0ca290fb6266ca4a9" -dependencies = [ - "lazy_static", - "libc", - "log", - "openssl", - "openssl-probe", - "openssl-sys", - "schannel", - "security-framework", - "security-framework-sys", - "tempfile", -] - [[package]] name = "netlink-packet-core" version = "0.4.2" @@ -3944,9 +3861,9 @@ dependencies = [ [[package]] name = "nix" -version = "0.24.1" +version = "0.24.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8f17df307904acd05aa8e32e97bb20f2a0df1728bbc2d771ae8f9a90463441e9" +checksum = "195cdbc1741b8134346d515b3a56a1c94b0912758009cfd53f99ea0f57b065fc" dependencies = [ "bitflags", "cfg-if", @@ -3986,7 +3903,7 @@ version = "0.1.45" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "225d3389fb3509a24c93f5c29eb6bde2586b98d9f016636dff58d7c6f7569cd9" dependencies = [ - "autocfg 1.1.0", + "autocfg", "num-traits", ] @@ -3996,7 +3913,7 @@ version = "0.2.15" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "578ede34cf02f8924ab9447f50c28075b4d3e5b269972345e7e0372b38c6cdcd" dependencies = [ - "autocfg 1.1.0", + "autocfg", ] [[package]] @@ -4036,56 +3953,11 @@ version = "0.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "624a8340c38c1b80fd549087862da4ba43e08858af025b236e509b6649fc13d5" -[[package]] -name = "openssl" -version = "0.10.40" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fb81a6430ac911acb25fe5ac8f1d2af1b4ea8a4fdfda0f1ee4292af2e2d8eb0e" -dependencies = [ - "bitflags", - "cfg-if", - "foreign-types", - "libc", - "once_cell", - "openssl-macros", - "openssl-sys", -] - -[[package]] -name = "openssl-macros" -version = "0.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b501e44f11665960c7e7fcf062c7d96a14ade4aa98116c004b2e37b5be7d736c" -dependencies = [ - "proc-macro2", - "quote", - "syn", -] - -[[package]] -name = "openssl-probe" -version = "0.1.5" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ff011a302c396a5197692431fc1948019154afc178baf7d8e37367442a4601cf" - -[[package]] -name = "openssl-sys" -version = "0.9.74" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "835363342df5fba8354c5b453325b110ffd54044e588c539cf2f20a8014e4cb1" -dependencies = [ - "autocfg 1.1.0", - "cc", - "libc", - "pkg-config", - "vcpkg", -] - [[package]] name = "os_str_bytes" -version = "6.1.0" +version = "6.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "21326818e99cfe6ce1e524c2a805c189a99b5ae555a35d19f9a284b427d86afa" +checksum = "648001efe5d5c0102d8cea768e348da85d90af8ba91f0bea908f157951493cd4" [[package]] name = "owning_ref" @@ -4098,12 +3970,12 @@ dependencies = [ [[package]] name = "parity-scale-codec" -version = "3.1.2" +version = "3.1.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e8b44461635bbb1a0300f100a841e571e7d919c81c73075ef5d152ffdb521066" +checksum = "9182e4a71cae089267ab03e67c99368db7cd877baf50f931e5d6d4b71e195ac0" dependencies = [ "arrayvec 0.7.2", - "bitvec 1.0.0", + "bitvec 1.0.1", "byte-slice-cast", "impl-trait-for-tuples", "parity-scale-codec-derive", @@ -4112,9 +3984,9 @@ dependencies = [ [[package]] name = "parity-scale-codec-derive" -version = "3.1.2" +version = "3.1.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c45ed1f39709f5a89338fab50e59816b2e8815f5bb58276e7ddf9afd495f73f8" +checksum = "9299338969a3d2f491d65f140b00ddec470858402f888af98e8642fb5e8965cd" dependencies = [ "proc-macro-crate", "proc-macro2", @@ -4187,9 +4059,9 @@ dependencies = [ [[package]] name = "password-hash" -version = "0.2.3" +version = "0.3.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "77e0b28ace46c5a396546bcf443bf422b57049617433d8854227352a4a9b24e7" +checksum = "1d791538a6dcc1e7cb7fe6f6b58aca40e7f79403c45b2bc274008b5e647af1d8" dependencies = [ "base64ct", "rand_core 0.6.3", @@ -4198,9 +4070,9 @@ dependencies = [ [[package]] name = "password-hash" -version = "0.3.2" +version = "0.4.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1d791538a6dcc1e7cb7fe6f6b58aca40e7f79403c45b2bc274008b5e647af1d8" +checksum = "7676374caaee8a325c9e7a2ae557f216c5563a171d6997b0ef8a65af35147700" dependencies = [ "base64ct", "rand_core 0.6.3", @@ -4215,26 +4087,25 @@ checksum = "0c520e05135d6e763148b6426a837e239041653ba7becd2e538c076c738025fc" [[package]] name = "pbkdf2" -version = "0.8.0" +version = "0.10.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d95f5254224e617595d2cc3cc73ff0a5eaf2637519e25f03388154e9378b6ffa" +checksum = "271779f35b581956db91a3e55737327a03aa051e90b1c47aeb189508533adfd7" dependencies = [ - "base64ct", - "crypto-mac 0.11.1", - "hmac 0.11.0", - "password-hash 0.2.3", - "sha2 0.9.9", + "digest 0.10.3", + "hmac 0.12.1", + "password-hash 0.3.2", + "sha2 0.10.2", ] [[package]] name = "pbkdf2" -version = "0.10.1" +version = "0.11.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "271779f35b581956db91a3e55737327a03aa051e90b1c47aeb189508533adfd7" +checksum = "83a0692ec44e4cf1ef28ca317f14f8f07da2d95ec3fa01f86e4467b725e60917" dependencies = [ "digest 0.10.3", "hmac 0.12.1", - "password-hash 0.3.2", + "password-hash 0.4.2", "sha2 0.10.2", ] @@ -4281,18 +4152,18 @@ dependencies = [ [[package]] name = "phf" -version = "0.10.1" +version = "0.11.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fabbf1ead8a5bcbc20f5f8b939ee3f5b0f6f281b6ad3468b84656b658b455259" +checksum = "4724fa946c8d1e7cd881bd3dbee63ce32fc1e9e191e35786b3dc1320a3f68131" dependencies = [ "phf_shared", ] [[package]] name = "phf_codegen" -version = "0.10.0" +version = "0.11.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4fb1c3a8bc4dd4e5cfce29b44ffc14bedd2ee294559a294e2a4d4c9e9a6a13cd" +checksum = "32ba0c43d7a1b6492b2924a62290cfd83987828af037b0743b38e6ab092aee58" dependencies = [ "phf_generator", "phf_shared", @@ -4300,9 +4171,9 @@ dependencies = [ [[package]] name = "phf_generator" -version = "0.10.0" +version = "0.11.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5d5285893bb5eb82e6aaf5d59ee909a06a16737a8970984dd7746ba9283498d6" +checksum = "5b450720b6f75cfbfabc195814bd3765f337a4f9a83186f8537297cac12f6705" dependencies = [ "phf_shared", "rand 0.8.5", @@ -4310,9 +4181,9 @@ dependencies = [ [[package]] name = "phf_shared" -version = "0.10.0" +version = "0.11.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b6796ad771acdc0123d2a88dc428b5e38ef24456743ddb1744ed628f9815c096" +checksum = "9dd5609d4b2df87167f908a32e1b146ce309c16cf35df76bc11f440b756048e4" dependencies = [ "siphasher", "uncased", @@ -4320,27 +4191,27 @@ dependencies = [ [[package]] name = "pin-project" -version = "0.4.29" +version = "0.4.30" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9615c18d31137579e9ff063499264ddc1278e7b1982757ebc111028c4d1dc909" +checksum = "3ef0f924a5ee7ea9cbcea77529dba45f8a9ba9f622419fe3386ca581a3ae9d5a" dependencies = [ - "pin-project-internal 0.4.29", + "pin-project-internal 0.4.30", ] [[package]] name = "pin-project" -version = "1.0.10" +version = "1.0.11" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "58ad3879ad3baf4e44784bc6a718a8698867bb991f8ce24d1bcbe2cfb4c3a75e" +checksum = "78203e83c48cffbe01e4a2d35d566ca4de445d79a85372fc64e378bfc812a260" dependencies = [ - "pin-project-internal 1.0.10", + "pin-project-internal 1.0.11", ] [[package]] name = "pin-project-internal" -version = "0.4.29" +version = "0.4.30" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "044964427019eed9d49d9d5bbce6047ef18f37100ea400912a9fa4a3523ab12a" +checksum = "851c8d0ce9bebe43790dedfc86614c23494ac9f423dd618d3a61fc693eafe61e" dependencies = [ "proc-macro2", "quote", @@ -4349,9 +4220,9 @@ dependencies = [ [[package]] name = "pin-project-internal" -version = "1.0.10" +version = "1.0.11" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "744b6f092ba29c3650faf274db506afd39944f48420f6c86b17cfe0ee1cb36bb" +checksum = "710faf75e1b33345361201d36d04e98ac1ed8909151a017ed384700836104c74" dependencies = [ "proc-macro2", "quote", @@ -4378,13 +4249,12 @@ checksum = "8b870d8c151b6f2fb93e84a13146138f05d02ed11c7e7c54f8826aaaf7c9f184" [[package]] name = "pkcs8" -version = "0.8.0" +version = "0.9.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7cabda3fb821068a9a4fab19a683eac3af12edf0f34b94a8be53c4972b8149d0" +checksum = "9eca2c590a5f85da82668fa685c09ce2888b9430e83299debf1f34b65fd4a4ba" dependencies = [ "der", "spki", - "zeroize", ] [[package]] @@ -4501,9 +4371,9 @@ checksum = "dbf0c48bc1d91375ae5c3cd81e3722dff1abcf81a30960240640d223f59fe0e5" [[package]] name = "proc-macro2" -version = "1.0.40" +version = "1.0.42" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "dd96a1e8ed2596c337f8eae5f24924ec83f5ad5ab21ea8e455d3566c69fbcaf7" +checksum = "c278e965f1d8cf32d6e0e96de3d3e79712178ae67986d9cf9151f51e95aac89b" dependencies = [ "unicode-ident", ] @@ -4643,25 +4513,6 @@ version = "0.7.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "dc33ff2d4973d518d823d61aa239014831e521c75da58e3df4840d3f47749d09" -[[package]] -name = "rand" -version = "0.6.5" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6d71dacdc3c88c1fde3885a3be3fbab9f35724e6ce99467f7d9c5026132184ca" -dependencies = [ - "autocfg 0.1.8", - "libc", - "rand_chacha 0.1.1", - "rand_core 0.4.2", - "rand_hc 0.1.0", - "rand_isaac", - "rand_jitter", - "rand_os", - "rand_pcg", - "rand_xorshift", - "winapi", -] - [[package]] name = "rand" version = "0.7.3" @@ -4672,7 +4523,7 @@ dependencies = [ "libc", "rand_chacha 0.2.2", "rand_core 0.5.1", - "rand_hc 0.2.0", + "rand_hc", ] [[package]] @@ -4686,16 +4537,6 @@ dependencies = [ "rand_core 0.6.3", ] -[[package]] -name = "rand_chacha" -version = "0.1.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "556d3a1ca6600bfcbab7c7c91ccb085ac7fbbcd70e008a98742e7847f4f7bcef" -dependencies = [ - "autocfg 0.1.8", - "rand_core 0.3.1", -] - [[package]] name = "rand_chacha" version = "0.2.2" @@ -4716,21 +4557,6 @@ dependencies = [ "rand_core 0.6.3", ] -[[package]] -name = "rand_core" -version = "0.3.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7a6fdeb83b075e8266dcc8762c22776f6877a63111121f5f8c7411e5be7eed4b" -dependencies = [ - "rand_core 0.4.2", -] - -[[package]] -name = "rand_core" -version = "0.4.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9c33a3c44ca05fa6f1807d8e6743f3824e8509beca625669633be0acbdf509dc" - [[package]] name = "rand_core" version = "0.5.1" @@ -4749,15 +4575,6 @@ dependencies = [ "getrandom 0.2.7", ] -[[package]] -name = "rand_hc" -version = "0.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7b40677c7be09ae76218dc623efbf7b18e34bced3f38883af07bb75630a21bc4" -dependencies = [ - "rand_core 0.3.1", -] - [[package]] name = "rand_hc" version = "0.2.0" @@ -4767,73 +4584,11 @@ dependencies = [ "rand_core 0.5.1", ] -[[package]] -name = "rand_isaac" -version = "0.1.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ded997c9d5f13925be2a6fd7e66bf1872597f759fd9dd93513dd7e92e5a5ee08" -dependencies = [ - "rand_core 0.3.1", -] - -[[package]] -name = "rand_jitter" -version = "0.1.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1166d5c91dc97b88d1decc3285bb0a99ed84b05cfd0bc2341bdf2d43fc41e39b" -dependencies = [ - "libc", - "rand_core 0.4.2", - "winapi", -] - -[[package]] -name = "rand_os" -version = "0.1.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7b75f676a1e053fc562eafbb47838d67c84801e38fc1ba459e8f180deabd5071" -dependencies = [ - "cloudabi", - "fuchsia-cprng", - "libc", - "rand_core 0.4.2", - "rdrand", - "winapi", -] - -[[package]] -name = "rand_pcg" -version = "0.1.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "abf9b09b01790cfe0364f52bf32995ea3c39f4d2dd011eac241d2914146d0b44" -dependencies = [ - "autocfg 0.1.8", - "rand_core 0.4.2", -] - -[[package]] -name = "rand_xorshift" -version = "0.1.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cbf7e9e623549b0e21f6e97cf8ecf247c1a8fd2e8a992ae265314300b2455d5c" -dependencies = [ - "rand_core 0.3.1", -] - -[[package]] -name = "rdrand" -version = "0.4.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "678054eb77286b51581ba43620cc911abf02758c91f93f479767aed0f90458b2" -dependencies = [ - "rand_core 0.3.1", -] - [[package]] name = "redox_syscall" -version = "0.2.13" +version = "0.2.16" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "62f25bc4c7e55e0b0b7a1d43fb893f4fa1361d0abe38b9ce4f323c2adfe6ef42" +checksum = "fb5a58c1855b4b6819d59012155603f0b22ad30cad752600aadfcb695265519a" dependencies = [ "bitflags", ] @@ -4851,9 +4606,9 @@ dependencies = [ [[package]] name = "regex" -version = "1.5.6" +version = "1.6.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d83f127d94bdbcda4c8cc2e50f6f84f4b611f69c902699ca385a39c3a75f9ff1" +checksum = "4c4eb3267174b8c6c2f654116623910a0fef09c4753f8dd83db29c48a0df988b" dependencies = [ "aho-corasick", "memchr", @@ -4871,9 +4626,9 @@ dependencies = [ [[package]] name = "regex-syntax" -version = "0.6.26" +version = "0.6.27" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "49b3de9ec5dc0a3417da371aab17d729997c15010e7fd24ff707773a33bddb64" +checksum = "a3f87b73ce11b1619a3c6332f45341e0047173771e8b8b73f87bfeefb7b56244" [[package]] name = "remove_dir_all" @@ -4900,13 +4655,11 @@ dependencies = [ "http-body", "hyper", "hyper-rustls", - "hyper-tls", "ipnet", "js-sys", "lazy_static", "log", "mime", - "native-tls", "percent-encoding", "pin-project-lite 0.2.9", "rustls 0.20.6", @@ -4915,14 +4668,13 @@ dependencies = [ "serde_json", "serde_urlencoded", "tokio", - "tokio-native-tls", "tokio-rustls", "tower-service", "url", "wasm-bindgen", "wasm-bindgen-futures", "web-sys", - "webpki-roots 0.22.3", + "webpki-roots 0.22.4", "winreg 0.10.1", ] @@ -4938,12 +4690,12 @@ dependencies = [ [[package]] name = "rfc6979" -version = "0.1.0" +version = "0.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "96ef608575f6392792f9ecf7890c00086591d29a83910939d430753f7c050525" +checksum = "88c86280f057430a52f4861551b092a01b419b8eacefc7c995eacb9dc132fe32" dependencies = [ "crypto-bigint", - "hmac 0.11.0", + "hmac 0.12.1", "zeroize", ] @@ -4963,14 +4715,12 @@ dependencies = [ ] [[package]] -name = "ripemd160" -version = "0.9.1" +name = "ripemd" +version = "0.1.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2eca4ecc81b7f313189bf73ce724400a07da2a6dac19588b03c8bd76a2dcc251" +checksum = "1facec54cb5e0dc08553501fa740091086d0259ad0067e0d4103448e4cb22ed3" dependencies = [ - "block-buffer 0.9.0", - "digest 0.9.0", - "opaque-debug 0.3.0", + "digest 0.10.3", ] [[package]] @@ -4996,9 +4746,9 @@ dependencies = [ [[package]] name = "rocksdb" -version = "0.18.0" +version = "0.19.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "620f4129485ff1a7128d184bc687470c21c7951b64779ebc9cfdad3dcd920290" +checksum = "7e9562ea1d70c0cc63a34a22d977753b50cca91cc6b6527750463bd5dd8697bc" dependencies = [ "libc", "librocksdb-sys", @@ -5083,7 +4833,7 @@ version = "0.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "bfa0f585226d2e68097d4f95d113b15b83a82e819ab25717ec0590d9584ef366" dependencies = [ - "semver 1.0.10", + "semver 1.0.12", ] [[package]] @@ -5122,9 +4872,9 @@ dependencies = [ [[package]] name = "rustversion" -version = "1.0.6" +version = "1.0.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f2cc38e8fa666e2de3c4aba7edeb5ffc5246c1c2ed0e3d17e560aeeba736b23f" +checksum = "24c8ad4f0c00e1eb5bc7614d236a7f1300e3dbd76b68cac8e06fb00b015ad8d8" [[package]] name = "rw-stream-sink" @@ -5133,7 +4883,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "4da5fcb054c46f5a5dff833b129285a93d3f0179531735e6c866e8cc307d2020" dependencies = [ "futures", - "pin-project 0.4.29", + "pin-project 0.4.30", "static_assertions", ] @@ -5161,16 +4911,6 @@ dependencies = [ "winapi-util", ] -[[package]] -name = "schannel" -version = "0.1.20" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "88d6731146462ea25d9244b2ed5fd1d716d25c52e4d54aa4fb0f3c4e9854dbe2" -dependencies = [ - "lazy_static", - "windows-sys", -] - [[package]] name = "schemafy_core" version = "0.5.2" @@ -5238,10 +4978,11 @@ dependencies = [ [[package]] name = "sec1" -version = "0.2.1" +version = "0.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "08da66b8b0965a5555b6bd6639e68ccba85e1e2506f5fbb089e93f8a04e1a2d1" +checksum = "3be24c1842290c45df0a7bf069e0c268a747ad05a192f2fd7dcfdbc1cba40928" dependencies = [ + "base16ct", "der", "generic-array 0.14.5", "pkcs8", @@ -5251,46 +4992,23 @@ dependencies = [ [[package]] name = "secp256k1" -version = "0.20.3" +version = "0.24.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "97d03ceae636d0fed5bae6a7f4f664354c5f4fcedf6eef053fef17e49f837d0a" +checksum = "b7649a0b3ffb32636e60c7ce0d70511eda9c52c658cd0634e194d5a19943aeff" dependencies = [ - "rand 0.6.5", + "rand 0.8.5", "secp256k1-sys", ] [[package]] name = "secp256k1-sys" -version = "0.4.2" +version = "0.6.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "957da2573cde917463ece3570eab4a0b3f19de6f1646cde62e6fd3868f566036" +checksum = "7058dc8eaf3f2810d7828680320acda0b25a288f6d288e19278e249bbf74226b" dependencies = [ "cc", ] -[[package]] -name = "security-framework" -version = "2.6.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2dc14f172faf8a0194a3aded622712b0de276821addc574fa54fc0a1167e10dc" -dependencies = [ - "bitflags", - "core-foundation", - "core-foundation-sys", - "libc", - "security-framework-sys", -] - -[[package]] -name = "security-framework-sys" -version = "2.6.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0160a13a177a45bfb43ce71c01580998474f556ad854dcbca936dd2841a5c556" -dependencies = [ - "core-foundation-sys", - "libc", -] - [[package]] name = "semver" version = "0.9.0" @@ -5302,9 +5020,9 @@ dependencies = [ [[package]] name = "semver" -version = "1.0.10" +version = "1.0.12" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a41d061efea015927ac527063765e73601444cdc344ba855bc7bd44578b25e1c" +checksum = "a2333e6df6d6598f2b1974829f853c2b4c5f4a6e503c10af918081aa6f8564e1" dependencies = [ "serde", ] @@ -5323,18 +5041,18 @@ checksum = "930c0acf610d3fdb5e2ab6213019aaa04e227ebe9547b0649ba599b16d788bd7" [[package]] name = "serde" -version = "1.0.138" +version = "1.0.140" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1578c6245786b9d168c5447eeacfb96856573ca56c9d68fdcf394be134882a47" +checksum = "fc855a42c7967b7c369eb5860f7164ef1f6f81c20c7cc1141f2a604e18723b03" dependencies = [ "serde_derive", ] [[package]] name = "serde-aux" -version = "3.0.1" +version = "3.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "93abf9799c576f004252b2a05168d58527fb7c54de12e94b4d12fe3475ffad24" +checksum = "d0a77223b653fa95f3f9864f3eb25b93e4ed170687eb42d85b6b98af21d5e1de" dependencies = [ "serde", "serde_json", @@ -5342,9 +5060,9 @@ dependencies = [ [[package]] name = "serde_derive" -version = "1.0.138" +version = "1.0.140" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "023e9b1467aef8a10fb88f25611870ada9800ef7e22afce356bb0d2387b6f27c" +checksum = "6f2122636b9fe3b81f1cb25099fcf2d3f542cdb1d45940d56c713158884a05da" dependencies = [ "proc-macro2", "quote", @@ -5353,9 +5071,9 @@ dependencies = [ [[package]] name = "serde_json" -version = "1.0.81" +version = "1.0.82" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9b7ce2b32a1aed03c558dc61a5cd328f15aff2dbc17daad8fb8af04d2100e15c" +checksum = "82c2c1fdcd807d1098552c5b9a36e425e42e9fbd7c6a37a8425f390f781f7fa7" dependencies = [ "itoa", "ryu", @@ -5401,7 +5119,7 @@ version = "1.5.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "e182d6ec6f05393cc0e5ed1bf81ad6db3a8feedf8ee515ecdd369809bcce8082" dependencies = [ - "darling", + "darling 0.13.4", "proc-macro2", "quote", "syn", @@ -5409,9 +5127,9 @@ dependencies = [ [[package]] name = "serde_yaml" -version = "0.8.24" +version = "0.8.26" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "707d15895415db6628332b737c838b88c598522e4dc70647e59b72312924aebc" +checksum = "578a7433b776b56a35785ed5ce9a7e777ac0598aac5a6dd1b4b18a307c7fc71b" dependencies = [ "indexmap", "ryu", @@ -5494,18 +5212,6 @@ dependencies = [ "digest 0.10.3", ] -[[package]] -name = "sha3" -version = "0.9.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f81199417d4e5de3f04b1e871023acea7389672c4135918f05aa9cbf2f2fa809" -dependencies = [ - "block-buffer 0.9.0", - "digest 0.9.0", - "keccak", - "opaque-debug 0.3.0", -] - [[package]] name = "sha3" version = "0.10.1" @@ -5552,11 +5258,11 @@ dependencies = [ [[package]] name = "signature" -version = "1.4.0" +version = "1.5.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "02658e48d89f2bec991f9a78e69cfa4c316f8d6a6c4ec12fae1aeb263d486788" +checksum = "f054c6c1a6e95179d6f23ed974060dcefb2d9388bb7256900badad682c499de4" dependencies = [ - "digest 0.9.0", + "digest 0.10.3", "rand_core 0.6.3", ] @@ -5583,9 +5289,12 @@ checksum = "7bd3e3206899af3f8b12af284fafc038cc1dc2b41d1b89dd17297221c5d225de" [[package]] name = "slab" -version = "0.4.6" +version = "0.4.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "eb703cfe953bccee95685111adeedb76fabe4e97549a58d16f03ea7b9367bb32" +checksum = "4614a76b2a8be0058caa9dbbaf66d988527d86d003c11a94fbd335d7661edcef" +dependencies = [ + "autocfg", +] [[package]] name = "smallvec" @@ -5600,9 +5309,9 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "774d05a3edae07ce6d68ea6984f3c05e9bba8927e3dd591e3b479e5b03213d0d" dependencies = [ "aes-gcm 0.9.4", - "blake2 0.10.4", + "blake2", "chacha20poly1305", - "curve25519-dalek 4.0.0-pre.2", + "curve25519-dalek 4.0.0-pre.1", "rand_core 0.6.3", "ring", "rustc_version 0.4.0", @@ -5644,15 +5353,15 @@ checksum = "6e63cff320ae2c57904679ba7cb63280a3dc4613885beafb148ee7bf9aa9042d" [[package]] name = "spin" -version = "0.9.3" +version = "0.9.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c530c2b0d0bf8b69304b39fe2001993e267461948b890cd037d8ad4293fa1a0d" +checksum = "7f6002a767bff9e83f8eeecf883ecb8011875a21ae8da43bffb817a57e78cc09" [[package]] name = "spki" -version = "0.5.4" +version = "0.6.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "44d01ac02a6ccf3e07db148d2be087da624fea0221a16152ed01f0496a6b0a27" +checksum = "67cf02bbac7a337dc36e4f5a693db6c21e7863f45070f7064577eb4367a3212b" dependencies = [ "base64ct", "der", @@ -5742,11 +5451,11 @@ checksum = "aaf86bbcfd1fa9670b7a129f64fc0c9fcbbfe4f1bc4210e9e98fe71ffc12cde2" [[package]] name = "strum" -version = "0.24.0" +version = "0.24.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e96acfc1b70604b8b2f1ffa4c57e59176c7dbb05d556c71ecd2f5498a1dee7f8" +checksum = "063e6045c0e62079840579a7e47a355ae92f60eb74daaf156fb1e84ba164e63f" dependencies = [ - "strum_macros 0.24.0", + "strum_macros 0.24.2", ] [[package]] @@ -5763,9 +5472,9 @@ dependencies = [ [[package]] name = "strum_macros" -version = "0.24.0" +version = "0.24.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6878079b17446e4d3eba6192bb0a2950d5b14f0ed8424b852310e5a94345d0ef" +checksum = "4faebde00e8ff94316c01800f9054fd2ba77d30d9e922541913051d1d978918b" dependencies = [ "heck 0.4.0", "proc-macro2", @@ -5852,6 +5561,12 @@ dependencies = [ "libc", ] +[[package]] +name = "tai64" +version = "4.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ed7401421025f4132e6c1f7af5e7f8287383969f36e6628016cd509b8d3da9dc" + [[package]] name = "tap" version = "1.0.1" @@ -5928,9 +5643,9 @@ dependencies = [ [[package]] name = "tikv-jemalloc-sys" -version = "0.4.3+5.2.1-patched.2" +version = "0.5.1+5.3.0-patched" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a1792ccb507d955b46af42c123ea8863668fae24d03721e40cad6a41773dbb49" +checksum = "931e876f91fed0827f863a2d153897790da0b24d882c721a79cb3beb0b903261" dependencies = [ "cc", "fs_extra", @@ -5939,11 +5654,12 @@ dependencies = [ [[package]] name = "time" -version = "0.1.43" +version = "0.1.44" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ca8a50ef2360fbd1eeb0ecd46795a87a19024eb4b53c5dc916ca1fd95fe62438" +checksum = "6db9e6914ab8b1ae1c260a4ae7a49b6c5611b40328a735b21862567685e73255" dependencies = [ "libc", + "wasi 0.10.0+wasi-snapshot-preview1", "winapi", ] @@ -5964,9 +5680,9 @@ dependencies = [ [[package]] name = "time" -version = "0.3.9" +version = "0.3.11" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c2702e08a7a860f005826c6815dcac101b19b5eb330c27fe4a5928fec1d20ddd" +checksum = "72c91f41dcb2f096c05f0873d667dceec1087ce5bcf984ec8ffb19acddbb3217" dependencies = [ "itoa", "libc", @@ -6022,10 +5738,11 @@ checksum = "cda74da7e1a664f795bb1f8a87ec406fb89a02522cf6e50620d016add6dbbf5c" [[package]] name = "tokio" -version = "1.19.2" +version = "1.20.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c51a52ed6686dd62c320f9b89299e9dfb46f730c7a48e635c19f21d116cb1439" +checksum = "7a8325f63a7d4774dd041e363b2409ed1c5cbbd0f867795e661df066b2b0a581" dependencies = [ + "autocfg", "bytes", "libc", "memchr", @@ -6051,16 +5768,6 @@ dependencies = [ "syn", ] -[[package]] -name = "tokio-native-tls" -version = "0.3.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f7d995660bd2b7f8c1568414c1126076c13fbb725c40112dc0120b78eb9b717b" -dependencies = [ - "native-tls", - "tokio", -] - [[package]] name = "tokio-rustls" version = "0.23.4" @@ -6074,9 +5781,9 @@ dependencies = [ [[package]] name = "tokio-tungstenite" -version = "0.17.1" +version = "0.17.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "06cda1232a49558c46f8a504d5b93101d42c0bf7f911f12a105ba48168f821ae" +checksum = "f714dd15bead90401d77e04243611caec13726c2408afd5b31901dfcdcb3b181" dependencies = [ "futures-util", "log", @@ -6085,7 +5792,7 @@ dependencies = [ "tokio-rustls", "tungstenite", "webpki 0.22.0", - "webpki-roots 0.22.3", + "webpki-roots 0.22.4", ] [[package]] @@ -6113,13 +5820,13 @@ dependencies = [ [[package]] name = "tower" -version = "0.4.12" +version = "0.4.13" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9a89fd63ad6adf737582df5db40d286574513c69a11dac5214dc3b5603d6713e" +checksum = "b8fa9be0de6cf49e536ce1851f987bd21a43b771b09473c3549a6c853db37c1c" dependencies = [ "futures-core", "futures-util", - "pin-project 1.0.10", + "pin-project 1.0.11", "pin-project-lite 0.2.9", "tokio", "tokio-util", @@ -6156,9 +5863,9 @@ checksum = "343bc9466d3fe6b0f960ef45960509f84480bf4fd96f92901afe7ff3df9d3a62" [[package]] name = "tower-service" -version = "0.3.1" +version = "0.3.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "360dfd1d6d30e05fda32ace2c8c70e9c0a9da713275777f5a4dbb8a1893930c6" +checksum = "b6bc1c9ce2b5135ac7f93c72918fc37feb872bdc6a5533a8b85eb4b86bfdae52" [[package]] name = "tracing" @@ -6180,15 +5887,15 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "09d48f71a791638519505cefafe162606f706c25592e4bde4d97600c0195312e" dependencies = [ "crossbeam-channel", - "time 0.3.9", + "time 0.3.11", "tracing-subscriber", ] [[package]] name = "tracing-attributes" -version = "0.1.21" +version = "0.1.22" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cc6b8ad3567499f98a1db7a752b07a7c8c7c7c34c332ec00effb2b0027974b7c" +checksum = "11c75893af559bc8e10716548bdef5cb2b983f8e637db9d0e15126b61b484ee2" dependencies = [ "proc-macro2", "quote", @@ -6197,9 +5904,9 @@ dependencies = [ [[package]] name = "tracing-core" -version = "0.1.27" +version = "0.1.28" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7709595b8878a4965ce5e87ebf880a7d39c9afc6837721b21a5a816a8117d921" +checksum = "7b7358be39f2f274f322d2aaed611acc57f382e8eb1e5b48cb9ae30933495ce7" dependencies = [ "once_cell", "valuable", @@ -6213,7 +5920,7 @@ checksum = "97d095ae15e245a057c8e8451bab9b3ee1e1f68e9ba2b4fbc18d0ac5237835f2" dependencies = [ "futures", "futures-task", - "pin-project 1.0.10", + "pin-project 1.0.11", "tracing", ] @@ -6240,13 +5947,13 @@ dependencies = [ [[package]] name = "tracing-subscriber" -version = "0.3.11" +version = "0.3.15" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4bc28f93baff38037f64e6f43d34cfa1605f27a49c34e8a04c5e78b0babf2596" +checksum = "60db860322da191b40952ad9affe65ea23e7dd6a5c442c2c42865810c6ab8e6b" dependencies = [ "ansi_term", - "lazy_static", "matchers", + "once_cell", "regex", "serde", "serde_json", @@ -6261,9 +5968,9 @@ dependencies = [ [[package]] name = "tracing-test" -version = "0.2.2" +version = "0.2.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f6992d8a98f570be1c729fe8b6f464fb18c4117054c10f1f952c22d533b48a74" +checksum = "9e3d272c44878d2bbc9f4a20ad463724f03e19dbc667c6e84ac433ab7ffcc70b" dependencies = [ "lazy_static", "tracing-core", @@ -6273,9 +5980,9 @@ dependencies = [ [[package]] name = "tracing-test-macro" -version = "0.2.2" +version = "0.2.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "719fa6c65bada6a7a3b8466702ec6fb4c4189b69339f78c9e597f796e493712e" +checksum = "744324b12d69a9fc1edea4b38b7b1311295b662d161ad5deac17bb1358224a08" dependencies = [ "lazy_static", "quote", @@ -6333,9 +6040,9 @@ checksum = "59547bce71d9c38b83d9c0e92b6066c4253371f15005def0c30d9657f50c7642" [[package]] name = "tungstenite" -version = "0.17.2" +version = "0.17.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d96a2dea40e7570482f28eb57afbe42d97551905da6a9400acc5c328d24004f5" +checksum = "e27992fd6a8c29ee7eef28fc78349aa244134e10ad447ce3b9f0ac0ed0fa4ce0" dependencies = [ "base64 0.13.0", "byteorder", @@ -6360,9 +6067,9 @@ checksum = "dcf81ac59edc17cc8697ff311e8f5ef2d99fcbd9817b34cec66f90b6c3dfd987" [[package]] name = "ucd-trie" -version = "0.1.3" +version = "0.1.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "56dee185309b50d1f11bfedef0fe6d036842e3fb77413abef29f8f8d1c5d4c1c" +checksum = "89570599c4fe5585de2b388aab47e99f7fa4e9238a1399f707a02e356058141c" [[package]] name = "uint" @@ -6402,15 +6109,15 @@ checksum = "099b7128301d285f79ddd55b9a83d5e6b9e97c92e0ea0daebee7263e932de992" [[package]] name = "unicode-ident" -version = "1.0.1" +version = "1.0.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5bd2fe26506023ed7b5e1e315add59d6f584c621d037f9368fea9cfb988f368c" +checksum = "15c61ba63f9235225a22310255a29b806b907c9b8c964bcbd0a2c70f3f2deea7" [[package]] name = "unicode-normalization" -version = "0.1.19" +version = "0.1.21" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d54590932941a9e9266f0832deed84ebe1bf2e4c9e4a3554d393d18f5e854bf9" +checksum = "854cbdc4f7bc6ae19c820d44abdc3277ac3e1b2b93db20a636825d9322fb60e6" dependencies = [ "tinyvec", ] @@ -6560,6 +6267,12 @@ version = "0.9.0+wasi-snapshot-preview1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "cccddf32554fecc6acb585f82a32a72e28b48f8c4c1883ddfeeeaa96f7d8e519" +[[package]] +name = "wasi" +version = "0.10.0+wasi-snapshot-preview1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1a143597ca7c7793eff794def352d41792a93c481eb1042423ff7ff72ba2c31f" + [[package]] name = "wasi" version = "0.11.0+wasi-snapshot-preview1" @@ -6568,9 +6281,9 @@ checksum = "9c8d87e72b64a3b4db28d11ce29237c246188f4f51057d65a7eab63b7987e423" [[package]] name = "wasm-bindgen" -version = "0.2.81" +version = "0.2.82" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7c53b543413a17a202f4be280a7e5c62a1c69345f5de525ee64f8cfdbc954994" +checksum = "fc7652e3f6c4706c8d9cd54832c4a4ccb9b5336e2c3bd154d5cccfbf1c1f5f7d" dependencies = [ "cfg-if", "wasm-bindgen-macro", @@ -6578,13 +6291,13 @@ dependencies = [ [[package]] name = "wasm-bindgen-backend" -version = "0.2.81" +version = "0.2.82" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5491a68ab4500fa6b4d726bd67408630c3dbe9c4fe7bda16d5c82a1fd8c7340a" +checksum = "662cd44805586bd52971b9586b1df85cdbbd9112e4ef4d8f41559c334dc6ac3f" dependencies = [ "bumpalo", - "lazy_static", "log", + "once_cell", "proc-macro2", "quote", "syn", @@ -6593,9 +6306,9 @@ dependencies = [ [[package]] name = "wasm-bindgen-futures" -version = "0.4.31" +version = "0.4.32" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "de9a9cec1733468a8c657e57fa2413d2ae2c0129b95e87c5b72b8ace4d13f31f" +checksum = "fa76fb221a1f8acddf5b54ace85912606980ad661ac7a503b4570ffd3a624dad" dependencies = [ "cfg-if", "js-sys", @@ -6605,9 +6318,9 @@ dependencies = [ [[package]] name = "wasm-bindgen-macro" -version = "0.2.81" +version = "0.2.82" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c441e177922bc58f1e12c022624b6216378e5febc2f0533e41ba443d505b80aa" +checksum = "b260f13d3012071dfb1512849c033b1925038373aea48ced3012c09df952c602" dependencies = [ "quote", "wasm-bindgen-macro-support", @@ -6615,9 +6328,9 @@ dependencies = [ [[package]] name = "wasm-bindgen-macro-support" -version = "0.2.81" +version = "0.2.82" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7d94ac45fcf608c1f45ef53e748d35660f168490c10b23704c7779ab8f5c3048" +checksum = "5be8e654bdd9b79216c2929ab90721aa82faf65c48cdf08bdc4e7f51357b80da" dependencies = [ "proc-macro2", "quote", @@ -6628,9 +6341,9 @@ dependencies = [ [[package]] name = "wasm-bindgen-shared" -version = "0.2.81" +version = "0.2.82" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6a89911bd99e5f3659ec4acf9c4d93b0a90fe4a2a11f15328472058edc5261be" +checksum = "6598dd0bd3c7d51095ff6531a5b23e02acdc81804e30d8f07afb77b7215a140a" [[package]] name = "wasm-timer" @@ -6649,9 +6362,9 @@ dependencies = [ [[package]] name = "web-sys" -version = "0.3.58" +version = "0.3.59" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2fed94beee57daf8dd7d51f2b15dc2bcde92d7a72304cdf662a4371008b71b90" +checksum = "ed055ab27f941423197eb86b2035720b1a3ce40504df082cac2ecc6ed73335a1" dependencies = [ "js-sys", "wasm-bindgen", @@ -6688,9 +6401,9 @@ dependencies = [ [[package]] name = "webpki-roots" -version = "0.22.3" +version = "0.22.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "44d8de8415c823c8abd270ad483c6feeac771fad964890779f9a8cb24fbbc1bf" +checksum = "f1c760f0d366a6c24a02ed7816e23e691f5d92291f94d15e836006fd11b04daf" dependencies = [ "webpki 0.22.0", ] @@ -6885,11 +6598,11 @@ dependencies = [ [[package]] name = "x25519-dalek" -version = "1.2.0" +version = "1.1.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2392b6b94a576b4e2bf3c5b2757d63f10ada8020a2e4d08ac849ebcf6ea8e077" +checksum = "5a0c105152107e3b96f6a00a65e86ce82d9b125230e1c4302940eca58ff71f4f" dependencies = [ - "curve25519-dalek 3.2.1", + "curve25519-dalek 3.2.0", "rand_core 0.5.1", "zeroize", ] @@ -6927,9 +6640,9 @@ dependencies = [ [[package]] name = "zeroize" -version = "1.3.0" +version = "1.5.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4756f7db3f7b5574938c3eb1c117038b8e07f95ee6718c0efad4ac21508f1efd" +checksum = "c394b5bd0c6f669e7275d9c20aa90ae064cb22e75a1cad54e1b34088034b149f" dependencies = [ "zeroize_derive", ] diff --git a/README.md b/README.md index bc16913e1b3..18fa932d9e0 100644 --- a/README.md +++ b/README.md @@ -77,6 +77,8 @@ Jul 12 23:28:47.238 INFO fuel_core: Binding GraphQL provider to 127.0.0.1:4000 #### Troubleshooting +##### Outdated database + If you encounter an error such as ```console @@ -85,6 +87,15 @@ thread 'main' panicked at 'unable to open database: DatabaseError(Error { messag Clear your local database using: `rm -rf ~/.fuel/db` +##### File descriptor limits + +On some macOS versions the default file descriptor limit is quite low, which can lead to IO errors with messages like `Too many open files` or even `fatal runtime error: Rust cannot catch foreign exceptions` when RocksDB encounters these issues. Use the following command to increase the open file limit. Note that this only affects the current shell session, so consider adding it to `~/.zshrc`. + +```bash +ulimit -n 10240 +``` + + #### Log level The service relies on the environment variable `RUST_LOG`. For more information, check the [EnvFilter examples](https://docs.rs/tracing-subscriber/latest/tracing_subscriber/struct.EnvFilter.html#examples) crate. @@ -133,5 +144,5 @@ RET(REG_ONE), ```console $ cargo run --bin fuel-gql-cli -- transaction submit \ -"{\"Script\":{\"byte_price\":0,\"gas_price\":0,\"gas_limit\":1000000,\"maturity\":0,\"script\":[80,64,0,202,80,68,0,186,51,65,16,0,36,4,0,0],\"script_data\":[],\"inputs\":[],\"outputs\":[],\"witnesses\":[],\"receipts_root\":\"0x6114142d12e0f58cfb8c72c270cd0535944fb1ba763dce83c17e882c482224a2\"}}" +"{\"Script\":{\"gas_price\":0,\"gas_limit\":1000000,\"maturity\":0,\"script\":[80,64,0,202,80,68,0,186,51,65,16,0,36,4,0,0],\"script_data\":[],\"inputs\":[],\"outputs\":[],\"witnesses\":[],\"receipts_root\":\"0x6114142d12e0f58cfb8c72c270cd0535944fb1ba763dce83c17e882c482224a2\"}}" ``` diff --git a/ci/Dockerfile.x86_64-unknown-linux-gnu-clang b/ci/Dockerfile.x86_64-unknown-linux-gnu-clang index 139fc8a2616..3aaf9bbf5fb 100644 --- a/ci/Dockerfile.x86_64-unknown-linux-gnu-clang +++ b/ci/Dockerfile.x86_64-unknown-linux-gnu-clang @@ -1,10 +1,4 @@ -FROM ghcr.io/cross-rs/x86_64-unknown-linux-gnu:main-centos +FROM ghcr.io/cross-rs/x86_64-unknown-linux-gnu:main -RUN yum -y update && \ - yum -y install centos-release-scl && \ - yum-config-manager --enable rhel-server-rhscl-8-rpms && \ - yum -y install llvm-toolset-7.0 - -COPY centos-entrypoint /usr/bin/entrypoint.sh -RUN chmod +x /usr/bin/entrypoint.sh -ENTRYPOINT [ "/usr/bin/entrypoint.sh" ] +RUN apt-get update && \ + apt-get install --assume-yes clang libclang-dev binutils-aarch64-linux-gnu diff --git a/ci/centos-entrypoint b/ci/centos-entrypoint deleted file mode 100755 index 2adf1f626d8..00000000000 --- a/ci/centos-entrypoint +++ /dev/null @@ -1,3 +0,0 @@ -#!/usr/bin/env bash -source scl_source enable llvm-toolset-7.0 -exec "$@" \ No newline at end of file diff --git a/deployment/charts/Chart.yaml b/deployment/charts/Chart.yaml index 734e08de89b..bc2378b0665 100644 --- a/deployment/charts/Chart.yaml +++ b/deployment/charts/Chart.yaml @@ -2,5 +2,5 @@ apiVersion: v2 name: fuel-core description: Fuel Core Helm Chart type: application -appVersion: "0.9.5" +appVersion: "0.10.0" version: 0.1.0 diff --git a/deployment/charts/templates/fuel-core-deploy.yaml b/deployment/charts/templates/fuel-core-deploy.yaml index 9ce69a4c559..77e7ff3e38e 100644 --- a/deployment/charts/templates/fuel-core-deploy.yaml +++ b/deployment/charts/templates/fuel-core-deploy.yaml @@ -88,8 +88,6 @@ spec: {{- end}} - "--min-gas-price" - "{{ .Values.app.min_gas_price }}" - - "--min-byte-price" - - "{{ .Values.app.min_byte_price }}" resources: {} imagePullPolicy: {{ .Values.app.image.pullPolicy }} ports: diff --git a/deployment/charts/values.yaml b/deployment/charts/values.yaml index 999a6cfb0f0..e688324544f 100644 --- a/deployment/charts/values.yaml +++ b/deployment/charts/values.yaml @@ -12,7 +12,6 @@ app: utxo_validation: ${fuel_core_utxo_validation} vm_backtrace: ${fuel_core_vm_backtrace} min_gas_price: ${fuel_core_min_gas_price} - min_byte_price: ${fuel_core_min_byte_price} predicates: ${fuel_core_predicates} image: repository: ${fuel_core_image_repository} diff --git a/deployment/scripts/.env b/deployment/scripts/.env index b0ff591f624..1522a162d4b 100644 --- a/deployment/scripts/.env +++ b/deployment/scripts/.env @@ -14,7 +14,6 @@ fuel_core_human_logging=false fuel_core_utxo_validation=true fuel_core_vm_backtrace=false fuel_core_min_gas_price=0 -fuel_core_min_byte_price=0 fuel_core_predicates=false # Ingress Environment variables diff --git a/fuel-block-executor/Cargo.toml b/fuel-block-executor/Cargo.toml index e57d7d5398e..3e0b0e114d2 100644 --- a/fuel-block-executor/Cargo.toml +++ b/fuel-block-executor/Cargo.toml @@ -1,6 +1,6 @@ [package] name = "fuel-block-executor" -version = "0.9.5" +version = "0.10.0" authors = ["Fuel Labs "] edition = "2021" homepage = "https://fuel.network/" @@ -11,5 +11,5 @@ description = "Fuel Block Executor" [dependencies] anyhow = "1.0" -fuel-core-interfaces = { path = "../fuel-core-interfaces", version = "0.9.5" } +fuel-core-interfaces = { path = "../fuel-core-interfaces", version = "0.10.0" } tokio = { version = "1.14", features = ["full"] } diff --git a/fuel-block-importer/Cargo.toml b/fuel-block-importer/Cargo.toml index bae8fa4e08e..199e97291a6 100644 --- a/fuel-block-importer/Cargo.toml +++ b/fuel-block-importer/Cargo.toml @@ -1,6 +1,6 @@ [package] name = "fuel-block-importer" -version = "0.9.5" +version = "0.10.0" authors = ["Fuel Labs "] edition = "2021" homepage = "https://fuel.network/" @@ -11,6 +11,6 @@ description = "Fuel Block Importer" [dependencies] anyhow = "1.0" -fuel-core-interfaces = { path = "../fuel-core-interfaces", version = "0.9.5" } +fuel-core-interfaces = { path = "../fuel-core-interfaces", version = "0.10.0" } parking_lot = "0.12" tokio = { version = "1.14", features = ["full"] } diff --git a/fuel-block-producer/Cargo.toml b/fuel-block-producer/Cargo.toml index 32a9f396a89..6e79c1646a2 100644 --- a/fuel-block-producer/Cargo.toml +++ b/fuel-block-producer/Cargo.toml @@ -1,6 +1,6 @@ [package] name = "fuel-block-producer" -version = "0.9.5" +version = "0.10.0" authors = ["Fuel Labs "] edition = "2021" homepage = "https://fuel.network/" @@ -11,6 +11,6 @@ description = "Fuel Block Producer" [dependencies] anyhow = "1.0" -fuel-core-interfaces = { path = "../fuel-core-interfaces", version = "0.9.5" } +fuel-core-interfaces = { path = "../fuel-core-interfaces", version = "0.10.0" } parking_lot = "0.12" tokio = { version = "1.14", features = ["full"] } diff --git a/fuel-client/Cargo.toml b/fuel-client/Cargo.toml index 3466b8714de..10b0ba6b7ae 100644 --- a/fuel-client/Cargo.toml +++ b/fuel-client/Cargo.toml @@ -1,6 +1,6 @@ [package] name = "fuel-gql-client" -version = "0.9.5" +version = "0.10.0" authors = ["Fuel Labs "] categories = ["concurrency", "cryptography::cryptocurrencies", "emulators"] edition = "2021" @@ -20,9 +20,9 @@ chrono = { version = "0.4", features = ["serde"] } clap = { version = "3.1", features = ["derive"] } cynic = { version = "1.0", features = ["surf"] } derive_more = { version = "0.99" } -fuel-tx = { version = "0.13", features = ["serde"] } +fuel-tx = { version = "0.18", features = ["serde"] } fuel-types = { version = "0.5", features = ["serde"] } -fuel-vm = { version = "0.12", features = ["serde"] } +fuel-vm = { version = "0.15", features = ["serde"] } futures = "0.3" hex = "0.4" itertools = "0.10" diff --git a/fuel-client/assets/schema.sdl b/fuel-client/assets/schema.sdl index 6664af13989..665ff4b0df3 100644 --- a/fuel-client/assets/schema.sdl +++ b/fuel-client/assets/schema.sdl @@ -17,6 +17,10 @@ type BalanceConnection { A list of edges. """ edges: [BalanceEdge!]! + """ + A list of nodes. + """ + nodes: [Balance!]! } """ @@ -28,7 +32,7 @@ type BalanceEdge { """ cursor: String! """ - "The item at the end of the edge + The item at the end of the edge """ node: Balance! } @@ -57,6 +61,10 @@ type BlockConnection { A list of edges. """ edges: [BlockEdge!]! + """ + A list of nodes. + """ + nodes: [Block!]! } """ @@ -68,7 +76,7 @@ type BlockEdge { """ cursor: String! """ - "The item at the end of the edge + The item at the end of the edge """ node: Block! } @@ -116,6 +124,10 @@ type CoinConnection { A list of edges. """ edges: [CoinEdge!]! + """ + A list of nodes. + """ + nodes: [Coin!]! } """ @@ -127,7 +139,7 @@ type CoinEdge { """ cursor: String! """ - "The item at the end of the edge + The item at the end of the edge """ node: Coin! } @@ -162,11 +174,12 @@ type ConsensusParameters { maxGasPerTx: U64! maxScriptLength: U64! maxScriptDataLength: U64! - maxStaticContracts: U64! maxStorageSlots: U64! maxPredicateLength: U64! maxPredicateDataLength: U64! gasPriceFactor: U64! + gasPerByte: U64! + maxMessageDataLength: U64! } type Contract { @@ -190,6 +203,10 @@ type ContractBalanceConnection { A list of edges. """ edges: [ContractBalanceEdge!]! + """ + A list of nodes. + """ + nodes: [ContractBalance!]! } """ @@ -201,7 +218,7 @@ type ContractBalanceEdge { """ cursor: String! """ - "The item at the end of the edge + The item at the end of the edge """ node: ContractBalance! } @@ -244,13 +261,14 @@ type FailureStatus { scalar HexString -union Input = | InputCoin | InputContract +union Input = InputCoin | InputContract | InputMessage type InputCoin { utxoId: UtxoId! owner: Address! amount: U64! assetId: AssetId! + txPointer: TxPointer! witnessIndex: Int! maturity: U64! predicate: HexString! @@ -261,9 +279,70 @@ type InputContract { utxoId: UtxoId! balanceRoot: Bytes32! stateRoot: Bytes32! + txPointer: TxPointer! contract: Contract! } +type InputMessage { + messageId: MessageId! + sender: Address! + recipient: Address! + amount: U64! + nonce: U64! + owner: Address! + witnessIndex: Int! + data: HexString! + predicate: HexString! + predicateData: HexString! +} + + +type Message { + amount: U64! + sender: Address! + recipient: Address! + owner: Address! + nonce: U64! + data: [Int!]! + daHeight: U64! + fuelBlockSpend: U64 +} + +type MessageConnection { + """ + Information to aid in pagination. + """ + pageInfo: PageInfo! + """ + A list of edges. + """ + edges: [MessageEdge!]! + """ + A list of nodes. + """ + nodes: [Message!]! +} + +""" +An edge in a connection. +""" +type MessageEdge { + """ + A cursor for use in pagination + """ + cursor: String! + """ + The item at the end of the edge + """ + node: Message! +} + +scalar MessageId + +type MessageOutput { + recipient: Address! + amount: U64! +} type Mutation { startSession: ID! @@ -290,13 +369,12 @@ type NodeInfo { predicates: Boolean! vmBacktrace: Boolean! minGasPrice: U64! - minBytePrice: U64! maxTx: U64! maxDepth: U64! nodeVersion: String! } -union Output = | CoinOutput | ContractOutput | WithdrawalOutput | ChangeOutput | VariableOutput | ContractCreated +union Output = CoinOutput | ContractOutput | MessageOutput | ChangeOutput | VariableOutput | ContractCreated """ A separate `Breakpoint` type to be used as an output, as a single @@ -362,6 +440,7 @@ type Query { contractBalance(contract: ContractId!, asset: AssetId!): ContractBalance! contractBalances(filter: ContractBalanceFilterInput!, first: Int, after: String, last: Int, before: String): ContractBalanceConnection! nodeInfo: NodeInfo! + messages(owner: Address, first: Int, after: String, last: Int, before: String): MessageConnection! } type Receipt { @@ -389,6 +468,10 @@ type Receipt { result: U64 gasUsed: U64 data: HexString + messageId: MessageId + sender: Address + recipient: Address + nonce: Bytes32 } enum ReceiptType { @@ -402,6 +485,7 @@ enum ReceiptType { TRANSFER TRANSFER_OUT SCRIPT_RESULT + MESSAGE_OUT } enum ReturnType { @@ -451,7 +535,6 @@ type Transaction { inputContracts: [Contract!]! gasPrice: U64! gasLimit: U64! - bytePrice: U64! maturity: U64! isScript: Boolean! inputs: [Input!]! @@ -465,7 +548,6 @@ type Transaction { bytecodeWitnessIndex: Int bytecodeLength: U64 salt: Salt - staticContracts: [Contract!] storageSlots: [HexString!] """ Return the transaction bytes using canonical encoding @@ -482,6 +564,10 @@ type TransactionConnection { A list of edges. """ edges: [TransactionEdge!]! + """ + A list of nodes. + """ + nodes: [Transaction!]! } """ @@ -493,14 +579,16 @@ type TransactionEdge { """ cursor: String! """ - "The item at the end of the edge + The item at the end of the edge """ node: Transaction! } scalar TransactionId -union TransactionStatus = | SubmittedStatus | SuccessStatus | FailureStatus +union TransactionStatus = SubmittedStatus | SuccessStatus | FailureStatus + +scalar TxPointer scalar U64 @@ -512,12 +600,6 @@ type VariableOutput { assetId: AssetId! } -type WithdrawalOutput { - to: Address! - amount: U64! - assetId: AssetId! -} - schema { query: Query mutation: Mutation diff --git a/fuel-client/src/client.rs b/fuel-client/src/client.rs index 3879b29ebb5..70949025868 100644 --- a/fuel-client/src/client.rs +++ b/fuel-client/src/client.rs @@ -446,6 +446,19 @@ impl FuelClient { Ok(balances) } + + pub async fn messages( + &self, + owner: Option<&str>, + request: PaginationRequest, + ) -> io::Result> { + let owner: Option = owner.map(|owner| owner.parse()).transpose()?; + let query = schema::message::OwnedMessageQuery::build(&(owner, request).into()); + + let messages = self.query(query).await?.messages.into(); + + Ok(messages) + } } #[cfg(any(test, feature = "test-helpers"))] diff --git a/fuel-client/src/client/schema.rs b/fuel-client/src/client/schema.rs index 7793b6907dc..92b7c9b6567 100644 --- a/fuel-client/src/client/schema.rs +++ b/fuel-client/src/client/schema.rs @@ -18,6 +18,7 @@ pub mod block; pub mod chain; pub mod coin; pub mod contract; +pub mod message; pub mod node_info; pub mod primitives; pub mod tx; @@ -221,7 +222,7 @@ pub struct OutputBreakpoint { } /// Generic graphql pagination query args -#[derive(cynic::FragmentArguments, Debug)] +#[derive(cynic::FragmentArguments, Debug, Default)] pub struct ConnectionArgs { /// Skip until cursor (forward pagination) pub after: Option, diff --git a/fuel-client/src/client/schema/chain.rs b/fuel-client/src/client/schema/chain.rs index e321ab6871b..980c48bac57 100644 --- a/fuel-client/src/client/schema/chain.rs +++ b/fuel-client/src/client/schema/chain.rs @@ -11,11 +11,12 @@ pub struct ConsensusParameters { pub max_gas_per_tx: U64, pub max_script_length: U64, pub max_script_data_length: U64, - pub max_static_contracts: U64, pub max_storage_slots: U64, pub max_predicate_length: U64, pub max_predicate_data_length: U64, pub gas_price_factor: U64, + pub gas_per_byte: U64, + pub max_message_data_length: U64, } impl From for TxConsensusParameters { @@ -28,11 +29,12 @@ impl From for TxConsensusParameters { max_gas_per_tx: params.max_gas_per_tx.into(), max_script_length: params.max_script_length.into(), max_script_data_length: params.max_script_data_length.into(), - max_static_contracts: params.max_static_contracts.into(), max_storage_slots: params.max_storage_slots.into(), max_predicate_length: params.max_predicate_length.into(), max_predicate_data_length: params.max_predicate_data_length.into(), gas_price_factor: params.gas_price_factor.into(), + gas_per_byte: params.gas_per_byte.into(), + max_message_data_length: params.max_message_data_length.into(), } } } diff --git a/fuel-client/src/client/schema/message.rs b/fuel-client/src/client/schema/message.rs new file mode 100644 index 00000000000..939c8a41baa --- /dev/null +++ b/fuel-client/src/client/schema/message.rs @@ -0,0 +1,107 @@ +use super::{PageDirection, PageInfo, PaginatedResult, PaginationRequest}; +use crate::client::schema::{schema, Address, U64}; + +#[derive(cynic::QueryFragment, Debug)] +#[cynic(schema_path = "./assets/schema.sdl")] +pub struct Message { + pub amount: U64, + pub sender: Address, + pub recipient: Address, + pub owner: Address, + pub nonce: U64, + pub data: Vec, + pub da_height: U64, + pub fuel_block_spend: Option, +} + +#[derive(cynic::QueryFragment, Debug)] +#[cynic( + schema_path = "./assets/schema.sdl", + graphql_type = "Query", + argument_struct = "OwnedMessagesConnectionArgs" +)] +pub struct OwnedMessageQuery { + #[arguments(owner = &args.owner, after = &args.after, before = &args.before, first = &args.first, last = &args.last)] + pub messages: MessageConnection, +} + +#[derive(cynic::QueryFragment, Debug)] +#[cynic(schema_path = "./assets/schema.sdl")] +pub struct MessageConnection { + pub edges: Vec, + pub page_info: PageInfo, +} + +#[derive(cynic::QueryFragment, Debug)] +#[cynic(schema_path = "./assets/schema.sdl")] +pub struct MessageEdge { + pub cursor: String, + pub node: Message, +} + +#[derive(cynic::FragmentArguments, Debug)] +pub struct OwnedMessagesConnectionArgs { + /// Filter messages based on an owner + pub owner: Option
, + /// Skip until coin id (forward pagination) + pub after: Option, + /// Skip until coin id (backward pagination) + pub before: Option, + /// Retrieve the first n coins in order (forward pagination) + pub first: Option, + /// Retrieve the last n coins in order (backward pagination). + /// Can't be used at the same time as `first`. + pub last: Option, +} + +impl From<(Option
, PaginationRequest)> for OwnedMessagesConnectionArgs { + fn from(r: (Option
, PaginationRequest)) -> Self { + match r.1.direction { + PageDirection::Forward => OwnedMessagesConnectionArgs { + owner: r.0, + after: r.1.cursor, + before: None, + first: Some(r.1.results as i32), + last: None, + }, + PageDirection::Backward => OwnedMessagesConnectionArgs { + owner: r.0, + after: None, + before: r.1.cursor, + first: None, + last: Some(r.1.results as i32), + }, + } + } +} + +impl From for PaginatedResult { + fn from(conn: MessageConnection) -> Self { + PaginatedResult { + cursor: conn.page_info.end_cursor, + has_next_page: conn.page_info.has_next_page, + has_previous_page: conn.page_info.has_previous_page, + results: conn.edges.into_iter().map(|e| e.node).collect(), + } + } +} + +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn owned_message_query_gql_output() { + use cynic::QueryBuilder; + + let operation = OwnedMessageQuery::build(OwnedMessagesConnectionArgs { + owner: Some(Address::default()), + after: None, + before: None, + first: None, + last: None, + }); + + insta::assert_snapshot!(operation.query) + } +} diff --git a/fuel-client/src/client/schema/node_info.rs b/fuel-client/src/client/schema/node_info.rs index 8ee5ac02200..3a8b65dfddd 100644 --- a/fuel-client/src/client/schema/node_info.rs +++ b/fuel-client/src/client/schema/node_info.rs @@ -7,7 +7,6 @@ pub struct NodeInfo { pub predicates: bool, pub vm_backtrace: bool, pub min_gas_price: U64, - pub min_byte_price: U64, pub max_tx: U64, pub max_depth: U64, pub node_version: String, diff --git a/fuel-client/src/client/schema/primitives.rs b/fuel-client/src/client/schema/primitives.rs index d713750b57a..bfc9c50bdb3 100644 --- a/fuel-client/src/client/schema/primitives.rs +++ b/fuel-client/src/client/schema/primitives.rs @@ -93,6 +93,7 @@ fuel_type_scalar!(AssetId, AssetId); fuel_type_scalar!(ContractId, ContractId); fuel_type_scalar!(Salt, Salt); fuel_type_scalar!(TransactionId, Bytes32); +fuel_type_scalar!(MessageId, MessageId); #[derive(cynic::Scalar, Debug, Clone, Default)] pub struct UtxoId(pub HexFormatted); @@ -118,6 +119,30 @@ impl LowerHex for UtxoId { } } +#[derive(cynic::Scalar, Debug, Clone, Default)] +pub struct TxPointer(pub HexFormatted); + +impl FromStr for TxPointer { + type Err = ConversionError; + + fn from_str(s: &str) -> Result { + let b = HexFormatted::::from_str(s)?; + Ok(TxPointer(b)) + } +} + +impl From for fuel_tx::TxPointer { + fn from(s: TxPointer) -> Self { + s.0 .0 + } +} + +impl LowerHex for TxPointer { + fn fmt(&self, f: &mut Formatter<'_>) -> fmt::Result { + LowerHex::fmt(&self.0 .0, f) + } +} + #[derive(cynic::Scalar, Debug, Clone)] pub struct HexString(pub Bytes); diff --git a/fuel-client/src/client/schema/snapshots/fuel_gql_client__client__schema__chain__tests__chain_gql_query_output.snap b/fuel-client/src/client/schema/snapshots/fuel_gql_client__client__schema__chain__tests__chain_gql_query_output.snap index 6bc4c541246..ef54513bd30 100644 --- a/fuel-client/src/client/schema/snapshots/fuel_gql_client__client__schema__chain__tests__chain_gql_query_output.snap +++ b/fuel-client/src/client/schema/snapshots/fuel_gql_client__client__schema__chain__tests__chain_gql_query_output.snap @@ -1,6 +1,6 @@ --- source: fuel-client/src/client/schema/chain.rs -assertion_line: 64 +assertion_line: 66 expression: operation.query --- query Query { @@ -25,11 +25,12 @@ query Query { maxGasPerTx maxScriptLength maxScriptDataLength - maxStaticContracts maxStorageSlots maxPredicateLength maxPredicateDataLength gasPriceFactor + gasPerByte + maxMessageDataLength } } } diff --git a/fuel-client/src/client/schema/snapshots/fuel_gql_client__client__schema__message__tests__owned_message_query_gql_output.snap b/fuel-client/src/client/schema/snapshots/fuel_gql_client__client__schema__message__tests__owned_message_query_gql_output.snap new file mode 100644 index 00000000000..68f889c82b0 --- /dev/null +++ b/fuel-client/src/client/schema/snapshots/fuel_gql_client__client__schema__message__tests__owned_message_query_gql_output.snap @@ -0,0 +1,29 @@ +--- +source: fuel-client/src/client/schema/message.rs +assertion_line: 105 +expression: operation.query +--- +query Query($_0: Address, $_1: Int, $_2: String, $_3: Int, $_4: String) { + messages(owner: $_0, first: $_1, after: $_2, last: $_3, before: $_4) { + edges { + cursor + node { + amount + sender + recipient + owner + nonce + data + daHeight + fuelBlockSpend + } + } + pageInfo { + endCursor + hasNextPage + hasPreviousPage + startCursor + } + } +} + diff --git a/fuel-client/src/client/schema/snapshots/fuel_gql_client__client__schema__node_info__tests__node_info_query_gql_output.snap b/fuel-client/src/client/schema/snapshots/fuel_gql_client__client__schema__node_info__tests__node_info_query_gql_output.snap index ac90d7722e7..41d3095e6af 100644 --- a/fuel-client/src/client/schema/snapshots/fuel_gql_client__client__schema__node_info__tests__node_info_query_gql_output.snap +++ b/fuel-client/src/client/schema/snapshots/fuel_gql_client__client__schema__node_info__tests__node_info_query_gql_output.snap @@ -1,5 +1,6 @@ --- source: fuel-client/src/client/schema/node_info.rs +assertion_line: 29 expression: operation.query --- query Query { @@ -8,7 +9,6 @@ query Query { predicates vmBacktrace minGasPrice - minBytePrice maxTx maxDepth nodeVersion diff --git a/fuel-client/src/client/schema/snapshots/fuel_gql_client__client__schema__tx__tests__transparent_transaction_by_id_query_gql_output.snap b/fuel-client/src/client/schema/snapshots/fuel_gql_client__client__schema__tx__tests__transparent_transaction_by_id_query_gql_output.snap index fe405eae770..151a58baeb9 100644 --- a/fuel-client/src/client/schema/snapshots/fuel_gql_client__client__schema__tx__tests__transparent_transaction_by_id_query_gql_output.snap +++ b/fuel-client/src/client/schema/snapshots/fuel_gql_client__client__schema__tx__tests__transparent_transaction_by_id_query_gql_output.snap @@ -1,14 +1,11 @@ --- source: fuel-client/src/client/schema/tx.rs -assertion_line: 273 expression: operation.query - --- query Query($_0: TransactionId!) { transaction(id: $_0) { gasLimit gasPrice - bytePrice id inputAssetIds inputContracts { @@ -21,6 +18,7 @@ query Query($_0: TransactionId!) { owner amount assetId + txPointer witnessIndex maturity predicate @@ -30,10 +28,23 @@ query Query($_0: TransactionId!) { utxoId balanceRoot stateRoot + txPointer contract { id } } + ... on InputMessage { + messageId + sender + recipient + amount + nonce + owner + witnessIndex + data + predicate + predicateData + } } isScript outputs { @@ -48,10 +59,9 @@ query Query($_0: TransactionId!) { balanceRoot stateRoot } - ... on WithdrawalOutput { - to + ... on MessageOutput { + recipient amount - assetId } ... on ChangeOutput { to @@ -128,13 +138,14 @@ query Query($_0: TransactionId!) { result gasUsed data + messageId + sender + recipient + nonce } script scriptData salt - staticContracts { - id - } storageSlots bytecodeWitnessIndex bytecodeLength diff --git a/fuel-client/src/client/schema/tx/tests/transparent_receipt.rs b/fuel-client/src/client/schema/tx/tests/transparent_receipt.rs index 96abfd2e130..0a6206336fa 100644 --- a/fuel-client/src/client/schema/tx/tests/transparent_receipt.rs +++ b/fuel-client/src/client/schema/tx/tests/transparent_receipt.rs @@ -1,6 +1,6 @@ use crate::client::schema::{ contract::ContractIdFragment, schema, Address, AssetId, Bytes32, ConversionError, - ConversionError::MissingField, HexString, U64, + ConversionError::MissingField, HexString, MessageId, U64, }; use fuel_types::Word; @@ -30,6 +30,10 @@ pub struct Receipt { pub result: Option, pub gas_used: Option, pub data: Option, + pub message_id: Option, + pub sender: Option
, + pub recipient: Option
, + pub nonce: Option, } #[derive(cynic::Enum, Clone, Copy, Debug)] @@ -45,6 +49,7 @@ pub enum ReceiptType { Transfer, TransferOut, ScriptResult, + MessageOut, } impl TryFrom for fuel_vm::prelude::Receipt { @@ -317,6 +322,40 @@ impl TryFrom for fuel_vm::prelude::Receipt { .ok_or_else(|| MissingField("gas_used".to_string()))? .into(), }, + ReceiptType::MessageOut => fuel_tx::Receipt::MessageOut { + message_id: schema + .message_id + .ok_or_else(|| MissingField("message_id".to_string()))? + .into(), + sender: schema + .sender + .ok_or_else(|| MissingField("sender".to_string()))? + .into(), + recipient: schema + .recipient + .ok_or_else(|| MissingField("recipient".to_string()))? + .into(), + amount: schema + .amount + .ok_or_else(|| MissingField("amount".to_string()))? + .into(), + nonce: schema + .nonce + .ok_or_else(|| MissingField("nonce".to_string()))? + .into(), + len: schema + .len + .ok_or_else(|| MissingField("len".to_string()))? + .into(), + digest: schema + .digest + .ok_or_else(|| MissingField("digest".to_string()))? + .into(), + data: schema + .data + .ok_or_else(|| MissingField("data".to_string()))? + .into(), + }, }) } } diff --git a/fuel-client/src/client/schema/tx/tests/transparent_tx.rs b/fuel-client/src/client/schema/tx/tests/transparent_tx.rs index 1761871a294..6607cc74efe 100644 --- a/fuel-client/src/client/schema/tx/tests/transparent_tx.rs +++ b/fuel-client/src/client/schema/tx/tests/transparent_tx.rs @@ -2,8 +2,8 @@ use crate::client::schema::{ contract::ContractIdFragment, schema, tx::{tests::transparent_receipt::Receipt, TransactionStatus, TxIdArgs}, - Address, AssetId, Bytes32, ConnectionArgs, ConversionError, HexString, PageInfo, Salt, - TransactionId, UtxoId, U64, + Address, AssetId, Bytes32, ConnectionArgs, ConversionError, HexString, MessageId, PageInfo, + Salt, TransactionId, TxPointer, UtxoId, U64, }; use core::convert::{TryFrom, TryInto}; use fuel_tx::StorageSlot; @@ -51,7 +51,6 @@ pub struct TransactionEdge { pub struct Transaction { pub gas_limit: U64, pub gas_price: U64, - pub byte_price: U64, pub id: TransactionId, pub input_asset_ids: Vec, pub input_contracts: Vec, @@ -66,7 +65,6 @@ pub struct Transaction { pub script: Option, pub script_data: Option, pub salt: Option, - pub static_contracts: Option>, pub storage_slots: Option>, pub bytecode_witness_index: Option, pub bytecode_length: Option, @@ -80,7 +78,6 @@ impl TryFrom for fuel_vm::prelude::Transaction { true => Self::Script { gas_price: tx.gas_price.into(), gas_limit: tx.gas_limit.into(), - byte_price: tx.byte_price.into(), maturity: tx.maturity.into(), receipts_root: tx .receipts_root @@ -110,7 +107,6 @@ impl TryFrom for fuel_vm::prelude::Transaction { false => Self::Create { gas_price: tx.gas_price.into(), gas_limit: tx.gas_limit.into(), - byte_price: tx.byte_price.into(), maturity: tx.maturity.into(), bytecode_length: tx .bytecode_length @@ -126,12 +122,6 @@ impl TryFrom for fuel_vm::prelude::Transaction { .salt .ok_or_else(|| ConversionError::MissingField("salt".to_string()))? .into(), - static_contracts: tx - .static_contracts - .ok_or_else(|| ConversionError::MissingField("static_contracts".to_string()))? - .into_iter() - .map(|c| c.id.into()) - .collect(), storage_slots: tx .storage_slots .ok_or_else(|| ConversionError::MissingField("storage_slots".to_string()))? @@ -173,6 +163,7 @@ impl TryFrom for fuel_vm::prelude::Transaction { pub enum Input { InputCoin(InputCoin), InputContract(InputContract), + InputMessage(InputMessage), } #[derive(cynic::QueryFragment, Debug)] @@ -182,6 +173,7 @@ pub struct InputCoin { pub owner: Address, pub amount: U64, pub asset_id: AssetId, + pub tx_pointer: TxPointer, pub witness_index: i32, pub maturity: U64, pub predicate: HexString, @@ -194,9 +186,25 @@ pub struct InputContract { pub utxo_id: UtxoId, pub balance_root: Bytes32, pub state_root: Bytes32, + pub tx_pointer: TxPointer, pub contract: ContractIdFragment, } +#[derive(cynic::QueryFragment, Debug)] +#[cynic(schema_path = "./assets/schema.sdl")] +pub struct InputMessage { + message_id: MessageId, + sender: Address, + recipient: Address, + amount: U64, + nonce: U64, + owner: Address, + witness_index: i32, + data: HexString, + predicate: HexString, + predicate_data: HexString, +} + impl TryFrom for fuel_tx::Input { type Error = ConversionError; @@ -209,6 +217,7 @@ impl TryFrom for fuel_tx::Input { owner: coin.owner.into(), amount: coin.amount.into(), asset_id: coin.asset_id.into(), + tx_pointer: coin.tx_pointer.into(), witness_index: coin.witness_index.try_into()?, maturity: coin.maturity.into(), } @@ -219,6 +228,7 @@ impl TryFrom for fuel_tx::Input { amount: coin.amount.into(), asset_id: coin.asset_id.into(), maturity: coin.maturity.into(), + tx_pointer: coin.tx_pointer.into(), predicate: coin.predicate.into(), predicate_data: coin.predicate_data.into(), } @@ -228,8 +238,35 @@ impl TryFrom for fuel_tx::Input { utxo_id: contract.utxo_id.into(), balance_root: contract.balance_root.into(), state_root: contract.state_root.into(), + tx_pointer: contract.tx_pointer.into(), contract_id: contract.contract.id.into(), }, + Input::InputMessage(message) => { + if message.predicate.0 .0.is_empty() { + fuel_tx::Input::MessageSigned { + message_id: message.message_id.into(), + sender: message.sender.into(), + recipient: message.recipient.into(), + amount: message.amount.into(), + nonce: message.nonce.into(), + owner: message.owner.into(), + witness_index: message.witness_index.try_into()?, + data: message.data.into(), + } + } else { + fuel_tx::Input::MessagePredicate { + message_id: message.message_id.into(), + sender: message.sender.into(), + recipient: message.recipient.into(), + amount: message.amount.into(), + nonce: message.nonce.into(), + owner: message.owner.into(), + data: message.data.into(), + predicate: message.predicate.into(), + predicate_data: message.predicate_data.into(), + } + } + } }) } } @@ -239,7 +276,7 @@ impl TryFrom for fuel_tx::Input { pub enum Output { CoinOutput(CoinOutput), ContractOutput(ContractOutput), - WithdrawalOutput(WithdrawalOutput), + MessageOutput(MessageOutput), ChangeOutput(ChangeOutput), VariableOutput(VariableOutput), ContractCreated(ContractCreated), @@ -255,10 +292,9 @@ pub struct CoinOutput { #[derive(cynic::QueryFragment, Debug)] #[cynic(schema_path = "./assets/schema.sdl")] -pub struct WithdrawalOutput { - pub to: Address, +pub struct MessageOutput { + pub recipient: Address, pub amount: U64, - pub asset_id: AssetId, } #[derive(cynic::QueryFragment, Debug)] @@ -307,10 +343,9 @@ impl TryFrom for fuel_tx::Output { balance_root: contract.balance_root.into(), state_root: contract.state_root.into(), }, - Output::WithdrawalOutput(withdrawal) => Self::Withdrawal { - to: withdrawal.to.into(), - amount: withdrawal.amount.into(), - asset_id: withdrawal.asset_id.into(), + Output::MessageOutput(message) => Self::Message { + recipient: message.recipient.into(), + amount: message.amount.into(), }, Output::ChangeOutput(change) => Self::Change { to: change.to.into(), diff --git a/fuel-core-bft/Cargo.toml b/fuel-core-bft/Cargo.toml index 5e2e8a8b696..f8db2a880cb 100644 --- a/fuel-core-bft/Cargo.toml +++ b/fuel-core-bft/Cargo.toml @@ -1,6 +1,6 @@ [package] name = "fuel-core-bft" -version = "0.9.5" +version = "0.10.0" authors = ["Fuel Labs "] edition = "2021" homepage = "https://fuel.network/" @@ -11,6 +11,6 @@ description = "Fuel Core BFT" [dependencies] anyhow = "1.0" -fuel-core-interfaces = { path = "../fuel-core-interfaces", version = "0.9.5" } +fuel-core-interfaces = { path = "../fuel-core-interfaces", version = "0.10.0" } parking_lot = "0.12" tokio = { version = "1.14", features = ["full"] } diff --git a/fuel-core-interfaces/Cargo.toml b/fuel-core-interfaces/Cargo.toml index 640c2f759cd..ba9181ce695 100644 --- a/fuel-core-interfaces/Cargo.toml +++ b/fuel-core-interfaces/Cargo.toml @@ -1,6 +1,6 @@ [package] name = "fuel-core-interfaces" -version = "0.9.5" +version = "0.10.0" authors = ["Fuel Labs "] categories = ["cryptography::cryptocurrencies"] edition = "2021" @@ -15,13 +15,13 @@ anyhow = "1.0" async-trait = "0.1" chrono = { version = "0.4" } derive_more = { version = "0.99" } -fuel-asm = "0.5" -fuel-crypto = { version = "0.5", default-features = false, features = [ "random" ] } +fuel-asm = "0.8" +fuel-crypto = { version = "0.6", default-features = false, features = [ "random" ] } fuel-merkle = { version = "0.3" } -fuel-storage = "0.1" -fuel-tx = { version = "0.13", default-features = false } +fuel-storage = "0.2" +fuel-tx = { version = "0.18", default-features = false } fuel-types = { version = "0.5", default-features = false } -fuel-vm = { version = "0.12", default-features = false } +fuel-vm = { version = "0.15", default-features = false } futures = "0.3" lazy_static = "1.4" parking_lot = "0.12" diff --git a/fuel-core-interfaces/src/db.rs b/fuel-core-interfaces/src/db.rs index d889bc63008..3dea0c35d55 100644 --- a/fuel-core-interfaces/src/db.rs +++ b/fuel-core-interfaces/src/db.rs @@ -71,7 +71,6 @@ pub mod helpers { // constants pub const TX1_GAS_PRICE: u64 = 10u64; - pub const TX1_BYTE_PRICE: u64 = 5u64; lazy_static! { pub static ref TX_ID_DB1: TxId = TxId::from_str("0x0000000000000000000000000000000000000000000000000000000000000000") @@ -122,13 +121,15 @@ pub mod helpers { use fuel_tx::{ Address, Bytes32, ContractId, Input, Metadata, Output, Transaction, TxId, UtxoId, }; + use fuel_types::MessageId; use fuel_vm::prelude::Contract; use std::collections::HashMap; use crate::{ model::{ - BlockHeight, Coin, CoinStatus, ConsensusId, DaBlockHeight, DepositCoin, FuelBlock, - FuelBlockConsensus, FuelBlockHeader, SealedFuelBlock, ValidatorId, ValidatorStake, + BlockHeight, Coin, CoinStatus, ConsensusId, DaBlockHeight, FuelBlock, + FuelBlockConsensus, FuelBlockHeader, Message, SealedFuelBlock, ValidatorId, + ValidatorStake, }, relayer::{RelayerDb, StakingDiff}, txpool::TxPoolDb, @@ -166,8 +167,8 @@ pub mod helpers { pub coins: HashMap, /// Dummy contracts pub contract: HashMap, - /// Dummy deposit coins. - pub deposit_coin: HashMap, + /// Dummy messages. + pub messages: HashMap, /// variable for last committed and finalized fuel height pub last_committed_finalized_fuel_height: BlockHeight, } @@ -192,7 +193,6 @@ pub mod helpers { let tx1 = Transaction::Script { gas_price: TX1_GAS_PRICE, gas_limit: 1_000_000, - byte_price: TX1_BYTE_PRICE, maturity: 0, receipts_root: Default::default(), script, @@ -202,6 +202,7 @@ pub mod helpers { owner: Address::default(), amount: 100, asset_id: Default::default(), + tx_pointer: Default::default(), witness_index: 0, maturity: 0, }], @@ -231,7 +232,6 @@ pub mod helpers { let tx1_faulty = Transaction::Script { gas_price: 10, gas_limit: 1_000_000, - byte_price: 10, maturity: 0, receipts_root: Default::default(), script, @@ -241,6 +241,7 @@ pub mod helpers { owner: Address::default(), amount: 100, asset_id: Default::default(), + tx_pointer: Default::default(), witness_index: 0, maturity: 0, }], @@ -263,7 +264,6 @@ pub mod helpers { let tx2 = Transaction::Script { gas_price: 9, gas_limit: 1_000_001, - byte_price: 9, maturity: 0, receipts_root: Default::default(), script, @@ -273,6 +273,7 @@ pub mod helpers { owner: Address::default(), amount: 100, asset_id: Default::default(), + tx_pointer: Default::default(), witness_index: 0, maturity: 0, }], @@ -296,7 +297,6 @@ pub mod helpers { let tx2_faulty = Transaction::Script { gas_price: 9, gas_limit: 1_000_001, - byte_price: 9, maturity: 0, receipts_root: Default::default(), script, @@ -306,6 +306,7 @@ pub mod helpers { owner: Address::default(), amount: 100, asset_id: Default::default(), + tx_pointer: Default::default(), witness_index: 0, maturity: 0, }], @@ -336,7 +337,6 @@ pub mod helpers { let tx3 = Transaction::Script { gas_price: 20, // more then tx1 gas_limit: 1_000_001, - byte_price: 20, maturity: 0, receipts_root: Default::default(), script, @@ -346,6 +346,7 @@ pub mod helpers { owner: Address::default(), amount: 100, asset_id: Default::default(), + tx_pointer: Default::default(), witness_index: 0, maturity: 0, }], @@ -370,7 +371,6 @@ pub mod helpers { let tx4 = Transaction::Script { gas_price: 20, // more then tx1 gas_limit: 1_000_001, - byte_price: 20, maturity: 0, receipts_root: Default::default(), script, @@ -380,6 +380,7 @@ pub mod helpers { owner: Address::default(), amount: 200, asset_id: Default::default(), + tx_pointer: Default::default(), witness_index: 0, maturity: 0, }], @@ -403,7 +404,6 @@ pub mod helpers { let tx5 = Transaction::Script { gas_price: 5, //lower then tx1 gas_limit: 1_000_000, - byte_price: 5, maturity: 0, receipts_root: Default::default(), script, @@ -412,6 +412,7 @@ pub mod helpers { utxo_id: UtxoId::default(), balance_root: Bytes32::default(), state_root: Bytes32::default(), + tx_pointer: Default::default(), contract_id: *CONTRACT_ID1, }], outputs: vec![ @@ -465,7 +466,6 @@ pub mod helpers { fun(Transaction::script( 10, 1000, - 10, 0, script.clone(), Vec::new(), @@ -480,7 +480,6 @@ pub mod helpers { fun(Transaction::script( 10, 1000, - 10, 0, script.clone(), Vec::new(), @@ -495,7 +494,6 @@ pub mod helpers { fun(Transaction::script( 10, 1000, - 10, 0, script, Vec::new(), @@ -510,7 +508,6 @@ pub mod helpers { fun(Transaction::script( 10, 1000, - 10, 0, Vec::new(), Vec::new(), @@ -616,7 +613,7 @@ pub mod helpers { tx: HashMap::from_iter(txs.into_iter().map(|tx| (tx.id(), Arc::new(tx)))), coins, contract: HashMap::new(), - deposit_coin: HashMap::new(), + messages: HashMap::new(), chain_height: BlockHeight::from(0u64), validators_height: 0, finalized_da_height: 0, @@ -755,36 +752,36 @@ pub mod helpers { } } - // token deposit. Used by relayer. - impl Storage for DummyDb { - type Error = crate::db::KvStoreError; + // bridge message. Used by relayer. + impl Storage for DummyDb { + type Error = KvStoreError; fn insert( &mut self, - key: &Bytes32, - value: &DepositCoin, - ) -> Result, Self::Error> { - Ok(self.data.lock().deposit_coin.insert(*key, value.clone())) + key: &MessageId, + value: &Message, + ) -> Result, Self::Error> { + Ok(self.data.lock().messages.insert(*key, value.clone())) } - fn remove(&mut self, key: &Bytes32) -> Result, Self::Error> { - Ok(self.data.lock().deposit_coin.remove(key)) + fn remove(&mut self, key: &MessageId) -> Result, Self::Error> { + Ok(self.data.lock().messages.remove(key)) } fn get<'a>( &'a self, - key: &Bytes32, - ) -> Result>, Self::Error> { + key: &MessageId, + ) -> Result>, Self::Error> { Ok(self .data .lock() - .deposit_coin + .messages .get(key) .map(|i| Cow::Owned(i.clone()))) } - fn contains_key(&self, key: &Bytes32) -> Result { - Ok(self.data.lock().deposit_coin.contains_key(key)) + fn contains_key(&self, key: &MessageId) -> Result { + Ok(self.data.lock().messages.contains_key(key)) } } @@ -956,13 +953,13 @@ mod tests { use crate::db::helpers::{DummyDb, CONTRACT_ID1}; use crate::model::{ - BlockHeight, Coin, CoinStatus, ConsensusId, DaBlockHeight, DepositCoin, ValidatorId, + BlockHeight, Coin, CoinStatus, ConsensusId, DaBlockHeight, Message, ValidatorId, ValidatorStake, }; use crate::relayer::StakingDiff; use fuel_storage::Storage; use fuel_tx::{Contract, Transaction, UtxoId}; - use fuel_types::{Address, Bytes32, ContractId}; + use fuel_types::{Address, ContractId}; #[test] fn coins_db() { @@ -1000,19 +997,21 @@ mod tests { } #[test] - fn deposit_coins_db() { + fn message_db() { let db = sample_db(); - let value = DepositCoin { + let value = Message { owner: Address::default(), amount: 400, - asset_id: Default::default(), - nonce: Bytes32::default(), - deposited_da_height: DaBlockHeight::default(), + nonce: 10, fuel_block_spend: Some(BlockHeight::default()), - }; - let key = value.id(); + sender: Address::default(), + recipient: Address::default(), + data: vec![], + da_height: Default::default(), + } + .check(); - assert!(execute_test(db, key, value).is_ok()); + assert!(execute_test(db, *value.id(), Message::from(value)).is_ok()); } #[test] diff --git a/fuel-core-interfaces/src/model.rs b/fuel-core-interfaces/src/model.rs index 18d152cfe35..9f82545bac1 100644 --- a/fuel-core-interfaces/src/model.rs +++ b/fuel-core-interfaces/src/model.rs @@ -1,19 +1,19 @@ mod block; mod block_height; mod coin; -mod deposit_coin; +mod messages; mod txpool; mod vote; pub use block::{FuelBlock, FuelBlockConsensus, FuelBlockDb, FuelBlockHeader, SealedFuelBlock}; pub use block_height::BlockHeight; pub use coin::{Coin, CoinStatus}; -pub use deposit_coin::DepositCoin; use fuel_types::{Address, Bytes32}; +pub use messages::*; pub use txpool::{ArcTx, TxInfo}; pub use vote::ConsensusVote; -pub type DaBlockHeight = u32; +pub type DaBlockHeight = u64; pub type ValidatorStake = u64; /// Validator address used for registration of validator on DA layer diff --git a/fuel-core-interfaces/src/model/block.rs b/fuel-core-interfaces/src/model/block.rs index 5a191b7f20b..268b4467479 100644 --- a/fuel-core-interfaces/src/model/block.rs +++ b/fuel-core-interfaces/src/model/block.rs @@ -12,7 +12,7 @@ use std::collections::HashMap; pub struct FuelBlockHeader { /// Fuel block height. pub height: BlockHeight, - /// The layer 1 height of deposits and events to include since the last layer 1 block number. + /// The layer 1 height of messages and events to include since the last layer 1 block number. /// This is not meant to represent the layer 1 block this was committed to. Validators will need /// to have some rules in place to ensure the block number was chosen in a reasonable way. For /// example, they should verify that the block number satisfies the finality requirements of the diff --git a/fuel-core-interfaces/src/model/deposit_coin.rs b/fuel-core-interfaces/src/model/deposit_coin.rs deleted file mode 100644 index 25098c4e4b6..00000000000 --- a/fuel-core-interfaces/src/model/deposit_coin.rs +++ /dev/null @@ -1,20 +0,0 @@ -use super::{BlockHeight, DaBlockHeight}; -use fuel_types::{Address, AssetId, Bytes32, Word}; - -/// Probably going to be superseded with bridge message https://github.com/FuelLabs/fuel-core/issues/366 -#[cfg_attr(feature = "serde", derive(serde::Serialize, serde::Deserialize))] -#[derive(Debug, Clone)] -pub struct DepositCoin { - pub owner: Address, - pub amount: Word, - pub asset_id: AssetId, - pub nonce: Bytes32, - pub deposited_da_height: DaBlockHeight, - pub fuel_block_spend: Option, -} - -impl DepositCoin { - pub fn id(&self) -> Bytes32 { - self.nonce - } -} diff --git a/fuel-core-interfaces/src/model/messages.rs b/fuel-core-interfaces/src/model/messages.rs new file mode 100644 index 00000000000..13d7500c6d0 --- /dev/null +++ b/fuel-core-interfaces/src/model/messages.rs @@ -0,0 +1,70 @@ +use super::BlockHeight; +use crate::model::DaBlockHeight; +use core::ops::Deref; +use fuel_crypto::Hasher; +use fuel_types::{Address, MessageId, Word}; + +/// Message send from Da layer to fuel by bridge +#[cfg_attr(feature = "serde", derive(serde::Serialize, serde::Deserialize))] +#[derive(Debug, Default, Clone, PartialEq, Eq)] +pub struct Message { + pub sender: Address, + pub recipient: Address, + pub owner: Address, + pub nonce: Word, + pub amount: Word, + pub data: Vec, + /// The block height from the parent da layer that originated this message + pub da_height: DaBlockHeight, + pub fuel_block_spend: Option, +} + +impl Message { + pub fn id(&self) -> MessageId { + let mut hasher = Hasher::default(); + hasher.input(self.sender); + hasher.input(self.recipient); + hasher.input(self.nonce.to_be_bytes()); + hasher.input(self.owner); + hasher.input(self.amount.to_be_bytes()); + hasher.input(&self.data); + MessageId::from(*hasher.digest()) + } + + pub fn check(self) -> CheckedMessage { + let id = self.id(); + CheckedMessage { message: self, id } + } +} + +#[derive(Debug, Clone, Eq, PartialEq)] +pub struct CheckedMessage { + message: Message, + id: MessageId, +} + +impl CheckedMessage { + pub fn id(&self) -> &MessageId { + &self.id + } +} + +impl From for Message { + fn from(checked_message: CheckedMessage) -> Self { + checked_message.message + } +} + +impl AsRef for CheckedMessage { + fn as_ref(&self) -> &Message { + &self.message + } +} + +impl Deref for CheckedMessage { + type Target = Message; + + fn deref(&self) -> &Self::Target { + &self.message + } +} diff --git a/fuel-core-interfaces/src/relayer.rs b/fuel-core-interfaces/src/relayer.rs index 5d96c78b9ac..67b871f55c6 100644 --- a/fuel-core-interfaces/src/relayer.rs +++ b/fuel-core-interfaces/src/relayer.rs @@ -1,7 +1,7 @@ use async_trait::async_trait; use derive_more::{Deref, DerefMut}; use fuel_storage::Storage; -use fuel_types::{Address, Bytes32}; +use fuel_types::{Address, MessageId}; use std::{collections::HashMap, sync::Arc}; use tokio::sync::{mpsc, oneshot}; @@ -36,12 +36,12 @@ impl StakingDiff { } } -// Database has two main functionalities, ValidatorSet and TokenDeposits. -// From relayer perspective TokenDeposits are just insert when they get finalized. +// Database has two main functionalities, ValidatorSet and Bridge Message. +// From relayer perspective messages are just inserted when they get finalized. // But for ValidatorSet, it is little bit different. #[async_trait] pub trait RelayerDb: - Storage // token deposit + Storage // bridge messages + Storage), Error = KvStoreError> // validator set + Storage,Error = KvStoreError> // delegate index + Storage // staking diff @@ -49,12 +49,12 @@ pub trait RelayerDb: + Sync { - /// deposit token to database. Token deposits are not revertable. - async fn insert_coin_deposit( + /// add bridge message to database. Messages are not revertible. + async fn insert_message( &mut self, - deposit: DepositCoin, + message: &CheckedMessage, ) { - let _ = Storage::::insert(self,&deposit.id(),&deposit); + let _ = Storage::::insert(self,message.id(),message.as_ref()); } /// Insert difference make on staking in this particular DA height. @@ -199,8 +199,8 @@ pub use thiserror::Error; use crate::{ db::KvStoreError, model::{ - BlockHeight, ConsensusId, DaBlockHeight, DepositCoin, SealedFuelBlock, ValidatorId, - ValidatorStake, + BlockHeight, CheckedMessage, ConsensusId, DaBlockHeight, Message, SealedFuelBlock, + ValidatorId, ValidatorStake, }, }; diff --git a/fuel-core-interfaces/src/txpool.rs b/fuel-core-interfaces/src/txpool.rs index 7f9d0f2295d..ffb9ef7fcc1 100644 --- a/fuel-core-interfaces/src/txpool.rs +++ b/fuel-core-interfaces/src/txpool.rs @@ -1,12 +1,14 @@ +use crate::model::ArcTx; use crate::{ db::{Error as DbStateError, KvStoreError}, - model::Coin, model::TxInfo, + model::{Coin, Message}, }; use derive_more::{Deref, DerefMut}; use fuel_storage::Storage; use fuel_tx::{ContractId, UtxoId}; use fuel_tx::{Transaction, TxId}; +use fuel_types::MessageId; use fuel_vm::prelude::Contract; use std::sync::Arc; use thiserror::Error; @@ -15,6 +17,7 @@ use tokio::sync::{mpsc, oneshot}; pub trait TxPoolDb: Storage + Storage + + Storage + Send + Sync { @@ -25,6 +28,10 @@ pub trait TxPoolDb: fn contract_exist(&self, contract_id: ContractId) -> Result { Storage::::contains_key(self, &contract_id) } + + fn message(&self, message_id: MessageId) -> Result, KvStoreError> { + Storage::::get(self, &message_id).map(|t| t.map(|t| t.as_ref().clone())) + } } #[derive(Clone, Deref, DerefMut)] @@ -46,40 +53,40 @@ impl Sender { pub async fn find(&self, ids: Vec) -> anyhow::Result>> { let (response, receiver) = oneshot::channel(); - let _ = self.send(TxPoolMpsc::Find { ids, response }).await; + self.send(TxPoolMpsc::Find { ids, response }).await?; receiver.await.map_err(Into::into) } pub async fn find_one(&self, id: TxId) -> anyhow::Result> { let (response, receiver) = oneshot::channel(); - let _ = self.send(TxPoolMpsc::FindOne { id, response }).await; + self.send(TxPoolMpsc::FindOne { id, response }).await?; receiver.await.map_err(Into::into) } pub async fn find_dependent(&self, ids: Vec) -> anyhow::Result>> { let (response, receiver) = oneshot::channel(); - let _ = self.send(TxPoolMpsc::FindDependent { ids, response }).await; + self.send(TxPoolMpsc::FindDependent { ids, response }) + .await?; receiver.await.map_err(Into::into) } pub async fn filter_by_negative(&self, ids: Vec) -> anyhow::Result> { let (response, receiver) = oneshot::channel(); - let _ = self - .send(TxPoolMpsc::FilterByNegative { ids, response }) - .await; + self.send(TxPoolMpsc::FilterByNegative { ids, response }) + .await?; receiver.await.map_err(Into::into) } pub async fn includable(&self) -> anyhow::Result>> { let (response, receiver) = oneshot::channel(); - let _ = self.send(TxPoolMpsc::Includable { response }).await; + self.send(TxPoolMpsc::Includable { response }).await?; receiver.await.map_err(Into::into) } - pub async fn remove(&self, ids: Vec) -> anyhow::Result<()> { - self.send(TxPoolMpsc::Remove { ids }) - .await - .map_err(Into::into) + pub async fn remove(&self, ids: Vec) -> anyhow::Result> { + let (response, receiver) = oneshot::channel(); + self.send(TxPoolMpsc::Remove { ids, response }).await?; + receiver.await.map_err(Into::into) } } @@ -116,7 +123,10 @@ pub enum TxPoolMpsc { response: oneshot::Sender>>, }, /// remove transaction from pool needed on user demand. Low priority - Remove { ids: Vec }, + Remove { + ids: Vec, + response: oneshot::Sender>>, + }, /// Iterate over `hashes` and return all hashes that we don't have. /// Needed when we receive list of new hashed from peer with /// **BroadcastTransactionHashes**, so txpool needs to return @@ -156,8 +166,6 @@ pub enum Error { NoMetadata, #[error("Transaction is not inserted. The gas price is too low.")] NotInsertedGasPriceTooLow, - #[error("Transaction is not inserted. The byte price is too low.")] - NotInsertedBytePriceTooLow, #[error( "Transaction is not inserted. More priced tx {0:#x} already spend this UTXO output: {1:#x}" )] @@ -166,6 +174,10 @@ pub enum Error { "Transaction is not inserted. More priced tx has created contract with ContractId {0:#x}" )] NotInsertedCollisionContractId(ContractId), + #[error( + "Transaction is not inserted. A higher priced tx {0:#x} is already spending this messageId: {1:#x}" + )] + NotInsertedCollisionMessageId(TxId, MessageId), #[error("Transaction is not inserted. Dependent UTXO output is not existing: {0:#x}")] NotInsertedOutputNotExisting(UtxoId), #[error("Transaction is not inserted. UTXO input contract is not existing: {0:#x}")] @@ -176,6 +188,10 @@ pub enum Error { NotInsertedInputUtxoIdNotExisting(UtxoId), #[error("Transaction is not inserted. UTXO is spent: {0:#x}")] NotInsertedInputUtxoIdSpent(UtxoId), + #[error("Transaction is not inserted. Message is spent: {0:#x}")] + NotInsertedInputMessageIdSpent(MessageId), + #[error("Transaction is not inserted. Message id {0:#x} does not match any received message from the DA layer.")] + NotInsertedInputMessageUnknown(MessageId), #[error( "Transaction is not inserted. UTXO requires Contract input {0:#x} that is priced lower" )] @@ -186,12 +202,16 @@ pub enum Error { NotInsertedIoWrongAmount, #[error("Transaction is not inserted. Input output mismatch. Coin output asset_id does not match expected inputs")] NotInsertedIoWrongAssetId, + #[error("Transaction is not inserted. The computed message id doesn't match the provided message id.")] + NotInsertedIoWrongMessageId, #[error( "Transaction is not inserted. Input output mismatch. Expected coin but output is contract" )] NotInsertedIoContractOutput, - #[error("Transaction is not inserted. Input output mismatch. Expected coin but output is withdrawal")] - NotInsertedIoWithdrawalInput, + #[error( + "Transaction is not inserted. Input output mismatch. Expected coin but output is message" + )] + NotInsertedIoMessageInput, #[error("Transaction is not inserted. Maximum depth of dependent transaction chain reached")] NotInsertedMaxDepth, // small todo for now it can pass but in future we should include better messages diff --git a/fuel-core/Cargo.toml b/fuel-core/Cargo.toml index 673b61da074..33eac6c4836 100644 --- a/fuel-core/Cargo.toml +++ b/fuel-core/Cargo.toml @@ -1,6 +1,6 @@ [package] name = "fuel-core" -version = "0.9.5" +version = "0.10.0" authors = ["Fuel Labs "] categories = ["concurrency", "cryptography::cryptocurrencies", "emulators"] edition = "2021" @@ -33,17 +33,17 @@ clap = { version = "3.2", features = ["env", "derive"] } derive_more = { version = "0.99" } dirs = "3.0" env_logger = "0.9" -fuel-block-executor = { path = "../fuel-block-executor", version = "0.9.5" } -fuel-block-importer = { path = "../fuel-block-importer", version = "0.9.5" } -fuel-block-producer = { path = "../fuel-block-producer", version = "0.9.5" } -fuel-core-bft = { path = "../fuel-core-bft", version = "0.9.5" } -fuel-core-interfaces = { path = "../fuel-core-interfaces", version = "0.9.5", features = [ +fuel-block-executor = { path = "../fuel-block-executor", version = "0.10.0" } +fuel-block-importer = { path = "../fuel-block-importer", version = "0.10.0" } +fuel-block-producer = { path = "../fuel-block-producer", version = "0.10.0" } +fuel-core-bft = { path = "../fuel-core-bft", version = "0.10.0" } +fuel-core-interfaces = { path = "../fuel-core-interfaces", version = "0.10.0", features = [ "serde", ] } -fuel-p2p = { path = "../fuel-p2p", version = "0.9.5" } -fuel-relayer = { path = "../fuel-relayer", version = "0.9.5" } -fuel-sync = { path = "../fuel-sync", version = "0.9.5" } -fuel-txpool = { path = "../fuel-txpool", version = "0.9.5" } +fuel-p2p = { path = "../fuel-p2p", version = "0.10.0" } +fuel-relayer = { path = "../fuel-relayer", version = "0.10.0" } +fuel-sync = { path = "../fuel-sync", version = "0.10.0" } +fuel-txpool = { path = "../fuel-txpool", version = "0.10.0" } futures = "0.3" graphql-parser = "0.3.0" hex = { version = "0.4", features = ["serde"] } @@ -52,7 +52,7 @@ itertools = "0.10" lazy_static = "1.4" prometheus = { version = "0.13", optional = true } rand = "0.8" -rocksdb = { version = "0.18", default-features = false, features = [ +rocksdb = { version = "0.19", default-features = false, features = [ "lz4", "multi-threaded-cf", "jemalloc" diff --git a/fuel-core/src/config/chain_config.rs b/fuel-core/src/chain_config.rs similarity index 82% rename from fuel-core/src/config/chain_config.rs rename to fuel-core/src/chain_config.rs index 90749f83b00..111d2828c6a 100644 --- a/fuel-core/src/config/chain_config.rs +++ b/fuel-core/src/chain_config.rs @@ -1,15 +1,21 @@ -use super::serialization::{HexNumber, HexType}; use crate::{database::Database, model::BlockHeight}; -use fuel_core_interfaces::common::{ - fuel_tx::ConsensusParameters, - fuel_types::{Address, AssetId, Bytes32, Salt}, +use fuel_core_interfaces::{ + common::{ + fuel_tx::ConsensusParameters, + fuel_types::{Address, AssetId, Bytes32, Salt}, + fuel_vm::fuel_types::Word, + }, + model::{DaBlockHeight, Message}, }; use itertools::Itertools; use rand::{rngs::StdRng, SeedableRng}; use serde::{Deserialize, Serialize}; use serde_with::{serde_as, skip_serializing_none}; +use serialization::{HexNumber, HexType}; use std::{io::ErrorKind, path::PathBuf, str::FromStr}; +pub mod serialization; + pub const LOCAL_TESTNET: &str = "local_testnet"; pub const TESTNET_INITIAL_BALANCE: u64 = 10_000_000; @@ -114,6 +120,8 @@ pub struct StateConfig { pub coins: Option>, /// Contract state pub contracts: Option>, + /// Messages from Layer 1 + pub messages: Option>, /// Starting block height (useful for flattened fork networks) #[serde_as(as = "Option")] #[serde(default)] @@ -125,6 +133,7 @@ impl StateConfig { Ok(StateConfig { coins: db.get_coin_config()?, contracts: db.get_contract_config()?, + messages: db.get_message_config()?, height: db.get_block_height()?, }) } @@ -172,6 +181,42 @@ pub struct ContractConfig { pub balances: Option>, } +#[skip_serializing_none] +#[serde_as] +#[derive(Clone, Debug, Default, Deserialize, Serialize, PartialEq)] +pub struct MessageConfig { + #[serde_as(as = "HexType")] + pub sender: Address, + #[serde_as(as = "HexType")] + pub recipient: Address, + #[serde_as(as = "HexType")] + pub owner: Address, + #[serde_as(as = "HexNumber")] + pub nonce: Word, + #[serde_as(as = "HexNumber")] + pub amount: Word, + #[serde_as(as = "HexType")] + pub data: Vec, + /// The block height from the parent da layer that originated this message + #[serde_as(as = "HexNumber")] + pub da_height: DaBlockHeight, +} + +impl From for Message { + fn from(msg: MessageConfig) -> Self { + Message { + sender: msg.sender, + recipient: msg.recipient, + owner: msg.owner, + nonce: msg.nonce, + amount: msg.amount, + data: msg.data, + da_height: msg.da_height, + fuel_block_spend: None, + } + } +} + #[cfg(test)] mod tests { use super::*; @@ -298,6 +343,21 @@ mod tests { assert_eq!(config, deserialized_config); } + #[test] + fn snapshot_simple_message_state() { + let config = test_message_config(); + let json = serde_json::to_string_pretty(&config).unwrap(); + insta::assert_snapshot!(json); + } + + #[test] + fn can_roundtrip_simple_message_state() { + let config = test_message_config(); + let json = serde_json::to_string(&config).unwrap(); + let deserialized_config: ChainConfig = serde_json::from_str(json.as_str()).unwrap(); + assert_eq!(config, deserialized_config); + } + fn test_config_contract(state: bool, balances: bool) -> ChainConfig { let mut rng = StdRng::seed_from_u64(1); let state = if state { @@ -358,6 +418,26 @@ mod tests { } } + fn test_message_config() -> ChainConfig { + let mut rng = StdRng::seed_from_u64(1); + + ChainConfig { + initial_state: Some(StateConfig { + messages: Some(vec![MessageConfig { + sender: rng.gen(), + recipient: rng.gen(), + owner: rng.gen(), + nonce: rng.gen(), + amount: rng.gen(), + data: vec![rng.gen()], + da_height: rng.gen(), + }]), + ..Default::default() + }), + ..ChainConfig::local_testnet() + } + } + fn tmp_path() -> PathBuf { let mut path = temp_dir(); path.push(rand::random::().to_string()); diff --git a/fuel-core/src/config/serialization.rs b/fuel-core/src/chain_config/serialization.rs similarity index 100% rename from fuel-core/src/config/serialization.rs rename to fuel-core/src/chain_config/serialization.rs diff --git a/fuel-core/src/cli/run.rs b/fuel-core/src/cli/run.rs index f0a444c63c5..365e4974166 100644 --- a/fuel-core/src/cli/run.rs +++ b/fuel-core/src/cli/run.rs @@ -1,7 +1,7 @@ use crate::cli::DEFAULT_DB_PATH; use crate::FuelService; use clap::Parser; -use fuel_core::config::{Config, DbType, VMConfig}; +use fuel_core::service::{Config, DbType, VMConfig}; use std::{env, io, net, path::PathBuf}; use strum::VariantNames; use tracing::{info, trace}; @@ -49,10 +49,6 @@ pub struct Command { #[clap(long = "min-gas-price", default_value = "0")] pub min_gas_price: u64, - /// The minimum allowed byte price - #[clap(long = "min-byte-price", default_value = "0")] - pub min_byte_price: u64, - /// Enable predicate execution on transaction inputs. /// Will reject any transactions with predicates if set to false. #[clap(long = "predicates")] @@ -77,7 +73,6 @@ impl Command { manual_blocks_enabled, utxo_validation, min_gas_price, - min_byte_price, predicates, relayer_args, p2p_args, @@ -104,7 +99,6 @@ impl Command { }, txpool: fuel_txpool::Config { min_gas_price, - min_byte_price, ..Default::default() }, predicates, diff --git a/fuel-core/src/cli/run/relayer.rs b/fuel-core/src/cli/run/relayer.rs index db4f3521254..4e91aedd32d 100644 --- a/fuel-core/src/cli/run/relayer.rs +++ b/fuel-core/src/cli/run/relayer.rs @@ -22,7 +22,7 @@ pub struct RelayerArgs { /// Number of da block after which messages/stakes/validators become finalized. #[clap(long = "relayer-da-finalization", default_value = "64")] - pub da_finalization: u32, + pub da_finalization: u64, /// contract to publish commit fuel block. #[clap(long = "relayer-v2-commit-contract", parse(try_from_str = parse_h160))] diff --git a/fuel-core/src/cli/snapshot.rs b/fuel-core/src/cli/snapshot.rs index 38488c7fdb8..9d699696557 100644 --- a/fuel-core/src/cli/snapshot.rs +++ b/fuel-core/src/cli/snapshot.rs @@ -28,9 +28,7 @@ pub async fn exec(command: Command) -> anyhow::Result<()> { #[cfg(feature = "rocksdb")] pub async fn exec(command: Command) -> anyhow::Result<()> { use anyhow::Context; - use fuel_core::{ - config::chain_config::ChainConfig, config::chain_config::StateConfig, database::Database, - }; + use fuel_core::{chain_config::ChainConfig, chain_config::StateConfig, database::Database}; let path = command.database_path; let config: ChainConfig = command.chain_config.parse()?; diff --git a/fuel-core/src/database.rs b/fuel-core/src/database.rs index 3cae943ead7..faefd99f090 100644 --- a/fuel-core/src/database.rs +++ b/fuel-core/src/database.rs @@ -8,6 +8,7 @@ use crate::state::{ in_memory::memory_store::MemoryStore, ColumnId, DataSource, Error, IterDirection, }; use async_trait::async_trait; +use fuel_core_interfaces::common::fuel_asm::Word; pub use fuel_core_interfaces::db::KvStoreError; use fuel_core_interfaces::{ common::{ @@ -42,7 +43,7 @@ pub mod code_root; pub mod coin; pub mod contracts; pub mod delegates_index; -pub mod deposit_coin; +pub mod message; pub mod metadata; mod receipts; pub mod staking_diffs; @@ -70,17 +71,19 @@ pub mod columns { pub const BLOCKS: u32 = 12; // maps block id -> block hash pub const BLOCK_IDS: u32 = 13; - pub const TOKEN_DEPOSITS: u32 = 14; + pub const MESSAGES: u32 = 14; /// contain current validator stake and it consensus_key if set. pub const VALIDATOR_SET: u32 = 15; /// contain diff between da blocks it contains new registers consensus key and new delegate sets. pub const STAKING_DIFFS: u32 = 16; /// Maps delegate address with validator_set_diff index where last delegate change happened pub const DELEGATES_INDEX: u32 = 17; + // (Owner, MessageId) => true + pub const OWNED_MESSAGE_IDS: u32 = 18; // Number of columns #[cfg(feature = "rocksdb")] - pub const COLUMN_NUM: u32 = 18; + pub const COLUMN_NUM: u32 = 19; } #[derive(Clone, Debug)] @@ -209,10 +212,12 @@ impl Database { { self.data .iter_all(column, prefix, start, direction.unwrap_or_default()) - .map(|(key, value)| { - let key = K::from(key); - let value: V = bincode::deserialize(&value).map_err(|_| Error::Codec)?; - Ok((key, value)) + .map(|val| { + val.and_then(|(key, value)| { + let key = K::from(key); + let value: V = bincode::deserialize(&value).map_err(|_| Error::Codec)?; + Ok((key, value)) + }) }) } @@ -266,6 +271,17 @@ impl InterpreterStorage for Database { Ok(height.into()) } + fn timestamp(&self, height: u32) -> Result { + let id = self.block_hash(height)?; + let block = Storage::::get(self, &id)?.unwrap_or_default(); + block + .headers + .time + .timestamp() + .try_into() + .map_err(|e| Self::DataError::DatabaseError(Box::new(e))) + } + fn block_hash(&self, block_height: u32) -> Result { let hash = self.get_block_id(block_height.into())?.unwrap_or_default(); Ok(hash) @@ -324,7 +340,7 @@ impl RelayerDb for Database { use byteorder::{BigEndian, ReadBytesExt}; use std::io::Cursor; let mut i = Cursor::new(i); - Self(i.read_u32::().unwrap_or_default()) + Self(i.read_u64::().unwrap_or_default()) } } let mut out = Vec::new(); diff --git a/fuel-core/src/database/coin.rs b/fuel-core/src/database/coin.rs index c1925cc7e43..88e4eef66a5 100644 --- a/fuel-core/src/database/coin.rs +++ b/fuel-core/src/database/coin.rs @@ -1,5 +1,5 @@ use crate::{ - config::chain_config::CoinConfig, + chain_config::CoinConfig, database::{ columns::{self, COIN, OWNED_COINS}, Database, KvStoreError, diff --git a/fuel-core/src/database/contracts.rs b/fuel-core/src/database/contracts.rs index c05149b5ec5..b1628838b23 100644 --- a/fuel-core/src/database/contracts.rs +++ b/fuel-core/src/database/contracts.rs @@ -1,6 +1,6 @@ use crate::database::InterpreterStorage; use crate::{ - config::chain_config::ContractConfig, + chain_config::ContractConfig, database::{ columns::{BALANCES, CONTRACTS, CONTRACTS_STATE, CONTRACT_UTXO_ID}, Database, diff --git a/fuel-core/src/database/deposit_coin.rs b/fuel-core/src/database/deposit_coin.rs deleted file mode 100644 index f6a6c790a8b..00000000000 --- a/fuel-core/src/database/deposit_coin.rs +++ /dev/null @@ -1,31 +0,0 @@ -use crate::database::{columns, Database, KvStoreError}; -use fuel_core_interfaces::{ - common::{fuel_storage::Storage, fuel_types::Bytes32}, - model::DepositCoin, -}; -use std::borrow::Cow; - -impl Storage for Database { - type Error = KvStoreError; - - fn insert( - &mut self, - key: &Bytes32, - value: &DepositCoin, - ) -> Result, KvStoreError> { - Database::insert(self, key.as_ref(), columns::TOKEN_DEPOSITS, value.clone()) - .map_err(Into::into) - } - - fn remove(&mut self, key: &Bytes32) -> Result, KvStoreError> { - Database::remove(self, key.as_ref(), columns::TOKEN_DEPOSITS).map_err(Into::into) - } - - fn get(&self, key: &Bytes32) -> Result>, KvStoreError> { - Database::get(self, key.as_ref(), columns::TOKEN_DEPOSITS).map_err(Into::into) - } - - fn contains_key(&self, key: &Bytes32) -> Result { - Database::exists(self, key.as_ref(), columns::TOKEN_DEPOSITS).map_err(Into::into) - } -} diff --git a/fuel-core/src/database/message.rs b/fuel-core/src/database/message.rs new file mode 100644 index 00000000000..f66645bef8e --- /dev/null +++ b/fuel-core/src/database/message.rs @@ -0,0 +1,158 @@ +use crate::{ + chain_config::MessageConfig, + database::{columns, Database, KvStoreError}, + state::{Error, IterDirection}, +}; +use fuel_core_interfaces::{ + common::{ + fuel_storage::Storage, + fuel_types::{Address, Bytes32, MessageId}, + }, + model::Message, +}; +use std::borrow::Cow; +use std::ops::Deref; + +impl Storage for Database { + type Error = KvStoreError; + + fn insert( + &mut self, + key: &MessageId, + value: &Message, + ) -> Result, KvStoreError> { + // insert primary record + let result = Database::insert(self, key.as_ref(), columns::MESSAGES, value.clone())?; + + // insert secondary record by owner + Database::insert( + self, + owner_msg_id_key(&value.owner, key), + columns::OWNED_MESSAGE_IDS, + true, + )?; + + Ok(result) + } + + fn remove(&mut self, key: &MessageId) -> Result, KvStoreError> { + let result: Option = Database::remove(self, key.as_ref(), columns::MESSAGES)?; + + if let Some(message) = &result { + Database::remove::( + self, + &owner_msg_id_key(&message.owner, key), + columns::OWNED_MESSAGE_IDS, + )?; + } + + Ok(result) + } + + fn get(&self, key: &MessageId) -> Result>, KvStoreError> { + Database::get(self, key.as_ref(), columns::MESSAGES).map_err(Into::into) + } + + fn contains_key(&self, key: &MessageId) -> Result { + Database::exists(self, key.as_ref(), columns::MESSAGES).map_err(Into::into) + } +} + +impl Database { + pub fn owned_message_ids( + &self, + owner: Address, + start_message_id: Option, + direction: Option, + ) -> impl Iterator> + '_ { + self.iter_all::, bool>( + columns::OWNED_MESSAGE_IDS, + Some(owner.as_ref().to_vec()), + start_message_id.map(|msg_id| owner_msg_id_key(&owner, &msg_id)), + direction, + ) + // Safety: key is always 64 bytes + .map(|res| { + res.map(|(key, _)| { + MessageId::new(unsafe { *Bytes32::from_slice_unchecked(&key[32..64]) }) + }) + }) + } + + pub fn all_messages( + &self, + start: Option, + direction: Option, + ) -> impl Iterator> + '_ { + let start = start.map(|v| v.deref().to_vec()); + self.iter_all::, Message>(columns::MESSAGES, None, start, direction) + .map(|res| res.map(|(_, message)| message)) + } + + pub fn get_message_config(&self) -> Result>, Error> { + let configs = self + .all_messages(None, None) + .map(|msg| -> Result { + let msg = msg?; + + Ok(MessageConfig { + sender: msg.sender, + recipient: msg.recipient, + owner: msg.owner, + nonce: msg.nonce, + amount: msg.amount, + data: msg.data, + da_height: msg.da_height, + }) + }) + .collect::, Error>>()?; + + Ok(Some(configs)) + } +} + +/// Get a Key by chaining Owner + MessageId +fn owner_msg_id_key(owner: &Address, msg_id: &MessageId) -> Vec { + owner + .as_ref() + .iter() + .chain(msg_id.as_ref().iter()) + .copied() + .collect() +} + +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn owned_message_ids() { + let mut db = Database::default(); + let message = Message::default(); + + // insert a message with the first id + let first_id = MessageId::new([1; 32]); + let _ = Storage::::insert(&mut db, &first_id, &message).unwrap(); + + // insert a message with the second id with the same Owner + let second_id = MessageId::new([2; 32]); + let _ = Storage::::insert(&mut db, &second_id, &message).unwrap(); + + // verify that 2 message IDs are associated with a single Owner + let owned_msg_ids = db.owned_message_ids(message.owner, None, None); + assert_eq!(owned_msg_ids.count(), 2); + + // remove the first message with its given id + let _ = Storage::::remove(&mut db, &first_id).unwrap(); + + // verify that only second ID is left + let owned_msg_ids: Vec<_> = db.owned_message_ids(message.owner, None, None).collect(); + assert_eq!(owned_msg_ids.first().unwrap().as_ref().unwrap(), &second_id); + assert_eq!(owned_msg_ids.len(), 1); + + // remove the second message with its given id + let _ = Storage::::remove(&mut db, &second_id).unwrap(); + let owned_msg_ids = db.owned_message_ids(message.owner, None, None); + assert_eq!(owned_msg_ids.count(), 0); + } +} diff --git a/fuel-core/src/database/metadata.rs b/fuel-core/src/database/metadata.rs index e7fda685677..74dac74a516 100644 --- a/fuel-core/src/database/metadata.rs +++ b/fuel-core/src/database/metadata.rs @@ -1,7 +1,7 @@ -use crate::config::Config; use crate::database::columns::METADATA; use crate::database::Database; use crate::model::BlockHeight; +use crate::service::config::Config; use crate::state::Error; pub(crate) const DB_VERSION_KEY: &[u8] = b"version"; diff --git a/fuel-core/src/executor.rs b/fuel-core/src/executor.rs index 12044e3cd6e..1ebb52cf946 100644 --- a/fuel-core/src/executor.rs +++ b/fuel-core/src/executor.rs @@ -1,10 +1,14 @@ use crate::{ - config::Config, database::{transaction::TransactionIndex, Database, KvStoreError}, model::{BlockHeight, Coin, CoinStatus, FuelBlock, FuelBlockDb}, + service::Config, tx_pool::TransactionStatus, }; use chrono::Utc; +use fuel_core_interfaces::{ + common::fuel_tx::{CheckedTransaction, TransactionFee}, + model::Message, +}; use fuel_core_interfaces::{ common::{ fuel_asm::Word, @@ -14,7 +18,7 @@ use fuel_core_interfaces::{ Address, AssetId, Bytes32, Input, Output, Receipt, Transaction, TxId, UtxoId, ValidationError, }, - fuel_types::{bytes::SerializableVec, ContractId}, + fuel_types::{bytes::SerializableVec, ContractId, MessageId}, fuel_vm::{ consts::REG_SP, prelude::{Backtrace as FuelBacktrace, Interpreter, PredicateStorage}, @@ -85,6 +89,13 @@ impl Executor { }; let mut block_db_transaction = self.database.transaction(); + // Insert the current headers (including time, block height, producer into the db tx) + Storage::::insert( + block_db_transaction.deref_mut(), + &Bytes32::zeroed(), // use id of zero as current block + &block.to_db_block(), + )?; + let mut txs_merkle = MerkleTree::new(); let mut tx_status = vec![]; let mut coinbase = 0u64; @@ -100,23 +111,33 @@ impl Executor { return Err(Error::TransactionIdCollision(tx_id)); } - self.verify_tx_predicates(tx)?; + self.compute_contract_input_utxo_ids(tx, &mode, block_db_transaction.deref())?; + + let checked_tx = CheckedTransaction::check_unsigned( + tx.clone(), + block.header.height.into(), + &self.config.chain_conf.transaction_parameters, + )?; + let min_fee = checked_tx.min_fee(); + let max_fee = checked_tx.max_fee(); + + self.verify_tx_predicates(&checked_tx)?; if self.config.utxo_validation { // validate transaction has at least one coin - self.verify_tx_has_at_least_one_coin(tx)?; + self.verify_tx_has_at_least_one_coin_or_message(tx)?; // validate utxos exist and maturity is properly set - self.verify_input_state(block_db_transaction.deref(), tx, block.header.height)?; + self.verify_input_state( + block_db_transaction.deref(), + tx, + block.header.height, + block.header.number, + )?; // validate transaction signature tx.validate_input_signature() .map_err(TransactionValidityError::from)?; } - self.compute_contract_input_utxo_ids(tx, &mode, block_db_transaction.deref())?; - - // verify that the tx has enough gas to cover committed costs - self.verify_gas(tx)?; - // index owners of inputs and outputs with tx-id, regardless of validity (hence block_tx instead of tx_db) self.persist_owners_index( block.header.height, @@ -136,7 +157,7 @@ impl Executor { self.config.chain_conf.transaction_parameters, ); let vm_result = vm - .transact(tx.clone()) + .transact(checked_tx) .map_err(|error| Error::VmExecution { error, transaction_id: tx_id, @@ -149,7 +170,8 @@ impl Executor { } // update block commitment - let tx_fee = self.total_fee_paid(tx, vm_result.receipts())?; + let tx_fee = + self.total_fee_paid(min_fee, max_fee, tx.gas_price(), vm_result.receipts())?; coinbase = coinbase.checked_add(tx_fee).ok_or(Error::FeeOverflow)?; // include the canonical serialization of the malleated tx into the commitment, @@ -184,7 +206,11 @@ impl Executor { )?; // change the spent status of the tx inputs - self.spend_inputs(vm_result.tx(), block_db_transaction.deref_mut())?; + self.spend_inputs( + vm_result.tx(), + block_db_transaction.deref_mut(), + block.header.height, + )?; // persist any outputs self.persist_outputs( @@ -266,6 +292,12 @@ impl Executor { block_db_transaction.deref_mut(), )?; + // cleanup unfinalized headers (block height + time + producer) + Storage::::remove( + block_db_transaction.deref_mut(), + &Bytes32::zeroed(), + )?; + // insert block into database Storage::::insert( block_db_transaction.deref_mut(), @@ -276,12 +308,12 @@ impl Executor { Ok(()) } - // Waiting until accounts and genesis block setup is working fn verify_input_state( &self, db: &Database, transaction: &Transaction, block_height: BlockHeight, + block_da_height: BlockHeight, ) -> Result<(), TransactionValidityError> { for input in transaction.inputs() { match input { @@ -298,6 +330,21 @@ impl Executor { } } Input::Contract { .. } => {} + Input::MessageSigned { message_id, .. } + | Input::MessagePredicate { message_id, .. } => { + if let Some(message) = Storage::::get(db, message_id)? { + if message.fuel_block_spend.is_some() { + return Err(TransactionValidityError::MessageAlreadySpent(*message_id)); + } + if BlockHeight::from(message.da_height) > block_da_height { + return Err(TransactionValidityError::MessageSpendTooEarly( + *message_id, + )); + } + } else { + return Err(TransactionValidityError::MessageDoesNotExist(*message_id)); + } + } } } @@ -305,13 +352,17 @@ impl Executor { } /// Verify all the predicates of a tx. - pub fn verify_tx_predicates(&self, tx: &Transaction) -> Result<(), Error> { + pub fn verify_tx_predicates(&self, tx: &CheckedTransaction) -> Result<(), Error> { // fail if tx contains any predicates when predicates are disabled if !self.config.predicates { - let has_predicate = tx.inputs().iter().any(|input| input.is_coin_predicate()); + let has_predicate = tx + .as_ref() + .inputs() + .iter() + .any(|input| input.is_coin_predicate()); if has_predicate { return Err(Error::TransactionValidity( - TransactionValidityError::PredicateExecutionDisabled(tx.id()), + TransactionValidityError::PredicateExecutionDisabled(tx.transaction().id()), )); } } else { @@ -321,7 +372,7 @@ impl Executor { self.config.chain_conf.transaction_parameters, ) { return Err(Error::TransactionValidity( - TransactionValidityError::InvalidPredicate(tx.id()), + TransactionValidityError::InvalidPredicate(tx.transaction().id()), )); } } @@ -332,124 +383,139 @@ impl Executor { /// /// TODO: This verification really belongs in fuel-tx, and can be removed once /// https://github.com/FuelLabs/fuel-tx/issues/118 is resolved. - fn verify_tx_has_at_least_one_coin(&self, tx: &Transaction) -> Result<(), Error> { - if tx.inputs().iter().filter(|input| input.is_coin()).count() == 0 { - Err(TransactionValidityError::NoCoinInput(tx.id()).into()) - } else { + fn verify_tx_has_at_least_one_coin_or_message(&self, tx: &Transaction) -> Result<(), Error> { + if tx + .inputs() + .iter() + .any(|input| input.is_coin() || input.is_message()) + { Ok(()) + } else { + Err(TransactionValidityError::NoCoinOrMessageInput(tx.id()).into()) } } /// Mark inputs as spent - fn spend_inputs(&self, tx: &Transaction, db: &mut Database) -> Result<(), Error> { + fn spend_inputs( + &self, + tx: &Transaction, + db: &mut Database, + block_height: BlockHeight, + ) -> Result<(), Error> { for input in tx.inputs() { - if let Input::CoinSigned { - utxo_id, - owner, - amount, - asset_id, - maturity, - .. - } - | Input::CoinPredicate { - utxo_id, - owner, - amount, - asset_id, - maturity, - .. - } = input - { - let block_created = if self.config.utxo_validation { - Storage::::get(db, utxo_id)? - .ok_or(Error::TransactionValidity( - TransactionValidityError::CoinDoesNotExist(*utxo_id), - ))? - .block_created - } else { - // if utxo validation is disabled, just assign this new input to the original block - Default::default() - }; - - Storage::::insert( - db, + match input { + Input::CoinSigned { utxo_id, - &Coin { - owner: *owner, - amount: *amount, - asset_id: *asset_id, - maturity: (*maturity).into(), - status: CoinStatus::Spent, - block_created, - }, - )?; - } - } - Ok(()) - } + owner, + amount, + asset_id, + maturity, + .. + } + | Input::CoinPredicate { + utxo_id, + owner, + amount, + asset_id, + maturity, + .. + } => { + let block_created = if self.config.utxo_validation { + Storage::::get(db, utxo_id)? + .ok_or(Error::TransactionValidity( + TransactionValidityError::CoinDoesNotExist(*utxo_id), + ))? + .block_created + } else { + // if utxo validation is disabled, just assign this new input to the original block + Default::default() + }; - /// verify that the transaction has enough gas to cover fees - fn verify_gas(&self, tx: &Transaction) -> Result<(), Error> { - if tx.gas_price() != 0 || tx.byte_price() != 0 { - let gas: Word = tx - .inputs() - .iter() - .filter_map(|input| match input { - Input::CoinSigned { - amount, asset_id, .. - } if asset_id == &AssetId::default() => Some(amount), - Input::CoinPredicate { - amount, asset_id, .. - } if asset_id == &AssetId::default() => Some(amount), - _ => None, - }) - .sum(); - let spent_gas: Word = tx - .outputs() - .iter() - .filter_map(|output| match output { - Output::Coin { - amount, asset_id, .. - } if asset_id == &AssetId::default() => Some(amount), - Output::Withdrawal { - amount, asset_id, .. - } if asset_id == &AssetId::default() => Some(amount), - _ => None, - }) - .sum(); - let factor = self - .config - .chain_conf - .transaction_parameters - .gas_price_factor as f64; - let byte_fees = - ((tx.metered_bytes_size() as Word * tx.byte_price()) as f64 / factor).ceil() as u64; - let gas_fees = ((tx.gas_limit() * tx.gas_price()) as f64 / factor).ceil() as u64; - let total_gas_required = spent_gas - .checked_add(byte_fees) - .ok_or(Error::FeeOverflow)? - .checked_add(gas_fees) - .ok_or(Error::FeeOverflow)?; - gas.checked_sub(total_gas_required) - .ok_or(Error::InsufficientFeeAmount { - provided: gas, - required: total_gas_required, - })?; - } + Storage::::insert( + db, + utxo_id, + &Coin { + owner: *owner, + amount: *amount, + asset_id: *asset_id, + maturity: (*maturity).into(), + status: CoinStatus::Spent, + block_created, + }, + )?; + } + Input::MessageSigned { + message_id, + sender, + recipient, + amount, + nonce, + owner, + data, + .. + } + | Input::MessagePredicate { + message_id, + sender, + recipient, + amount, + nonce, + owner, + data, + .. + } => { + let da_height = if self.config.utxo_validation { + Storage::::get(db, message_id)? + .ok_or(Error::TransactionValidity( + TransactionValidityError::MessageDoesNotExist(*message_id), + ))? + .da_height + } else { + // if utxo validation is disabled, just assignto the original block + Default::default() + }; + Storage::::insert( + db, + message_id, + &Message { + da_height, + fuel_block_spend: Some(block_height), + sender: *sender, + recipient: *recipient, + owner: *owner, + nonce: *nonce, + amount: *amount, + data: data.clone(), + }, + )?; + } + _ => {} + } + } Ok(()) } - fn total_fee_paid(&self, tx: &Transaction, receipts: &[Receipt]) -> Result { - let mut fee = tx.metered_bytes_size() as Word * tx.byte_price(); - + fn total_fee_paid( + &self, + min_fee: u64, + max_fee: u64, + gas_price: u64, + receipts: &[Receipt], + ) -> Result { for r in receipts { if let Receipt::ScriptResult { gas_used, .. } = r { - fee = fee.checked_add(*gas_used).ok_or(Error::FeeOverflow)?; + return TransactionFee::gas_refund_value( + &self.config.chain_conf.transaction_parameters, + *gas_used, + gas_price, + ) + .and_then(|refund| max_fee.checked_sub(refund)) + .ok_or(Error::FeeOverflow); } } - - Ok(fee) + // if there's no script result (i.e. create) then fee == base amount + Ok(min_fee) } /// In production mode, lookup and set the proper utxo ids for contract inputs @@ -543,8 +609,8 @@ impl Executor { )); } } - Output::Withdrawal { .. } => { - // TODO: Handle withdrawals somehow (new field on the block type?) + Output::Message { .. } => { + // TODO: Handle message outputs somehow (new field on the block type?) } Output::Change { to, @@ -624,7 +690,7 @@ impl Executor { for output in tx.outputs() { match output { Output::Coin { to, .. } - | Output::Withdrawal { to, .. } + | Output::Message { recipient: to, .. } | Output::Change { to, .. } | Output::Variable { to, .. } => { owners.push(to); @@ -675,10 +741,16 @@ pub enum TransactionValidityError { CoinHasNotMatured(UtxoId), #[error("The specified coin doesn't exist")] CoinDoesNotExist(UtxoId), + #[error("The specified message was already spent")] + MessageAlreadySpent(MessageId), + #[error("Message is not yet spendable, as it's DA height is newer than this block allows")] + MessageSpendTooEarly(MessageId), + #[error("The specified message doesn't exist")] + MessageDoesNotExist(MessageId), #[error("Contract output index isn't valid: {0:#x}")] InvalidContractInputIndex(UtxoId), - #[error("The transaction must have at least one coin input type: {0:#x}")] - NoCoinInput(TxId), + #[error("The transaction must have at least one coin or message input type: {0:#x}")] + NoCoinOrMessageInput(TxId), #[error("The transaction contains predicate inputs which aren't enabled: {0:#x}")] PredicateExecutionDisabled(TxId), #[error( @@ -704,8 +776,6 @@ pub enum Error { TransactionIdCollision(Bytes32), #[error("output already exists")] OutputAlreadyExists, - #[error("Transaction doesn't include enough value to pay for gas: {provided} < {required}")] - InsufficientFeeAmount { provided: Word, required: Word }, #[error("The computed fee caused an integer overflow")] FeeOverflow, #[error("Invalid transaction: {0}")] @@ -722,6 +792,8 @@ pub enum Error { error: fuel_core_interfaces::common::fuel_vm::prelude::InterpreterError, transaction_id: Bytes32, }, + #[error(transparent)] + InvalidTransaction(#[from] ValidationError), #[error("Execution error with backtrace")] Backtrace(Box), #[error("Transaction doesn't match expected result: {transaction_id:#x}")] @@ -758,18 +830,22 @@ impl From for Error { mod tests { use super::*; use crate::model::FuelBlockHeader; - use fuel_core_interfaces::common::fuel_tx::ConsensusParameters; - use fuel_core_interfaces::common::{ - fuel_asm::Opcode, - fuel_crypto::SecretKey, - fuel_tx::{self, TransactionBuilder}, - fuel_types::{ContractId, Immediate12, Immediate18, Salt}, - fuel_vm::{ - consts::{REG_CGAS, REG_FP, REG_ONE, REG_ZERO}, - prelude::{Call, CallFrame}, - script_with_data_offset, - util::test_helpers::TestBuilder as TxBuilder, + use chrono::TimeZone; + use fuel_core_interfaces::model::{CheckedMessage, Message}; + use fuel_core_interfaces::{ + common::{ + fuel_asm::Opcode, + fuel_crypto::SecretKey, + fuel_tx::{self, ConsensusParameters, Transaction, TransactionBuilder}, + fuel_types::{ContractId, Immediate12, Immediate18, Salt}, + fuel_vm::{ + consts::{REG_CGAS, REG_FP, REG_ONE, REG_ZERO}, + prelude::{Call, CallFrame}, + script_with_data_offset, + util::test_helpers::TestBuilder as TxBuilder, + }, }, + relayer::RelayerDb, }; use itertools::Itertools; use rand::prelude::StdRng; @@ -785,6 +861,8 @@ mod tests { .coin_output(AssetId::default(), (i as Word) * 50) .change_output(AssetId::default()) .build() + .transaction() + .clone() }) .collect_vec(); @@ -806,11 +884,9 @@ mod tests { 0, 0, 0, - 0, salt, vec![], vec![], - vec![], vec![Output::ContractCreated { contract_id, state_root, @@ -899,7 +975,7 @@ mod tests { .await; assert!(matches!( produce_result, - Err(Error::InsufficientFeeAmount { required, .. }) if required == (gas_limit as f64 / factor).ceil() as u64 + Err(Error::InvalidTransaction(ValidationError::InsufficientFeeAmount { expected, .. })) if expected == (gas_limit as f64 / factor).ceil() as u64 )); let verify_result = verifier @@ -907,7 +983,7 @@ mod tests { .await; assert!(matches!( verify_result, - Err(Error::InsufficientFeeAmount {required, ..}) if required == (gas_limit as f64 / factor).ceil() as u64 + Err(Error::InvalidTransaction(ValidationError::InsufficientFeeAmount { expected, ..})) if expected == (gas_limit as f64 / factor).ceil() as u64 )) } @@ -970,7 +1046,15 @@ mod tests { Storage::::insert(&mut db, &spent_utxo_id, &coin).unwrap(); // create an input referring to a coin that is already spent - let input = Input::coin_signed(spent_utxo_id, owner, amount, asset_id, 0, 0); + let input = Input::coin_signed( + spent_utxo_id, + owner, + amount, + asset_id, + Default::default(), + 0, + 0, + ); let output = Output::Change { to: owner, amount: 0, @@ -980,7 +1064,6 @@ mod tests { 0, 0, 0, - 0, vec![], vec![], vec![input], @@ -1038,10 +1121,11 @@ mod tests { let tx = TransactionBuilder::script(vec![Opcode::RET(REG_ONE)].into_iter().collect(), vec![]) .add_unsigned_coin_input( + SecretKey::random(&mut rng), rng.gen(), - &SecretKey::random(&mut rng), 10, Default::default(), + Default::default(), 0, ) .add_output(Output::Change { @@ -1099,11 +1183,12 @@ mod tests { let input_amount = 10; let fake_output_amount = 100; - let tx = TxBuilder::new(2322u64) + let tx: Transaction = TxBuilder::new(2322u64) .gas_limit(1) .coin_input(Default::default(), input_amount) .change_output(Default::default()) - .build(); + .build() + .into(); let tx_id = tx.id(); @@ -1148,11 +1233,12 @@ mod tests { #[tokio::test] async fn executor_invalidates_blocks_with_diverging_tx_commitment() { let mut rng = StdRng::seed_from_u64(2322u64); - let tx = TxBuilder::new(2322u64) + let tx: Transaction = TxBuilder::new(2322u64) .gas_limit(1) .coin_input(Default::default(), 10) .change_output(Default::default()) - .build(); + .build() + .into(); let producer = Executor { database: Default::default(), @@ -1187,7 +1273,7 @@ mod tests { // invalidate a block if a tx is missing at least one coin input #[tokio::test] async fn executor_invalidates_missing_coin_input() { - let tx = TxBuilder::new(2322u64).build(); + let tx: Transaction = TxBuilder::new(2322u64).build().into(); let tx_id = tx.id(); let executor = Executor { @@ -1212,17 +1298,18 @@ mod tests { // assert block failed to validate when transaction didn't contain any coin inputs assert!(matches!( err, - Error::TransactionValidity(TransactionValidityError::NoCoinInput(id)) if id == tx_id + Error::TransactionValidity(TransactionValidityError::NoCoinOrMessageInput(id)) if id == tx_id )); } #[tokio::test] async fn input_coins_are_marked_as_spent() { // ensure coins are marked as spent after tx is processed - let tx = TxBuilder::new(2322u64) + let tx: Transaction = TxBuilder::new(2322u64) .coin_input(AssetId::default(), 100) .change_output(AssetId::default()) - .build(); + .build() + .into(); let db = Database::default(); let executor = Executor { @@ -1241,9 +1328,10 @@ mod tests { .unwrap(); // assert the tx coin is spent - let coin = Storage::::get(&db, block.transactions[0].inputs()[0].utxo_id()) - .unwrap() - .unwrap(); + let coin = + Storage::::get(&db, block.transactions[0].inputs()[0].utxo_id().unwrap()) + .unwrap() + .unwrap(); assert_eq!(coin.status, CoinStatus::Spent); } @@ -1256,10 +1344,11 @@ mod tests { let tx = TransactionBuilder::script(vec![Opcode::RET(REG_ONE)].into_iter().collect(), vec![]) .add_unsigned_coin_input( + SecretKey::random(&mut rng), rng.gen(), - &SecretKey::random(&mut rng), 100, Default::default(), + Default::default(), 0, ) .add_output(Output::Change { @@ -1324,9 +1413,10 @@ mod tests { .unwrap(); // assert the tx coin is spent - let coin = Storage::::get(&db, block.transactions[0].inputs()[0].utxo_id()) - .unwrap() - .unwrap(); + let coin = + Storage::::get(&db, block.transactions[0].inputs()[0].utxo_id().unwrap()) + .unwrap() + .unwrap(); assert_eq!(coin.status, CoinStatus::Spent); // assert block created from coin before spend is still intact (only a concern when utxo-validation is enabled) assert_eq!(coin.block_created, starting_block) @@ -1343,11 +1433,13 @@ mod tests { transactions: vec![tx], }; - let tx2 = TxBuilder::new(2322) - .script(vec![Opcode::RET(1)]) + let tx2: Transaction = TxBuilder::new(2322) + .start_script(vec![Opcode::RET(1)], vec![]) .contract_input(contract_id) .contract_output(&contract_id) - .build(); + .build() + .into(); + let mut second_block = FuelBlock { header: FuelBlockHeader { height: 2u64.into(), @@ -1396,22 +1488,30 @@ mod tests { // create a contract in block 1 // verify a block 2 containing contract id from block 1, with wrong input contract utxo_id let (tx, contract_id) = create_contract(vec![], &mut rng); - let tx2 = TxBuilder::new(2322) - .script(vec![Opcode::ADDI(0x10, REG_ZERO, 0), Opcode::RET(1)]) + let tx2: Transaction = TxBuilder::new(2322) + .start_script( + vec![Opcode::ADDI(0x10, REG_ZERO, 0), Opcode::RET(1)], + vec![], + ) .contract_input(contract_id) .contract_output(&contract_id) - .build(); + .build() + .into(); let mut first_block = FuelBlock { header: Default::default(), transactions: vec![tx, tx2], }; - let tx3 = TxBuilder::new(2322) - .script(vec![Opcode::ADDI(0x10, REG_ZERO, 1), Opcode::RET(1)]) + let tx3: Transaction = TxBuilder::new(2322) + .start_script( + vec![Opcode::ADDI(0x10, REG_ZERO, 1), Opcode::RET(1)], + vec![], + ) .contract_input(contract_id) .contract_output(&contract_id) - .build(); + .build() + .into(); let tx_id = tx3.id(); let mut second_block = FuelBlock { @@ -1528,17 +1628,17 @@ mod tests { .copied() .collect(); - let tx2 = TxBuilder::new(2322) + let tx2: Transaction = TxBuilder::new(2322) .gas_limit(ConsensusParameters::DEFAULT.max_gas_per_tx) - .script(script) - .script_data(script_data) + .start_script(script, script_data) .contract_input(contract_id) .coin_input(asset_id, input_amount) .variable_output(Default::default()) .coin_output(asset_id, coin_output_amount) .change_output(asset_id) .contract_output(&contract_id) - .build(); + .build() + .into(); let tx2_id = tx2.id(); let database = Database::default(); @@ -1579,12 +1679,13 @@ mod tests { let input_amount = 0; let coin_output_amount = 0; - let tx = TxBuilder::new(2322) + let tx: Transaction = TxBuilder::new(2322) .coin_input(asset_id, input_amount) .variable_output(Default::default()) .coin_output(asset_id, coin_output_amount) .change_output(asset_id) - .build(); + .build() + .into(); let tx_id = tx.id(); let database = Database::default(); @@ -1604,9 +1705,327 @@ mod tests { .unwrap(); for idx in 0..2 { - let id = fuel_tx::UtxoId::new(tx_id, idx); + let id = UtxoId::new(tx_id, idx); let maybe_utxo = Storage::::get(&database, &id).unwrap(); assert!(maybe_utxo.is_none()); } } + + /// Helper to build transactions and a message in it for some of the message tests + fn make_tx_and_message(rng: &mut StdRng, da_height: u64) -> (Transaction, CheckedMessage) { + let mut message = Message { + sender: rng.gen(), + recipient: rng.gen(), + owner: rng.gen(), + nonce: rng.gen(), + amount: 1000, + data: vec![], + da_height, + fuel_block_spend: None, + }; + + let tx = TransactionBuilder::script(vec![], vec![]) + .add_unsigned_message_input( + rng.gen(), + message.sender, + message.recipient, + message.nonce, + message.amount, + vec![], + ) + .finalize(); + + if let Input::MessageSigned { owner, .. } = tx.inputs()[0] { + message.owner = owner; + } else { + unreachable!(); + } + + (tx, message.check()) + } + + /// Helper to build database and executor for some of the message tests + async fn make_executor(messages: &[&CheckedMessage]) -> Executor { + let mut database = Database::default(); + + for message in messages { + database.insert_message(message).await; + } + + Executor { + database, + config: Config { + utxo_validation: true, + ..Config::local_node() + }, + } + } + + #[tokio::test] + async fn unspent_message_succeeds_when_msg_da_height_lt_block_da_height() { + let mut rng = StdRng::seed_from_u64(2322); + + let (tx, message) = make_tx_and_message(&mut rng, 0); + + let mut block = FuelBlock { + header: Default::default(), + transactions: vec![tx.clone()], + }; + + make_executor(&[&message]) + .await + .execute(&mut block, ExecutionMode::Production) + .await + .expect("block execution failed unexpectedly"); + + make_executor(&[&message]) + .await + .execute(&mut block, ExecutionMode::Validation) + .await + .expect("block validation failed unexpectedly"); + } + + #[tokio::test] + async fn message_fails_when_spending_nonexistent_message_id() { + let mut rng = StdRng::seed_from_u64(2322); + + let (tx, _message) = make_tx_and_message(&mut rng, 0); + + let mut block = FuelBlock { + header: Default::default(), + transactions: vec![tx.clone()], + }; + + let res = make_executor(&[]) // No messages in the db + .await + .execute(&mut block, ExecutionMode::Production) + .await; + assert!(matches!( + res, + Err(Error::TransactionValidity( + TransactionValidityError::MessageDoesNotExist(_) + )) + )); + + let res = make_executor(&[]) // No messages in the db + .await + .execute(&mut block, ExecutionMode::Validation) + .await; + assert!(matches!( + res, + Err(Error::TransactionValidity( + TransactionValidityError::MessageDoesNotExist(_) + )) + )); + } + + #[tokio::test] + async fn message_fails_when_spending_da_height_gt_block_da_height() { + let mut rng = StdRng::seed_from_u64(2322); + + let (tx, message) = make_tx_and_message(&mut rng, 1); // Block has zero da_height + + let mut block = FuelBlock { + header: Default::default(), + transactions: vec![tx], + }; + + let res = make_executor(&[&message]) + .await + .execute(&mut block, ExecutionMode::Production) + .await; + assert!(matches!( + res, + Err(Error::TransactionValidity( + TransactionValidityError::MessageSpendTooEarly(_) + )) + )); + + let res = make_executor(&[&message]) + .await + .execute(&mut block, ExecutionMode::Validation) + .await; + assert!(matches!( + res, + Err(Error::TransactionValidity( + TransactionValidityError::MessageSpendTooEarly(_) + )) + )); + } + + #[tokio::test] + async fn message_fails_when_spending_already_spent_message_id() { + let mut rng = StdRng::seed_from_u64(2322); + + // Create two transactions with the same message + let (tx1, message) = make_tx_and_message(&mut rng, 0); + let (mut tx2, _) = make_tx_and_message(&mut rng, 0); + tx2.inputs_mut()[0] = tx1.inputs()[0].clone(); + + let mut block = FuelBlock { + header: Default::default(), + transactions: vec![tx1, tx2], + }; + + let res = make_executor(&[&message]) + .await + .execute(&mut block, ExecutionMode::Production) + .await; + assert!(matches!( + res, + Err(Error::TransactionValidity( + TransactionValidityError::MessageAlreadySpent(_) + )) + )); + + let res = make_executor(&[&message]) + .await + .execute(&mut block, ExecutionMode::Validation) + .await; + assert!(matches!( + res, + Err(Error::TransactionValidity( + TransactionValidityError::MessageAlreadySpent(_) + )) + )); + } + + #[tokio::test] + async fn get_block_height_returns_current_executing_block() { + let mut rng = StdRng::seed_from_u64(1234); + + // return current block height + let script = vec![Opcode::BHEI(0x10), Opcode::RET(0x10)]; + let tx = TransactionBuilder::script(script.into_iter().collect(), vec![]) + .gas_limit(10000) + .add_unsigned_coin_input( + rng.gen(), + rng.gen(), + 1000, + AssetId::zeroed(), + Default::default(), + 0, + ) + .finalize(); + + // setup block + let block_height = rng.gen_range(5u32..1000u32); + + let mut block = FuelBlock { + header: FuelBlockHeader { + height: block_height.into(), + ..Default::default() + }, + transactions: vec![tx.clone()], + }; + + // setup db with coin to spend + let mut database = Database::default(); + let coin_input = &tx.inputs()[0]; + Storage::::insert( + &mut database, + coin_input.utxo_id().unwrap(), + &Coin { + owner: *coin_input.input_owner().unwrap(), + amount: coin_input.amount().unwrap(), + asset_id: *coin_input.asset_id().unwrap(), + maturity: (coin_input.maturity().unwrap()).into(), + block_created: 0u64.into(), + status: CoinStatus::Unspent, + }, + ) + .unwrap(); + + // make executor with db + let executor = Executor { + database: database.clone(), + config: Config { + utxo_validation: true, + ..Config::local_node() + }, + }; + + executor + .execute(&mut block, ExecutionMode::Production) + .await + .unwrap(); + + let receipts = Storage::>::get(&database, &tx.id()) + .unwrap() + .unwrap(); + assert_eq!(block_height as u64, receipts[0].val().unwrap()); + } + + #[tokio::test] + async fn get_time_returns_current_executing_block_time() { + let mut rng = StdRng::seed_from_u64(1234); + + // return current block height + let script = vec![ + Opcode::BHEI(0x10), + Opcode::TIME(0x11, 0x10), + Opcode::RET(0x11), + ]; + let tx = TransactionBuilder::script(script.into_iter().collect(), vec![]) + .gas_limit(10000) + .add_unsigned_coin_input( + rng.gen(), + rng.gen(), + 1000, + AssetId::zeroed(), + Default::default(), + 0, + ) + .finalize(); + + // setup block + let block_height = rng.gen_range(5u32..1000u32); + let time = rng.gen_range(1u32..u32::MAX); + + let mut block = FuelBlock { + header: FuelBlockHeader { + height: block_height.into(), + time: Utc.timestamp(time as i64, 0), + ..Default::default() + }, + transactions: vec![tx.clone()], + }; + + // setup db with coin to spend + let mut database = Database::default(); + let coin_input = &tx.inputs()[0]; + Storage::::insert( + &mut database, + coin_input.utxo_id().unwrap(), + &Coin { + owner: *coin_input.input_owner().unwrap(), + amount: coin_input.amount().unwrap(), + asset_id: *coin_input.asset_id().unwrap(), + maturity: (coin_input.maturity().unwrap()).into(), + block_created: 0u64.into(), + status: CoinStatus::Unspent, + }, + ) + .unwrap(); + + // make executor with db + let executor = Executor { + database: database.clone(), + config: Config { + utxo_validation: true, + ..Config::local_node() + }, + }; + + executor + .execute(&mut block, ExecutionMode::Production) + .await + .unwrap(); + + let receipts = Storage::>::get(&database, &tx.id()) + .unwrap() + .unwrap(); + + assert_eq!(time as u64, receipts[0].val().unwrap()); + } } diff --git a/fuel-core/src/lib.rs b/fuel-core/src/lib.rs index 97e81432ca3..d3bdb628128 100644 --- a/fuel-core/src/lib.rs +++ b/fuel-core/src/lib.rs @@ -1,5 +1,5 @@ +pub mod chain_config; pub mod coin_query; -pub mod config; pub mod database; pub mod executor; pub mod model; diff --git a/fuel-core/src/schema.rs b/fuel-core/src/schema.rs index 8aba43b9b39..198903a5fdb 100644 --- a/fuel-core/src/schema.rs +++ b/fuel-core/src/schema.rs @@ -7,6 +7,7 @@ pub mod coin; pub mod contract; pub mod dap; pub mod health; +pub mod message; pub mod node_info; pub mod scalars; pub mod tx; @@ -23,6 +24,7 @@ pub struct Query( contract::ContractQuery, contract::ContractBalanceQuery, node_info::NodeQuery, + message::MessageQuery, ); #[derive(MergedObject, Default)] @@ -39,6 +41,6 @@ pub fn build_schema() -> SchemaBuilder { Query::default(), Mutation::default(), EmptySubscription::default(), - ["TransactionConnection"], + ["TransactionConnection", "MessageConnection"], ) } diff --git a/fuel-core/src/schema/balance.rs b/fuel-core/src/schema/balance.rs index fd5a9c858c0..8a02308fb08 100644 --- a/fuel-core/src/schema/balance.rs +++ b/fuel-core/src/schema/balance.rs @@ -2,6 +2,7 @@ use crate::database::{Database, KvStoreError}; use crate::model::{Coin as CoinModel, CoinStatus}; use crate::schema::scalars::{Address, AssetId, U64}; use crate::state::{Error, IterDirection}; +use anyhow::anyhow; use async_graphql::InputObject; use async_graphql::{ connection::{query, Connection, Edge, EmptyFields}, @@ -143,6 +144,13 @@ impl BalanceQuery { (0, IterDirection::Forward) }; + if (first.is_some() && before.is_some()) + || (after.is_some() && before.is_some()) + || (last.is_some() && after.is_some()) + { + return Err(anyhow!("Wrong argument combination")); + } + let after = after.map(fuel_tx::AssetId::from); let before = before.map(fuel_tx::AssetId::from); @@ -199,7 +207,7 @@ impl BalanceQuery { .map(|item| Edge::new(item.asset_id.into(), item)), ); - Ok::, KvStoreError>(connection) + Ok::, anyhow::Error>(connection) }, ) .await diff --git a/fuel-core/src/schema/block.rs b/fuel-core/src/schema/block.rs index 970eac22d2c..fd7beed661f 100644 --- a/fuel-core/src/schema/block.rs +++ b/fuel-core/src/schema/block.rs @@ -1,10 +1,10 @@ -use crate::config::Config; use crate::database::Database; use crate::executor::{ExecutionMode, Executor}; use crate::schema::{ scalars::{BlockId, U64}, tx::types::Transaction, }; +use crate::service::Config; use crate::{ database::KvStoreError, model::{BlockHeight, FuelBlock, FuelBlockDb, FuelBlockHeader}, @@ -121,6 +121,13 @@ impl BlockQuery { (0, IterDirection::Forward) }; + if (first.is_some() && before.is_some()) + || (after.is_some() && before.is_some()) + || (last.is_some() && after.is_some()) + { + return Err(anyhow!("Wrong argument combination")); + } + let start; let end; @@ -172,7 +179,7 @@ impl BlockQuery { Edge::new(item.headers.height.to_usize(), Block(item.into_owned())) })); - Ok::, KvStoreError>(connection) + Ok::, anyhow::Error>(connection) }, ) .await diff --git a/fuel-core/src/schema/chain.rs b/fuel-core/src/schema/chain.rs index c584b313b3d..3a4e9abe8e9 100644 --- a/fuel-core/src/schema/chain.rs +++ b/fuel-core/src/schema/chain.rs @@ -1,6 +1,6 @@ use crate::{ - config::Config, database::Database, model::FuelBlockDb, schema::block::Block, - schema::scalars::U64, + database::Database, model::FuelBlockDb, schema::block::Block, schema::scalars::U64, + service::Config, }; use async_graphql::{Context, Object}; use fuel_core_interfaces::common::{fuel_storage::Storage, fuel_tx, fuel_types}; @@ -41,10 +41,6 @@ impl ConsensusParameters { self.0.max_script_data_length.into() } - async fn max_static_contracts(&self) -> U64 { - self.0.max_static_contracts.into() - } - async fn max_storage_slots(&self) -> U64 { self.0.max_storage_slots.into() } @@ -60,6 +56,14 @@ impl ConsensusParameters { async fn gas_price_factor(&self) -> U64 { self.0.gas_price_factor.into() } + + async fn gas_per_byte(&self) -> U64 { + self.0.gas_per_byte.into() + } + + async fn max_message_data_length(&self) -> U64 { + self.0.max_message_data_length.into() + } } #[Object] diff --git a/fuel-core/src/schema/coin.rs b/fuel-core/src/schema/coin.rs index bdc3d5c948e..9f7bdc59940 100644 --- a/fuel-core/src/schema/coin.rs +++ b/fuel-core/src/schema/coin.rs @@ -1,10 +1,11 @@ use crate::{ coin_query::{random_improve, SpendQueryElement}, - config::Config, database::{Database, KvStoreError}, schema::scalars::{Address, AssetId, UtxoId, U64}, + service::Config, state::IterDirection, }; +use anyhow::anyhow; use async_graphql::{ connection::{query, Connection, Edge, EmptyFields}, Context, Enum, InputObject, Object, @@ -113,6 +114,13 @@ impl CoinQuery { (0, IterDirection::Forward) }; + if (first.is_some() && before.is_some()) + || (after.is_some() && before.is_some()) + || (last.is_some() && after.is_some()) + { + return Err(anyhow!("Wrong argument combination")); + } + let after = after.map(fuel_tx::UtxoId::from); let before = before.map(fuel_tx::UtxoId::from); @@ -181,7 +189,7 @@ impl CoinQuery { .map(|item| Edge::new(UtxoId::from(item.0), item)), ); - Ok::, KvStoreError>(connection) + Ok::, anyhow::Error>(connection) }, ) .await diff --git a/fuel-core/src/schema/dap.rs b/fuel-core/src/schema/dap.rs index 771a4d3ff77..28a66ec283b 100644 --- a/fuel-core/src/schema/dap.rs +++ b/fuel-core/src/schema/dap.rs @@ -46,11 +46,16 @@ impl ConcreteStorage { &mut self, txs: &[Transaction], storage: DatabaseTransaction, - ) -> Result { + ) -> anyhow::Result { let id = Uuid::new_v4(); let id = ID::from(id); let tx = txs.first().cloned().unwrap_or_default(); + let checked_tx = CheckedTransaction::check_unsigned( + tx, + storage.get_block_height()?.unwrap_or_default().into(), + &self.params, + )?; self.tx .get_mut(&id) .map(|tx| tx.extend_from_slice(txs)) @@ -59,7 +64,7 @@ impl ConcreteStorage { }); let mut vm = Interpreter::with_storage(storage.as_ref().clone(), self.params); - vm.transact(tx)?; + vm.transact(checked_tx)?; self.vm.insert(id.clone(), vm); self.db.insert(id.clone(), storage); @@ -72,7 +77,7 @@ impl ConcreteStorage { self.db.remove(id).is_some() } - pub fn reset(&mut self, id: &ID, storage: DatabaseTransaction) -> Result<(), InterpreterError> { + pub fn reset(&mut self, id: &ID, storage: DatabaseTransaction) -> anyhow::Result<()> { let tx = self .tx .get(id) @@ -80,8 +85,14 @@ impl ConcreteStorage { .cloned() .unwrap_or_default(); + let checked_tx = CheckedTransaction::check_unsigned( + tx, + storage.get_block_height()?.unwrap_or_default().into(), + &self.params, + )?; + let mut vm = Interpreter::with_storage(storage.as_ref().clone(), self.params); - vm.transact(tx)?; + vm.transact(checked_tx)?; self.vm.insert(id.clone(), vm).ok_or_else(|| { InterpreterError::Io(io::Error::new( io::ErrorKind::NotFound, @@ -281,13 +292,22 @@ impl DapMutation { .map_err(|_| async_graphql::Error::new("Invalid transaction JSON"))?; let mut locked = ctx.data_unchecked::().lock().await; + + let db = locked.db.get(&id).ok_or("Invalid debugging session ID")?; + + let checked_tx = CheckedTransaction::check_unsigned( + tx, + db.get_block_height()?.unwrap_or_default().into(), + &locked.params, + )?; + let vm = locked .vm .get_mut(&id) .ok_or_else(|| async_graphql::Error::new("VM not found"))?; let state_ref = vm - .transact(tx) + .transact(checked_tx) .map_err(|err| async_graphql::Error::new(format!("Transaction failed: {err:?}")))?; let json_receipts = state_ref diff --git a/fuel-core/src/schema/message.rs b/fuel-core/src/schema/message.rs new file mode 100644 index 00000000000..c94b3213fee --- /dev/null +++ b/fuel-core/src/schema/message.rs @@ -0,0 +1,151 @@ +use super::scalars::{Address, MessageId, U64}; +use crate::{database::Database, state::IterDirection}; +use anyhow::anyhow; +use async_graphql::{ + connection::{self, Connection, Edge, EmptyFields}, + Context, Object, +}; +use fuel_core_interfaces::{ + common::{fuel_storage::Storage, fuel_types}, + db::KvStoreError, + model, +}; +use itertools::Itertools; + +pub struct Message(pub(crate) model::Message); + +#[Object] +impl Message { + async fn amount(&self) -> U64 { + self.0.amount.into() + } + + async fn sender(&self) -> Address { + self.0.sender.into() + } + + async fn recipient(&self) -> Address { + self.0.recipient.into() + } + + async fn owner(&self) -> Address { + self.0.owner.into() + } + + async fn nonce(&self) -> U64 { + self.0.nonce.into() + } + + async fn data(&self) -> &Vec { + &self.0.data + } + + async fn da_height(&self) -> U64 { + self.0.da_height.into() + } + + async fn fuel_block_spend(&self) -> Option { + self.0.fuel_block_spend.map(|v| v.into()) + } +} + +#[derive(Default)] +pub struct MessageQuery {} + +#[Object] +impl MessageQuery { + async fn messages( + &self, + ctx: &Context<'_>, + #[graphql(desc = "address of the owner")] owner: Option
, + first: Option, + after: Option, + last: Option, + before: Option, + ) -> async_graphql::Result> { + let db = ctx.data_unchecked::().clone(); + + connection::query( + after, + before, + first, + last, + |after: Option, before: Option, first, last| async move { + let (records_to_fetch, direction) = if let Some(first) = first { + (first, IterDirection::Forward) + } else if let Some(last) = last { + (last, IterDirection::Reverse) + } else { + (0, IterDirection::Forward) + }; + + if (first.is_some() && before.is_some()) + || (after.is_some() && before.is_some()) + || (last.is_some() && after.is_some()) + { + return Err(anyhow!("Wrong argument combination")); + } + + let start = if direction == IterDirection::Forward { + after + } else { + before + }; + + let (mut messages, has_next_page, has_previous_page) = if let Some(owner) = owner { + let mut message_ids = + db.owned_message_ids(owner.into(), start.map(Into::into), Some(direction)); + let mut started = None; + if start.is_some() { + // skip initial result + started = message_ids.next(); + } + let message_ids = message_ids.take(records_to_fetch + 1); + let message_ids: Vec = message_ids.try_collect()?; + let has_next_page = message_ids.len() > records_to_fetch; + + let messages: Vec = message_ids + .iter() + .take(records_to_fetch) + .map(|msg_id| { + Storage::::get(&db, msg_id) + .transpose() + .ok_or(KvStoreError::NotFound)? + .map(|f| f.into_owned()) + }) + .try_collect()?; + (messages, has_next_page, started.is_some()) + } else { + let mut messages = db.all_messages(start.map(Into::into), Some(direction)); + let mut started = None; + if start.is_some() { + // skip initial result + started = messages.next(); + } + let messages: Vec = + messages.take(records_to_fetch + 1).try_collect()?; + let has_next_page = messages.len() > records_to_fetch; + let messages = messages.into_iter().take(records_to_fetch).collect(); + (messages, has_next_page, started.is_some()) + }; + + // reverse after filtering next page test record to maintain consistent ordering + // in the response regardless of whether first or last was used. + if direction == IterDirection::Forward { + messages.reverse(); + } + + let mut connection = Connection::new(has_previous_page, has_next_page); + + connection.edges.extend( + messages + .into_iter() + .map(|message| Edge::new(message.id().into(), Message(message))), + ); + + Ok::, anyhow::Error>(connection) + }, + ) + .await + } +} diff --git a/fuel-core/src/schema/node_info.rs b/fuel-core/src/schema/node_info.rs index a32c9d2ca9a..6f8862eb939 100644 --- a/fuel-core/src/schema/node_info.rs +++ b/fuel-core/src/schema/node_info.rs @@ -1,5 +1,5 @@ use super::scalars::U64; -use crate::config::Config; +use crate::service::Config; use async_graphql::{Context, Object}; pub struct NodeInfo { @@ -7,7 +7,6 @@ pub struct NodeInfo { predicates: bool, vm_backtrace: bool, min_gas_price: U64, - min_byte_price: U64, max_tx: U64, max_depth: U64, node_version: String, @@ -31,10 +30,6 @@ impl NodeInfo { self.min_gas_price } - async fn min_byte_price(&self) -> U64 { - self.min_byte_price - } - async fn max_tx(&self) -> U64 { self.max_tx } @@ -69,7 +64,6 @@ impl NodeQuery { predicates: *predicates, vm_backtrace: vm.backtrace, min_gas_price: txpool.min_gas_price.into(), - min_byte_price: txpool.min_byte_price.into(), max_tx: (txpool.max_tx as u64).into(), max_depth: (txpool.max_depth as u64).into(), node_version: VERSION.to_owned(), diff --git a/fuel-core/src/schema/scalars.rs b/fuel-core/src/schema/scalars.rs index 18e4a45f4e5..42aca0008af 100644 --- a/fuel-core/src/schema/scalars.rs +++ b/fuel-core/src/schema/scalars.rs @@ -3,12 +3,17 @@ use crate::model::BlockHeight; use async_graphql::{ connection::CursorType, InputValueError, InputValueResult, Scalar, ScalarType, Value, }; -use fuel_core_interfaces::common::{fuel_tx, fuel_types}; +use fuel_core_interfaces::common::fuel_types; use std::{ convert::TryInto, fmt::{Display, Formatter}, str::FromStr, }; +pub use tx_pointer::TxPointer; +pub use utxo_id::UtxoId; + +pub mod tx_pointer; +pub mod utxo_id; /// Need our own u64 type since GraphQL integers are restricted to i32. #[derive(Copy, Clone, Debug, derive_more::Into, derive_more::From)] @@ -136,65 +141,6 @@ impl FromStr for HexString { } } -#[derive(Clone, Copy, Debug)] -pub struct UtxoId(pub(crate) fuel_tx::UtxoId); - -#[Scalar(name = "UtxoId")] -impl ScalarType for UtxoId { - fn parse(value: Value) -> InputValueResult { - if let Value::String(value) = &value { - UtxoId::from_str(value.as_str()).map_err(Into::into) - } else { - Err(InputValueError::expected_type(value)) - } - } - - fn to_value(&self) -> Value { - Value::String(self.to_string()) - } -} - -impl FromStr for UtxoId { - type Err = String; - - fn from_str(s: &str) -> Result { - fuel_tx::UtxoId::from_str(s) - .map(Self) - .map_err(str::to_owned) - } -} - -impl Display for UtxoId { - fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { - let s = format!("{:#x}", self.0); - s.fmt(f) - } -} - -impl From for fuel_tx::UtxoId { - fn from(s: UtxoId) -> Self { - s.0 - } -} - -impl From for UtxoId { - fn from(utxo_id: fuel_tx::UtxoId) -> Self { - Self(utxo_id) - } -} - -impl CursorType for UtxoId { - type Error = String; - - fn decode_cursor(s: &str) -> Result { - Self::from_str(s) - } - - fn encode_cursor(&self) -> String { - self.to_string() - } -} - macro_rules! fuel_type_scalar { ($name:literal, $id:ident, $ft_id:ident, $len:expr) => { #[derive(Copy, Clone, Debug)] @@ -274,3 +220,4 @@ fuel_type_scalar!("AssetId", AssetId, AssetId, 32); fuel_type_scalar!("ContractId", ContractId, ContractId, 32); fuel_type_scalar!("Salt", Salt, Salt, 32); fuel_type_scalar!("TransactionId", TransactionId, Bytes32, 32); +fuel_type_scalar!("MessageId", MessageId, MessageId, 32); diff --git a/fuel-core/src/schema/scalars/tx_pointer.rs b/fuel-core/src/schema/scalars/tx_pointer.rs new file mode 100644 index 00000000000..75157d66622 --- /dev/null +++ b/fuel-core/src/schema/scalars/tx_pointer.rs @@ -0,0 +1,67 @@ +use async_graphql::{ + connection::CursorType, InputValueError, InputValueResult, Scalar, ScalarType, Value, +}; +use fuel_core_interfaces::common::fuel_tx; +use std::{ + fmt::{Display, Formatter}, + str::FromStr, +}; + +#[derive(Clone, Copy, Debug)] +pub struct TxPointer(pub(crate) fuel_tx::TxPointer); + +#[Scalar(name = "TxPointer")] +impl ScalarType for TxPointer { + fn parse(value: Value) -> InputValueResult { + if let Value::String(value) = &value { + TxPointer::from_str(value.as_str()).map_err(Into::into) + } else { + Err(InputValueError::expected_type(value)) + } + } + + fn to_value(&self) -> Value { + Value::String(self.to_string()) + } +} + +impl FromStr for TxPointer { + type Err = String; + + fn from_str(s: &str) -> Result { + fuel_tx::TxPointer::from_str(s) + .map(Self) + .map_err(str::to_owned) + } +} + +impl Display for TxPointer { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + let s = format!("{:#x}", self.0); + s.fmt(f) + } +} + +impl From for fuel_tx::TxPointer { + fn from(s: TxPointer) -> Self { + s.0 + } +} + +impl From for TxPointer { + fn from(tx_pointer: fuel_tx::TxPointer) -> Self { + Self(tx_pointer) + } +} + +impl CursorType for TxPointer { + type Error = String; + + fn decode_cursor(s: &str) -> Result { + Self::from_str(s) + } + + fn encode_cursor(&self) -> String { + self.to_string() + } +} diff --git a/fuel-core/src/schema/scalars/utxo_id.rs b/fuel-core/src/schema/scalars/utxo_id.rs new file mode 100644 index 00000000000..47532a4eba4 --- /dev/null +++ b/fuel-core/src/schema/scalars/utxo_id.rs @@ -0,0 +1,67 @@ +use async_graphql::{ + connection::CursorType, InputValueError, InputValueResult, Scalar, ScalarType, Value, +}; +use fuel_core_interfaces::common::fuel_tx; +use std::{ + fmt::{Display, Formatter}, + str::FromStr, +}; + +#[derive(Clone, Copy, Debug)] +pub struct UtxoId(pub(crate) fuel_tx::UtxoId); + +#[Scalar(name = "UtxoId")] +impl ScalarType for UtxoId { + fn parse(value: Value) -> InputValueResult { + if let Value::String(value) = &value { + UtxoId::from_str(value.as_str()).map_err(Into::into) + } else { + Err(InputValueError::expected_type(value)) + } + } + + fn to_value(&self) -> Value { + Value::String(self.to_string()) + } +} + +impl FromStr for UtxoId { + type Err = String; + + fn from_str(s: &str) -> Result { + fuel_tx::UtxoId::from_str(s) + .map(Self) + .map_err(str::to_owned) + } +} + +impl Display for UtxoId { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + let s = format!("{:#x}", self.0); + s.fmt(f) + } +} + +impl From for fuel_tx::UtxoId { + fn from(s: UtxoId) -> Self { + s.0 + } +} + +impl From for UtxoId { + fn from(utxo_id: fuel_tx::UtxoId) -> Self { + Self(utxo_id) + } +} + +impl CursorType for UtxoId { + type Error = String; + + fn decode_cursor(s: &str) -> Result { + Self::from_str(s) + } + + fn encode_cursor(&self) -> String { + self.to_string() + } +} diff --git a/fuel-core/src/schema/tx.rs b/fuel-core/src/schema/tx.rs index a28b1c19600..e0b21cab433 100644 --- a/fuel-core/src/schema/tx.rs +++ b/fuel-core/src/schema/tx.rs @@ -1,9 +1,10 @@ -use crate::config::Config; use crate::database::{transaction::OwnedTransactionIndexCursor, Database, KvStoreError}; use crate::executor::Executor; use crate::model::{BlockHeight, FuelBlockDb}; use crate::schema::scalars::{Address, Bytes32, HexString, SortedTxCursor, TransactionId}; +use crate::service::Config; use crate::state::IterDirection; +use anyhow::anyhow; use async_graphql::{ connection::{query, Connection, Edge, EmptyFields}, Context, Object, @@ -82,6 +83,13 @@ impl TxQuery { (0, IterDirection::Forward) }; + if (first.is_some() && before.is_some()) + || (after.is_some() && before.is_some()) + || (last.is_some() && after.is_some()) + { + return Err(anyhow!("Wrong argument combination")); + } + let block_id; let tx_id; @@ -161,7 +169,7 @@ impl TxQuery { ) })); - Ok::, KvStoreError>(connection) + Ok::, anyhow::Error>(connection) }, ) .await @@ -193,6 +201,13 @@ impl TxQuery { (0, IterDirection::Forward) }; + if (first.is_some() && before.is_some()) + || (after.is_some() && before.is_some()) + || (last.is_some() && after.is_some()) + { + return Err(anyhow!("Wrong argument combination")); + } + let after = after.map(OwnedTransactionIndexCursor::from); let before = before.map(OwnedTransactionIndexCursor::from); @@ -246,7 +261,7 @@ impl TxQuery { .map(|item| Edge::new(HexString::from(item.0), Transaction(item.1))), ); - Ok::, KvStoreError>(connection) + Ok::, anyhow::Error>(connection) }, ) .await diff --git a/fuel-core/src/schema/tx/input.rs b/fuel-core/src/schema/tx/input.rs index d32c12813ed..f86c3d66754 100644 --- a/fuel-core/src/schema/tx/input.rs +++ b/fuel-core/src/schema/tx/input.rs @@ -1,23 +1,26 @@ +use crate::schema::scalars::TxPointer; use crate::schema::{ contract::Contract, - scalars::{Address, AssetId, Bytes32, ContractId, HexString, UtxoId, U64}, + scalars::{Address, AssetId, Bytes32, ContractId, HexString, MessageId, UtxoId, U64}, }; use async_graphql::{Object, Union}; -use fuel_core_interfaces::common::{fuel_asm::Word, fuel_tx}; +use fuel_core_interfaces::common::fuel_tx; #[derive(Union)] pub enum Input { Coin(InputCoin), Contract(InputContract), + Message(InputMessage), } pub struct InputCoin { utxo_id: UtxoId, owner: Address, - amount: Word, + amount: U64, asset_id: AssetId, + tx_pointer: TxPointer, witness_index: u8, - maturity: Word, + maturity: U64, predicate: HexString, predicate_data: HexString, } @@ -33,19 +36,23 @@ impl InputCoin { } async fn amount(&self) -> U64 { - self.amount.into() + self.amount } async fn asset_id(&self) -> AssetId { self.asset_id } + async fn tx_pointer(&self) -> TxPointer { + self.tx_pointer + } + async fn witness_index(&self) -> u8 { self.witness_index } async fn maturity(&self) -> U64 { - self.maturity.into() + self.maturity } async fn predicate(&self) -> HexString { @@ -61,6 +68,7 @@ pub struct InputContract { utxo_id: UtxoId, balance_root: Bytes32, state_root: Bytes32, + tx_pointer: TxPointer, contract_id: ContractId, } @@ -78,11 +86,71 @@ impl InputContract { self.state_root } + async fn tx_pointer(&self) -> TxPointer { + self.tx_pointer + } + async fn contract(&self) -> Contract { self.contract_id.0.into() } } +pub struct InputMessage { + message_id: MessageId, + sender: Address, + recipient: Address, + amount: U64, + nonce: U64, + owner: Address, + witness_index: u8, + data: HexString, + predicate: HexString, + predicate_data: HexString, +} + +#[Object] +impl InputMessage { + async fn message_id(&self) -> MessageId { + self.message_id + } + + async fn sender(&self) -> Address { + self.sender + } + + async fn recipient(&self) -> Address { + self.recipient + } + + async fn amount(&self) -> U64 { + self.amount + } + + async fn nonce(&self) -> U64 { + self.nonce + } + + async fn owner(&self) -> Address { + self.owner + } + + async fn witness_index(&self) -> u8 { + self.witness_index + } + + async fn data(&self) -> &HexString { + &self.data + } + + async fn predicate(&self) -> &HexString { + &self.predicate + } + + async fn predicate_data(&self) -> &HexString { + &self.predicate_data + } +} + impl From<&fuel_tx::Input> for Input { fn from(input: &fuel_tx::Input) -> Self { match input { @@ -91,15 +159,17 @@ impl From<&fuel_tx::Input> for Input { owner, amount, asset_id, + tx_pointer, witness_index, maturity, } => Input::Coin(InputCoin { utxo_id: UtxoId(*utxo_id), owner: Address(*owner), - amount: *amount, + amount: (*amount).into(), asset_id: AssetId(*asset_id), + tx_pointer: TxPointer(*tx_pointer), witness_index: *witness_index, - maturity: *maturity, + maturity: (*maturity).into(), predicate: HexString(Default::default()), predicate_data: HexString(Default::default()), }), @@ -108,16 +178,18 @@ impl From<&fuel_tx::Input> for Input { owner, amount, asset_id, + tx_pointer, maturity, predicate, predicate_data, } => Input::Coin(InputCoin { utxo_id: UtxoId(*utxo_id), owner: Address(*owner), - amount: *amount, + amount: (*amount).into(), asset_id: AssetId(*asset_id), + tx_pointer: TxPointer(*tx_pointer), witness_index: Default::default(), - maturity: *maturity, + maturity: (*maturity).into(), predicate: HexString(predicate.clone()), predicate_data: HexString(predicate_data.clone()), }), @@ -125,13 +197,58 @@ impl From<&fuel_tx::Input> for Input { utxo_id, balance_root, state_root, + tx_pointer, contract_id, } => Input::Contract(InputContract { utxo_id: UtxoId(*utxo_id), balance_root: Bytes32(*balance_root), state_root: Bytes32(*state_root), + tx_pointer: TxPointer(*tx_pointer), contract_id: ContractId(*contract_id), }), + fuel_tx::Input::MessageSigned { + message_id, + sender, + recipient, + amount, + nonce, + owner, + witness_index, + data, + } => Input::Message(InputMessage { + message_id: MessageId(*message_id), + sender: Address(*sender), + recipient: Address(*recipient), + amount: (*amount).into(), + nonce: (*nonce).into(), + owner: Address(*owner), + witness_index: *witness_index, + data: HexString(data.clone()), + predicate: HexString(Default::default()), + predicate_data: HexString(Default::default()), + }), + fuel_tx::Input::MessagePredicate { + message_id, + sender, + recipient, + amount, + nonce, + owner, + data, + predicate, + predicate_data, + } => Input::Message(InputMessage { + message_id: MessageId(*message_id), + sender: Address(*sender), + recipient: Address(*recipient), + amount: (*amount).into(), + nonce: (*nonce).into(), + owner: Address(*owner), + witness_index: Default::default(), + data: HexString(data.clone()), + predicate: HexString(predicate.clone()), + predicate_data: HexString(predicate_data.clone()), + }), } } } diff --git a/fuel-core/src/schema/tx/output.rs b/fuel-core/src/schema/tx/output.rs index 9793b81d9ab..75d419a5438 100644 --- a/fuel-core/src/schema/tx/output.rs +++ b/fuel-core/src/schema/tx/output.rs @@ -7,7 +7,7 @@ use fuel_core_interfaces::common::{fuel_asm::Word, fuel_tx, fuel_types}; pub enum Output { Coin(CoinOutput), Contract(ContractOutput), - Withdrawal(WithdrawalOutput), + Message(MessageOutput), Change(ChangeOutput), Variable(VariableOutput), ContractCreated(ContractCreated), @@ -34,20 +34,19 @@ impl CoinOutput { } } -pub struct WithdrawalOutput(CoinOutput); +pub struct MessageOutput { + amount: Word, + recipient: fuel_types::Address, +} #[Object] -impl WithdrawalOutput { - async fn to(&self) -> Address { - self.0.to.into() +impl MessageOutput { + async fn recipient(&self) -> Address { + self.recipient.into() } async fn amount(&self) -> U64 { - self.0.amount.into() - } - - async fn asset_id(&self) -> AssetId { - self.0.asset_id.into() + self.amount.into() } } @@ -143,15 +142,10 @@ impl From<&fuel_tx::Output> for Output { balance_root: *balance_root, state_root: *state_root, }), - fuel_tx::Output::Withdrawal { - to, - amount, - asset_id, - } => Output::Withdrawal(WithdrawalOutput(CoinOutput { - to: *to, + fuel_tx::Output::Message { recipient, amount } => Output::Message(MessageOutput { + recipient: *recipient, amount: *amount, - asset_id: *asset_id, - })), + }), fuel_tx::Output::Change { to, amount, diff --git a/fuel-core/src/schema/tx/receipt.rs b/fuel-core/src/schema/tx/receipt.rs index 3f8024a68e7..95df42baa4e 100644 --- a/fuel-core/src/schema/tx/receipt.rs +++ b/fuel-core/src/schema/tx/receipt.rs @@ -1,3 +1,4 @@ +use crate::schema::scalars::MessageId; use crate::schema::{ contract::Contract, scalars::{Address, AssetId, Bytes32, HexString, U64}, @@ -18,6 +19,7 @@ pub enum ReceiptType { Transfer, TransferOut, ScriptResult, + MessageOut, } impl From<&fuel_tx::Receipt> for ReceiptType { @@ -33,6 +35,7 @@ impl From<&fuel_tx::Receipt> for ReceiptType { fuel_tx::Receipt::Transfer { .. } => ReceiptType::Transfer, fuel_tx::Receipt::TransferOut { .. } => ReceiptType::TransferOut, fuel_tx::Receipt::ScriptResult { .. } => ReceiptType::ScriptResult, + fuel_tx::Receipt::MessageOut { .. } => ReceiptType::MessageOut, } } } @@ -113,6 +116,18 @@ impl Receipt { async fn data(&self) -> Option { self.0.data().map(|d| d.to_vec().into()) } + async fn message_id(&self) -> Option { + self.0.message_id().copied().map(MessageId) + } + async fn sender(&self) -> Option
{ + self.0.sender().copied().map(Address) + } + async fn recipient(&self) -> Option
{ + self.0.recipient().copied().map(Address) + } + async fn nonce(&self) -> Option { + self.0.nonce().copied().map(Bytes32) + } } impl From<&fuel_tx::Receipt> for Receipt { diff --git a/fuel-core/src/schema/tx/types.rs b/fuel-core/src/schema/tx/types.rs index f78f5d99e17..97d540ede31 100644 --- a/fuel-core/src/schema/tx/types.rs +++ b/fuel-core/src/schema/tx/types.rs @@ -194,10 +194,6 @@ impl Transaction { self.0.gas_limit().into() } - async fn byte_price(&self) -> U64 { - self.0.byte_price().into() - } - async fn maturity(&self) -> U64 { self.0.maturity().into() } @@ -295,15 +291,6 @@ impl Transaction { } } - async fn static_contracts(&self) -> Option> { - match &self.0 { - fuel_tx::Transaction::Script { .. } => None, - fuel_tx::Transaction::Create { - static_contracts, .. - } => Some(static_contracts.iter().cloned().map(Into::into).collect()), - } - } - async fn storage_slots(&self) -> Option> { match &self.0 { fuel_tx::Transaction::Script { .. } => None, diff --git a/fuel-core/src/service.rs b/fuel-core/src/service.rs index c6c0c2eec48..7ad8c1b6542 100644 --- a/fuel-core/src/service.rs +++ b/fuel-core/src/service.rs @@ -1,7 +1,4 @@ -use crate::{ - config::{Config, DbType}, - database::Database, -}; +use crate::database::Database; use anyhow::Error as AnyError; use modules::Modules; use std::{net::SocketAddr, panic}; @@ -9,6 +6,9 @@ use thiserror::Error; use tokio::task::JoinHandle; use tracing::log::warn; +pub use config::{Config, DbType, VMConfig}; + +pub mod config; pub(crate) mod genesis; pub mod graph_api; pub mod metrics; diff --git a/fuel-core/src/config.rs b/fuel-core/src/service/config.rs similarity index 96% rename from fuel-core/src/config.rs rename to fuel-core/src/service/config.rs index c77769cd3f6..59b699ad316 100644 --- a/fuel-core/src/config.rs +++ b/fuel-core/src/service/config.rs @@ -1,12 +1,10 @@ -pub mod chain_config; -pub mod serialization; - -use chain_config::ChainConfig; +use crate::chain_config::ChainConfig; use std::{ net::{Ipv4Addr, SocketAddr}, path::PathBuf, }; use strum_macros::{Display, EnumString, EnumVariantNames}; + #[derive(Clone, Debug)] pub struct Config { pub addr: SocketAddr, diff --git a/fuel-core/src/service/genesis.rs b/fuel-core/src/service/genesis.rs index db2a008a8b0..311880beab8 100644 --- a/fuel-core/src/service/genesis.rs +++ b/fuel-core/src/service/genesis.rs @@ -1,16 +1,14 @@ use crate::{ - config::{ - chain_config::{ContractConfig, StateConfig}, - Config, - }, + chain_config::{ContractConfig, StateConfig}, database::Database, - service::FuelService, + service::{config::Config, FuelService}, }; use anyhow::Result; +use fuel_core_interfaces::model::Message; use fuel_core_interfaces::{ common::{ fuel_storage::{MerkleStorage, Storage}, - fuel_tx::{Contract, UtxoId}, + fuel_tx::{Contract, MessageId, UtxoId}, fuel_types::{bytes::WORD_SIZE, AssetId, Bytes32, ContractId, Salt, Word}, }, model::{Coin, CoinStatus}, @@ -32,6 +30,7 @@ impl FuelService { if let Some(initial_state) = &config.chain_conf.initial_state { Self::init_coin_state(database, initial_state)?; Self::init_contracts(database, initial_state)?; + Self::init_da_messages(database, initial_state)?; } } @@ -137,6 +136,27 @@ impl FuelService { Ok(()) } + fn init_da_messages(db: &mut Database, state: &StateConfig) -> Result<()> { + if let Some(message_state) = &state.messages { + for msg in message_state { + let message = Message { + sender: msg.sender, + recipient: msg.recipient, + owner: msg.owner, + nonce: msg.nonce, + amount: msg.amount, + data: msg.data.clone(), + da_height: msg.da_height, + fuel_block_spend: None, + }; + + Storage::::insert(db, &message.id(), &message)?; + } + } + + Ok(()) + } + fn init_contract_balance( db: &mut Database, contract_id: &ContractId, @@ -154,14 +174,19 @@ impl FuelService { #[cfg(test)] mod tests { + use std::vec; + use super::*; - use crate::config::chain_config::{ChainConfig, CoinConfig, ContractConfig, StateConfig}; - use crate::config::Config; + use crate::chain_config::{ + ChainConfig, CoinConfig, ContractConfig, MessageConfig, StateConfig, + }; use crate::model::BlockHeight; + use crate::service::config::Config; use fuel_core_interfaces::common::{ fuel_asm::Opcode, fuel_types::{Address, AssetId, Word}, }; + use fuel_core_interfaces::model::Message; use itertools::Itertools; use rand::{rngs::StdRng, Rng, RngCore, SeedableRng}; @@ -352,6 +377,40 @@ mod tests { assert_eq!(test_value, ret) } + #[tokio::test] + async fn tests_init_da_msgs() { + let mut rng = StdRng::seed_from_u64(32492); + let mut config = Config::local_node(); + + let msg = MessageConfig { + sender: rng.gen(), + recipient: rng.gen(), + owner: rng.gen(), + nonce: rng.gen(), + amount: rng.gen(), + data: vec![rng.gen()], + da_height: 0, + }; + + config.chain_conf.initial_state = Some(StateConfig { + messages: Some(vec![msg.clone()]), + ..Default::default() + }); + + let db = Database::default(); + + FuelService::initialize_state(&config, &db).unwrap(); + + let expected_msg: Message = msg.into(); + + let ret_msg = Storage::::get(&db, &expected_msg.id()) + .unwrap() + .unwrap() + .into_owned(); + + assert_eq!(expected_msg, ret_msg); + } + #[tokio::test] async fn config_state_initializes_contract_balance() { let mut rng = StdRng::seed_from_u64(10); diff --git a/fuel-core/src/service/graph_api.rs b/fuel-core/src/service/graph_api.rs index a4d63e06ec3..f0b4f4c746f 100644 --- a/fuel-core/src/service/graph_api.rs +++ b/fuel-core/src/service/graph_api.rs @@ -1,8 +1,8 @@ use super::modules::Modules; -use crate::config::Config; use crate::database::Database; use crate::schema::{build_schema, dap, CoreSchema}; use crate::service::metrics::metrics; +use crate::service::Config; use anyhow::Result; use async_graphql::{ extensions::Tracing, http::playground_source, http::GraphQLPlaygroundConfig, Request, Response, diff --git a/fuel-core/src/service/modules.rs b/fuel-core/src/service/modules.rs index bb663780464..e86cebec1dd 100644 --- a/fuel-core/src/service/modules.rs +++ b/fuel-core/src/service/modules.rs @@ -1,6 +1,6 @@ #![allow(clippy::let_unit_value)] -use crate::config::Config; use crate::database::Database; +use crate::service::Config; use anyhow::Result; use fuel_core_interfaces::p2p::P2pDb; use fuel_core_interfaces::relayer::RelayerDb; diff --git a/fuel-core/src/config/snapshots/fuel_core__config__chain_config__tests__snapshot_configurable_block_height.snap b/fuel-core/src/snapshots/fuel_core__chain_config__tests__snapshot_configurable_block_height.snap similarity index 77% rename from fuel-core/src/config/snapshots/fuel_core__config__chain_config__tests__snapshot_configurable_block_height.snap rename to fuel-core/src/snapshots/fuel_core__chain_config__tests__snapshot_configurable_block_height.snap index 8fc163abebb..41efdefdbda 100644 --- a/fuel-core/src/config/snapshots/fuel_core__config__chain_config__tests__snapshot_configurable_block_height.snap +++ b/fuel-core/src/snapshots/fuel_core__chain_config__tests__snapshot_configurable_block_height.snap @@ -1,5 +1,6 @@ --- -source: fuel-core/src/config/chain_config.rs +source: fuel-core/src/genesis.rs +assertion_line: 223 expression: json --- { @@ -16,10 +17,11 @@ expression: json "max_gas_per_tx": 100000000, "max_script_length": 1048576, "max_script_data_length": 1048576, - "max_static_contracts": 255, "max_storage_slots": 255, "max_predicate_length": 1048576, "max_predicate_data_length": 1048576, - "gas_price_factor": 1000000000 + "gas_price_factor": 1000000000, + "gas_per_byte": 4, + "max_message_data_length": 1048576 } } diff --git a/fuel-core/src/config/snapshots/fuel_core__config__chain_config__tests__snapshot_contract_with_balances.snap b/fuel-core/src/snapshots/fuel_core__chain_config__tests__snapshot_contract_with_balances.snap similarity index 84% rename from fuel-core/src/config/snapshots/fuel_core__config__chain_config__tests__snapshot_contract_with_balances.snap rename to fuel-core/src/snapshots/fuel_core__chain_config__tests__snapshot_contract_with_balances.snap index cb42771ffe2..7682b756d56 100644 --- a/fuel-core/src/config/snapshots/fuel_core__config__chain_config__tests__snapshot_contract_with_balances.snap +++ b/fuel-core/src/snapshots/fuel_core__chain_config__tests__snapshot_contract_with_balances.snap @@ -1,5 +1,6 @@ --- -source: fuel-core/src/config/chain_config.rs +source: fuel-core/src/genesis.rs +assertion_line: 275 expression: json --- { @@ -27,10 +28,11 @@ expression: json "max_gas_per_tx": 100000000, "max_script_length": 1048576, "max_script_data_length": 1048576, - "max_static_contracts": 255, "max_storage_slots": 255, "max_predicate_length": 1048576, "max_predicate_data_length": 1048576, - "gas_price_factor": 1000000000 + "gas_price_factor": 1000000000, + "gas_per_byte": 4, + "max_message_data_length": 1048576 } } diff --git a/fuel-core/src/config/snapshots/fuel_core__config__chain_config__tests__snapshot_contract_with_state.snap b/fuel-core/src/snapshots/fuel_core__chain_config__tests__snapshot_contract_with_state.snap similarity index 84% rename from fuel-core/src/config/snapshots/fuel_core__config__chain_config__tests__snapshot_contract_with_state.snap rename to fuel-core/src/snapshots/fuel_core__chain_config__tests__snapshot_contract_with_state.snap index 22bbd285e05..6465f021af4 100644 --- a/fuel-core/src/config/snapshots/fuel_core__config__chain_config__tests__snapshot_contract_with_state.snap +++ b/fuel-core/src/snapshots/fuel_core__chain_config__tests__snapshot_contract_with_state.snap @@ -1,5 +1,6 @@ --- -source: fuel-core/src/config/chain_config.rs +source: fuel-core/src/genesis.rs +assertion_line: 260 expression: json --- { @@ -27,10 +28,11 @@ expression: json "max_gas_per_tx": 100000000, "max_script_length": 1048576, "max_script_data_length": 1048576, - "max_static_contracts": 255, "max_storage_slots": 255, "max_predicate_length": 1048576, "max_predicate_data_length": 1048576, - "gas_price_factor": 1000000000 + "gas_price_factor": 1000000000, + "gas_per_byte": 4, + "max_message_data_length": 1048576 } } diff --git a/fuel-core/src/config/snapshots/fuel_core__config__chain_config__tests__snapshot_local_testnet_config.snap b/fuel-core/src/snapshots/fuel_core__chain_config__tests__snapshot_local_testnet_config.snap similarity index 91% rename from fuel-core/src/config/snapshots/fuel_core__config__chain_config__tests__snapshot_local_testnet_config.snap rename to fuel-core/src/snapshots/fuel_core__chain_config__tests__snapshot_local_testnet_config.snap index 38417bcce9d..87b69c139c5 100644 --- a/fuel-core/src/config/snapshots/fuel_core__config__chain_config__tests__snapshot_local_testnet_config.snap +++ b/fuel-core/src/snapshots/fuel_core__chain_config__tests__snapshot_local_testnet_config.snap @@ -1,5 +1,6 @@ --- -source: fuel-core/src/config/chain_config.rs +source: fuel-core/src/genesis.rs +assertion_line: 201 expression: json --- { @@ -42,10 +43,11 @@ expression: json "max_gas_per_tx": 100000000, "max_script_length": 1048576, "max_script_data_length": 1048576, - "max_static_contracts": 255, "max_storage_slots": 255, "max_predicate_length": 1048576, "max_predicate_data_length": 1048576, - "gas_price_factor": 1000000000 + "gas_price_factor": 1000000000, + "gas_per_byte": 4, + "max_message_data_length": 1048576 } } diff --git a/fuel-core/src/config/snapshots/fuel_core__config__chain_config__tests__snapshot_simple_coin_state.snap b/fuel-core/src/snapshots/fuel_core__chain_config__tests__snapshot_simple_coin_state.snap similarity index 86% rename from fuel-core/src/config/snapshots/fuel_core__config__chain_config__tests__snapshot_simple_coin_state.snap rename to fuel-core/src/snapshots/fuel_core__chain_config__tests__snapshot_simple_coin_state.snap index fc5fc3f8715..6b456b14dfe 100644 --- a/fuel-core/src/config/snapshots/fuel_core__config__chain_config__tests__snapshot_simple_coin_state.snap +++ b/fuel-core/src/snapshots/fuel_core__chain_config__tests__snapshot_simple_coin_state.snap @@ -1,5 +1,6 @@ --- -source: fuel-core/src/config/chain_config.rs +source: fuel-core/src/genesis.rs +assertion_line: 290 expression: json --- { @@ -26,10 +27,11 @@ expression: json "max_gas_per_tx": 100000000, "max_script_length": 1048576, "max_script_data_length": 1048576, - "max_static_contracts": 255, "max_storage_slots": 255, "max_predicate_length": 1048576, "max_predicate_data_length": 1048576, - "gas_price_factor": 1000000000 + "gas_price_factor": 1000000000, + "gas_per_byte": 4, + "max_message_data_length": 1048576 } } diff --git a/fuel-core/src/config/snapshots/fuel_core__config__chain_config__tests__snapshot_simple_contract.snap b/fuel-core/src/snapshots/fuel_core__chain_config__tests__snapshot_simple_contract.snap similarity index 80% rename from fuel-core/src/config/snapshots/fuel_core__config__chain_config__tests__snapshot_simple_contract.snap rename to fuel-core/src/snapshots/fuel_core__chain_config__tests__snapshot_simple_contract.snap index c8d83927cc6..ebf5dc510ff 100644 --- a/fuel-core/src/config/snapshots/fuel_core__config__chain_config__tests__snapshot_simple_contract.snap +++ b/fuel-core/src/snapshots/fuel_core__chain_config__tests__snapshot_simple_contract.snap @@ -1,5 +1,6 @@ --- -source: fuel-core/src/config/chain_config.rs +source: fuel-core/src/genesis.rs +assertion_line: 245 expression: json --- { @@ -21,10 +22,11 @@ expression: json "max_gas_per_tx": 100000000, "max_script_length": 1048576, "max_script_data_length": 1048576, - "max_static_contracts": 255, "max_storage_slots": 255, "max_predicate_length": 1048576, "max_predicate_data_length": 1048576, - "gas_price_factor": 1000000000 + "gas_price_factor": 1000000000, + "gas_per_byte": 4, + "max_message_data_length": 1048576 } } diff --git a/fuel-core/src/snapshots/fuel_core__chain_config__tests__snapshot_simple_message_state.snap b/fuel-core/src/snapshots/fuel_core__chain_config__tests__snapshot_simple_message_state.snap new file mode 100644 index 00000000000..146597b70fb --- /dev/null +++ b/fuel-core/src/snapshots/fuel_core__chain_config__tests__snapshot_simple_message_state.snap @@ -0,0 +1,37 @@ +--- +source: fuel-core/src/genesis.rs +assertion_line: 348 +expression: json +--- +{ + "chain_name": "local_testnet", + "block_production": "Instant", + "initial_state": { + "messages": [ + { + "sender": "0x61644a25da5ad31754e23bc1eca8c766a61ceaa8c60ddc65f158a30209256e35", + "recipient": "0xc9fba417a2d8d9ed086c0f3f03f94ec89a67bfdc6a1e22b69d667e5ac8055d00", + "owner": "0xa3f57027bcb10d053242a9719dca480b4f4910fa60671f2068086498c72e30fc", + "nonce": "0x0e1ef2963832068b", + "amount": "0xb04f3c08f59b309e", + "data": "0x39", + "da_height": "0x3322237b00655632" + } + ] + }, + "transaction_parameters": { + "contract_max_size": 16777216, + "max_inputs": 255, + "max_outputs": 255, + "max_witnesses": 255, + "max_gas_per_tx": 100000000, + "max_script_length": 1048576, + "max_script_data_length": 1048576, + "max_storage_slots": 255, + "max_predicate_length": 1048576, + "max_predicate_data_length": 1048576, + "gas_price_factor": 1000000000, + "gas_per_byte": 4, + "max_message_data_length": 1048576 + } +} diff --git a/fuel-core/src/state.rs b/fuel-core/src/state.rs index 6bba616b1e7..26cc8587308 100644 --- a/fuel-core/src/state.rs +++ b/fuel-core/src/state.rs @@ -40,6 +40,8 @@ impl, K2: AsRef<[u8]>> From> for Vec { } } +pub type KVItem = Result<(Vec, Vec)>; + pub trait KeyValueStore { fn get(&self, key: &[u8], column: ColumnId) -> Result>>; fn put(&self, key: Vec, column: ColumnId, value: Vec) -> Result>>; @@ -51,7 +53,7 @@ pub trait KeyValueStore { prefix: Option>, start: Option>, direction: IterDirection, - ) -> Box, Vec)> + '_>; + ) -> Box + '_>; } #[derive(Copy, Clone, Debug, PartialOrd, PartialEq)] diff --git a/fuel-core/src/state/in_memory/memory_store.rs b/fuel-core/src/state/in_memory/memory_store.rs index e9026095f0d..315dcd13c02 100644 --- a/fuel-core/src/state/in_memory/memory_store.rs +++ b/fuel-core/src/state/in_memory/memory_store.rs @@ -50,7 +50,7 @@ impl KeyValueStore for MemoryStore { prefix: Option>, start: Option>, direction: IterDirection, - ) -> Box, Vec)> + '_> { + ) -> Box, Vec)>> + '_> { // clone entire set so we can drop the lock let mut copy: Vec<(Vec, Vec)> = self .inner @@ -78,10 +78,11 @@ impl KeyValueStore for MemoryStore { if let Some(start) = start { Box::new( copy.into_iter() - .skip_while(move |(key, _)| key.as_slice() != start.as_slice()), + .skip_while(move |(key, _)| key.as_slice() != start.as_slice()) + .map(Ok), ) } else { - Box::new(copy.into_iter()) + Box::new(copy.into_iter().map(Ok)) } } } diff --git a/fuel-core/src/state/in_memory/transaction.rs b/fuel-core/src/state/in_memory/transaction.rs index 16acaef42ce..777a009cae5 100644 --- a/fuel-core/src/state/in_memory/transaction.rs +++ b/fuel-core/src/state/in_memory/transaction.rs @@ -4,6 +4,7 @@ use crate::state::{ TransactionError, TransactionResult, WriteOperation, }; use itertools::{EitherOrBoth, Itertools}; +use std::cmp::Ordering; use std::collections::HashMap; use std::fmt::Debug; use std::sync::{Arc, Mutex}; @@ -98,7 +99,7 @@ impl KeyValueStore for MemoryTransactionView { prefix: Option>, start: Option>, direction: IterDirection, - ) -> Box, Vec)> + '_> { + ) -> Box, Vec)>> + '_> { // iterate over inmemory + db while also filtering deleted entries let changes = self.changes.clone(); Box::new( @@ -109,10 +110,19 @@ impl KeyValueStore for MemoryTransactionView { .merge_join_by( self.data_source.iter_all(column, prefix, start, direction), move |i, j| { - if IterDirection::Forward == direction { - i.0.cmp(&j.0) + if let (Ok(i), Ok(j)) = (i, j) { + if IterDirection::Forward == direction { + i.0.cmp(&j.0) + } else { + j.0.cmp(&i.0) + } } else { - j.0.cmp(&i.0) + // prioritize errors from db result first + if j.is_err() { + Ordering::Greater + } else { + Ordering::Less + } } }, ) @@ -125,14 +135,19 @@ impl KeyValueStore for MemoryTransactionView { } }) // filter entries which have been deleted over the course of this transaction - .filter(move |(key, _)| { - !matches!( - changes - .lock() - .expect("poisoned") - .get(&column_key(key, column)), - Some(WriteOperation::Remove(_, _)) - ) + .filter(move |item| { + if let Ok((key, _)) = item { + !matches!( + changes + .lock() + .expect("poisoned") + .get(&column_key(key, column)), + Some(WriteOperation::Remove(_, _)) + ) + } else { + // ensure errors are propagated + true + } }), ) } @@ -381,10 +396,11 @@ mod tests { view.put(vec![i], 0, vec![2]).unwrap(); }); - let ret = view + let ret: Vec<_> = view .iter_all(0, None, None, IterDirection::Forward) - .map(|(k, _)| k[0]) - .collect_vec(); + .map_ok(|(k, _)| k[0]) + .try_collect() + .unwrap(); // verify assert_eq!(ret, vec![0, 2, 3, 4, 6, 8, 9]) } @@ -403,10 +419,11 @@ mod tests { view.put(vec![i], 0, vec![2]).unwrap(); }); - let ret = view + let ret: Vec<_> = view .iter_all(0, None, None, IterDirection::Reverse) - .map(|(k, _)| k[0]) - .collect_vec(); + .map_ok(|(k, _)| k[0]) + .try_collect() + .unwrap(); // verify assert_eq!(ret, vec![9, 8, 6, 4, 3, 2, 0]) } @@ -425,11 +442,12 @@ mod tests { view.put(vec![i], 0, vec![0xB]).unwrap(); }); - let ret = view + let ret: Vec<_> = view .iter_all(0, None, None, IterDirection::Forward) // return all the values from the iterator - .map(|(_, v)| v[0]) - .collect_vec(); + .map_ok(|(_, v)| v[0]) + .try_collect() + .unwrap(); // verify assert_eq!(ret, vec![0xB, 0xB, 0xB, 0xB, 0xB]) } @@ -447,11 +465,12 @@ mod tests { let _ = view.delete(&[0], 0).unwrap(); let _ = view.delete(&[6], 0).unwrap(); - let ret = view + let ret: Vec<_> = view .iter_all(0, None, None, IterDirection::Forward) // return all the values from the iterator - .map(|(k, _)| k[0]) - .collect_vec(); + .map_ok(|(k, _)| k[0]) + .try_collect() + .unwrap(); // verify assert_eq!(ret, vec![2, 4, 8]) } diff --git a/fuel-core/src/state/rocks_db.rs b/fuel-core/src/state/rocks_db.rs index 390be0e6695..f67c250df54 100644 --- a/fuel-core/src/state/rocks_db.rs +++ b/fuel-core/src/state/rocks_db.rs @@ -1,5 +1,6 @@ #[cfg(feature = "prometheus")] use crate::service::metrics::prometheus_metrics::DATABASE_METRICS; +use crate::state::KVItem; use crate::{ database::{ columns, @@ -166,7 +167,7 @@ impl KeyValueStore for RocksDb { prefix: Option>, start: Option>, direction: IterDirection, - ) -> Box, Vec)> + '_> { + ) -> Box + '_> { let iter_mode = start.as_ref().map_or_else( || { prefix.as_ref().map_or_else( @@ -194,23 +195,32 @@ impl KeyValueStore for RocksDb { let iter = self .db .iterator_cf_opt(&self.cf(column), opts, iter_mode) - .map(|(key, value)| { - let value_as_vec = value.to_vec(); - let key_as_vec = key.to_vec(); - #[cfg(feature = "prometheus")] - { - DATABASE_METRICS.read_meter.inc(); - DATABASE_METRICS - .bytes_read_meter - .inc_by(key_as_vec.len() as u64 + value_as_vec.len() as u64); - } - (key_as_vec, value_as_vec) + .map(|item| { + item.map(|(key, value)| { + let value_as_vec = value.to_vec(); + let key_as_vec = key.to_vec(); + #[cfg(feature = "prometheus")] + { + DATABASE_METRICS.read_meter.inc(); + DATABASE_METRICS + .bytes_read_meter + .inc_by(key_as_vec.len() as u64 + value_as_vec.len() as u64); + } + (key_as_vec, value_as_vec) + }) + .map_err(|e| Error::DatabaseError(Box::new(e))) }); if let Some(prefix) = prefix { let prefix = prefix.to_vec(); // end iterating when we've gone outside the prefix - Box::new(iter.take_while(move |(key, _)| key.starts_with(prefix.as_slice()))) + Box::new(iter.take_while(move |item| { + if let Ok((key, _)) = item { + key.starts_with(prefix.as_slice()) + } else { + true + } + })) } else { Box::new(iter) } diff --git a/fuel-p2p/Cargo.toml b/fuel-p2p/Cargo.toml index 317dd7776e5..fb246d699b5 100644 --- a/fuel-p2p/Cargo.toml +++ b/fuel-p2p/Cargo.toml @@ -1,6 +1,6 @@ [package] name = "fuel-p2p" -version = "0.9.5" +version = "0.10.0" authors = ["Fuel Labs "] categories = ["cryptography::cryptocurrencies", "network-programming"] edition = "2021" @@ -14,7 +14,7 @@ description = "Fuel client networking" anyhow = "1.0" async-trait = "0.1" bincode = "1.3" -fuel-core-interfaces = { path = "../fuel-core-interfaces", features = ["serde"], version = "0.9.5" } +fuel-core-interfaces = { path = "../fuel-core-interfaces", features = ["serde"], version = "0.10.0" } futures = "0.3" futures-timer = "3.0" ip_network = "0.4" diff --git a/fuel-relayer/Cargo.toml b/fuel-relayer/Cargo.toml index bd2abe4b017..fb7e27c714a 100644 --- a/fuel-relayer/Cargo.toml +++ b/fuel-relayer/Cargo.toml @@ -1,6 +1,6 @@ [package] name = "fuel-relayer" -version = "0.9.5" +version = "0.10.0" authors = ["Fuel Labs "] edition = "2021" homepage = "https://fuel.network/" @@ -15,18 +15,18 @@ async-trait = "0.1" bytes = "1.1" chrono = "0.4" env_logger = "0.9" -ethers-contract = { version = "0.13", default-features = false, features = [ +ethers-contract = { version = "0.17", default-features = false, features = [ "abigen", ] } -ethers-core = { version = "0.13", default-features = false } -ethers-middleware = { version = "0.13", default-features = false } -ethers-providers = { version = "0.13", default-features = false, features = [ +ethers-core = { version = "0.17", default-features = false } +ethers-middleware = { version = "0.17", default-features = false } +ethers-providers = { version = "0.17", default-features = false, features = [ "ws", "rustls", ] } -ethers-signers = { version = "0.13", default-features = false } +ethers-signers = { version = "0.17", default-features = false } features = "0.10" -fuel-core-interfaces = { path = "../fuel-core-interfaces", package = "fuel-core-interfaces", version = "0.9.5" } +fuel-core-interfaces = { path = "../fuel-core-interfaces", package = "fuel-core-interfaces", version = "0.10.0" } futures = "0.3" hex = "0.4" once_cell = "1.4" diff --git a/fuel-relayer/README.md b/fuel-relayer/README.md index 078fef0f680..633574d6d40 100644 --- a/fuel-relayer/README.md +++ b/fuel-relayer/README.md @@ -24,7 +24,7 @@ Second finality that we have is related to fuel block attestation time limit, ho Example of sliding window: ![Sliding Window](../docs/diagrams/fuel_v2_relayer_sliding_window.jpg) -* Problem: How to choose when token deposit event gets enabled for use in fuel, at what exact fuel block does this happen? (Note that we have sliding window) +* Problem: How to choose when bridge message event gets enabled for use in fuel, at what exact fuel block does this happen? (Note that we have sliding window) * Solution: introduce `da_height` variable inside fuel block header that will tell at what block we are including validator delegation/withdrawal and token deposits. There are few rules that `da_height` (da as data availability) need to follow and can be enforced with v2 contract: @@ -73,7 +73,7 @@ On the database side we have: * `StakingDIffs` table: for every da_block it contain staking diff. Staking diff contains un/registration of validator and delegations * `ValidatorSet` table: validator set at last finalized da block. * `DelegatesIndex` table: One delegation has a list of validators and delegation stakes, when undelegating or adding a new delegation, the old delegation needs to be removed. This index contains list of blocks where delegation happened so that we can traverse it and get that old delegation. -* `DepositCoin`: table of finalized coins. It will become the table of messages in the new bridging architecture. +* `Message`: table of finalized bridge messages. And some DB variables: diff --git a/fuel-relayer/abi/IFuelMessageOutbox.json b/fuel-relayer/abi/IFuelMessageOutbox.json new file mode 100644 index 00000000000..a0e565afa2c --- /dev/null +++ b/fuel-relayer/abi/IFuelMessageOutbox.json @@ -0,0 +1,108 @@ +{ + "_format": "hh-sol-artifact-1", + "contractName": "IFuelMessageOutbox", + "sourceName": "contracts/messaging/IFuelMessageOutbox.sol", + "abi": [ + { + "anonymous": false, + "inputs": [ + { + "indexed": true, + "internalType": "bytes32", + "name": "sender", + "type": "bytes32" + }, + { + "indexed": true, + "internalType": "bytes32", + "name": "recipient", + "type": "bytes32" + }, + { + "indexed": false, + "internalType": "bytes32", + "name": "owner", + "type": "bytes32" + }, + { + "indexed": false, + "internalType": "uint64", + "name": "nonce", + "type": "uint64" + }, + { + "indexed": false, + "internalType": "uint64", + "name": "amount", + "type": "uint64" + }, + { + "indexed": false, + "internalType": "bytes", + "name": "data", + "type": "bytes" + } + ], + "name": "SentMessage", + "type": "event" + }, + { + "inputs": [ + { + "internalType": "bytes32", + "name": "recipient", + "type": "bytes32" + } + ], + "name": "sendETH", + "outputs": [], + "stateMutability": "payable", + "type": "function" + }, + { + "inputs": [ + { + "internalType": "bytes32", + "name": "recipient", + "type": "bytes32" + }, + { + "internalType": "bytes", + "name": "data", + "type": "bytes" + } + ], + "name": "sendMessage", + "outputs": [], + "stateMutability": "payable", + "type": "function" + }, + { + "inputs": [ + { + "internalType": "bytes32", + "name": "recipient", + "type": "bytes32" + }, + { + "internalType": "bytes32", + "name": "owner", + "type": "bytes32" + }, + { + "internalType": "bytes", + "name": "data", + "type": "bytes" + } + ], + "name": "sendMessageWithOwner", + "outputs": [], + "stateMutability": "payable", + "type": "function" + } + ], + "bytecode": "0x", + "deployedBytecode": "0x", + "linkReferences": {}, + "deployedLinkReferences": {} +} diff --git a/fuel-relayer/src/abi.rs b/fuel-relayer/src/abi.rs index 2b3756d2dd6..d0b5661402d 100644 --- a/fuel-relayer/src/abi.rs +++ b/fuel-relayer/src/abi.rs @@ -7,5 +7,10 @@ pub mod validators { super::abigen!(ValidatorSet, "abi/ValidatorSet.json"); } +pub mod bridge { + super::abigen!(Message, "abi/IFuelMessageOutbox.json"); +} + +pub use bridge::Message; pub use fuel::Fuel; pub use validators::ValidatorSet; diff --git a/fuel-relayer/src/config.rs b/fuel-relayer/src/config.rs index 1236a9c1c72..42d45727da7 100644 --- a/fuel-relayer/src/config.rs +++ b/fuel-relayer/src/config.rs @@ -12,11 +12,8 @@ pub fn keccak256(data: &'static str) -> H256 { H256::from_slice(out.as_slice()) } -pub(crate) static ETH_LOG_ASSET_DEPOSIT: Lazy = - Lazy::new(|| keccak256("DepositMade(uint32,address,address,uint8,uint256,uint256)")); -#[allow(dead_code)] -pub(crate) static ETH_LOG_ASSET_WITHDRAWAL: Lazy = - Lazy::new(|| keccak256("WithdrawalMade(address,address,address,uint256)")); +pub(crate) static ETH_LOG_MESSAGE: Lazy = + Lazy::new(|| keccak256("SentMessage(bytes32,bytes32,bytes32,uint64,uint64,bytes)")); pub(crate) static ETH_LOG_VALIDATOR_REGISTRATION: Lazy = Lazy::new(|| keccak256("ValidatorRegistration(bytes,bytes)")); pub(crate) static ETH_LOG_VALIDATOR_UNREGISTRATION: Lazy = @@ -31,7 +28,7 @@ pub(crate) static ETH_FUEL_BLOCK_COMMITTED: Lazy = #[derive(Clone, Debug)] pub struct Config { - /// Number of da block after which deposits/stakes/validators become finalized. + /// Number of da block after which messages/stakes/validators become finalized. pub da_finalization: DaBlockHeight, /// Uri address to ethereum client. pub eth_client: Option, @@ -54,7 +51,8 @@ impl Default for Config { fn default() -> Self { Self { da_finalization: 64, - eth_client: Some(String::from("http://localhost:8545")), + // Some(String::from("http://localhost:8545")) + eth_client: None, eth_chain_id: 1, // ethereum mainnet eth_v2_commit_contract: None, eth_v2_listening_contracts: vec![H160::from_str( @@ -109,13 +107,8 @@ mod tests { #[test] pub fn test_function_signatures() { assert_eq!( - *ETH_LOG_ASSET_DEPOSIT, - H256::from_str("0x34dccbe410bb771d28929a3f1ada2323bfb6ae501200c02dc871b287fb558759") - .unwrap() - ); - assert_eq!( - *ETH_LOG_ASSET_WITHDRAWAL, - H256::from_str("0x779c18fbb35b88ab773ee6b3d87e1d10eb58021e64e0d7808db646f49403d20b") + *ETH_LOG_MESSAGE, + H256::from_str("0x6e777c34951035560591fac300515942821cca139ab8a514eb117129048e21b2") .unwrap() ); assert_eq!( diff --git a/fuel-relayer/src/finalization_queue.rs b/fuel-relayer/src/finalization_queue.rs index 1a2fa977cde..5d5722157e6 100644 --- a/fuel-relayer/src/finalization_queue.rs +++ b/fuel-relayer/src/finalization_queue.rs @@ -1,14 +1,15 @@ use crate::{ - log::{AssetDepositLog, EthEventLog}, + log::EthEventLog, pending_blocks::{IsReverted, PendingBlocks}, validators::Validators, }; use ethers_core::types::{Log, H160}; use ethers_providers::Middleware; use fuel_core_interfaces::{ - common::fuel_tx::{Address, Bytes32}, + common::{fuel_tx::Address, fuel_types::MessageId}, model::{ - BlockHeight, ConsensusId, DaBlockHeight, SealedFuelBlock, ValidatorId, ValidatorStake, + BlockHeight, CheckedMessage, ConsensusId, DaBlockHeight, Message, SealedFuelBlock, + ValidatorId, ValidatorStake, }, relayer::{RelayerDb, StakingDiff, ValidatorDiff}, }; @@ -41,8 +42,8 @@ pub struct DaBlockDiff { pub validators: HashMap>, // Delegation diff contains new delegation list, if we did just withdrawal option will be None. pub delegations: HashMap>>, - /// erc-20 pending deposit. - pub assets: HashMap, + /// bridge messages (e.g. erc20 or nft assets) + pub messages: HashMap, } impl DaBlockDiff { @@ -51,7 +52,7 @@ impl DaBlockDiff { da_height, validators: HashMap::new(), delegations: HashMap::new(), - assets: HashMap::new(), + messages: HashMap::new(), } } } @@ -174,15 +175,15 @@ impl FinalizationQueue { /// Handle eth log events pub async fn append_eth_log(&mut self, log: Log) { + if log.block_number.is_none() { + error!(target:"relayer", "Block number not found in eth log"); + return; + } let event = EthEventLog::try_from(&log); if let Err(err) = event { warn!(target:"relayer", "Eth Event not formatted properly:{}",err); return; } - if log.block_number.is_none() { - error!(target:"relayer", "Block number not found in eth log"); - return; - } let removed = log.removed.unwrap_or(false); let da_height = log.block_number.unwrap().as_u64() as DaBlockHeight; let event = event.unwrap(); @@ -208,8 +209,9 @@ impl FinalizationQueue { } let last_diff = self.pending.back_mut().unwrap(); match fuel_event { - EthEventLog::AssetDeposit(deposit) => { - last_diff.assets.insert(deposit.deposit_nonce, deposit); + EthEventLog::Message(message) => { + let msg = Message::from(&message).check(); + last_diff.messages.insert(*msg.id(), msg); } EthEventLog::Deposit { .. } => { // It is fine to do nothing. This is only related to contract, @@ -313,8 +315,8 @@ impl FinalizationQueue { } // push finalized assets to db - for (_, deposit) in diff.assets.iter() { - db.insert_coin_deposit(deposit.into()).await + for (_, message) in diff.messages.iter() { + db.insert_message(message).await } // insert height index into delegations. @@ -342,22 +344,17 @@ mod tests { use super::*; use crate::log::tests::*; - use fuel_core_interfaces::{ - common::fuel_types::{Address, AssetId}, - db::helpers::DummyDb, - }; + use fuel_core_interfaces::{common::fuel_types::Address, db::helpers::DummyDb}; use rand::rngs::StdRng; use rand::{Rng, SeedableRng}; #[tokio::test] - pub async fn check_token_deposits_on_multiple_eth_blocks() { + pub async fn check_messages_on_multiple_eth_blocks() { let mut rng = StdRng::seed_from_u64(3020); let acc1: Address = rng.gen(); - let token1 = AssetId::zeroed(); - let nonce1: Bytes32 = rng.gen(); - let nonce2: Bytes32 = rng.gen(); - let nonce3: Bytes32 = rng.gen(); + let receipient = rng.gen(); + let owner = rng.gen(); let mut queue = FinalizationQueue::new( 0, @@ -368,29 +365,35 @@ mod tests { BlockHeight::from(0u64), ); - let deposit1 = eth_log_asset_deposit(0, acc1, token1, 0, 10, nonce1, 0); - let deposit2 = eth_log_asset_deposit(1, acc1, token1, 1, 20, nonce2, 0); - let deposit3 = eth_log_asset_deposit(1, acc1, token1, 1, 40, nonce3, 0); + let message1 = eth_log_message(0, acc1, receipient, owner, 0, 10, vec![]); + let message2 = eth_log_message(1, acc1, receipient, owner, 1, 14, vec![]); + let message3 = eth_log_message(1, acc1, receipient, owner, 2, 16, vec![]); - let deposit1_db = EthEventLog::try_from(&deposit1).unwrap(); - let deposit2_db = EthEventLog::try_from(&deposit2).unwrap(); - let deposit3_db = EthEventLog::try_from(&deposit3).unwrap(); + let message1_db = EthEventLog::try_from(&message1).unwrap(); + let message2_db = EthEventLog::try_from(&message2).unwrap(); + let message3_db = EthEventLog::try_from(&message3).unwrap(); queue - .append_eth_logs(vec![deposit1, deposit2, deposit3]) + .append_eth_logs(vec![message1, message2, message3]) .await; let diff1 = queue.pending[0].clone(); let diff2 = queue.pending[1].clone(); - if let EthEventLog::AssetDeposit(deposit) = &deposit1_db { - assert_eq!(diff1.assets.get(&nonce1), Some(deposit),); + if let EthEventLog::Message(message) = &message1_db { + let msg = Message::from(message).check(); + assert_eq!(msg.da_height, 0); + assert_eq!(diff1.messages.get(msg.id()), Some(&msg)); } - if let EthEventLog::AssetDeposit(deposit) = &deposit2_db { - assert_eq!(diff2.assets.get(&nonce2), Some(deposit),); + if let EthEventLog::Message(message) = &message2_db { + let msg = Message::from(message).check(); + assert_eq!(msg.da_height, 1); + assert_eq!(diff2.messages.get(msg.id()), Some(&msg)); } - if let EthEventLog::AssetDeposit(deposit) = &deposit3_db { - assert_eq!(diff2.assets.get(&nonce3), Some(deposit),); + if let EthEventLog::Message(message) = &message3_db { + let msg = Message::from(message).check(); + assert_eq!(msg.da_height, 1); + assert_eq!(diff2.messages.get(msg.id()), Some(&msg)); } } @@ -427,7 +430,7 @@ mod tests { } #[tokio::test] - pub async fn check_deposit_and_validator_finalization() { + pub async fn check_message_and_validator_finalization() { let mut rng = StdRng::seed_from_u64(3020); let v1: ValidatorId = rng.gen(); let c1: ConsensusId = rng.gen(); @@ -435,8 +438,8 @@ mod tests { let c2: ConsensusId = rng.gen(); let acc1: Address = rng.gen(); - let token1 = AssetId::zeroed(); - let nonce1: Bytes32 = rng.gen(); + let recipient = rng.gen(); + let sender = rng.gen(); let mut queue = FinalizationQueue::new( 0, @@ -447,31 +450,53 @@ mod tests { BlockHeight::from(0u64), ); + let test_message = Message { + sender: acc1, + recipient, + owner: sender, + nonce: 40, + amount: 0, + data: vec![], + da_height: 2, + fuel_block_spend: None, + }; queue .append_eth_logs(vec![ eth_log_validator_registration(1, v1, c1), eth_log_validator_registration(2, v2, c2), - eth_log_asset_deposit(2, acc1, token1, 1, 40, nonce1, 0), + eth_log_message( + 2, + test_message.sender, + test_message.recipient, + test_message.owner, + test_message.nonce as u32, + test_message.amount as u32, + test_message.data.clone(), + ), eth_log_validator_unregistration(3, v1), ]) .await; let mut db = DummyDb::filled(); - //let db_ref = &mut db as &mut dyn RelayerDb; queue.commit_diffs(&mut db, 1).await; assert_eq!(db.data.lock().validators.get(&v1), Some(&(0, Some(c1))),); assert_eq!(db.data.lock().validators.get(&v2), None,); - assert_eq!(db.data.lock().deposit_coin.len(), 0,); + assert_eq!(db.data.lock().messages.len(), 0,); queue.commit_diffs(&mut db, 2).await; assert_eq!(db.data.lock().validators.get(&v2), Some(&(0, Some(c2))),); - assert_eq!(db.data.lock().deposit_coin.len(), 1,); + assert_eq!(db.data.lock().messages.len(), 1,); + // ensure committed message id matches message id from the log + assert_eq!( + db.data.lock().messages.values().next().unwrap().id(), + test_message.id() + ); queue.commit_diffs(&mut db, 3).await; assert_eq!(db.data.lock().validators.get(&v1), Some(&(0, None)),); assert_eq!(db.data.lock().validators.get(&v2), Some(&(0, Some(c2))),); - assert_eq!(db.data.lock().deposit_coin.len(), 1,); + assert_eq!(db.data.lock().messages.len(), 1,); } #[tokio::test] diff --git a/fuel-relayer/src/log.rs b/fuel-relayer/src/log.rs index 99251c97de6..2a181eb5f44 100644 --- a/fuel-relayer/src/log.rs +++ b/fuel-relayer/src/log.rs @@ -5,30 +5,34 @@ use ethers_core::{ abi::RawLog, types::{Log, U256}, }; +use fuel_core_interfaces::model::DaBlockHeight; use fuel_core_interfaces::{ - common::fuel_types::{Address, AssetId, Bytes32, Word}, - model::{ConsensusId, DepositCoin, ValidatorId}, + common::fuel_types::{Address, Bytes32, Word}, + model::{ConsensusId, Message, ValidatorId}, }; -/// This is going to be superseded with MessageLog: https://github.com/FuelLabs/fuel-core/issues/366 +/// Bridge message send from da to fuel network. #[derive(Debug, Clone, PartialEq)] -pub struct AssetDepositLog { - pub account: Address, - pub token: AssetId, +pub struct MessageLog { + pub sender: Address, + pub recipient: Address, + pub owner: Address, + pub nonce: Word, pub amount: Word, - pub precision_factor: u8, - pub block_number: u32, - pub deposit_nonce: Bytes32, + pub data: Vec, + pub da_height: DaBlockHeight, } -impl From<&AssetDepositLog> for DepositCoin { - fn from(asset: &AssetDepositLog) -> Self { +impl From<&MessageLog> for Message { + fn from(message: &MessageLog) -> Self { Self { - owner: asset.account, - amount: asset.amount, - asset_id: asset.token, // TODO should this be hash of token_id and precision factor - nonce: asset.deposit_nonce, - deposited_da_height: asset.block_number, + sender: message.sender, + recipient: message.recipient, + owner: message.owner, + nonce: message.nonce, + amount: message.amount, + data: message.data.clone(), + da_height: message.da_height, fuel_block_spend: None, } } @@ -36,7 +40,8 @@ impl From<&AssetDepositLog> for DepositCoin { #[derive(Debug, Clone, PartialEq)] pub enum EthEventLog { - AssetDeposit(AssetDepositLog), + // Bridge message from da side + Message(MessageLog), // save it in validator set ValidatorRegistration { staking_key: ValidatorId, @@ -69,10 +74,6 @@ pub enum EthEventLog { Unknown, } -/// block_number(32bits) | precisionFactor(8bits) | depositNonce(256bits) -/// data is packet as three 256bit/32bytes values -const ASSET_DEPOSIT_DATA_LEN: usize = 32 + 32 + 32; - impl TryFrom<&Log> for EthEventLog { type Error = anyhow::Error; @@ -82,58 +83,35 @@ impl TryFrom<&Log> for EthEventLog { } let log = match log.topics[0] { - n if n == *config::ETH_LOG_ASSET_DEPOSIT => { - if log.topics.len() != 4 { - return Err(anyhow!("Malformed topics for AssetDeposit")); - } - let account = unsafe { Address::from_slice_unchecked(log.topics[1].as_ref()) }; - let token = unsafe { AssetId::from_slice_unchecked(log.topics[2].as_ref()) }; - - if !log.topics[3][..24].iter().all(|&b| b == 0) { - return Err(anyhow!( - "Malformed amount for AssetDeposit. Amount bigger then u64", - )); - } - let amount = <[u8; 8]>::try_from(&log.topics[3][24..]) - .map(u64::from_be_bytes) - .expect("We have checked slice bounds"); - - // data is contains: block_number(32bits) | precisionFactor(8bits) | depositNonce(256bits) - let data = &log.data.0; - - if data.len() != ASSET_DEPOSIT_DATA_LEN { - return Err(anyhow!( - "Malformed data length for AssetDeposit: {}", - data.len() - )); - } - if !data[..28].iter().all(|&b| b == 0) { - return Err(anyhow!( - "Malformed amount for AssetDeposit. Amount bigger then u64", - )); - } - - let block_number = <[u8; 4]>::try_from(&data[28..32]) - .map(u32::from_be_bytes) - .expect("We have checked slice bounds"); - - if !data[32..63].iter().all(|&b| b == 0) { - return Err(anyhow!( - "Malformed amount for AssetDeposit. Amount bigger then u64", - )); + n if n == *config::ETH_LOG_MESSAGE => { + if log.topics.len() != 3 { + return Err(anyhow!("Malformed topics for Message")); } - let precision_factor = data[63]; + let raw_log = RawLog { + topics: log.topics.clone(), + data: log.data.to_vec(), + }; - let deposit_nonce = unsafe { Bytes32::from_slice_unchecked(&data[64..]) }; + let message = abi::bridge::SentMessageFilter::decode_log(&raw_log)?; + let amount = message.amount; + let data = message.data.to_vec(); + let nonce = message.nonce; + let owner = Address::from(message.owner); + let recipient = Address::from(message.recipient); + let sender = Address::from(message.sender); - Self::AssetDeposit(AssetDepositLog { - block_number, - account, + Self::Message(MessageLog { amount, - token, - precision_factor, - deposit_nonce, + data, + nonce, + sender, + recipient, + owner, + // Safety: logs without block numbers are rejected by + // FinalizationQueue::append_eth_log before the conversion to EthEventLog happens. + // If block_number is none, that means the log is pending. + da_height: log.block_number.unwrap().as_u64(), }) } n if n == *config::ETH_LOG_VALIDATOR_REGISTRATION => { @@ -334,11 +312,15 @@ pub mod tests { del_data.extend(&bytes); } + // index for first item data.extend(H256::from_low_u64_be(64).as_ref()); + // index of end of del data data.extend(H256::from_low_u64_be(64 + del_data.len() as u64).as_ref()); + // del data data.extend(del_data); + // index of end of amount data data.extend(H256::from_low_u64_be(amounts.len() as u64).as_ref()); for amount in amounts { data.extend(H256::from_low_u64_be(amount).as_ref()); @@ -356,33 +338,42 @@ pub mod tests { ) } - pub fn eth_log_asset_deposit( + pub fn eth_log_message( eth_block: u64, - account: Address, - token: AssetId, - block_number: u32, - amount: Word, - deposit_nonce: Bytes32, - precision_factor: u8, + sender: Address, + receipient: Address, + owner: Address, + nonce: u32, + amount: u32, + data: Vec, ) -> Log { - //block_number(32bits) | precision_factor(256bits) | depositNonce(256bits) - // 32+32+32 - let mut b = BytesMut::new(); - b.resize(ASSET_DEPOSIT_DATA_LEN, 0); - //let mut b: [u8; 68] = [0; 68]; - b[28..32].copy_from_slice(&block_number.to_be_bytes()); - // 4..28 are zeroes - b[63] = precision_factor; - b[64..96].copy_from_slice(deposit_nonce.as_ref()); + let mut b: Vec = Vec::new(); + // owner nonce amount data + // 32 + 32 + 32 + dyn + + b.extend(owner.as_ref()); + b.extend(H256::from_low_u64_be(nonce as u64).as_ref()); + b.extend(H256::from_low_u64_be(amount as u64).as_ref()); + b.extend(H256::from_low_u64_be(128).as_ref()); + b.extend(H256::from_low_u64_be(data.len() as u64).as_ref()); + + // data takes as lest 32 bytes; + let data_size = ((data.len() / 32) + 1) * 32; + let start = b.len(); + // resize buffer to be able to extend data. + b.resize(b.len() + data_size, 0); + for (i, data) in data.iter().enumerate() { + b[start + i] = *data; + } + log_default( eth_block, vec![ - *config::ETH_LOG_ASSET_DEPOSIT, - H256::from_slice(account.as_ref()), - H256::from_slice(token.as_ref()), - H256::from_low_u64_be(amount), + *config::ETH_LOG_MESSAGE, + H256::from_slice(sender.as_ref()), + H256::from_slice(receipient.as_ref()), ], - b.freeze(), + BytesMut::from_iter(b.into_iter()).freeze(), ) } @@ -576,24 +567,24 @@ pub mod tests { } #[test] - fn eth_event_asset_deposit_try_from_log() { + fn eth_event_message_try_from_log() { let rng = &mut StdRng::seed_from_u64(2322u64); - let eth_block = rng.gen(); - let account = rng.gen(); - let token = rng.gen(); - let block_number: u32 = rng.gen(); - let amount = rng.gen(); - let deposit_nonce = rng.gen(); - let precision_factor = rng.gen(); - - let log = eth_log_asset_deposit( + let eth_block: u64 = rng.gen(); + let sender: Address = rng.gen(); + let receipient: Address = rng.gen(); + let owner: Address = rng.gen(); + let nonce: u32 = rng.gen(); + let amount: u32 = rng.gen(); + let data: Vec = vec![1u8]; + + let log = eth_log_message( eth_block, - account, - token, - block_number, + sender, + receipient, + owner, + nonce, amount, - deposit_nonce, - precision_factor, + data.clone(), ); assert_eq!( Some(U64([eth_block])), @@ -605,13 +596,14 @@ pub mod tests { assert_eq!( fuel_log.unwrap(), - EthEventLog::AssetDeposit(AssetDepositLog { - account, - token, - block_number, - precision_factor, - amount, - deposit_nonce + EthEventLog::Message(MessageLog { + sender, + recipient: receipient, + owner, + nonce: nonce as u64, + amount: amount as u64, + data, + da_height: eth_block }), "Decoded log does not match data we encoded" ); diff --git a/fuel-relayer/src/relayer.rs b/fuel-relayer/src/relayer.rs index d344307afef..93203bfd505 100644 --- a/fuel-relayer/src/relayer.rs +++ b/fuel-relayer/src/relayer.rs @@ -110,8 +110,8 @@ impl Relayer { ); // should be always more then last finalized_da_heights - let best_finalized_block = (provider.get_block_number().await?.as_u64() as u32) - - self.ctx.config.da_finalization(); + let best_finalized_block = + provider.get_block_number().await?.as_u64() - self.ctx.config.da_finalization(); // 1. sync from HardCoddedContractCreatingBlock->BestEthBlock-100) let step = self.ctx.config.initial_sync_step(); // do some stats on optimal value @@ -515,9 +515,11 @@ mod test { data.best_block.number = Some(U64([134])); data.best_block.number = Some(U64([134])); - data.logs_batch = vec![ - vec![log::tests::eth_log_deposit(136, Address::zeroed(), 10)], //Log::] - ]; + data.logs_batch = vec![vec![log::tests::eth_log_deposit( + 136, + Address::zeroed(), + 10, + )]]; data.blocks_batch = vec![vec![H256::zero()]]; } pub struct Handle { diff --git a/fuel-relayer/src/service.rs b/fuel-relayer/src/service.rs index f29323dc402..88c56bf8183 100644 --- a/fuel-relayer/src/service.rs +++ b/fuel-relayer/src/service.rs @@ -96,7 +96,7 @@ pub struct Context { pub receiver: mpsc::Receiver, /// Private. pub private_key: Vec, - /// Db connector to apply stake and token deposit. + /// Db connector to apply stake and messages. pub db: Box, /// Notification of new block event. pub new_block_event: broadcast::Receiver, diff --git a/fuel-sync/Cargo.toml b/fuel-sync/Cargo.toml index b96003875a2..7828ccc17b6 100644 --- a/fuel-sync/Cargo.toml +++ b/fuel-sync/Cargo.toml @@ -1,6 +1,6 @@ [package] name = "fuel-sync" -version = "0.9.5" +version = "0.10.0" authors = ["Fuel Labs "] edition = "2021" homepage = "https://fuel.network/" @@ -11,6 +11,6 @@ description = "Fuel Synchronizer" [dependencies] anyhow = "1.0" -fuel-core-interfaces = { path = "../fuel-core-interfaces", version = "0.9.5" } +fuel-core-interfaces = { path = "../fuel-core-interfaces", version = "0.10.0" } parking_lot = "0.12" tokio = { version = "1.14", features = ["full"] } diff --git a/fuel-tests/Cargo.toml b/fuel-tests/Cargo.toml index 55251ad451c..8c98aee7e57 100644 --- a/fuel-tests/Cargo.toml +++ b/fuel-tests/Cargo.toml @@ -28,14 +28,14 @@ async-std = "1.12" chrono = { version = "0.4", features = ["serde"] } fuel-core = { path = "../fuel-core", features = ["test-helpers"], default-features = false } fuel-core-interfaces = { path = "../fuel-core-interfaces", features = ["test-helpers"] } -fuel-crypto = { version = "0.5", features = ["random"] } +fuel-crypto = { version = "0.6", features = ["random"] } fuel-gql-client = { path = "../fuel-client", features = ["test-helpers"] } fuel-txpool = { path = "../fuel-txpool" } futures = "0.3" insta = "1.8" itertools = "0.10" rand = "0.8" -reqwest = "0.11" +reqwest = { version = "0.11", default-features = false, features = ["rustls-tls"] } rstest = { version = "0.13" } serde_json = "1.0" tempfile = "3.3" diff --git a/fuel-tests/tests/balances.rs b/fuel-tests/tests/balances.rs index 4ad74d296a1..cf646669191 100644 --- a/fuel-tests/tests/balances.rs +++ b/fuel-tests/tests/balances.rs @@ -1,9 +1,6 @@ use fuel_core::{ - config::{ - chain_config::{CoinConfig, StateConfig}, - Config, - }, - service::FuelService, + chain_config::{CoinConfig, StateConfig}, + service::{Config, FuelService}, }; use fuel_core_interfaces::common::{ fuel_tx::{AssetId, Input, Output}, @@ -42,6 +39,7 @@ async fn balance() { }) .collect(), ), + messages: None, }); // setup server & client @@ -80,6 +78,7 @@ async fn balance() { asset_id: coin.asset_id.into(), maturity: coin.maturity.into(), witness_index: 0, + tx_pointer: Default::default(), }); } let tx = tx @@ -142,6 +141,7 @@ async fn first_5_balances() { }) .collect(), ), + messages: None, }); // setup server & client diff --git a/fuel-tests/tests/blocks.rs b/fuel-tests/tests/blocks.rs index a44884473a1..3f2f4fcb3a2 100644 --- a/fuel-tests/tests/blocks.rs +++ b/fuel-tests/tests/blocks.rs @@ -1,10 +1,9 @@ use chrono::{TimeZone, Utc}; use fuel_core::{ - config::Config, database::Database, model::{FuelBlockDb, FuelBlockHeader}, schema::scalars::BlockId, - service::FuelService, + service::{Config, FuelService}, }; use fuel_core_interfaces::common::fuel_tx; use fuel_core_interfaces::common::{fuel_storage::Storage, fuel_types}; diff --git a/fuel-tests/tests/chain.rs b/fuel-tests/tests/chain.rs index 6dad619dfe7..34fc00de116 100644 --- a/fuel-tests/tests/chain.rs +++ b/fuel-tests/tests/chain.rs @@ -1,4 +1,4 @@ -use fuel_core::{config::Config, service::FuelService}; +use fuel_core::service::{Config, FuelService}; use fuel_gql_client::client::FuelClient; #[tokio::test] diff --git a/fuel-tests/tests/coin.rs b/fuel-tests/tests/coin.rs index d66bcf78ecf..2dd5d6f490d 100644 --- a/fuel-tests/tests/coin.rs +++ b/fuel-tests/tests/coin.rs @@ -1,11 +1,8 @@ use fuel_core::{ - config::{ - chain_config::{CoinConfig, StateConfig}, - Config, - }, + chain_config::{CoinConfig, StateConfig}, database::Database, model::{Coin, CoinStatus}, - service::FuelService, + service::{Config, FuelService}, }; use fuel_core_interfaces::common::{ fuel_storage::Storage, @@ -238,6 +235,7 @@ async fn coins_to_spend() { }) .collect(), ), + messages: None, }); // setup server & client diff --git a/fuel-tests/tests/dap.rs b/fuel-tests/tests/dap.rs index 5b498c9e291..949b84dfdf0 100644 --- a/fuel-tests/tests/dap.rs +++ b/fuel-tests/tests/dap.rs @@ -1,4 +1,4 @@ -use fuel_core::{config::Config, service::FuelService}; +use fuel_core::service::{Config, FuelService}; use fuel_core_interfaces::common::fuel_vm::{consts::*, prelude::*}; use fuel_gql_client::client::FuelClient; use std::convert::TryInto; diff --git a/fuel-tests/tests/debugger.rs b/fuel-tests/tests/debugger.rs index 0555da8d03f..152edf588ba 100755 --- a/fuel-tests/tests/debugger.rs +++ b/fuel-tests/tests/debugger.rs @@ -1,6 +1,6 @@ #![cfg(feature = "debug")] -use fuel_core::{config::Config, service::FuelService}; +use fuel_core::service::{Config, FuelService}; use fuel_core_interfaces::common::fuel_vm::prelude::*; use fuel_gql_client::client::FuelClient; diff --git a/fuel-tests/tests/example_tx.json b/fuel-tests/tests/example_tx.json index 6ef39a0a8d5..8e5d930bded 100755 --- a/fuel-tests/tests/example_tx.json +++ b/fuel-tests/tests/example_tx.json @@ -1,6 +1,5 @@ { "Script": { - "byte_price": 0, "gas_price": 0, "gas_limit": 1000000, "maturity": 0, diff --git a/fuel-tests/tests/health.rs b/fuel-tests/tests/health.rs index 976b5b4a857..47c9253c2df 100644 --- a/fuel-tests/tests/health.rs +++ b/fuel-tests/tests/health.rs @@ -1,4 +1,4 @@ -use fuel_core::{config::Config, service::FuelService}; +use fuel_core::service::{Config, FuelService}; use fuel_gql_client::client::FuelClient; #[tokio::test] diff --git a/fuel-tests/tests/helpers.rs b/fuel-tests/tests/helpers.rs index b5a80f966f0..75c8478d936 100644 --- a/fuel-tests/tests/helpers.rs +++ b/fuel-tests/tests/helpers.rs @@ -1,9 +1,6 @@ use fuel_core::{ - config::{ - chain_config::{ChainConfig, CoinConfig, ContractConfig, StateConfig}, - Config, - }, - service::FuelService, + chain_config::{ChainConfig, CoinConfig, ContractConfig, StateConfig}, + service::{Config, FuelService}, }; use fuel_core_interfaces::common::{fuel_tx::Contract, fuel_tx::Transaction, fuel_vm::prelude::*}; use fuel_gql_client::client::FuelClient; @@ -32,7 +29,6 @@ pub struct TestSetupBuilder { pub contracts: HashMap, pub initial_coins: Vec, pub min_gas_price: u64, - pub min_byte_price: u64, pub predicates: bool, } @@ -118,7 +114,6 @@ impl TestSetupBuilder { utxo_validation: true, predicates: self.predicates, txpool: fuel_txpool::Config { - min_byte_price: self.min_byte_price, min_gas_price: self.min_gas_price, ..Default::default() }, @@ -150,7 +145,6 @@ impl Default for TestSetupBuilder { contracts: Default::default(), initial_coins: vec![], min_gas_price: 0, - min_byte_price: 0, predicates: false, } } diff --git a/fuel-tests/tests/lib.rs b/fuel-tests/tests/lib.rs index f05bef1889b..ae464ff9fa1 100644 --- a/fuel-tests/tests/lib.rs +++ b/fuel-tests/tests/lib.rs @@ -7,6 +7,7 @@ mod dap; mod debugger; mod health; mod helpers; +mod messages; mod node_info; mod snapshot; mod tx; diff --git a/fuel-tests/tests/messages.rs b/fuel-tests/tests/messages.rs new file mode 100644 index 00000000000..788bb5acc0c --- /dev/null +++ b/fuel-tests/tests/messages.rs @@ -0,0 +1,186 @@ +use fuel_core::{ + chain_config::{MessageConfig, StateConfig}, + service::{Config, FuelService}, +}; +use fuel_core_interfaces::common::fuel_tx::TransactionBuilder; +use fuel_crypto::fuel_types::Address; +use fuel_crypto::SecretKey; +use fuel_gql_client::client::{FuelClient, PageDirection, PaginationRequest}; +use rand::{rngs::StdRng, Rng, SeedableRng}; +use std::ops::Deref; + +#[tokio::test] +async fn can_submit_genesis_message() { + let mut rng = StdRng::seed_from_u64(1234); + + let secret_key: SecretKey = rng.gen(); + let owner = secret_key.public_key().hash(); + + let msg1 = MessageConfig { + sender: rng.gen(), + recipient: rng.gen(), + owner: (*owner.deref()).into(), + nonce: rng.gen(), + amount: rng.gen(), + data: vec![rng.gen()], + da_height: 0, + }; + let tx1 = TransactionBuilder::script(vec![], vec![]) + .add_unsigned_message_input( + secret_key, + msg1.sender, + msg1.recipient, + msg1.nonce, + msg1.amount, + msg1.data.clone(), + ) + .finalize(); + + let mut node_config = Config::local_node(); + node_config.chain_conf.initial_state = Some(StateConfig { + messages: Some(vec![msg1]), + ..Default::default() + }); + node_config.utxo_validation = true; + + let srv = FuelService::new_node(node_config.clone()).await.unwrap(); + let client = FuelClient::from(srv.bound_address); + + client.submit(&tx1).await.unwrap(); +} + +#[tokio::test] +async fn messages_returns_messages_for_all_owners() { + // create some owners + let owner_a = Address::new([1; 32]); + let owner_b = Address::new([2; 32]); + + // create some messages for owner A + let first_msg = MessageConfig { + owner: owner_a, + nonce: 1, + ..Default::default() + }; + let second_msg = MessageConfig { + owner: owner_a, + nonce: 2, + ..Default::default() + }; + + // create a message for owner B + let third_msg = MessageConfig { + owner: owner_b, + nonce: 3, + ..Default::default() + }; + + // configure the messages + let mut config = Config::local_node(); + config.chain_conf.initial_state = Some(StateConfig { + messages: Some(vec![first_msg, second_msg, third_msg]), + ..Default::default() + }); + + // setup server & client + let srv = FuelService::new_node(config).await.unwrap(); + let client = FuelClient::from(srv.bound_address); + + // get the messages + let request = PaginationRequest { + cursor: None, + results: 5, + direction: PageDirection::Forward, + }; + let result = client.messages(None, request).await.unwrap(); + + // verify that there are 3 messages stored in total + assert_eq!(result.results.len(), 3); +} + +#[tokio::test] +async fn messages_by_owner_returns_messages_for_the_given_owner() { + // create some owners + let owner_a = Address::new([1; 32]); + let owner_b = Address::new([2; 32]); + + // create some messages for owner A + let first_msg = MessageConfig { + owner: owner_a, + nonce: 1, + ..Default::default() + }; + let second_msg = MessageConfig { + owner: owner_a, + nonce: 2, + ..Default::default() + }; + + // create a message for owner B + let third_msg = MessageConfig { + owner: owner_b, + nonce: 3, + ..Default::default() + }; + + // configure the messages + let mut config = Config::local_node(); + config.chain_conf.initial_state = Some(StateConfig { + messages: Some(vec![first_msg, second_msg, third_msg]), + ..Default::default() + }); + + // setup server & client + let srv = FuelService::new_node(config).await.unwrap(); + let client = FuelClient::from(srv.bound_address); + + let request = PaginationRequest { + cursor: None, + results: 5, + direction: PageDirection::Forward, + }; + + // get the messages from Owner A + let result = client + .messages(Some(&owner_a.to_string()), request.clone()) + .await + .unwrap(); + + // verify that Owner A has 2 messages + assert_eq!(result.results.len(), 2); + + // verify messages owner matches + for message in result.results { + assert_eq!(message.owner.0 .0, owner_a) + } + + // get the messages from Owner B + let result = client + .messages(Some(&owner_b.to_string()), request.clone()) + .await + .unwrap(); + + // verify that Owner B has 1 message + assert_eq!(result.results.len(), 1); + + assert_eq!(result.results[0].owner.0 .0, owner_b); +} + +#[tokio::test] +async fn messages_empty_results_for_owner_with_no_messages() { + let srv = FuelService::new_node(Config::local_node()).await.unwrap(); + let client = FuelClient::from(srv.bound_address); + + let owner = Address::new([1; 32]); + let request = PaginationRequest { + cursor: None, + results: 5, + direction: PageDirection::Forward, + }; + + let result = client + .messages(Some(&owner.to_string()), request) + .await + .unwrap(); + + assert_eq!(result.results.len(), 0); +} diff --git a/fuel-tests/tests/metrics.rs b/fuel-tests/tests/metrics.rs index 7db4a54d81d..256db50500f 100644 --- a/fuel-tests/tests/metrics.rs +++ b/fuel-tests/tests/metrics.rs @@ -1,7 +1,4 @@ -use fuel_core::{ - config::{Config, DbType}, - service::FuelService, -}; +use fuel_core::{service::Config, service::DbType, service::FuelService}; use fuel_core_interfaces::common::{ fuel_tx, fuel_tx::{Address, AssetId}, @@ -45,7 +42,6 @@ async fn test_database_metrics() { 0, 1000000, 0, - 0, script, vec![], vec![], diff --git a/fuel-tests/tests/node_info.rs b/fuel-tests/tests/node_info.rs index 1ff172fd276..1b1ec5d0904 100644 --- a/fuel-tests/tests/node_info.rs +++ b/fuel-tests/tests/node_info.rs @@ -1,4 +1,4 @@ -use fuel_core::{config::Config, service::FuelService}; +use fuel_core::service::{Config, FuelService}; use fuel_gql_client::client::{schema::node_info::NodeInfo, FuelClient}; #[tokio::test] @@ -11,7 +11,6 @@ async fn node_info() { utxo_validation, predicates, vm_backtrace, - min_byte_price, min_gas_price, max_depth, max_tx, @@ -22,7 +21,6 @@ async fn node_info() { assert_eq!(predicates, node_config.predicates); assert_eq!(vm_backtrace, node_config.vm.backtrace); assert_eq!(min_gas_price, node_config.txpool.min_gas_price.into()); - assert_eq!(min_byte_price, node_config.txpool.min_byte_price.into()); assert_eq!(max_depth, node_config.txpool.max_depth.into()); assert_eq!(max_tx, node_config.txpool.max_tx.into()); } diff --git a/fuel-tests/tests/snapshot.rs b/fuel-tests/tests/snapshot.rs index 9b2c54f3238..d35993cba37 100644 --- a/fuel-tests/tests/snapshot.rs +++ b/fuel-tests/tests/snapshot.rs @@ -1,10 +1,7 @@ -use fuel_core::database::Database; use fuel_core::{ - config::{ - chain_config::{CoinConfig, ContractConfig, StateConfig}, - Config, - }, - service::FuelService, + chain_config::{CoinConfig, ContractConfig, MessageConfig, StateConfig}, + database::Database, + service::{Config, FuelService}, }; use fuel_core_interfaces::{ common::{ @@ -13,9 +10,11 @@ use fuel_core_interfaces::{ }, model::BlockHeight, }; +use rand::{rngs::StdRng, Rng, SeedableRng}; #[tokio::test] async fn snapshot_state_config() { + let mut rng = StdRng::seed_from_u64(1234); let db = Database::default(); let owner = Address::default(); @@ -54,6 +53,15 @@ async fn snapshot_state_config() { }) .collect(), ), + messages: Some(vec![MessageConfig { + sender: rng.gen(), + recipient: rng.gen(), + owner: rng.gen(), + nonce: rng.gen_range(0..1000), + amount: rng.gen_range(0..1000), + data: vec![], + da_height: rng.gen_range(0..1000), + }]), }; config.chain_conf.initial_state = Some(starting_state.clone()); @@ -84,4 +92,6 @@ async fn snapshot_state_config() { assert_eq!(state_conf.height, starting_state.height); assert_eq!(state_conf.contracts, starting_state.contracts); + + assert_eq!(state_conf.messages, starting_state.messages) } diff --git a/fuel-tests/tests/tx.rs b/fuel-tests/tests/tx.rs index 9ee34b1518a..222f1b74995 100644 --- a/fuel-tests/tests/tx.rs +++ b/fuel-tests/tests/tx.rs @@ -1,11 +1,10 @@ use crate::helpers::TestContext; use chrono::Utc; use fuel_core::{ - config::Config, database::Database, executor::{ExecutionMode, Executor}, model::{FuelBlock, FuelBlockHeader}, - service::FuelService, + service::{Config, FuelService}, }; use fuel_core_interfaces::common::{ fuel_tx, @@ -45,7 +44,6 @@ async fn dry_run() { let gas_price = 0; let gas_limit = 1_000_000; - let byte_price = 0; let maturity = 0; let script = vec![ @@ -62,7 +60,6 @@ async fn dry_run() { let tx = fuel_tx::Transaction::script( gas_price, gas_limit, - byte_price, maturity, script, vec![], @@ -93,7 +90,6 @@ async fn submit() { let gas_price = 0; let gas_limit = 1_000_000; let maturity = 0; - let byte_price = 0; let script = vec![ Opcode::ADDI(0x10, REG_ZERO, 0xca), @@ -109,7 +105,6 @@ async fn submit() { let tx = fuel_tx::Transaction::script( gas_price, gas_limit, - byte_price, maturity, script, vec![], @@ -459,7 +454,6 @@ impl TestContext { let tx = Transaction::Script { gas_price: 0, gas_limit: 1_000_000, - byte_price: 0, maturity: 0, receipts_root: Default::default(), script, @@ -469,6 +463,7 @@ impl TestContext { owner: from, amount, asset_id: Default::default(), + tx_pointer: Default::default(), witness_index: 0, maturity: 0, }], @@ -506,7 +501,6 @@ fn create_mock_tx(val: u64) -> Transaction { 0, 0, 0, - 0, Default::default(), val.to_be_bytes().to_vec(), Default::default(), diff --git a/fuel-tests/tests/tx/predicates.rs b/fuel-tests/tests/tx/predicates.rs index ad168a2df22..9f16aa661d8 100644 --- a/fuel-tests/tests/tx/predicates.rs +++ b/fuel-tests/tests/tx/predicates.rs @@ -19,6 +19,7 @@ async fn transaction_with_predicates_is_rejected_when_feature_disabled() { rng.gen(), 500, asset_id, + Default::default(), 0, rng.gen::<[u8; 32]>().to_vec(), rng.gen::<[u8; 32]>().to_vec(), @@ -55,6 +56,7 @@ async fn transaction_with_predicate_is_executed_when_feature_enabled() { owner, amount, asset_id, + Default::default(), 0, predicate, vec![], @@ -103,6 +105,7 @@ async fn transaction_with_invalid_predicate_is_rejected_when_feature_is_enabled( owner, amount, asset_id, + Default::default(), 0, predicate, vec![], diff --git a/fuel-tests/tests/tx/utxo_validation.rs b/fuel-tests/tests/tx/utxo_validation.rs index 8ba88e3ad54..d08eea069ce 100644 --- a/fuel-tests/tests/tx/utxo_validation.rs +++ b/fuel-tests/tests/tx/utxo_validation.rs @@ -21,19 +21,25 @@ async fn submit_utxo_verified_tx_with_min_gas_price() { let transactions = (1..=10) .into_iter() .map(|i| { - let secret = SecretKey::random(&mut rng); TransactionBuilder::script( Opcode::RET(REG_ONE).to_bytes().into_iter().collect(), vec![], ) .gas_limit(100) .gas_price(1) - .byte_price(1) - .add_unsigned_coin_input(rng.gen(), &secret, 1000 + i, Default::default(), 0) + .add_unsigned_coin_input( + SecretKey::random(&mut rng), + rng.gen(), + 1000 + i, + Default::default(), + Default::default(), + 0, + ) .add_input(Input::Contract { utxo_id: Default::default(), balance_root: Default::default(), state_root: Default::default(), + tx_pointer: Default::default(), contract_id, }) .add_output(Output::Change { @@ -96,12 +102,10 @@ async fn submit_utxo_verified_tx_below_min_gas_price_fails() { ) .gas_limit(100) .gas_price(1) - .byte_price(1) .finalize(); // initialize node with higher minimum gas price let mut test_builder = TestSetupBuilder::new(2322u64); - test_builder.min_byte_price = 10; test_builder.min_gas_price = 10; let TestContext { client, .. } = test_builder.finalize().await; @@ -131,6 +135,7 @@ async fn dry_run_override_utxo_validation() { rng.gen(), 1000, AssetId::default(), + Default::default(), 0, Default::default(), )) @@ -139,6 +144,7 @@ async fn dry_run_override_utxo_validation() { rng.gen(), rng.gen(), asset_id, + Default::default(), 0, Default::default(), )) @@ -174,6 +180,7 @@ async fn dry_run_no_utxo_validation_override() { rng.gen(), 1000, AssetId::default(), + Default::default(), 0, Default::default(), )) @@ -182,6 +189,7 @@ async fn dry_run_no_utxo_validation_override() { rng.gen(), rng.gen(), asset_id, + Default::default(), 0, Default::default(), )) @@ -214,10 +222,11 @@ async fn concurrent_tx_submission_produces_expected_blocks() { ) .gas_limit(1000 + i as u64) .add_unsigned_coin_input( + secret, rng.gen(), - &secret, rng.gen_range(1..1000), Default::default(), + Default::default(), 0, ) .add_output(Output::change(rng.gen(), 0, Default::default())) diff --git a/fuel-txpool/Cargo.toml b/fuel-txpool/Cargo.toml index 055ca5029b3..ea4c5c4dc20 100644 --- a/fuel-txpool/Cargo.toml +++ b/fuel-txpool/Cargo.toml @@ -1,6 +1,6 @@ [package] name = "fuel-txpool" -version = "0.9.5" +version = "0.10.0" authors = ["Fuel Labs "] categories = ["cryptography::cryptocurrencies"] edition = "2021" @@ -14,7 +14,7 @@ description = "Transaction pool" anyhow = "1.0" async-trait = "0.1" chrono = "0.4" -fuel-core-interfaces = { path = "../fuel-core-interfaces", version = "0.9.5" } +fuel-core-interfaces = { path = "../fuel-core-interfaces", version = "0.10.0" } futures = "0.3" parking_lot = "0.11" thiserror = "1.0" @@ -22,6 +22,6 @@ tokio = { version = "1.14", default-features = false, features = ["sync"] } tracing = "0.1" [dev-dependencies] -fuel-core-interfaces = { path = "../fuel-core-interfaces", version = "0.9.5", features = [ +fuel-core-interfaces = { path = "../fuel-core-interfaces", version = "0.10.0", features = [ "test-helpers", ] } diff --git a/fuel-txpool/src/config.rs b/fuel-txpool/src/config.rs index 4969d52b84f..039214c2195 100644 --- a/fuel-txpool/src/config.rs +++ b/fuel-txpool/src/config.rs @@ -6,8 +6,6 @@ pub struct Config { pub max_depth: usize, /// The minimum allowed gas price pub min_gas_price: u64, - /// The minimum allowed byte price - pub min_byte_price: u64, } impl Default for Config { @@ -16,7 +14,6 @@ impl Default for Config { max_tx: 4064, max_depth: 10, min_gas_price: 0, - min_byte_price: 0, } } } diff --git a/fuel-txpool/src/containers/dependency.rs b/fuel-txpool/src/containers/dependency.rs index 94513d71af6..5f6579386f1 100644 --- a/fuel-txpool/src/containers/dependency.rs +++ b/fuel-txpool/src/containers/dependency.rs @@ -1,7 +1,10 @@ use crate::{types::*, Error}; use anyhow::anyhow; use fuel_core_interfaces::{ - common::fuel_tx::{Input, Output, UtxoId}, + common::{ + fuel_tx::{Input, Output, UtxoId}, + fuel_types::MessageId, + }, model::{ArcTx, Coin, CoinStatus, TxInfo}, txpool::TxPoolDb, }; @@ -16,6 +19,8 @@ pub struct Dependency { coins: HashMap, /// Contract-> Tx mapping. contracts: HashMap, + /// messageId -> tx mapping + messages: HashMap, /// max depth of dependency. max_depth: usize, } @@ -54,11 +59,20 @@ impl ContractState { } } +/// Always in database. No need for optional spenders, as this state would just be removed from +/// the hashmap if the message id isn't being spent. +#[derive(Debug, Clone)] +pub struct MessageState { + spent_by: TxId, + gas_price: GasPrice, +} + impl Dependency { pub fn new(max_depth: usize) -> Self { Self { coins: HashMap::new(), contracts: HashMap::new(), + messages: HashMap::new(), max_depth, } } @@ -111,6 +125,9 @@ impl Dependency { check.push(*origin.tx_id()); } } + Input::MessageSigned { .. } | Input::MessagePredicate { .. } => { + // Message inputs do not depend on any other fuel transactions + } } } } @@ -189,8 +206,8 @@ impl Dependency { } } Output::Contract { .. } => return Err(Error::NotInsertedIoContractOutput.into()), - Output::Withdrawal { .. } => { - return Err(Error::NotInsertedIoWithdrawalInput.into()); + Output::Message { .. } => { + return Err(Error::NotInsertedIoMessageInput.into()); } Output::Change { to, @@ -235,6 +252,41 @@ impl Dependency { Ok(()) } + /// Verifies the integrity of the message ID + fn check_if_message_input_matches_id(input: &Input) -> anyhow::Result<()> { + match input { + Input::MessageSigned { + message_id, + sender, + recipient, + nonce, + owner, + amount, + data, + .. + } + | Input::MessagePredicate { + message_id, + sender, + recipient, + nonce, + owner, + amount, + data, + .. + } => { + let computed_id = + Input::compute_message_id(sender, recipient, *nonce, owner, *amount, data); + if message_id != &computed_id { + return Err(Error::NotInsertedIoWrongMessageId.into()); + } + } + _ => {} + } + + Ok(()) + } + /// Check for collision. Used only inside insert function. /// Id doesn't change any dependency it just checks if it has possibility to be included. /// Returns: (max_depth, db_coins, db_contracts, collided_transactions); @@ -248,6 +300,7 @@ impl Dependency { usize, HashMap, HashMap, + HashMap, Vec, )> { let mut collided: Vec = Vec::new(); @@ -255,6 +308,7 @@ impl Dependency { let mut max_depth = 0; let mut db_coins: HashMap = HashMap::new(); let mut db_contracts: HashMap = HashMap::new(); + let mut db_messages: HashMap = HashMap::new(); for input in tx.inputs() { // check if all required inputs are here. match input { @@ -315,6 +369,40 @@ impl Dependency { // yey we got our coin } + Input::MessagePredicate { message_id, .. } + | Input::MessageSigned { message_id, .. } => { + // verify message id integrity + Self::check_if_message_input_matches_id(input)?; + // since message id is derived, we don't need to double check all the fields + if let Some(msg) = db.message(*message_id)? { + // return an error if spent block is set + if msg.fuel_block_spend.is_some() { + return Err(Error::NotInsertedInputMessageIdSpent(*message_id).into()); + } + } else { + return Err(Error::NotInsertedInputMessageUnknown(*message_id).into()); + } + + if let Some(state) = self.messages.get(message_id) { + // some other is already attempting to spend this message, compare gas price + if state.gas_price >= tx.gas_price() { + return Err(Error::NotInsertedCollisionMessageId( + state.spent_by, + *message_id, + ) + .into()); + } else { + collided.push(state.spent_by); + } + } + db_messages.insert( + *message_id, + MessageState { + spent_by: tx.id(), + gas_price: tx.gas_price(), + }, + ); + } Input::Contract { contract_id, .. } => { // Does contract exist. We don't need to do any check here other then if contract_id exist or not. if let Some(state) = self.contracts.get(contract_id) { @@ -376,7 +464,7 @@ impl Dependency { // collision of other outputs is not possible. } - Ok((max_depth, db_coins, db_contracts, collided)) + Ok((max_depth, db_coins, db_contracts, db_messages, collided)) } /// insert tx inside dependency @@ -387,7 +475,7 @@ impl Dependency { db: &dyn TxPoolDb, tx: &'a ArcTx, ) -> anyhow::Result> { - let (max_depth, db_coins, db_contracts, collided) = + let (max_depth, db_coins, db_contracts, db_messages, collided) = self.check_for_collision(txs, db, tx)?; // now we are sure that transaction can be included. remove all collided transactions @@ -416,6 +504,7 @@ impl Dependency { state.used_by.insert(tx.id()); } } + Input::MessageSigned { .. } | Input::MessagePredicate { .. } => {} } } @@ -424,6 +513,8 @@ impl Dependency { // for contracts from db that are not found in dependency, we already inserted used_by // and are okay to just extend current list self.contracts.extend(db_contracts.into_iter()); + // insert / overwrite all applicable message id spending relations + self.messages.extend(db_messages.into_iter()); // iterate over all outputs and insert them, marking them as available. for (index, output) in tx.outputs().iter().enumerate() { @@ -451,7 +542,7 @@ impl Dependency { }, ); } - Output::Withdrawal { .. } => { + Output::Message { .. } => { // withdrawal does nothing and it should not be found in dependency. } Output::Contract { .. } => { @@ -475,7 +566,7 @@ impl Dependency { // recursively remove all transactions that depend on the outputs of the current tx for (index, output) in tx.outputs().iter().enumerate() { match output { - Output::Withdrawal { .. } | Output::Contract { .. } => { + Output::Message { .. } | Output::Contract { .. } => { // no other transactions can depend on these types of outputs } Output::Coin { .. } | Output::Change { .. } | Output::Variable { .. } => { @@ -517,19 +608,15 @@ impl Dependency { // 1. coin state was already removed if parent tx was also removed, no cleanup required. // 2. coin state spent_by needs to be freed from this tx if parent tx isn't being removed // 3. coin state can be removed if this is a database coin, as no other txs are involved. - let mut rem_coin = false; if let Some(state) = self.coins.get_mut(utxo_id) { if !state.is_in_database() { // case 2 state.is_spend_by = None; } else { // case 3 - rem_coin = true; + self.coins.remove(utxo_id); } } - if rem_coin { - self.coins.remove(utxo_id); - } } Input::Contract { contract_id, .. } => { // Input contract cases @@ -538,17 +625,17 @@ impl Dependency { // 2. contract state exists and this tx needs to be removed as a user of it. // 2.a. contract state can be removed if it's from the database and this is the // last tx to use it, since no other txs are involved. - let mut rem_contract = false; if let Some(state) = self.contracts.get_mut(contract_id) { state.used_by.remove(&tx.id()); // if contract list is empty and is in db, flag contract state for removal. if state.used_by.is_empty() && state.is_in_database() { - rem_contract = true; + self.contracts.remove(contract_id); } } - if rem_contract { - self.contracts.remove(contract_id); - } + } + Input::MessageSigned { message_id, .. } + | Input::MessagePredicate { message_id, .. } => { + self.messages.remove(message_id); } } } @@ -581,6 +668,7 @@ mod tests { owner: Address::default(), amount: 10, asset_id: AssetId::default(), + tx_pointer: Default::default(), witness_index: 0, maturity: 0, }; @@ -666,17 +754,16 @@ mod tests { "test6" ); - let output = Output::Withdrawal { - to: Default::default(), - amount: Default::default(), - asset_id: Default::default(), + let output = Output::Message { + recipient: Default::default(), + amount: 0, }; let out = Dependency::check_if_coin_input_can_spend_output(&output, &input, false); assert!(out.is_err(), "test7 There should be error"); assert_eq!( out.err().unwrap().downcast_ref(), - Some(&Error::NotInsertedIoWithdrawalInput), + Some(&Error::NotInsertedIoMessageInput), "test7" ); diff --git a/fuel-txpool/src/service.rs b/fuel-txpool/src/service.rs index 2f30e3296f0..f4031ccceab 100644 --- a/fuel-txpool/src/service.rs +++ b/fuel-txpool/src/service.rs @@ -124,8 +124,8 @@ impl Context { TxPoolMpsc::FilterByNegative { ids, response } => { let _ = response.send(TxPool::filter_by_negative(txpool,&ids).await); } - TxPoolMpsc::Remove { ids } => { - TxPool::remove(txpool,broadcast,&ids).await; + TxPoolMpsc::Remove { ids, response } => { + let _ = response.send(TxPool::remove(txpool,broadcast,&ids).await); } TxPoolMpsc::Stop => {} }}); @@ -382,12 +382,15 @@ pub mod tests { ); // remove them + let (response, receiver) = oneshot::channel(); let _ = service .sender() .send(TxPoolMpsc::Remove { ids: vec![tx1_hash, tx2_hash], + response, }) .await; + let _rem = receiver.await.unwrap(); assert_eq!( tokio::time::timeout(std::time::Duration::from_secs(2), subscribe.recv()).await, diff --git a/fuel-txpool/src/txpool.rs b/fuel-txpool/src/txpool.rs index 56aaede470f..6708dd30a29 100644 --- a/fuel-txpool/src/txpool.rs +++ b/fuel-txpool/src/txpool.rs @@ -49,8 +49,6 @@ impl TxPool { // verify gas price is at least the minimum self.verify_tx_min_gas_price(&tx)?; - // verify byte price is at least the minimum - self.verify_tx_min_byte_price(&tx)?; if self.by_hash.contains_key(&tx.id()) { return Err(Error::NotInsertedTxKnown.into()); @@ -129,13 +127,6 @@ impl TxPool { Ok(()) } - fn verify_tx_min_byte_price(&mut self, tx: &Transaction) -> Result<(), Error> { - if tx.byte_price() < self.config.min_byte_price { - return Err(Error::NotInsertedBytePriceTooLow); - } - Ok(()) - } - /// Import a set of transactions from network gossip or GraphQL endpoints. pub async fn insert( txpool: &RwLock, @@ -240,30 +231,182 @@ impl TxPool { txpool: &RwLock, broadcast: broadcast::Sender, tx_ids: &[TxId], - ) { + ) -> Vec { let mut removed = Vec::new(); for tx_id in tx_ids { let rem = { txpool.write().await.remove_by_tx_id(tx_id) }; removed.extend(rem.into_iter()); } - for tx in removed { + for tx in &removed { let _ = broadcast.send(TxStatusBroadcast { - tx, + tx: tx.clone(), status: TxStatus::SqueezedOut { reason: Error::Removed, }, }); } + removed } } #[cfg(test)] pub mod tests { + mod helpers { + use fuel_core_interfaces::common::fuel_tx::Input; + use fuel_core_interfaces::{ + common::{ + fuel_storage::Storage, + fuel_tx::{Contract, ContractId, MessageId, UtxoId}, + }, + db::{self, KvStoreError}, + model::{Coin, Message}, + txpool::TxPoolDb, + }; + use std::{ + borrow::Cow, + collections::HashMap, + sync::{Arc, Mutex}, + }; + + #[derive(Default)] + pub(crate) struct Data { + pub coins: HashMap, + pub contracts: HashMap, + pub messages: HashMap, + } + + #[derive(Default)] + pub(crate) struct MockDb { + pub data: Arc>, + } + + impl Storage for MockDb { + type Error = KvStoreError; + + fn insert(&mut self, key: &UtxoId, value: &Coin) -> Result, Self::Error> { + Ok(self.data.lock().unwrap().coins.insert(*key, value.clone())) + } + + fn remove(&mut self, key: &UtxoId) -> Result, Self::Error> { + Ok(self.data.lock().unwrap().coins.remove(key)) + } + + fn get<'a>(&'a self, key: &UtxoId) -> Result>, Self::Error> { + Ok(self + .data + .lock() + .unwrap() + .coins + .get(key) + .map(|i| Cow::Owned(i.clone()))) + } + + fn contains_key(&self, key: &UtxoId) -> Result { + Ok(self.data.lock().unwrap().coins.contains_key(key)) + } + } + + impl Storage for MockDb { + type Error = db::Error; + + fn insert( + &mut self, + key: &ContractId, + value: &Contract, + ) -> Result, Self::Error> { + Ok(self + .data + .lock() + .unwrap() + .contracts + .insert(*key, value.clone())) + } + + fn remove(&mut self, key: &ContractId) -> Result, Self::Error> { + Ok(self.data.lock().unwrap().contracts.remove(key)) + } + + fn get<'a>( + &'a self, + key: &ContractId, + ) -> Result>, Self::Error> { + Ok(self + .data + .lock() + .unwrap() + .contracts + .get(key) + .map(|i| Cow::Owned(i.clone()))) + } + + fn contains_key(&self, key: &ContractId) -> Result { + Ok(self.data.lock().unwrap().contracts.contains_key(key)) + } + } + + impl Storage for MockDb { + type Error = db::KvStoreError; + + fn insert( + &mut self, + key: &MessageId, + value: &Message, + ) -> Result, Self::Error> { + Ok(self + .data + .lock() + .unwrap() + .messages + .insert(*key, value.clone())) + } + + fn remove(&mut self, key: &MessageId) -> Result, Self::Error> { + Ok(self.data.lock().unwrap().messages.remove(key)) + } + + fn get<'a>(&'a self, key: &MessageId) -> Result>, Self::Error> { + Ok(self + .data + .lock() + .unwrap() + .messages + .get(key) + .map(|i| Cow::Owned(i.clone()))) + } + + fn contains_key(&self, key: &MessageId) -> Result { + Ok(self.data.lock().unwrap().messages.contains_key(key)) + } + } + + impl TxPoolDb for MockDb {} + + pub(crate) fn create_message_predicate_from_message(message: &Message) -> Input { + Input::message_predicate( + message.id(), + message.sender, + message.recipient, + message.amount, + message.nonce, + message.owner, + message.data.clone(), + Default::default(), + Default::default(), + ) + } + } + use super::*; use crate::Error; - use fuel_core_interfaces::{common::fuel_tx::UtxoId, db::helpers::*, model::CoinStatus}; - use std::cmp::Reverse; - use std::sync::Arc; + use fuel_core_interfaces::{ + common::{ + fuel_storage::Storage, + fuel_tx::{TransactionBuilder, UtxoId}, + }, + db::helpers::*, + model::{CoinStatus, Message}, + }; + use std::{cmp::Reverse, sync::Arc}; #[tokio::test] async fn simple_insertion() { @@ -640,39 +783,231 @@ pub mod tests { } #[tokio::test] - async fn tx_above_min_byte_price_is_insertable() { - let config = Config { - min_byte_price: TX1_BYTE_PRICE, - ..Config::default() + async fn tx_inserted_into_pool_when_input_message_id_exists_in_db() { + let message = Message { + ..Default::default() }; - let db = DummyDb::filled(); - let tx1_hash = *TX_ID1; - let tx1 = Arc::new(DummyDb::dummy_tx(tx1_hash)); + let tx = TransactionBuilder::script(vec![], vec![]) + .add_input(helpers::create_message_predicate_from_message(&message)) + .finalize(); - let mut txpool = TxPool::new(config); + let mut db = helpers::MockDb::default(); + db.insert(&message.id(), &message).unwrap(); + let mut txpool = TxPool::new(Default::default()); - let out = txpool.insert_inner(tx1, &db).await; - assert!(out.is_ok(), "Tx1 should be OK, get err:{:?}", out); + txpool + .insert_inner(Arc::new(tx.clone()), &db) + .await + .expect("should succeed"); + + let returned_tx = TxPool::find_one(&RwLock::new(txpool), &tx.id()).await; + let tx_info = returned_tx.unwrap(); + assert_eq!(tx_info.tx().id(), tx.id()); } #[tokio::test] - async fn tx_below_min_byte_price_is_not_insertable() { - let config = Config { - min_byte_price: TX1_BYTE_PRICE + 1, - ..Config::default() + async fn tx_rejected_when_input_message_id_is_spent() { + let message = Message { + fuel_block_spend: Some(1u64.into()), + ..Default::default() }; - let db = DummyDb::filled(); - let tx1_hash = *TX_ID1; - let tx1 = Arc::new(DummyDb::dummy_tx(tx1_hash)); + let tx = TransactionBuilder::script(vec![], vec![]) + .add_input(helpers::create_message_predicate_from_message(&message)) + .finalize(); - let mut txpool = TxPool::new(config); + let mut db = helpers::MockDb::default(); + db.insert(&message.id(), &message).unwrap(); + let mut txpool = TxPool::new(Default::default()); - let err = txpool.insert_inner(tx1, &db).await.err().unwrap(); + let err = txpool + .insert_inner(Arc::new(tx.clone()), &db) + .await + .expect_err("should fail"); + + // check error assert!(matches!( - err.root_cause().downcast_ref::().unwrap(), - Error::NotInsertedBytePriceTooLow + err.downcast_ref::(), + Some(Error::NotInsertedInputMessageIdSpent(msg_id)) if msg_id == &message.id() + )); + } + + #[tokio::test] + async fn tx_rejected_from_pool_when_input_message_id_does_not_exist_in_db() { + let message = Message::default(); + let tx = TransactionBuilder::script(vec![], vec![]) + .add_input(helpers::create_message_predicate_from_message(&message)) + .finalize(); + + let db = helpers::MockDb::default(); + // Do not insert any messages into the DB to ensure there is no matching message for the + // tx. + + let mut txpool = TxPool::new(Default::default()); + + let err = txpool + .insert_inner(Arc::new(tx.clone()), &db) + .await + .expect_err("should fail"); + + // check error + assert!(matches!( + err.downcast_ref::(), + Some(Error::NotInsertedInputMessageUnknown(msg_id)) if msg_id == &message.id() + )); + } + + #[tokio::test] + async fn tx_rejected_from_pool_when_gas_price_is_lower_than_another_tx_with_same_message_id() { + let message_amount = 10_000; + let message = Message { + amount: message_amount, + ..Default::default() + }; + + let conflicting_message_input = helpers::create_message_predicate_from_message(&message); + let gas_price_high = 2u64; + let gas_price_low = 1u64; + + let tx_high = TransactionBuilder::script(vec![], vec![]) + .gas_price(gas_price_high) + .add_input(conflicting_message_input.clone()) + .finalize(); + + let tx_low = TransactionBuilder::script(vec![], vec![]) + .gas_price(gas_price_low) + .add_input(conflicting_message_input) + .finalize(); + + let mut db = helpers::MockDb::default(); + db.insert(&message.id(), &message).unwrap(); + + let mut txpool = TxPool::new(Default::default()); + + // Insert a tx for the message id with a high gas amount + txpool + .insert_inner(Arc::new(tx_high.clone()), &db) + .await + .expect("expected successful insertion"); + + // Insert a tx for the message id with a low gas amount + // Because the new transaction's id matches an existing transaction, we compare the gas + // prices of both the new and existing transactions. Since the existing transaction's gas + // price is higher, we must now reject the new transaction. + let err = txpool + .insert_inner(Arc::new(tx_low.clone()), &db) + .await + .expect_err("expected failure"); + + // check error + assert!(matches!( + err.downcast_ref::(), + Some(Error::NotInsertedCollisionMessageId(tx_id, msg_id)) if tx_id == &tx_high.id() && msg_id == &message.id() )); } + + #[tokio::test] + async fn higher_priced_tx_squeezes_out_lower_priced_tx_with_same_message_id() { + let message_amount = 10_000; + let message = Message { + amount: message_amount, + ..Default::default() + }; + + let conflicting_message_input = helpers::create_message_predicate_from_message(&message); + let gas_price_high = 2u64; + let gas_price_low = 1u64; + + // Insert a tx for the message id with a low gas amount + let tx_low = TransactionBuilder::script(vec![], vec![]) + .gas_price(gas_price_low) + .add_input(conflicting_message_input.clone()) + .finalize(); + + let mut db = helpers::MockDb::default(); + db.insert(&message.id(), &message).unwrap(); + + let mut txpool = TxPool::new(Default::default()); + + txpool + .insert_inner(Arc::new(tx_low.clone()), &db) + .await + .expect("should succeed"); + + // Insert a tx for the message id with a high gas amount + // Because the new transaction's id matches an existing transaction, we compare the gas + // prices of both the new and existing transactions. Since the existing transaction's gas + // price is lower, we accept the new transaction and squeeze out the old transaction. + let tx_high = TransactionBuilder::script(vec![], vec![]) + .gas_price(gas_price_high) + .add_input(conflicting_message_input) + .finalize(); + + let squeezed_out_txs = txpool + .insert_inner(Arc::new(tx_high.clone()), &db) + .await + .expect("should succeed"); + + assert_eq!(squeezed_out_txs.len(), 1); + assert_eq!(squeezed_out_txs[0].id(), tx_low.id()); + } + + #[tokio::test] + async fn message_of_squeezed_out_tx_can_be_resubmitted_at_lower_gas_price() { + // tx1 (message 1, message 2) gas_price 2 + // tx2 (message 1) gas_price 3 + // squeezes tx1 with higher gas price + // tx3 (message 2) gas_price 1 + // works since tx1 is no longer part of txpool state even though gas price is less + + let message_1 = Message { + amount: 10_000, + ..Default::default() + }; + let message_2 = Message { + amount: 20_000, + ..Default::default() + }; + + let message_input_1 = helpers::create_message_predicate_from_message(&message_1); + let message_input_2 = helpers::create_message_predicate_from_message(&message_2); + + // Insert a tx for the message id with a low gas amount + let tx_1 = TransactionBuilder::script(vec![], vec![]) + .gas_price(2) + .add_input(message_input_1.clone()) + .add_input(message_input_2.clone()) + .finalize(); + + let tx_2 = TransactionBuilder::script(vec![], vec![]) + .gas_price(3) + .add_input(message_input_1.clone()) + .finalize(); + + let tx_3 = TransactionBuilder::script(vec![], vec![]) + .gas_price(1) + .add_input(message_input_2.clone()) + .finalize(); + + let mut db = helpers::MockDb::default(); + db.insert(&message_1.id(), &message_1).unwrap(); + db.insert(&message_2.id(), &message_2).unwrap(); + let mut txpool = TxPool::new(Default::default()); + + txpool + .insert_inner(Arc::new(tx_1.clone()), &db) + .await + .expect("should succeed"); + + txpool + .insert_inner(Arc::new(tx_2.clone()), &db) + .await + .expect("should succeed"); + + txpool + .insert_inner(Arc::new(tx_3.clone()), &db) + .await + .expect("should succeed"); + } } From df7aeee74de5a9f7ed49ae5063b465ff96ec3de4 Mon Sep 17 00:00:00 2001 From: leviathanbeak88 Date: Thu, 11 Aug 2022 13:53:27 +0200 Subject: [PATCH 23/29] add some comments --- fuel-p2p/src/orchestrator.rs | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/fuel-p2p/src/orchestrator.rs b/fuel-p2p/src/orchestrator.rs index eb4b4c1c0c4..5fe84160568 100644 --- a/fuel-p2p/src/orchestrator.rs +++ b/fuel-p2p/src/orchestrator.rs @@ -137,8 +137,11 @@ impl NetworkOrchestrator { } pub struct Service { - join: Mutex>>>, + /// Network Orchestrator that handles p2p network and inter-module communication network_orchestrator: Arc>>, + /// Holds the spawned task when Netowrk Orchestrator is started + join: Mutex>>>, + /// Used for notifying the Network Orchestrator to stop tx_request_event: Sender, } From 8d817bd5280d9567f5d63c310de668d6c01aa81a Mon Sep 17 00:00:00 2001 From: leviathanbeak88 Date: Mon, 15 Aug 2022 16:00:58 +0200 Subject: [PATCH 24/29] clean up --- fuel-p2p/src/service.rs | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/fuel-p2p/src/service.rs b/fuel-p2p/src/service.rs index 6e8b711b1e6..fa5f3f35950 100644 --- a/fuel-p2p/src/service.rs +++ b/fuel-p2p/src/service.rs @@ -179,9 +179,7 @@ mod tests { /// helper function for building FuelP2PService async fn build_fuel_p2p_service(mut p2p_config: P2PConfig) -> FuelP2PService { p2p_config.local_keypair = Keypair::generate_secp256k1(); // change keypair for each Node - let fuel_p2p_service = FuelP2PService::new(p2p_config).await.unwrap(); - - fuel_p2p_service + FuelP2PService::new(p2p_config).await.unwrap() } /// attaches PeerId to the Multiaddr From 9d8e16cc69a37b0baf1cfce68d11da82265781b8 Mon Sep 17 00:00:00 2001 From: leviathanbeak88 Date: Wed, 17 Aug 2022 17:37:53 +0200 Subject: [PATCH 25/29] clean extra space --- fuel-core/src/cli/run/p2p.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/fuel-core/src/cli/run/p2p.rs b/fuel-core/src/cli/run/p2p.rs index 9ae56d009f2..1fddd3799da 100644 --- a/fuel-core/src/cli/run/p2p.rs +++ b/fuel-core/src/cli/run/p2p.rs @@ -54,7 +54,7 @@ pub struct P2pArgs { pub allow_private_addresses: bool, /// Choose how long will connection keep alive if idle - #[clap(long = "connection_idle_timeout ", default_value = "120")] + #[clap(long = "connection_idle_timeout", default_value = "120")] pub connection_idle_timeout: u64, /// Choose how often to recieve PeerInfo from other nodes From 418ed65f3e625b5ed5fd18112324c922ec4be549 Mon Sep 17 00:00:00 2001 From: leviathanbeak88 Date: Wed, 17 Aug 2022 18:01:37 +0200 Subject: [PATCH 26/29] set default values within clap --- fuel-core/src/cli/run/p2p.rs | 12 ++++++------ fuel-p2p/src/behavior.rs | 11 +++-------- fuel-p2p/src/config.rs | 10 +++++----- fuel-p2p/src/service.rs | 2 +- 4 files changed, 15 insertions(+), 20 deletions(-) diff --git a/fuel-core/src/cli/run/p2p.rs b/fuel-core/src/cli/run/p2p.rs index 1fddd3799da..a798034dafc 100644 --- a/fuel-core/src/cli/run/p2p.rs +++ b/fuel-core/src/cli/run/p2p.rs @@ -83,12 +83,12 @@ pub struct P2pArgs { pub ideal_mesh_size: usize, /// Choose timeout for sent requests in RequestResponse protocol - #[clap(long = "request_timeout")] - pub request_timeout: Option, + #[clap(long = "request_timeout", default_value = "20")] + pub request_timeout: u64, /// Choose how long RequestResponse protocol connections will live if idle - #[clap(long = "connection_keep_alive")] - pub connection_keep_alive: Option, + #[clap(long = "connection_keep_alive", default_value = "20")] + pub connection_keep_alive: u64, } impl From for anyhow::Result { @@ -121,8 +121,8 @@ impl From for anyhow::Result { max_mesh_size: args.max_mesh_size, min_mesh_size: args.min_mesh_size, ideal_mesh_size: args.ideal_mesh_size, - set_request_timeout: args.request_timeout.map(Duration::from_secs), - set_connection_keep_alive: args.connection_keep_alive.map(Duration::from_secs), + set_request_timeout: Duration::from_secs(args.request_timeout), + set_connection_keep_alive: Duration::from_secs(args.connection_keep_alive), info_interval: Some(Duration::from_secs(args.info_interval)), identify_interval: Some(Duration::from_secs(args.identify_interval)), }) diff --git a/fuel-p2p/src/behavior.rs b/fuel-p2p/src/behavior.rs index ee63ef0de57..4e64078a5b5 100644 --- a/fuel-p2p/src/behavior.rs +++ b/fuel-p2p/src/behavior.rs @@ -1,6 +1,6 @@ use crate::{ codecs::NetworkCodec, - config::{P2PConfig, REQ_RES_TIMEOUT}, + config::P2PConfig, discovery::{DiscoveryBehaviour, DiscoveryConfig, DiscoveryEvent}, gossipsub::{ self, @@ -131,13 +131,8 @@ impl FuelBehaviour { std::iter::once((codec.get_req_res_protocol(), ProtocolSupport::Full)); let mut req_res_config = RequestResponseConfig::default(); - req_res_config - .set_request_timeout(p2p_config.set_request_timeout.unwrap_or(REQ_RES_TIMEOUT)); - req_res_config.set_connection_keep_alive( - p2p_config - .set_connection_keep_alive - .unwrap_or(REQ_RES_TIMEOUT), - ); + req_res_config.set_request_timeout(p2p_config.set_request_timeout); + req_res_config.set_connection_keep_alive(p2p_config.set_connection_keep_alive); let request_response = RequestResponse::new(codec.clone(), req_res_protocol, req_res_config); diff --git a/fuel-p2p/src/config.rs b/fuel-p2p/src/config.rs index d4bff32c667..be42fe35968 100644 --- a/fuel-p2p/src/config.rs +++ b/fuel-p2p/src/config.rs @@ -8,7 +8,7 @@ use std::{ time::Duration, }; -pub const REQ_RES_TIMEOUT: Duration = Duration::from_secs(20); +const REQ_RES_TIMEOUT: Duration = Duration::from_secs(20); /// Maximum number of frames buffered per substream. const MAX_NUM_OF_FRAMES_BUFFERED: usize = 256; @@ -57,9 +57,9 @@ pub struct P2PConfig { // RequestResponse related fields /// Sets the timeout for inbound and outbound requests. - pub set_request_timeout: Option, + pub set_request_timeout: Duration, /// Sets the keep-alive timeout of idle connections. - pub set_connection_keep_alive: Option, + pub set_connection_keep_alive: Duration, } impl P2PConfig { @@ -82,8 +82,8 @@ impl P2PConfig { max_mesh_size: 12, min_mesh_size: 4, ideal_mesh_size: 6, - set_request_timeout: None, - set_connection_keep_alive: None, + set_request_timeout: REQ_RES_TIMEOUT, + set_connection_keep_alive: REQ_RES_TIMEOUT, info_interval: Some(Duration::from_secs(3)), identify_interval: Some(Duration::from_secs(5)), } diff --git a/fuel-p2p/src/service.rs b/fuel-p2p/src/service.rs index fa5f3f35950..d2f84a97679 100644 --- a/fuel-p2p/src/service.rs +++ b/fuel-p2p/src/service.rs @@ -545,7 +545,7 @@ mod tests { // Node A // setup request timeout to 0 in order for the Request to fail - p2p_config.set_request_timeout = Some(Duration::from_secs(0)); + p2p_config.set_request_timeout = Duration::from_secs(0); let mut node_a = build_fuel_p2p_service(p2p_config.clone()).await; let node_a_address = match node_a.next_event().await { From 53581e58ac9ebd49d5996e013dc2a32ea925b83e Mon Sep 17 00:00:00 2001 From: leviathanbeak88 Date: Thu, 18 Aug 2022 21:54:42 +0200 Subject: [PATCH 27/29] use fuel-crypto for key generation --- Cargo.lock | 1 + fuel-core/Cargo.toml | 1 + fuel-core/src/cli/run/p2p.rs | 19 +++++++++++++++---- fuel-core/src/service/config.rs | 4 ++-- fuel-p2p/src/config.rs | 12 ++++++++++-- fuel-p2p/src/lib.rs | 6 +++--- 6 files changed, 32 insertions(+), 11 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 6ea27d113ea..bd94eeba5c1 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -2117,6 +2117,7 @@ dependencies = [ "fuel-block-producer", "fuel-core-bft", "fuel-core-interfaces", + "fuel-crypto", "fuel-p2p", "fuel-relayer", "fuel-sync", diff --git a/fuel-core/Cargo.toml b/fuel-core/Cargo.toml index 0a7e42604e9..189c52b1b43 100644 --- a/fuel-core/Cargo.toml +++ b/fuel-core/Cargo.toml @@ -40,6 +40,7 @@ fuel-core-bft = { path = "../fuel-core-bft", version = "0.10.1" } fuel-core-interfaces = { path = "../fuel-core-interfaces", version = "0.10.1", features = [ "serde", ] } +fuel-crypto = { version = "0.6", default-features = false, features = [ "random" ] } fuel-p2p = { path = "../fuel-p2p", version = "0.10.1" } fuel-relayer = { path = "../fuel-relayer", version = "0.10.1" } fuel-sync = { path = "../fuel-sync", version = "0.10.1" } diff --git a/fuel-core/src/cli/run/p2p.rs b/fuel-core/src/cli/run/p2p.rs index a798034dafc..0360467ace3 100644 --- a/fuel-core/src/cli/run/p2p.rs +++ b/fuel-core/src/cli/run/p2p.rs @@ -6,7 +6,7 @@ use std::{ use clap::Args; -use fuel_p2p::{Keypair, Multiaddr, P2PConfig}; +use fuel_p2p::{config::P2PConfig, Multiaddr}; #[derive(Debug, Clone, Args)] pub struct P2pArgs { @@ -96,10 +96,21 @@ impl From for anyhow::Result { let local_keypair = { match args.keypair { Some(path) => { - let mut bytes = std::fs::read(path)?; - Keypair::secp256k1_from_der(&mut bytes)? + let phrase = std::fs::read_to_string(path)?; + + let secret_key = fuel_crypto::SecretKey::new_from_mnemonic_phrase_with_path( + &phrase, + "m/44'/60'/0'/0/0", + )?; + + fuel_p2p::config::convert_to_libp2p_keypair(&mut secret_key.to_vec())? + } + _ => { + let mut rand = fuel_crypto::rand::thread_rng(); + let secret_key = fuel_crypto::SecretKey::random(&mut rand); + + fuel_p2p::config::convert_to_libp2p_keypair(&mut secret_key.to_vec())? } - _ => Keypair::generate_secp256k1(), } }; diff --git a/fuel-core/src/service/config.rs b/fuel-core/src/service/config.rs index 61aafd16fb6..e088311b206 100644 --- a/fuel-core/src/service/config.rs +++ b/fuel-core/src/service/config.rs @@ -24,7 +24,7 @@ pub struct Config { pub bft: fuel_core_bft::Config, pub sync: fuel_sync::Config, pub relayer: fuel_relayer::Config, - pub p2p: fuel_p2p::P2PConfig, + pub p2p: fuel_p2p::config::P2PConfig, } impl Config { @@ -45,7 +45,7 @@ impl Config { bft: Default::default(), sync: Default::default(), relayer: Default::default(), - p2p: fuel_p2p::P2PConfig::default_with_network("test_network"), + p2p: fuel_p2p::config::P2PConfig::default_with_network("test_network"), } } } diff --git a/fuel-p2p/src/config.rs b/fuel-p2p/src/config.rs index be42fe35968..1cdc4e0e1a2 100644 --- a/fuel-p2p/src/config.rs +++ b/fuel-p2p/src/config.rs @@ -1,6 +1,6 @@ use libp2p::{ core::{muxing::StreamMuxerBox, transport::Boxed}, - identity::Keypair, + identity::{secp256k1::SecretKey, Keypair}, mplex, noise, yamux, Multiaddr, PeerId, Transport, }; use std::{ @@ -62,6 +62,14 @@ pub struct P2PConfig { pub set_connection_keep_alive: Duration, } +/// Takes secret key bytes generated outside of libp2p. +/// And converts it into libp2p's `Keypair::Secp256k1`. +pub fn convert_to_libp2p_keypair(secret_key_bytes: impl AsMut<[u8]>) -> anyhow::Result { + let secret_key = SecretKey::from_bytes(secret_key_bytes)?; + + Ok(Keypair::Secp256k1(secret_key.into())) +} + impl P2PConfig { pub fn default_with_network(network_name: &str) -> Self { let local_keypair = Keypair::generate_secp256k1(); @@ -94,7 +102,7 @@ impl P2PConfig { /// TCP/IP, Websocket /// Noise as encryption layer /// mplex or yamux for multiplexing -pub async fn build_transport(local_keypair: Keypair) -> Boxed<(PeerId, StreamMuxerBox)> { +pub(crate) async fn build_transport(local_keypair: Keypair) -> Boxed<(PeerId, StreamMuxerBox)> { let transport = { let tcp = libp2p::tcp::TcpConfig::new().nodelay(true); let ws_tcp = libp2p::websocket::WsConfig::new(tcp.clone()).or_transport(tcp); diff --git a/fuel-p2p/src/lib.rs b/fuel-p2p/src/lib.rs index 01a280270b9..7cd14cae7e8 100644 --- a/fuel-p2p/src/lib.rs +++ b/fuel-p2p/src/lib.rs @@ -1,11 +1,11 @@ mod behavior; mod codecs; -mod config; +pub mod config; mod discovery; mod gossipsub; pub mod orchestrator; mod peer_info; mod request_response; mod service; -pub use config::P2PConfig; -pub use libp2p::{identity::Keypair, Multiaddr, PeerId}; + +pub use libp2p::{Multiaddr, PeerId}; From 06e0dafa38738d58f2f2ff0ac79e9e5da29b4eee Mon Sep 17 00:00:00 2001 From: leviathanbeak88 Date: Mon, 22 Aug 2022 13:09:17 +0200 Subject: [PATCH 28/29] add p2p optional feature --- fuel-core/Cargo.toml | 3 ++- fuel-core/src/cli/run.rs | 6 +++++- fuel-core/src/service/config.rs | 5 +++++ fuel-core/src/service/modules.rs | 17 ++++++++++++++++- 4 files changed, 28 insertions(+), 3 deletions(-) diff --git a/fuel-core/Cargo.toml b/fuel-core/Cargo.toml index 189c52b1b43..bb592fd5d32 100644 --- a/fuel-core/Cargo.toml +++ b/fuel-core/Cargo.toml @@ -41,7 +41,7 @@ fuel-core-interfaces = { path = "../fuel-core-interfaces", version = "0.10.1", f "serde", ] } fuel-crypto = { version = "0.6", default-features = false, features = [ "random" ] } -fuel-p2p = { path = "../fuel-p2p", version = "0.10.1" } +fuel-p2p = { path = "../fuel-p2p", version = "0.10.1", optional = true } fuel-relayer = { path = "../fuel-relayer", version = "0.10.1" } fuel-sync = { path = "../fuel-sync", version = "0.10.1" } fuel-txpool = { path = "../fuel-txpool", version = "0.10.1" } @@ -89,3 +89,4 @@ prometheus = ["dep:prometheus"] default = ["rocksdb", "prometheus", "debug"] debug = ["fuel-core-interfaces/debug"] test-helpers = [] +p2p = [] diff --git a/fuel-core/src/cli/run.rs b/fuel-core/src/cli/run.rs index 365e4974166..ef6345cfc4c 100644 --- a/fuel-core/src/cli/run.rs +++ b/fuel-core/src/cli/run.rs @@ -5,7 +5,7 @@ use fuel_core::service::{Config, DbType, VMConfig}; use std::{env, io, net, path::PathBuf}; use strum::VariantNames; use tracing::{info, trace}; - +#[cfg(feature = "p2p")] mod p2p; mod relayer; @@ -57,6 +57,7 @@ pub struct Command { #[clap(flatten)] pub relayer_args: relayer::RelayerArgs, + #[cfg(feature = "p2p")] #[clap(flatten)] pub p2p_args: p2p::P2pArgs, } @@ -75,11 +76,13 @@ impl Command { min_gas_price, predicates, relayer_args, + #[cfg(feature = "p2p")] p2p_args, } = self; let addr = net::SocketAddr::new(ip, port); + #[cfg(feature = "p2p")] let p2p = { match p2p_args.into() { Ok(value) => value, @@ -108,6 +111,7 @@ impl Command { relayer: relayer_args.into(), bft: Default::default(), sync: Default::default(), + #[cfg(feature = "p2p")] p2p, }) } diff --git a/fuel-core/src/service/config.rs b/fuel-core/src/service/config.rs index e088311b206..d01abe334ff 100644 --- a/fuel-core/src/service/config.rs +++ b/fuel-core/src/service/config.rs @@ -5,6 +5,9 @@ use std::{ }; use strum_macros::{Display, EnumString, EnumVariantNames}; +#[cfg(feature = "p2p")] +use fuel_p2p; + #[derive(Clone, Debug)] pub struct Config { pub addr: SocketAddr, @@ -24,6 +27,7 @@ pub struct Config { pub bft: fuel_core_bft::Config, pub sync: fuel_sync::Config, pub relayer: fuel_relayer::Config, + #[cfg(feature = "p2p")] pub p2p: fuel_p2p::config::P2PConfig, } @@ -45,6 +49,7 @@ impl Config { bft: Default::default(), sync: Default::default(), relayer: Default::default(), + #[cfg(feature = "p2p")] p2p: fuel_p2p::config::P2PConfig::default_with_network("test_network"), } } diff --git a/fuel-core/src/service/modules.rs b/fuel-core/src/service/modules.rs index e86cebec1dd..e0bacfb201e 100644 --- a/fuel-core/src/service/modules.rs +++ b/fuel-core/src/service/modules.rs @@ -2,6 +2,7 @@ use crate::database::Database; use crate::service::Config; use anyhow::Result; +#[cfg(feature = "p2p")] use fuel_core_interfaces::p2p::P2pDb; use fuel_core_interfaces::relayer::RelayerDb; use fuel_core_interfaces::txpool::TxPoolDb; @@ -17,6 +18,7 @@ pub struct Modules { pub bft: Arc, pub sync: Arc, pub relayer: Arc, + #[cfg(feature = "p2p")] pub network_service: Arc, } @@ -28,6 +30,7 @@ impl Modules { self.block_producer.stop().await, self.bft.stop().await, self.sync.stop().await, + #[cfg(feature = "p2p")] self.network_service.stop().await, ] .into_iter() @@ -65,9 +68,16 @@ pub async fn start_modules(config: &Config, database: &Database) -> Result) .import_block_event(block_importer.subscribe()); + #[cfg(feature = "p2p")] let (tx_request_event, rx_request_event) = mpsc::channel(100); + #[cfg(feature = "p2p")] let (tx_block, rx_block) = mpsc::channel(100); + #[cfg(not(feature = "p2p"))] + let (tx_request_event, _) = mpsc::channel(100); + #[cfg(not(feature = "p2p"))] + let (_, rx_block) = mpsc::channel(100); + block_importer.start().await; block_producer.start(txpool_builder.sender().clone()).await; @@ -99,11 +109,14 @@ pub async fn start_modules(config: &Config, database: &Database) -> Result = Arc::new(database.clone()); - + #[cfg(feature = "p2p")] let (tx_consensus, _) = mpsc::channel(100); + #[cfg(feature = "p2p")] let (tx_transaction, _) = mpsc::channel(100); + #[cfg(feature = "p2p")] let network_service = fuel_p2p::orchestrator::Service::new( config.p2p.clone(), p2p_db, @@ -114,6 +127,7 @@ pub async fn start_modules(config: &Config, database: &Database) -> Result Result Date: Tue, 23 Aug 2022 07:11:47 -0700 Subject: [PATCH 29/29] Update fuel-core/Cargo.toml --- fuel-core/Cargo.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/fuel-core/Cargo.toml b/fuel-core/Cargo.toml index bb592fd5d32..1e5d24324cf 100644 --- a/fuel-core/Cargo.toml +++ b/fuel-core/Cargo.toml @@ -89,4 +89,4 @@ prometheus = ["dep:prometheus"] default = ["rocksdb", "prometheus", "debug"] debug = ["fuel-core-interfaces/debug"] test-helpers = [] -p2p = [] +p2p = ["dep:fuel-p2p"]