From 69da965f13b43e8b8ed356b891c05868494cc797 Mon Sep 17 00:00:00 2001 From: grumbach Date: Thu, 5 Dec 2024 17:43:43 +0900 Subject: [PATCH 01/33] feat: quoting upgrade nodeside with WIP client side --- ant-evm/src/data_payments.rs | 65 +---- ant-evm/src/lib.rs | 3 +- ant-networking/src/cmd.rs | 26 +- ant-networking/src/error.rs | 2 +- ant-networking/src/lib.rs | 163 ++--------- ant-networking/src/log_markers.rs | 8 +- ant-networking/src/metrics/mod.rs | 16 +- ant-networking/src/record_store.rs | 379 +------------------------ ant-networking/src/record_store_api.rs | 14 +- ant-node/src/error.rs | 5 +- ant-node/src/node.rs | 36 ++- ant-node/src/put_validation.rs | 17 +- ant-node/src/quote.rs | 30 +- ant-protocol/src/error.rs | 2 +- ant-protocol/src/messages/query.rs | 10 +- ant-protocol/src/messages/response.rs | 18 +- autonomi/src/client/data/public.rs | 18 +- autonomi/src/client/data_private.rs | 129 +++++++++ autonomi/src/client/mod.rs | 1 + autonomi/src/client/payment.rs | 8 +- autonomi/src/client/quote.rs | 130 +++++++++ autonomi/src/client/registers.rs | 9 +- autonomi/src/client/utils.rs | 109 ++----- autonomi/src/client/vault.rs | 17 +- autonomi/src/lib.rs | 1 - autonomi/src/utils.rs | 39 --- evmlib/src/lib.rs | 12 +- evmlib/src/quoting_metrics.rs | 47 +++ evmlib/src/transaction.rs | 4 +- 29 files changed, 480 insertions(+), 838 deletions(-) create mode 100644 autonomi/src/client/data_private.rs create mode 100644 autonomi/src/client/quote.rs delete mode 100644 autonomi/src/utils.rs create mode 100644 evmlib/src/quoting_metrics.rs diff --git a/ant-evm/src/data_payments.rs b/ant-evm/src/data_payments.rs index 89751e4d23..f091d65290 100644 --- a/ant-evm/src/data_payments.rs +++ b/ant-evm/src/data_payments.rs @@ -6,11 +6,9 @@ // KIND, either express or implied. Please review the Licences for the specific language governing // permissions and limitations relating to use of the SAFE Network Software. -use crate::{AttoTokens, EvmError}; -use evmlib::common::TxHash; +use crate::EvmError; use evmlib::{ - common::{Address as RewardsAddress, QuoteHash}, - utils::dummy_address, + common::{Address as RewardsAddress, QuoteHash, TxHash}, quoting_metrics::QuotingMetrics, utils::dummy_address }; use libp2p::{identity::PublicKey, PeerId}; use serde::{Deserialize, Serialize}; @@ -42,46 +40,6 @@ impl ProofOfPayment { } } -/// Quoting metrics that got used to generate a quote, or to track peer's status. -#[derive( - Clone, Eq, PartialEq, PartialOrd, Ord, Hash, Serialize, Deserialize, custom_debug::Debug, -)] -pub struct QuotingMetrics { - /// the records stored - pub close_records_stored: usize, - /// the max_records configured - pub max_records: usize, - /// number of times that got paid - pub received_payment_count: usize, - /// the duration that node keeps connected to the network, measured in hours - pub live_time: u64, - /// network density from this node's perspective, which is the responsible_range as well - /// This could be calculated via sampling, or equation calculation. - pub network_density: Option<[u8; 32]>, - /// estimated network size - pub network_size: Option, -} - -impl QuotingMetrics { - /// construct an empty QuotingMetrics - pub fn new() -> Self { - Self { - close_records_stored: 0, - max_records: 0, - received_payment_count: 0, - live_time: 0, - network_density: None, - network_size: None, - } - } -} - -impl Default for QuotingMetrics { - fn default() -> Self { - Self::new() - } -} - /// A payment quote to store data given by a node to a client /// Note that the PaymentQuote is a contract between the node and itself to make sure the clients aren’t mispaying. /// It is NOT a contract between the client and the node. @@ -89,17 +47,10 @@ impl Default for QuotingMetrics { pub struct PaymentQuote { /// the content paid for pub content: XorName, - /// how much the node demands for storing the content - /// TODO: to be removed once swtich to `client querying smart_contract` - pub cost: AttoTokens, /// the local node time when the quote was created pub timestamp: SystemTime, /// quoting metrics being used to generate this quote pub quoting_metrics: QuotingMetrics, - /// list of bad_nodes that client shall not pick as a payee - /// in `serialised` format to avoid cyclic dependent on ant_protocol - #[debug(skip)] - pub bad_nodes: Vec, /// the node's wallet address pub rewards_address: RewardsAddress, /// the node's libp2p identity public key in bytes (PeerId) @@ -115,10 +66,8 @@ impl PaymentQuote { pub fn zero() -> Self { Self { content: Default::default(), - cost: AttoTokens::zero(), timestamp: SystemTime::now(), quoting_metrics: Default::default(), - bad_nodes: vec![], rewards_address: dummy_address(), pub_key: vec![], signature: vec![], @@ -135,14 +84,11 @@ impl PaymentQuote { /// returns the bytes to be signed from the given parameters pub fn bytes_for_signing( xorname: XorName, - cost: AttoTokens, timestamp: SystemTime, quoting_metrics: &QuotingMetrics, - serialised_bad_nodes: &[u8], rewards_address: &RewardsAddress, ) -> Vec { let mut bytes = xorname.to_vec(); - bytes.extend_from_slice(&cost.to_bytes()); bytes.extend_from_slice( ×tamp .duration_since(SystemTime::UNIX_EPOCH) @@ -152,7 +98,6 @@ impl PaymentQuote { ); let serialised_quoting_metrics = rmp_serde::to_vec(quoting_metrics).unwrap_or_default(); bytes.extend_from_slice(&serialised_quoting_metrics); - bytes.extend_from_slice(serialised_bad_nodes); bytes.extend_from_slice(rewards_address.as_slice()); bytes } @@ -161,10 +106,8 @@ impl PaymentQuote { pub fn bytes_for_sig(&self) -> Vec { Self::bytes_for_signing( self.content, - self.cost, self.timestamp, &self.quoting_metrics, - &self.bad_nodes, &self.rewards_address, ) } @@ -217,13 +160,11 @@ impl PaymentQuote { } /// test utility to create a dummy quote - pub fn test_dummy(xorname: XorName, cost: AttoTokens) -> Self { + pub fn test_dummy(xorname: XorName) -> Self { Self { content: xorname, - cost, timestamp: SystemTime::now(), quoting_metrics: Default::default(), - bad_nodes: vec![], pub_key: vec![], signature: vec![], rewards_address: dummy_address(), diff --git a/ant-evm/src/lib.rs b/ant-evm/src/lib.rs index 45185101fb..d32ad1858f 100644 --- a/ant-evm/src/lib.rs +++ b/ant-evm/src/lib.rs @@ -28,7 +28,8 @@ mod amount; mod data_payments; mod error; -pub use data_payments::{PaymentQuote, ProofOfPayment, QuotingMetrics, QUOTE_EXPIRATION_SECS}; +pub use evmlib::quoting_metrics::QuotingMetrics; +pub use data_payments::{PaymentQuote, ProofOfPayment, QUOTE_EXPIRATION_SECS}; /// Types used in the public API pub use amount::{Amount, AttoTokens}; diff --git a/ant-networking/src/cmd.rs b/ant-networking/src/cmd.rs index 31987e8e72..0e10fbf0eb 100644 --- a/ant-networking/src/cmd.rs +++ b/ant-networking/src/cmd.rs @@ -13,7 +13,7 @@ use crate::{ log_markers::Marker, multiaddr_pop_p2p, GetRecordCfg, GetRecordError, MsgResponder, NetworkEvent, CLOSE_GROUP_SIZE, }; -use ant_evm::{AttoTokens, PaymentQuote, QuotingMetrics}; +use ant_evm::{PaymentQuote, QuotingMetrics}; use ant_protocol::{ messages::{Cmd, Request, Response}, storage::{RecordHeader, RecordKind, RecordType}, @@ -98,10 +98,11 @@ pub enum LocalSwarmCmd { key: RecordKey, sender: oneshot::Sender>, }, - /// GetLocalStoreCost for this node, also with the bad_node list close to the target - GetLocalStoreCost { + /// GetLocalQuotingMetrics for this node + /// Returns the quoting metrics and whether the record at `key` is already stored locally + GetLocalQuotingMetrics { key: RecordKey, - sender: oneshot::Sender<(AttoTokens, QuotingMetrics, Vec)>, + sender: oneshot::Sender<(QuotingMetrics, bool)>, }, /// Notify the node received a payment. PaymentReceived, @@ -241,8 +242,8 @@ impl Debug for LocalSwarmCmd { "LocalSwarmCmd::GetCloseGroupLocalPeers {{ key: {key:?} }}" ) } - LocalSwarmCmd::GetLocalStoreCost { .. } => { - write!(f, "LocalSwarmCmd::GetLocalStoreCost") + LocalSwarmCmd::GetLocalQuotingMetrics { .. } => { + write!(f, "LocalSwarmCmd::GetLocalQuotingMetrics") } LocalSwarmCmd::PaymentReceived => { write!(f, "LocalSwarmCmd::PaymentReceived") @@ -573,8 +574,8 @@ impl SwarmDriver { cmd_string = "TriggerIntervalReplication"; self.try_interval_replication()?; } - LocalSwarmCmd::GetLocalStoreCost { key, sender } => { - cmd_string = "GetLocalStoreCost"; + LocalSwarmCmd::GetLocalQuotingMetrics { key, sender } => { + cmd_string = "GetLocalQuotingMetrics"; let ( _index, _total_peers, @@ -584,15 +585,14 @@ impl SwarmDriver { ) = self.kbuckets_status(); let estimated_network_size = Self::estimate_network_size(peers_in_non_full_buckets, num_of_full_buckets); - let (cost, quoting_metrics) = self + let (quoting_metrics, is_already_stored) = self .swarm .behaviour_mut() .kademlia .store_mut() - .store_cost(&key, Some(estimated_network_size as u64)); + .quoting_metrics(&key, Some(estimated_network_size as u64)); - self.record_metrics(Marker::StoreCost { - cost: cost.as_atto(), + self.record_metrics(Marker::QuotingMetrics { quoting_metrics: "ing_metrics, }); @@ -630,7 +630,7 @@ impl SwarmDriver { .retain(|peer_addr| key_address.distance(peer_addr) < boundary_distance); } - let _res = sender.send((cost, quoting_metrics, bad_nodes)); + let _res = sender.send((quoting_metrics, is_already_stored)); } LocalSwarmCmd::PaymentReceived => { cmd_string = "PaymentReceived"; diff --git a/ant-networking/src/error.rs b/ant-networking/src/error.rs index 9835e8f1d2..c683ff4432 100644 --- a/ant-networking/src/error.rs +++ b/ant-networking/src/error.rs @@ -178,7 +178,7 @@ pub enum NetworkError { OutgoingResponseDropped(Response), #[error("Error setting up behaviour: {0}")] - BahviourErr(String), + BehaviourErr(String), #[error("Register already exists at this address")] RegisterAlreadyExists, diff --git a/ant-networking/src/lib.rs b/ant-networking/src/lib.rs index cfe81e6b0b..eb4c3dea2a 100644 --- a/ant-networking/src/lib.rs +++ b/ant-networking/src/lib.rs @@ -40,7 +40,7 @@ pub use self::{ }, error::{GetRecordError, NetworkError}, event::{MsgResponder, NetworkEvent}, - record_store::{calculate_cost_for_records, NodeRecordStore}, + record_store::NodeRecordStore, transactions::get_transactions_from_record, }; #[cfg(feature = "open-metrics")] @@ -48,7 +48,7 @@ pub use metrics::service::MetricsRegistries; pub use target_arch::{interval, sleep, spawn, Instant, Interval}; use self::{cmd::NetworkSwarmCmd, error::Result}; -use ant_evm::{AttoTokens, PaymentQuote, QuotingMetrics, RewardsAddress}; +use ant_evm::{PaymentQuote, QuotingMetrics}; use ant_protocol::{ error::Error as ProtocolError, messages::{ChunkProof, Cmd, Nonce, Query, QueryResponse, Request, Response}, @@ -84,7 +84,7 @@ use { }; /// The type of quote for a selected payee. -pub type PayeeQuote = (PeerId, RewardsAddress, PaymentQuote); +pub type PayeeQuote = (PeerId, PaymentQuote); /// Majority of a given group (i.e. > 1/2). #[inline] @@ -378,11 +378,11 @@ impl Network { /// /// Ignore the quote from any peers from `ignore_peers`. /// This is useful if we want to repay a different PeerId on failure. - pub async fn get_store_costs_from_network( + pub async fn get_store_quote_from_network( &self, record_address: NetworkAddress, ignore_peers: Vec, - ) -> Result { + ) -> Result> { // The requirement of having at least CLOSE_GROUP_SIZE // close nodes will be checked internally automatically. let mut close_nodes = self @@ -397,7 +397,7 @@ impl Network { } // Client shall decide whether to carry out storage verification or not. - let request = Request::Query(Query::GetStoreCost { + let request = Request::Query(Query::GetStoreQuote { key: record_address.clone(), nonce: None, difficulty: 0, @@ -406,54 +406,51 @@ impl Network { .send_and_get_responses(&close_nodes, &request, true) .await; - // loop over responses, generating an average fee and storing all responses along side - let mut all_costs = vec![]; + // loop over responses let mut all_quotes = vec![]; - for response in responses.into_values().flatten() { + let mut quotes_to_pay = vec![]; + for (peer, response) in responses { info!( - "StoreCostReq for {record_address:?} received response: {:?}", - response - ); + "StoreCostReq for {record_address:?} received response: {response:?}"); match response { - Response::Query(QueryResponse::GetStoreCost { + Ok(Response::Query(QueryResponse::GetStoreQuote { quote: Ok(quote), - payment_address, peer_address, storage_proofs, - }) => { + })) => { if !storage_proofs.is_empty() { - debug!("Storage proofing during GetStoreCost to be implemented."); + debug!("Storage proofing during GetStoreQuote to be implemented."); } // Check the quote itself is valid. - if quote.cost - != AttoTokens::from_u64(calculate_cost_for_records( - quote.quoting_metrics.close_records_stored, - )) - { + if !quote.check_is_signed_by_claimed_peer(peer) { warn!("Received invalid quote from {peer_address:?}, {quote:?}"); continue; } - all_costs.push((peer_address.clone(), payment_address, quote.clone())); - all_quotes.push((peer_address, quote)); + all_quotes.push((peer_address.clone(), quote.clone())); + quotes_to_pay.push((peer, quote)); } - Response::Query(QueryResponse::GetStoreCost { + Ok(Response::Query(QueryResponse::GetStoreQuote { quote: Err(ProtocolError::RecordExists(_)), - payment_address, peer_address, storage_proofs, - }) => { + })) => { if !storage_proofs.is_empty() { - debug!("Storage proofing during GetStoreCost to be implemented."); + debug!("Storage proofing during GetStoreQuote to be implemented."); } - all_costs.push((peer_address, payment_address, PaymentQuote::zero())); + info!("Address {record_address:?} was already paid for according to {peer_address:?}, ending quote request"); + return Ok(vec![]); + } + Err(err) => { + error!("Got an error while requesting quote from peer {peer:?}: {err}"); } _ => { - error!("Non store cost response received, was {:?}", response); + error!("Got an unexpected response while requesting quote from peer {peer:?}: {response:?}"); } } } + // send the quotes to the other peers for verification for peer_id in close_nodes.iter() { let request = Request::Cmd(Cmd::QuoteVerification { target: NetworkAddress::from_peer(*peer_id), @@ -463,7 +460,7 @@ impl Network { self.send_req_ignore_reply(request, *peer_id); } - get_fees_from_store_cost_responses(all_costs) + Ok(quotes_to_pay) } /// Get register from network. @@ -776,13 +773,13 @@ impl Network { Ok(None) } - /// Get the cost of storing the next record from the network - pub async fn get_local_storecost( + /// Get the quoting metrics for storing the next record from the network + pub async fn get_local_quoting_metrics( &self, key: RecordKey, - ) -> Result<(AttoTokens, QuotingMetrics, Vec)> { + ) -> Result<(QuotingMetrics, bool)> { let (sender, receiver) = oneshot::channel(); - self.send_local_swarm_cmd(LocalSwarmCmd::GetLocalStoreCost { key, sender }); + self.send_local_swarm_cmd(LocalSwarmCmd::GetLocalQuotingMetrics { key, sender }); receiver .await @@ -1209,42 +1206,6 @@ impl Network { } } -/// Given `all_costs` it will return the closest / lowest cost -/// Closest requiring it to be within CLOSE_GROUP nodes -fn get_fees_from_store_cost_responses( - all_costs: Vec<(NetworkAddress, RewardsAddress, PaymentQuote)>, -) -> Result { - // Find the minimum cost using a linear scan with random tie break - let mut rng = rand::thread_rng(); - let payee = all_costs - .into_iter() - .min_by( - |(_address_a, _main_key_a, cost_a), (_address_b, _main_key_b, cost_b)| { - let cmp = cost_a.cost.cmp(&cost_b.cost); - if cmp == std::cmp::Ordering::Equal { - if rng.gen() { - std::cmp::Ordering::Less - } else { - std::cmp::Ordering::Greater - } - } else { - cmp - } - }, - ) - .ok_or(NetworkError::NoStoreCostResponses)?; - - info!("Final fees calculated as: {payee:?}"); - // we dont need to have the address outside of here for now - let payee_id = if let Some(peer_id) = payee.0.as_peer_id() { - peer_id - } else { - error!("Can't get PeerId from payee {:?}", payee.0); - return Err(NetworkError::NoStoreCostResponses); - }; - Ok((payee_id, payee.1, payee.2)) -} - /// Get the value of the provided Quorum pub fn get_quorum_value(quorum: &Quorum) -> usize { match quorum { @@ -1369,69 +1330,7 @@ pub(crate) fn send_network_swarm_cmd( #[cfg(test)] mod tests { - use eyre::bail; - use super::*; - use ant_evm::PaymentQuote; - - #[test] - fn test_get_fee_from_store_cost_responses() -> Result<()> { - // for a vec of different costs of CLOSE_GROUP size - // ensure we return the CLOSE_GROUP / 2 indexed price - let mut costs = vec![]; - for i in 1..CLOSE_GROUP_SIZE { - let addr = ant_evm::utils::dummy_address(); - costs.push(( - NetworkAddress::from_peer(PeerId::random()), - addr, - PaymentQuote::test_dummy(Default::default(), AttoTokens::from_u64(i as u64)), - )); - } - let expected_price = costs[0].2.cost.as_atto(); - let (_peer_id, _key, price) = get_fees_from_store_cost_responses(costs)?; - - assert_eq!( - price.cost.as_atto(), - expected_price, - "price should be {expected_price}" - ); - - Ok(()) - } - - #[test] - fn test_get_some_fee_from_store_cost_responses_even_if_one_errs_and_sufficient( - ) -> eyre::Result<()> { - // for a vec of different costs of CLOSE_GROUP size - let responses_count = CLOSE_GROUP_SIZE as u64 - 1; - let mut costs = vec![]; - for i in 1..responses_count { - // push random addr and Nano - let addr = ant_evm::utils::dummy_address(); - costs.push(( - NetworkAddress::from_peer(PeerId::random()), - addr, - PaymentQuote::test_dummy(Default::default(), AttoTokens::from_u64(i)), - )); - println!("price added {i}"); - } - - // this should be the lowest price - let expected_price = costs[0].2.cost.as_atto(); - - let (_peer_id, _key, price) = match get_fees_from_store_cost_responses(costs) { - Err(_) => bail!("Should not have errored as we have enough responses"), - Ok(cost) => cost, - }; - - assert_eq!( - price.cost.as_atto(), - expected_price, - "price should be {expected_price}" - ); - - Ok(()) - } #[test] fn test_network_sign_verify() -> eyre::Result<()> { diff --git a/ant-networking/src/log_markers.rs b/ant-networking/src/log_markers.rs index 99bcd6726d..c8ce2ce744 100644 --- a/ant-networking/src/log_markers.rs +++ b/ant-networking/src/log_markers.rs @@ -6,7 +6,7 @@ // KIND, either express or implied. Please review the Licences for the specific language governing // permissions and limitations relating to use of the SAFE Network Software. -use ant_evm::{Amount, QuotingMetrics}; +use ant_evm::QuotingMetrics; use libp2p::PeerId; // this gets us to_string easily enough use strum::Display; @@ -19,10 +19,8 @@ use strum::Display; pub enum Marker<'a> { /// Close records held (Used in VDash) CloseRecordsLen(usize), - /// Store cost - StoreCost { - /// Cost - cost: Amount, + /// Quoting metrics + QuotingMetrics { quoting_metrics: &'a QuotingMetrics, }, /// The peer has been considered as bad diff --git a/ant-networking/src/metrics/mod.rs b/ant-networking/src/metrics/mod.rs index 43a5b73f16..cb90d9b28e 100644 --- a/ant-networking/src/metrics/mod.rs +++ b/ant-networking/src/metrics/mod.rs @@ -45,8 +45,7 @@ pub(crate) struct NetworkMetricsRecorder { pub(crate) peers_in_routing_table: Gauge, pub(crate) records_stored: Gauge, - // store cost - store_cost: Gauge, + // quoting metrics relevant_records: Gauge, max_records: Gauge, received_payment_count: Gauge, @@ -149,13 +148,7 @@ impl NetworkMetricsRecorder { process_cpu_usage_percentage.clone(), ); - // store cost - let store_cost = Gauge::default(); - sub_registry.register( - "store_cost", - "The store cost of the node", - store_cost.clone(), - ); + // quoting metrics let relevant_records = Gauge::default(); sub_registry.register( "relevant_records", @@ -222,7 +215,6 @@ impl NetworkMetricsRecorder { connected_peers, open_connections, peers_in_routing_table, - store_cost, relevant_records, max_records, received_payment_count, @@ -292,11 +284,9 @@ impl NetworkMetricsRecorder { } }); } - Marker::StoreCost { - cost, + Marker::QuotingMetrics { quoting_metrics, } => { - let _ = self.store_cost.set(cost.try_into().unwrap_or(i64::MAX)); let _ = self.relevant_records.set( quoting_metrics .close_records_stored diff --git a/ant-networking/src/record_store.rs b/ant-networking/src/record_store.rs index 744a7fd807..16f7917abe 100644 --- a/ant-networking/src/record_store.rs +++ b/ant-networking/src/record_store.rs @@ -16,7 +16,7 @@ use aes_gcm_siv::{ aead::{Aead, KeyInit}, Aes256GcmSiv, Key as AesKey, Nonce, }; -use ant_evm::{AttoTokens, QuotingMetrics}; +use ant_evm::QuotingMetrics; use ant_protocol::{ storage::{RecordHeader, RecordKind, RecordType}, NetworkAddress, PrettyPrintRecordKey, @@ -61,12 +61,6 @@ const MAX_RECORDS_CACHE_SIZE: usize = 25; /// File name of the recorded historical quoting metrics. const HISTORICAL_QUOTING_METRICS_FILENAME: &str = "historic_quoting_metrics"; -/// Max store cost for a chunk. -const MAX_STORE_COST: u64 = 1_000_000; - -// Min store cost for a chunk. -const MIN_STORE_COST: u64 = 1; - fn derive_aes256gcm_siv_from_seed(seed: &[u8; 16]) -> (Aes256GcmSiv, [u8; 4]) { // shall be unique for purpose. let salt = b"autonomi_record_store"; @@ -724,12 +718,13 @@ impl NodeRecordStore { Ok(()) } - /// Calculate the cost to store data for our current store state - pub(crate) fn store_cost( + /// Return the quoting metrics used to calculate the cost of storing a record + /// and whether the record is already stored locally + pub(crate) fn quoting_metrics( &self, key: &Key, network_size: Option, - ) -> (AttoTokens, QuotingMetrics) { + ) -> (QuotingMetrics, bool) { let records_stored = self.records.len(); let live_time = if let Ok(elapsed) = self.timestamp.elapsed() { @@ -758,15 +753,12 @@ impl NodeRecordStore { info!("Basing cost of _total_ records stored."); }; - let cost = if self.contains(key) { - 0 - } else { - calculate_cost_for_records(quoting_metrics.close_records_stored) - }; + // NB TODO tell happybeing! // vdash metric (if modified please notify at https://github.com/happybeing/vdash/issues): - info!("Cost is now {cost:?} for quoting_metrics {quoting_metrics:?}"); + info!("Quoting_metrics {quoting_metrics:?}"); - (AttoTokens::from_u64(cost), quoting_metrics) + let is_stored = self.contains(key); + (quoting_metrics, is_stored) } /// Notify the node received a payment. @@ -1002,39 +994,13 @@ impl RecordStore for ClientRecordStore { fn remove_provider(&mut self, _key: &Key, _provider: &PeerId) {} } -// Using a linear growth function tweaked by `max_records`, -// and gives an exponential pricing curve when storage reaches high. -// and give extra reward (lower the quoting price to gain a better chance) to long lived nodes. -pub fn calculate_cost_for_records(records_stored: usize) -> u64 { - use std::cmp::{max, min}; - - let max_records = MAX_RECORDS_COUNT; - - let ori_cost = positive_input_0_1_sigmoid(records_stored as f64 / max_records as f64) - * MAX_STORE_COST as f64; - - // Deploy a lower cap safe_guard to the store_cost - let charge = max(MIN_STORE_COST, ori_cost as u64); - // Deploy an upper cap safe_guard to the store_cost - min(MAX_STORE_COST, charge) -} - -fn positive_input_0_1_sigmoid(x: f64) -> f64 { - 1.0 / (1.0 + (-30.0 * (x - 0.5)).exp()) -} - #[expect(trivial_casts)] #[cfg(test)] mod tests { - - use crate::get_fees_from_store_cost_responses; - use super::*; use bls::SecretKey; use xor_name::XorName; - use ant_evm::utils::dummy_address; - use ant_evm::{PaymentQuote, RewardsAddress}; use ant_protocol::storage::{ try_deserialize_record, try_serialize_record, Chunk, ChunkAddress, Scratchpad, }; @@ -1043,12 +1009,9 @@ mod tests { TempDir, }; use bytes::Bytes; - use eyre::{bail, ContextCompat}; - use libp2p::kad::K_VALUE; + use eyre::ContextCompat; use libp2p::{core::multihash::Multihash, kad::RecordKey}; use quickcheck::*; - use std::collections::BTreeMap; - use std::sync::atomic::{AtomicU64, AtomicUsize, Ordering}; use tokio::runtime::Runtime; use tokio::time::{sleep, Duration}; @@ -1087,70 +1050,6 @@ mod tests { } } - #[test] - fn test_calculate_max_cost_for_records() { - let sut = calculate_cost_for_records(MAX_RECORDS_COUNT + 1); - assert_eq!(sut, MAX_STORE_COST - 1); - } - - #[test] - fn test_calculate_50_percent_cost_for_records() { - let percent = MAX_RECORDS_COUNT * 50 / 100; - let sut = calculate_cost_for_records(percent); - - // at this point we should be at max cost - assert_eq!(sut, 500000); - } - #[test] - fn test_calculate_60_percent_cost_for_records() { - let percent = MAX_RECORDS_COUNT * 60 / 100; - let sut = calculate_cost_for_records(percent); - - // at this point we should be at max cost - assert_eq!(sut, 952541); - } - - #[test] - fn test_calculate_65_percent_cost_for_records() { - let percent = MAX_RECORDS_COUNT * 65 / 100; - let sut = calculate_cost_for_records(percent); - - // at this point we should be at max cost - assert_eq!(sut, 989001); - } - - #[test] - fn test_calculate_70_percent_cost_for_records() { - let percent = MAX_RECORDS_COUNT * 70 / 100; - let sut = calculate_cost_for_records(percent); - - // at this point we should be at max cost - assert_eq!(sut, 997523); - } - - #[test] - fn test_calculate_80_percent_cost_for_records() { - let percent = MAX_RECORDS_COUNT * 80 / 100; - let sut = calculate_cost_for_records(percent); - - // at this point we should be at max cost - assert_eq!(sut, 999876); - } - - #[test] - fn test_calculate_90_percent_cost_for_records() { - let percent = MAX_RECORDS_COUNT * 90 / 100; - let sut = calculate_cost_for_records(percent); - // at this point we should be at max cost - assert_eq!(sut, 999993); - } - - #[test] - fn test_calculate_min_cost_for_records() { - let sut = calculate_cost_for_records(0); - assert_eq!(sut, MIN_STORE_COST); - } - #[test] fn put_get_remove_record() { fn prop(r: ArbitraryRecord) { @@ -1176,16 +1075,9 @@ mod tests { swarm_cmd_sender, ); - let store_cost_before = store.store_cost(&r.key, None); // An initial unverified put should not write to disk assert!(store.put(r.clone()).is_ok()); assert!(store.get(&r.key).is_none()); - // Store cost should not change if no PUT has been added - assert_eq!( - store.store_cost(&r.key, None).0, - store_cost_before.0, - "store cost should not change over unverified put" - ); let returned_record = if let Some(event) = network_event_receiver.recv().await { if let NetworkEvent::UnverifiedRecord(record) = event { @@ -1736,255 +1628,4 @@ mod tests { Ok(()) } - - struct PeerStats { - address: NetworkAddress, - rewards_addr: RewardsAddress, - records_stored: AtomicUsize, - nanos_earned: AtomicU64, - payments_received: AtomicUsize, - } - - // takes a long time to run - #[ignore] - #[test] - fn address_distribution_sim() { - use rayon::prelude::*; - - // as network saturates, we can see that peers all eventually earn similarly - let num_of_peers = 5_000; - let num_of_chunks_per_hour = 1_000_000; - let max_hours = 50; - - // - let k = K_VALUE.get(); - - let replication_group_size = k / 3; - - // Initialize peers with random addresses - let mut peers: Vec = (0..num_of_peers) - .into_par_iter() - .map(|_| PeerStats { - address: NetworkAddress::from_peer(PeerId::random()), - records_stored: AtomicUsize::new(0), - nanos_earned: AtomicU64::new(0), - payments_received: AtomicUsize::new(0), - rewards_addr: dummy_address(), - }) - .collect(); - - let mut hour = 0; - let mut total_received_payment_count = 0; - - let peers_len = peers.len(); - - // Generate a random sorting target address - let sorting_target_address = - NetworkAddress::from_chunk_address(ChunkAddress::new(XorName::default())); - - // Sort all peers based on their distance to the sorting target - peers.par_sort_by(|a, b| { - sorting_target_address - .distance(&a.address) - .cmp(&sorting_target_address.distance(&b.address)) - }); - - loop { - // Parallel processing of chunks - let _chunk_results: Vec<_> = (0..num_of_chunks_per_hour) - .into_par_iter() - .map(|_| { - // Generate a random chunk address - let name = xor_name::rand::random(); - let chunk_address = NetworkAddress::from_chunk_address(ChunkAddress::new(name)); - - let chunk_distance_to_sorting = sorting_target_address.distance(&chunk_address); - // Binary search to find the insertion point for the chunk - let partition_point = peers.partition_point(|peer| { - sorting_target_address.distance(&peer.address) < chunk_distance_to_sorting - }); - - // Collect close_group_size closest peers - let mut close_group = Vec::with_capacity(replication_group_size); - let mut left = partition_point; - let mut right = partition_point; - - while close_group.len() < replication_group_size - && (left > 0 || right < peers_len) - { - if left > 0 { - left -= 1; - close_group.push(left); - } - if close_group.len() < replication_group_size && right < peers_len { - close_group.push(right); - right += 1; - } - } - - // Truncate to ensure we have exactly close_group_size peers - close_group.truncate(replication_group_size); - - // Find the cheapest payee among the close group - let Ok((payee_index, cost)) = pick_cheapest_payee(&peers, &close_group) else { - bail!("Failed to find a payee"); - }; - - for &peer_index in &close_group { - let peer = &peers[peer_index]; - peer.records_stored.fetch_add(1, Ordering::Relaxed); - - if peer_index == payee_index { - peer.nanos_earned.fetch_add( - cost.as_atto().try_into().unwrap_or(u64::MAX), - Ordering::Relaxed, - ); - peer.payments_received.fetch_add(1, Ordering::Relaxed); - } - } - - Ok(()) - }) - .collect(); - - // Parallel reduction to calculate statistics - let ( - received_payment_count, - empty_earned_nodes, - min_earned, - max_earned, - min_store_cost, - max_store_cost, - ) = peers - .par_iter() - .map(|peer| { - let cost = - calculate_cost_for_records(peer.records_stored.load(Ordering::Relaxed)); - let earned = peer.nanos_earned.load(Ordering::Relaxed); - ( - peer.payments_received.load(Ordering::Relaxed), - if earned == 0 { 1 } else { 0 }, - earned, - earned, - cost, - cost, - ) - }) - .reduce( - || (0, 0, u64::MAX, 0, u64::MAX, 0), - |a, b| { - let ( - a_received_payment_count, - a_empty_earned_nodes, - a_min_earned, - a_max_earned, - a_min_store_cost, - a_max_store_cost, - ) = a; - let ( - b_received_payment_count, - b_empty_earned_nodes, - b_min_earned, - b_max_earned, - b_min_store_cost, - b_max_store_cost, - ) = b; - ( - a_received_payment_count + b_received_payment_count, - a_empty_earned_nodes + b_empty_earned_nodes, - a_min_earned.min(b_min_earned), - a_max_earned.max(b_max_earned), - a_min_store_cost.min(b_min_store_cost), - a_max_store_cost.max(b_max_store_cost), - ) - }, - ); - - total_received_payment_count += num_of_chunks_per_hour; - assert_eq!(total_received_payment_count, received_payment_count); - - println!("After the completion of hour {hour} with {num_of_chunks_per_hour} chunks put, there are {empty_earned_nodes} nodes which earned nothing"); - println!("\t\t with storecost variation of (min {min_store_cost} - max {max_store_cost}), and earned variation of (min {min_earned} - max {max_earned})"); - - hour += 1; - - // Check termination condition - if hour == max_hours { - let acceptable_percentage = 0.01; //% - - // Calculate acceptable empty nodes based on % of total nodes - let acceptable_empty_nodes = - (num_of_peers as f64 * acceptable_percentage).ceil() as usize; - - // Assert conditions for termination - assert!( - empty_earned_nodes <= acceptable_empty_nodes, - "More than {acceptable_percentage}% of nodes ({acceptable_empty_nodes}) still not earning: {empty_earned_nodes}" - ); - assert!( - (max_store_cost / min_store_cost) < 1000000, - "store cost is not 'balanced', expected ratio max/min to be < 1000000, but was {}", - max_store_cost / min_store_cost - ); - assert!( - (max_earned / min_earned) < 500000000, - "earning distribution is not balanced, expected to be < 500000000, but was {}", - max_earned / min_earned - ); - break; - } - } - } - - fn pick_cheapest_payee( - peers: &[PeerStats], - close_group: &[usize], - ) -> eyre::Result<(usize, AttoTokens)> { - let mut costs_vec = Vec::with_capacity(close_group.len()); - let mut address_to_index = BTreeMap::new(); - - for &i in close_group { - let peer = &peers[i]; - address_to_index.insert(peer.address.clone(), i); - - let close_records_stored = peer.records_stored.load(Ordering::Relaxed); - let cost = AttoTokens::from(calculate_cost_for_records(close_records_stored)); - - let quote = PaymentQuote { - content: XorName::default(), // unimportant for cost calc - cost, - timestamp: std::time::SystemTime::now(), - quoting_metrics: QuotingMetrics { - close_records_stored: peer.records_stored.load(Ordering::Relaxed), - max_records: MAX_RECORDS_COUNT, - received_payment_count: 1, // unimportant for cost calc - live_time: 0, // unimportant for cost calc - network_density: None, - network_size: None, - }, - bad_nodes: vec![], - pub_key: bls::SecretKey::random().public_key().to_bytes().to_vec(), - signature: vec![], - rewards_address: peer.rewards_addr, // unimportant for cost calc - }; - - costs_vec.push((peer.address.clone(), peer.rewards_addr, quote)); - } - - // sort by address first - costs_vec.sort_by(|(a_addr, _, _), (b_addr, _, _)| a_addr.cmp(b_addr)); - - let Ok((recip_id, _pk, q)) = get_fees_from_store_cost_responses(costs_vec) else { - bail!("Failed to get fees from store cost responses") - }; - - let Some(index) = address_to_index - .get(&NetworkAddress::from_peer(recip_id)) - .copied() - else { - bail!("Cannot find the index for the cheapest payee"); - }; - - Ok((index, q.cost)) - } } diff --git a/ant-networking/src/record_store_api.rs b/ant-networking/src/record_store_api.rs index 7923c0d1b3..2aeb33a9a2 100644 --- a/ant-networking/src/record_store_api.rs +++ b/ant-networking/src/record_store_api.rs @@ -8,7 +8,7 @@ #![allow(clippy::mutable_key_type)] // for the Bytes in NetworkAddress use crate::record_store::{ClientRecordStore, NodeRecordStore}; -use ant_evm::{AttoTokens, QuotingMetrics}; +use ant_evm::QuotingMetrics; use ant_protocol::{storage::RecordType, NetworkAddress}; use libp2p::kad::{ store::{RecordStore, Result}, @@ -111,17 +111,19 @@ impl UnifiedRecordStore { } } - pub(crate) fn store_cost( + /// Return the quoting metrics used to calculate the cost of storing a record + /// and whether the record is already stored locally + pub(crate) fn quoting_metrics( &self, key: &RecordKey, network_size: Option, - ) -> (AttoTokens, QuotingMetrics) { + ) -> (QuotingMetrics, bool) { match self { Self::Client(_) => { - warn!("Calling store cost calculation at Client. This should not happen"); - (AttoTokens::zero(), Default::default()) + warn!("Calling quoting metrics calculation at Client. This should not happen"); + Default::default() } - Self::Node(store) => store.store_cost(key, network_size), + Self::Node(store) => store.quoting_metrics(key, network_size), } } diff --git a/ant-node/src/error.rs b/ant-node/src/error.rs index 86aba2df5c..e17c4ab111 100644 --- a/ant-node/src/error.rs +++ b/ant-node/src/error.rs @@ -69,9 +69,8 @@ pub enum Error { /// Missing network royalties payment #[error("Missing network royalties payment in proof received with record: {0:?}.")] NoNetworkRoyaltiesPayment(PrettyPrintRecordKey<'static>), - /// The amount paid by payment proof is not the required for the received content - #[error("The amount paid by payment proof is not the required for the received content, paid {paid}, expected {expected}")] - PaymentProofInsufficientAmount { + #[error("The amount paid is less than the storecost, paid {paid}, expected {expected}")] + PaymentInsufficientAmount { paid: AttoTokens, expected: AttoTokens, }, diff --git a/ant-node/src/node.rs b/ant-node/src/node.rs index 018ef4596a..7c34c0cfa9 100644 --- a/ant-node/src/node.rs +++ b/ant-node/src/node.rs @@ -571,16 +571,16 @@ impl Node { payment_address: RewardsAddress, ) -> Response { let resp: QueryResponse = match query { - Query::GetStoreCost { + Query::GetStoreQuote { key, nonce, difficulty, } => { - debug!("Got GetStoreCost request for {key:?} with difficulty {difficulty}"); + debug!("Got GetStoreQuote request for {key:?} with difficulty {difficulty}"); let record_key = key.to_record_key(); let self_id = network.peer_id(); - let store_cost = network.get_local_storecost(record_key.clone()).await; + let maybe_quoting_metrics = network.get_local_quoting_metrics(record_key.clone()).await; let storage_proofs = if let Some(nonce) = nonce { Self::respond_x_closest_record_proof( @@ -595,39 +595,37 @@ impl Node { vec![] }; - match store_cost { - Ok((cost, quoting_metrics, bad_nodes)) => { - if cost == AttoTokens::zero() { - QueryResponse::GetStoreCost { + match maybe_quoting_metrics { + Ok((quoting_metrics, is_already_stored)) => { + if is_already_stored { + QueryResponse::GetStoreQuote { quote: Err(ProtocolError::RecordExists( PrettyPrintRecordKey::from(&record_key).into_owned(), )), - payment_address, peer_address: NetworkAddress::from_peer(self_id), storage_proofs, } } else { - QueryResponse::GetStoreCost { - quote: Self::create_quote_for_storecost( + QueryResponse::GetStoreQuote { + quote: Self::create_quote_for_storecost( network, - cost, &key, "ing_metrics, - bad_nodes, &payment_address, ), - payment_address, peer_address: NetworkAddress::from_peer(self_id), storage_proofs, } } } - Err(_) => QueryResponse::GetStoreCost { - quote: Err(ProtocolError::GetStoreCostFailed), - payment_address, - peer_address: NetworkAddress::from_peer(self_id), - storage_proofs, - }, + Err(err) => { + warn!("GetStoreQuote failed for {key:?}: {err}"); + QueryResponse::GetStoreQuote { + quote: Err(ProtocolError::GetStoreQuoteFailed), + peer_address: NetworkAddress::from_peer(self_id), + storage_proofs, + } + } } } Query::GetRegisterRecord { requester, key } => { diff --git a/ant-node/src/put_validation.rs b/ant-node/src/put_validation.rs index 002652faa0..ff9c5b3974 100644 --- a/ant-node/src/put_validation.rs +++ b/ant-node/src/put_validation.rs @@ -7,7 +7,7 @@ // permissions and limitations relating to use of the SAFE Network Software. use crate::{node::Node, Error, Marker, Result}; -use ant_evm::{ProofOfPayment, QUOTE_EXPIRATION_SECS}; +use ant_evm::{AttoTokens, ProofOfPayment, QUOTE_EXPIRATION_SECS}; use ant_networking::NetworkError; use ant_protocol::storage::Transaction; use ant_protocol::{ @@ -652,7 +652,6 @@ impl Node { debug!("Validating record payment for {pretty_key}"); // check if the quote is valid - let storecost = payment.quote.cost; let self_peer_id = self.network().peer_id(); if !payment.quote.check_is_signed_by_claimed_peer(self_peer_id) { warn!("Payment quote signature is not valid for record {pretty_key}"); @@ -676,17 +675,17 @@ impl Node { // check if payment is valid on chain debug!("Verifying payment for record {pretty_key}"); - self.evm_network() + let reward_amount = self.evm_network() .verify_data_payment( payment.tx_hash, payment.quote.hash(), + payment.quote.quoting_metrics, *self.reward_address(), - storecost.as_atto(), quote_expiration_time_in_secs, ) .await .map_err(|e| Error::EvmNetwork(format!("Failed to verify chunk payment: {e}")))?; - debug!("Payment is valid for record {pretty_key}"); + debug!("Payment of {reward_amount:?} is valid for record {pretty_key}"); // Notify `record_store` that the node received a payment. self.network().notify_payment_received(); @@ -696,22 +695,22 @@ impl Node { // FIXME: We would reach the MAX if the storecost is scaled up. let current_value = metrics_recorder.current_reward_wallet_balance.get(); let new_value = - current_value.saturating_add(storecost.as_atto().try_into().unwrap_or(i64::MAX)); + current_value.saturating_add(reward_amount.try_into().unwrap_or(i64::MAX)); let _ = metrics_recorder .current_reward_wallet_balance .set(new_value); } self.events_channel() - .broadcast(crate::NodeEvent::RewardReceived(storecost, address.clone())); + .broadcast(crate::NodeEvent::RewardReceived(AttoTokens::from(reward_amount), address.clone())); // vdash metric (if modified please notify at https://github.com/happybeing/vdash/issues): - info!("Total payment of {storecost:?} atto tokens accepted for record {pretty_key}"); + info!("Total payment of {reward_amount:?} atto tokens accepted for record {pretty_key}"); // loud mode: print a celebratory message to console #[cfg(feature = "loud")] { println!("🌟🌟🌟🌟🌟🌟🌟🌟🌟🌟 RECEIVED REWARD 🌟🌟🌟🌟🌟🌟🌟🌟🌟🌟"); - println!("Total payment of {storecost:?} atto tokens accepted for record {pretty_key}"); + println!("Total payment of {reward_amount:?} atto tokens accepted for record {pretty_key}"); println!("🌟🌟🌟🌟🌟🌟🌟🌟🌟🌟🌟🌟🌟🌟🌟🌟🌟🌟🌟🌟🌟🌟🌟🌟🌟🌟🌟🌟🌟🌟🌟🌟🌟"); } diff --git a/ant-node/src/quote.rs b/ant-node/src/quote.rs index fa3defd843..4a11fd2ef7 100644 --- a/ant-node/src/quote.rs +++ b/ant-node/src/quote.rs @@ -7,8 +7,8 @@ // permissions and limitations relating to use of the SAFE Network Software. use crate::{node::Node, Error, Result}; -use ant_evm::{AttoTokens, PaymentQuote, QuotingMetrics, RewardsAddress}; -use ant_networking::{calculate_cost_for_records, Network, NodeIssue}; +use ant_evm::{PaymentQuote, QuotingMetrics, RewardsAddress}; +use ant_networking::Network; use ant_protocol::{error::Error as ProtocolError, storage::ChunkAddress, NetworkAddress}; use libp2p::PeerId; use std::time::Duration; @@ -16,21 +16,16 @@ use std::time::Duration; impl Node { pub(crate) fn create_quote_for_storecost( network: &Network, - cost: AttoTokens, address: &NetworkAddress, quoting_metrics: &QuotingMetrics, - bad_nodes: Vec, payment_address: &RewardsAddress, ) -> Result { let content = address.as_xorname().unwrap_or_default(); let timestamp = std::time::SystemTime::now(); - let serialised_bad_nodes = rmp_serde::to_vec(&bad_nodes).unwrap_or_default(); let bytes = PaymentQuote::bytes_for_signing( content, - cost, timestamp, quoting_metrics, - &serialised_bad_nodes, payment_address, ); @@ -40,10 +35,8 @@ impl Node { let quote = PaymentQuote { content, - cost, timestamp, quoting_metrics: quoting_metrics.clone(), - bad_nodes: serialised_bad_nodes, pub_key: network.get_pub_key(), rewards_address: *payment_address, signature, @@ -87,8 +80,7 @@ pub(crate) fn verify_quote_for_storecost( // 3, quote is no longer valid // // Following metrics will be considered as node's bad quote. -// 1, Price calculation is incorrect -// 2, QuoteMetrics doesn't match the historical quotes collected by self +// 1, QuoteMetrics doesn't match the historical quotes collected by self pub(crate) async fn quotes_verification(network: &Network, quotes: Vec<(PeerId, PaymentQuote)>) { // Do nothing if self is not one of the quoters. if let Some((_, self_quote)) = quotes @@ -98,12 +90,11 @@ pub(crate) async fn quotes_verification(network: &Network, quotes: Vec<(PeerId, let target_address = NetworkAddress::from_chunk_address(ChunkAddress::new(self_quote.content)); if verify_quote_for_storecost(network, self_quote.clone(), &target_address).is_ok() { - let mut quotes_for_nodes_duty: Vec<_> = quotes + let quotes_for_nodes_duty: Vec<_> = quotes .iter() .filter(|(peer_id, quote)| { let is_same_target = quote.content == self_quote.content; let is_not_self = *peer_id != network.peer_id(); - let is_not_zero_quote = quote.cost != AttoTokens::zero(); let time_gap = Duration::from_secs(10); let is_around_same_time = if quote.timestamp > self_quote.timestamp { @@ -117,25 +108,12 @@ pub(crate) async fn quotes_verification(network: &Network, quotes: Vec<(PeerId, is_same_target && is_not_self - && is_not_zero_quote && is_around_same_time && is_signed_by_the_claimed_peer }) .cloned() .collect(); - quotes_for_nodes_duty.retain(|(peer_id, quote)| { - let cost = calculate_cost_for_records(quote.quoting_metrics.close_records_stored); - let is_same_as_expected = quote.cost == AttoTokens::from_u64(cost); - - if !is_same_as_expected { - info!("Quote from {peer_id:?} using a different quoting_metrics to achieve the claimed cost. Quote {quote:?} can only result in cost {cost:?}"); - network.record_node_issues(*peer_id, NodeIssue::BadQuoting); - } - - is_same_as_expected - }); - // Pass down to swarm_driver level for further bad quote detection // against historical collected quotes. network.historical_verify_quotes(quotes_for_nodes_duty); diff --git a/ant-protocol/src/error.rs b/ant-protocol/src/error.rs index 7db10f9612..bc784860e1 100644 --- a/ant-protocol/src/error.rs +++ b/ant-protocol/src/error.rs @@ -57,7 +57,7 @@ pub enum Error { // ---------- payment errors #[error("There was an error getting the storecost from kademlia store")] - GetStoreCostFailed, + GetStoreQuoteFailed, #[error("There was an error generating the payment quote")] QuoteGenerationFailed, diff --git a/ant-protocol/src/messages/query.rs b/ant-protocol/src/messages/query.rs index 60392d7651..b685ad524e 100644 --- a/ant-protocol/src/messages/query.rs +++ b/ant-protocol/src/messages/query.rs @@ -18,9 +18,9 @@ use serde::{Deserialize, Serialize}; /// [`protocol`]: crate #[derive(Eq, PartialEq, PartialOrd, Clone, Serialize, Deserialize, Debug)] pub enum Query { - /// Retrieve the cost of storing a record at the given address. + /// Retrieve the quote to store a record at the given address. /// The storage verification is optional to be undertaken - GetStoreCost { + GetStoreQuote { /// The Address of the record to be stored. key: NetworkAddress, /// The random nonce that nodes use to produce the Proof (i.e., hash(record+nonce)) @@ -87,7 +87,7 @@ impl Query { Query::CheckNodeInProblem(address) => address.clone(), // Shall not be called for this, as this is a `one-to-one` message, // and the destination shall be decided by the requester already. - Query::GetStoreCost { key, .. } + Query::GetStoreQuote { key, .. } | Query::GetReplicatedRecord { key, .. } | Query::GetRegisterRecord { key, .. } | Query::GetChunkExistenceProof { key, .. } @@ -99,12 +99,12 @@ impl Query { impl std::fmt::Display for Query { fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { match self { - Query::GetStoreCost { + Query::GetStoreQuote { key, nonce, difficulty, } => { - write!(f, "Query::GetStoreCost({key:?} {nonce:?} {difficulty})") + write!(f, "Query::GetStoreQuote({key:?} {nonce:?} {difficulty})") } Query::GetReplicatedRecord { key, requester } => { write!(f, "Query::GetReplicatedRecord({requester:?} {key:?})") diff --git a/ant-protocol/src/messages/response.rs b/ant-protocol/src/messages/response.rs index a7f8bf9220..d3fc29ab31 100644 --- a/ant-protocol/src/messages/response.rs +++ b/ant-protocol/src/messages/response.rs @@ -9,7 +9,7 @@ use crate::{error::Result, NetworkAddress}; use super::ChunkProof; -use ant_evm::{PaymentQuote, RewardsAddress}; +use ant_evm::PaymentQuote; use bytes::Bytes; use core::fmt; use libp2p::Multiaddr; @@ -19,16 +19,14 @@ use std::fmt::Debug; /// The response to a query, containing the query result. #[derive(Clone, PartialEq, Eq, Serialize, Deserialize)] pub enum QueryResponse { - // ===== GetStoreCost ===== + // ===== GetStoreQuote ===== // - /// Response to [`GetStoreCost`] + /// Response to [`GetStoreQuote`] /// - /// [`GetStoreCost`]: crate::messages::Query::GetStoreCost - GetStoreCost { + /// [`GetStoreQuote`]: crate::messages::Query::GetStoreQuote + GetStoreQuote { /// The store cost quote for storing the next record. quote: Result, - /// The rewards address to pay this node's store cost to. - payment_address: RewardsAddress, /// Node's Peer Address peer_address: NetworkAddress, /// Storage proofs based on requested target address and difficulty @@ -80,15 +78,15 @@ pub enum QueryResponse { impl Debug for QueryResponse { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { match self { - QueryResponse::GetStoreCost { + QueryResponse::GetStoreQuote { quote, - payment_address, peer_address, storage_proofs, } => { + let payment_address = quote.as_ref().map(|q| q.rewards_address).ok(); write!( f, - "GetStoreCost(quote: {quote:?}, from {peer_address:?} w/ payment_address: {payment_address:?}, and {} storage proofs)", + "GetStoreQuote(quote: {quote:?}, from {peer_address:?} w/ payment_address: {payment_address:?}, and {} storage proofs)", storage_proofs.len() ) } diff --git a/autonomi/src/client/data/public.rs b/autonomi/src/client/data/public.rs index a4ff4e1a40..0a374f5c4b 100644 --- a/autonomi/src/client/data/public.rs +++ b/autonomi/src/client/data/public.rs @@ -96,8 +96,7 @@ impl Client { if let Some(channel) = self.client_event_sender.as_ref() { let tokens_spent = receipt .values() - .map(|proof| proof.quote.cost.as_atto()) - .sum::(); + .fold(Amount::ZERO, |acc, (_, cost)| acc + cost.as_atto()); let summary = UploadSummary { record_count, @@ -167,21 +166,18 @@ impl Client { .get_store_quotes(content_addrs.into_iter()) .await .inspect_err(|err| error!("Error getting store quotes: {err:?}"))?; - let total_cost = AttoTokens::from_atto( - cost_map - .values() - .map(|quote| quote.2.cost.as_atto()) - .sum::(), - ); + let total_cost = cost_map + .values() + .fold(Amount::ZERO, |acc, q| acc + q.total_cost.as_atto()); debug!("Total cost calculated: {total_cost:?}"); - Ok(total_cost) + Ok(AttoTokens::from_atto(total_cost)) } // Upload chunks and retry failed uploads up to `RETRY_ATTEMPTS` times. pub(crate) async fn upload_chunks_with_retries<'a>( &self, mut chunks: Vec<&'a Chunk>, - receipt: &HashMap, + receipt: &HashMap, AttoTokens)>, ) -> Vec<(&'a Chunk, PutError)> { let mut current_attempt: usize = 1; @@ -198,7 +194,7 @@ impl Client { upload_tasks.push(async move { self_clone - .chunk_upload_with_payment(chunk, proof.clone()) + .chunk_upload_with_payment(chunk, proof.0.clone()) .await .inspect_err(|err| error!("Error uploading chunk {address:?} :{err:?}")) // Return chunk reference too, to re-use it next attempt/iteration diff --git a/autonomi/src/client/data_private.rs b/autonomi/src/client/data_private.rs new file mode 100644 index 0000000000..a04670b448 --- /dev/null +++ b/autonomi/src/client/data_private.rs @@ -0,0 +1,129 @@ +// Copyright 2024 MaidSafe.net limited. +// +// This SAFE Network Software is licensed to you under The General Public License (GPL), version 3. +// Unless required by applicable law or agreed to in writing, the SAFE Network Software distributed +// under the GPL Licence is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. Please review the Licences for the specific language governing +// permissions and limitations relating to use of the SAFE Network Software. + +use std::hash::{DefaultHasher, Hash, Hasher}; + +use ant_evm::Amount; +use ant_protocol::storage::Chunk; +use bytes::Bytes; +use serde::{Deserialize, Serialize}; + +use super::data::{GetError, PutError}; +use crate::client::payment::PaymentOption; +use crate::client::{ClientEvent, UploadSummary}; +use crate::{self_encryption::encrypt, Client}; + +/// Private data on the network can be accessed with this +#[derive(Debug, Clone, Serialize, Deserialize, PartialEq, Eq, Hash, PartialOrd, Ord)] +pub struct PrivateDataAccess(Chunk); + +impl PrivateDataAccess { + pub fn to_hex(&self) -> String { + hex::encode(self.0.value()) + } + + pub fn from_hex(hex: &str) -> Result { + let data = hex::decode(hex)?; + Ok(Self(Chunk::new(Bytes::from(data)))) + } + + /// Get a private address for [`PrivateDataAccess`]. Note that this is not a network address, it is only used for refering to private data client side. + pub fn address(&self) -> String { + hash_to_short_string(&self.to_hex()) + } +} + +fn hash_to_short_string(input: &str) -> String { + let mut hasher = DefaultHasher::new(); + input.hash(&mut hasher); + let hash_value = hasher.finish(); + hash_value.to_string() +} + +impl Client { + /// Fetch a blob of private data from the network + pub async fn private_data_get(&self, data_map: PrivateDataAccess) -> Result { + info!( + "Fetching private data from Data Map {:?}", + data_map.0.address() + ); + let data = self.fetch_from_data_map_chunk(data_map.0.value()).await?; + + Ok(data) + } + + /// Upload a piece of private data to the network. This data will be self-encrypted. + /// Returns the [`PrivateDataAccess`] containing the map to the encrypted chunks. + /// This data is private and only accessible with the [`PrivateDataAccess`]. + pub async fn private_data_put( + &self, + data: Bytes, + payment_option: PaymentOption, + ) -> Result { + let now = ant_networking::target_arch::Instant::now(); + let (data_map_chunk, chunks) = encrypt(data)?; + debug!("Encryption took: {:.2?}", now.elapsed()); + + // Pay for all chunks + let xor_names: Vec<_> = chunks.iter().map(|chunk| *chunk.name()).collect(); + info!("Paying for {} addresses", xor_names.len()); + let receipt = self + .pay_for_content_addrs(xor_names.into_iter(), payment_option) + .await + .inspect_err(|err| error!("Error paying for data: {err:?}"))?; + + // Upload the chunks with the payments + debug!("Uploading {} chunks", chunks.len()); + + let mut failed_uploads = self + .upload_chunks_with_retries(chunks.iter().collect(), &receipt) + .await; + + // Return the last chunk upload error + if let Some(last_chunk_fail) = failed_uploads.pop() { + tracing::error!( + "Error uploading chunk ({:?}): {:?}", + last_chunk_fail.0.address(), + last_chunk_fail.1 + ); + return Err(last_chunk_fail.1); + } + + let record_count = chunks.len(); + + // Reporting + if let Some(channel) = self.client_event_sender.as_ref() { + let tokens_spent = receipt + .values() + .fold(Amount::ZERO, |acc, (_, cost)| acc + cost.as_atto()); + + let summary = UploadSummary { + record_count, + tokens_spent, + }; + if let Err(err) = channel.send(ClientEvent::UploadComplete(summary)).await { + error!("Failed to send client event: {err:?}"); + } + } + + Ok(PrivateDataAccess(data_map_chunk)) + } +} + +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn test_hex() { + let data_map = PrivateDataAccess(Chunk::new(Bytes::from_static(b"hello"))); + let hex = data_map.to_hex(); + let data_map2 = PrivateDataAccess::from_hex(&hex).expect("Failed to decode hex"); + assert_eq!(data_map, data_map2); + } +} diff --git a/autonomi/src/client/mod.rs b/autonomi/src/client/mod.rs index acc62981da..7ca25bd7a2 100644 --- a/autonomi/src/client/mod.rs +++ b/autonomi/src/client/mod.rs @@ -11,6 +11,7 @@ pub mod address; pub mod payment; +pub mod quote; pub mod data; #[cfg(feature = "external-signer")] diff --git a/autonomi/src/client/payment.rs b/autonomi/src/client/payment.rs index f91f71678f..b8fc399c38 100644 --- a/autonomi/src/client/payment.rs +++ b/autonomi/src/client/payment.rs @@ -1,11 +1,11 @@ use crate::client::data::PayError; use crate::Client; -use ant_evm::{EvmWallet, ProofOfPayment}; +use ant_evm::{AttoTokens, EvmWallet, ProofOfPayment}; use std::collections::HashMap; use xor_name::XorName; -/// Contains the proof of payment for XOR addresses. -pub type Receipt = HashMap; +/// Contains the proof of payments for XOR addresses as well as the total cost. +pub type Receipt = HashMap, AttoTokens)>; /// Payment options for data payments. #[derive(Clone)] @@ -40,7 +40,7 @@ impl Client { ) -> Result { match payment_option { PaymentOption::Wallet(wallet) => { - let (receipt, _) = self.pay(content_addrs, &wallet).await?; + let receipt = self.pay(content_addrs, &wallet).await?; debug!( "Paid for content addresses with wallet and the receipt is {:?}", receipt diff --git a/autonomi/src/client/quote.rs b/autonomi/src/client/quote.rs new file mode 100644 index 0000000000..8b257f74d6 --- /dev/null +++ b/autonomi/src/client/quote.rs @@ -0,0 +1,130 @@ +// Copyright 2024 MaidSafe.net limited. +// +// This SAFE Network Software is licensed to you under The General Public License (GPL), version 3. +// Unless required by applicable law or agreed to in writing, the SAFE Network Software distributed +// under the GPL Licence is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. Please review the Licences for the specific language governing +// permissions and limitations relating to use of the SAFE Network Software. + +use ant_evm::{PaymentQuote, ProofOfPayment, QuoteHash, TxHash}; +use ant_evm::{Amount, AttoTokens, QuotePayment}; +use ant_networking::{Network, NetworkError, PayeeQuote}; +use ant_protocol::{ + storage::ChunkAddress, + NetworkAddress, +}; +use xor_name::XorName; +use std::collections::{BTreeMap, HashMap}; + +use crate::client::payment::Receipt; +use super::{data::CostError, Client}; + +pub struct QuotesToPay { + pub nodes_to_pay: Vec, + pub nodes_to_upload_to: Vec, + pub cost_per_node: AttoTokens, + pub total_cost: AttoTokens, +} + +impl Client { + pub(crate) async fn get_store_quotes( + &self, + content_addrs: impl Iterator, + ) -> Result, CostError> { + let futures: Vec<_> = content_addrs + .into_iter() + .map(|content_addr| fetch_store_quote_with_retries(&self.network, content_addr)) + .collect(); + + let quotes = futures::future::try_join_all(futures).await?; + + let mut quotes_to_pay_per_addr = HashMap::new(); + for (content_addr, quotes) in quotes { + // NB TODO: get cost from smart contract for each quote and set this value to the median of all quotes! + let cost_per_node = Amount::from(1); + + // NB TODO: that's all the nodes except the invalid ones (rejected by smart contract) + let nodes_to_pay: Vec<_> = quotes.iter().map(|(_, q)| (q.hash(), q.rewards_address, cost_per_node)).collect(); + + // NB TODO: that's the lower half (quotes under or equal to the median price) + let nodes_to_upload_to = quotes.clone(); + + let total_cost = cost_per_node * Amount::from(nodes_to_pay.len()); + quotes_to_pay_per_addr.insert(content_addr, QuotesToPay { + nodes_to_pay, + nodes_to_upload_to, + cost_per_node: AttoTokens::from_atto(cost_per_node), + total_cost: AttoTokens::from_atto(total_cost), + }); + } + + Ok(quotes_to_pay_per_addr) + } +} + +/// Fetch a store quote for a content address. +async fn fetch_store_quote( + network: &Network, + content_addr: XorName, +) -> Result, NetworkError> { + network + .get_store_quote_from_network( + NetworkAddress::from_chunk_address(ChunkAddress::new(content_addr)), + vec![], + ) + .await +} + +/// Fetch a store quote for a content address with a retry strategy. +async fn fetch_store_quote_with_retries( + network: &Network, + content_addr: XorName, +) -> Result<(XorName, Vec), CostError> { + let mut retries = 0; + + loop { + match fetch_store_quote(network, content_addr).await { + Ok(quote) => { + break Ok((content_addr, quote)); + } + Err(err) if retries < 2 => { + retries += 1; + error!("Error while fetching store quote: {err:?}, retry #{retries}"); + } + Err(err) => { + error!( + "Error while fetching store quote: {err:?}, stopping after {retries} retries" + ); + break Err(CostError::CouldNotGetStoreQuote(content_addr)); + } + } + } +} + +pub fn receipt_from_quotes_and_payments( + quotes_map: HashMap, + payments: &BTreeMap, +) -> Receipt { + let quotes = cost_map_to_quotes(quotes_map); + receipt_from_quotes_and_payments("es, payments) +} + +pub fn receipt_from_quotes_and_payments( + quotes: &HashMap, + payments: &BTreeMap, +) -> Receipt { + quotes + .iter() + .filter_map(|(xor_name, quote)| { + payments.get("e.hash()).map(|tx_hash| { + ( + *xor_name, + ProofOfPayment { + quote: quote.clone(), + tx_hash: *tx_hash, + }, + ) + }) + }) + .collect() +} diff --git a/autonomi/src/client/registers.rs b/autonomi/src/client/registers.rs index 0d19fb27fe..19447ce078 100644 --- a/autonomi/src/client/registers.rs +++ b/autonomi/src/client/registers.rs @@ -302,7 +302,7 @@ impl Client { let reg_xor = address.xorname(); debug!("Paying for register at address: {address}"); - let (payment_proofs, _skipped) = self + let payment_proofs = self .pay(std::iter::once(reg_xor), wallet) .await .inspect_err(|err| { @@ -317,6 +317,11 @@ impl Client { }; let payee = proof + // NB TODO only pay the first one for now, but we should try all of them if first one fails + .0 + .first() + .expect("Missing proof of payment") + // TODO remove the tmp hack above and upload to all of them one by one until one succeeds .to_peer_id_payee() .ok_or(RegisterError::InvalidQuote) .inspect_err(|err| error!("Failed to get payee from payment proof: {err}"))?; @@ -359,7 +364,7 @@ impl Client { if let Some(channel) = self.client_event_sender.as_ref() { let summary = UploadSummary { record_count: 1, - tokens_spent: proof.quote.cost.as_atto(), + tokens_spent: proof.1.as_atto(), }; if let Err(err) = channel.send(ClientEvent::UploadComplete(summary)).await { error!("Failed to send client event: {err}"); diff --git a/autonomi/src/client/utils.rs b/autonomi/src/client/utils.rs index 9207b035c2..886f007af0 100644 --- a/autonomi/src/client/utils.rs +++ b/autonomi/src/client/utils.rs @@ -6,27 +6,26 @@ // KIND, either express or implied. Please review the Licences for the specific language governing // permissions and limitations relating to use of the SAFE Network Software. -use crate::client::payment::Receipt; -use crate::utils::receipt_from_cost_map_and_payments; use ant_evm::{EvmWallet, ProofOfPayment, QuotePayment}; use ant_networking::{ - GetRecordCfg, Network, NetworkError, PayeeQuote, PutRecordCfg, VerificationKind, + GetRecordCfg, PutRecordCfg, VerificationKind, }; use ant_protocol::{ messages::ChunkProof, - storage::{try_serialize_record, Chunk, ChunkAddress, RecordKind, RetryStrategy}, - NetworkAddress, + storage::{try_serialize_record, Chunk, RecordKind, RetryStrategy}, }; use bytes::Bytes; use futures::stream::{FuturesUnordered, StreamExt}; use libp2p::kad::{Quorum, Record}; use rand::{thread_rng, Rng}; use self_encryption::{decrypt_full_set, DataMap, EncryptedChunk}; -use std::{collections::HashMap, future::Future, num::NonZero}; +use std::{future::Future, num::NonZero}; use xor_name::XorName; use super::{ - data::{CostError, GetError, PayError, PutError, CHUNK_DOWNLOAD_BATCH_SIZE}, + quote::receipt_from_quotes_and_payments, + data::{GetError, PayError, PutError, CHUNK_DOWNLOAD_BATCH_SIZE}, + payment::Receipt, Client, }; use crate::self_encryption::DataMapLevel; @@ -102,9 +101,11 @@ impl Client { pub(crate) async fn chunk_upload_with_payment( &self, chunk: &Chunk, - payment: ProofOfPayment, + payment: Vec, ) -> Result<(), PutError> { - let storing_node = payment.to_peer_id_payee().expect("Missing node Peer ID"); + // NB TODO only pay the first one for now, but we should try all of them if first one fails + // NB TODO remove expects!! + let storing_node = payment.first().expect("Missing proof of payment").to_peer_id_payee().expect("Missing node Peer ID"); debug!("Storing chunk: {chunk:?} to {:?}", storing_node); @@ -164,10 +165,9 @@ impl Client { &self, content_addrs: impl Iterator, wallet: &EvmWallet, - ) -> Result<(Receipt, Vec), PayError> { - let cost_map = self.get_store_quotes(content_addrs).await?; - - let (quote_payments, skipped_chunks) = extract_quote_payments(&cost_map); + ) -> Result { + let quotes = self.get_store_quotes(content_addrs).await?; + let quotes_to_pay: Vec = quotes.values().map(|q| q.nodes_to_pay.iter()).flatten().cloned().collect(); // Make sure nobody else can use the wallet while we are paying debug!("Waiting for wallet lock"); @@ -178,7 +178,7 @@ impl Client { // TODO: retry when it fails? // Execute chunk payments let payments = wallet - .pay_for_quotes(quote_payments) + .pay_for_quotes(quotes_to_pay.into_iter()) .await .map_err(|err| PayError::from(err.0))?; @@ -186,89 +186,18 @@ impl Client { drop(lock_guard); debug!("Unlocked wallet"); - let proofs = receipt_from_cost_map_and_payments(cost_map, &payments); + let proofs = receipt_from_quotes_and_payments(quotes, &payments); + let already_paid_for = content_addrs.count() - quotes.len(); trace!( - "Chunk payments of {} chunks completed. {} chunks were free / already paid for", - proofs.len(), - skipped_chunks.len() + "Chunk payments of {} chunks completed. {already_paid_for} chunks were free / already paid for", + proofs.len() ); - Ok((proofs, skipped_chunks)) - } - - pub(crate) async fn get_store_quotes( - &self, - content_addrs: impl Iterator, - ) -> Result, CostError> { - let futures: Vec<_> = content_addrs - .into_iter() - .map(|content_addr| fetch_store_quote_with_retries(&self.network, content_addr)) - .collect(); - - let quotes = futures::future::try_join_all(futures).await?; - - Ok(quotes.into_iter().collect::>()) + Ok(proofs) } } -/// Fetch a store quote for a content address with a retry strategy. -async fn fetch_store_quote_with_retries( - network: &Network, - content_addr: XorName, -) -> Result<(XorName, PayeeQuote), CostError> { - let mut retries = 0; - - loop { - match fetch_store_quote(network, content_addr).await { - Ok(quote) => { - break Ok((content_addr, quote)); - } - Err(err) if retries < 2 => { - retries += 1; - error!("Error while fetching store quote: {err:?}, retry #{retries}"); - } - Err(err) => { - error!( - "Error while fetching store quote: {err:?}, stopping after {retries} retries" - ); - break Err(CostError::CouldNotGetStoreQuote(content_addr)); - } - } - } -} - -/// Fetch a store quote for a content address. -async fn fetch_store_quote( - network: &Network, - content_addr: XorName, -) -> Result { - network - .get_store_costs_from_network( - NetworkAddress::from_chunk_address(ChunkAddress::new(content_addr)), - vec![], - ) - .await -} - -/// Form to be executed payments and already executed payments from a cost map. -pub(crate) fn extract_quote_payments( - cost_map: &HashMap, -) -> (Vec, Vec) { - let mut to_be_paid = vec![]; - let mut already_paid = vec![]; - - for (chunk_address, (_, _, quote)) in cost_map.iter() { - if quote.cost.is_zero() { - already_paid.push(*chunk_address); - } else { - to_be_paid.push((quote.hash(), quote.rewards_address, quote.cost.as_atto())); - } - } - - (to_be_paid, already_paid) -} - pub(crate) async fn process_tasks_with_max_concurrency(tasks: I, batch_size: usize) -> Vec where I: IntoIterator, diff --git a/autonomi/src/client/vault.rs b/autonomi/src/client/vault.rs index 83553e3e16..01b6dd66b7 100644 --- a/autonomi/src/client/vault.rs +++ b/autonomi/src/client/vault.rs @@ -153,14 +153,9 @@ impl Client { // NB TODO: vault should be priced differently from other data let cost_map = self.get_store_quotes(std::iter::once(vault_xor)).await?; - let total_cost = AttoTokens::from_atto( - cost_map - .values() - .map(|quote| quote.2.cost.as_atto()) - .sum::(), - ); + let total_cost = cost_map.values().fold(Amount::ZERO, |acc, q| acc + q.total_cost.as_atto()); - Ok(total_cost) + Ok(AttoTokens::from_atto(total_cost)) } /// Put data into the client's VaultPacket @@ -198,12 +193,14 @@ impl Client { })?; let proof = match receipt.values().next() { - Some(proof) => proof, + Some(proof) => { + // NB TODO only use the first one for now, but we should try the others if first one fails + total_cost = proof.1; + proof.0.first().expect("Missing proof of payment") + }, None => return Err(PutError::PaymentUnexpectedlyInvalid(scratch_address)), }; - total_cost = proof.quote.cost; - Record { key: scratch_key, value: try_serialize_record(&(proof, scratch), RecordKind::ScratchpadWithPayment) diff --git a/autonomi/src/lib.rs b/autonomi/src/lib.rs index 7f200df9cc..7dd7aeb1a8 100644 --- a/autonomi/src/lib.rs +++ b/autonomi/src/lib.rs @@ -64,7 +64,6 @@ extern crate tracing; pub mod client; mod self_encryption; -mod utils; pub use ant_evm::get_evm_network_from_env; pub use ant_evm::EvmNetwork as Network; diff --git a/autonomi/src/utils.rs b/autonomi/src/utils.rs deleted file mode 100644 index 1348c0c685..0000000000 --- a/autonomi/src/utils.rs +++ /dev/null @@ -1,39 +0,0 @@ -use crate::client::payment::Receipt; -use ant_evm::{PaymentQuote, ProofOfPayment, QuoteHash, TxHash}; -use ant_networking::PayeeQuote; -use std::collections::{BTreeMap, HashMap}; -use xor_name::XorName; - -pub fn cost_map_to_quotes( - cost_map: HashMap, -) -> HashMap { - cost_map.into_iter().map(|(k, (_, _, v))| (k, v)).collect() -} - -pub fn receipt_from_cost_map_and_payments( - cost_map: HashMap, - payments: &BTreeMap, -) -> Receipt { - let quotes = cost_map_to_quotes(cost_map); - receipt_from_quotes_and_payments("es, payments) -} - -pub fn receipt_from_quotes_and_payments( - quotes: &HashMap, - payments: &BTreeMap, -) -> Receipt { - quotes - .iter() - .filter_map(|(xor_name, quote)| { - payments.get("e.hash()).map(|tx_hash| { - ( - *xor_name, - ProofOfPayment { - quote: quote.clone(), - tx_hash: *tx_hash, - }, - ) - }) - }) - .collect() -} diff --git a/evmlib/src/lib.rs b/evmlib/src/lib.rs index e0df96d466..fb3303fd47 100644 --- a/evmlib/src/lib.rs +++ b/evmlib/src/lib.rs @@ -6,10 +6,12 @@ // KIND, either express or implied. Please review the Licences for the specific language governing // permissions and limitations relating to use of the SAFE Network Software. -use crate::common::{Address, QuoteHash, TxHash, U256}; +use crate::common::{Address, QuoteHash, TxHash}; use crate::transaction::verify_data_payment; use alloy::primitives::address; use alloy::transports::http::reqwest; +use common::Amount; +use quoting_metrics::QuotingMetrics; use serde::{Deserialize, Serialize}; use serde_with::{serde_as, DisplayFromStr}; use std::str::FromStr; @@ -24,6 +26,7 @@ pub mod cryptography; pub(crate) mod event; #[cfg(feature = "external-signer")] pub mod external_signer; +pub mod quoting_metrics; pub mod testnet; pub mod transaction; pub mod utils; @@ -138,16 +141,17 @@ impl Network { &self, tx_hash: TxHash, quote_hash: QuoteHash, + _quoting_metrics: QuotingMetrics, reward_addr: Address, - amount: U256, quote_expiration_timestamp_in_secs: u64, - ) -> Result<(), transaction::Error> { + ) -> Result { verify_data_payment( self, tx_hash, quote_hash, + // quoting_metrics, // NB TODO use them here @Mick reward_addr, - amount, + Default::default(), // NB TODO remove amounts @Mick quote_expiration_timestamp_in_secs, ) .await diff --git a/evmlib/src/quoting_metrics.rs b/evmlib/src/quoting_metrics.rs new file mode 100644 index 0000000000..801ee4c97c --- /dev/null +++ b/evmlib/src/quoting_metrics.rs @@ -0,0 +1,47 @@ +// Copyright 2024 MaidSafe.net limited. +// +// This SAFE Network Software is licensed to you under The General Public License (GPL), version 3. +// Unless required by applicable law or agreed to in writing, the SAFE Network Software distributed +// under the GPL Licence is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. Please review the Licences for the specific language governing +// permissions and limitations relating to use of the SAFE Network Software. + +use serde::{Deserialize, Serialize}; + +/// Quoting metrics used to generate a quote, or to track peer's status. +#[derive(Clone, Eq, PartialEq, PartialOrd, Ord, Hash, Serialize, Deserialize, Debug)] +pub struct QuotingMetrics { + /// the records stored + pub close_records_stored: usize, + /// the max_records configured + pub max_records: usize, + /// number of times that got paid + pub received_payment_count: usize, + /// the duration that node keeps connected to the network, measured in hours + pub live_time: u64, + /// network density from this node's perspective, which is the responsible_range as well + /// This could be calculated via sampling, or equation calculation. + pub network_density: Option<[u8; 32]>, + /// estimated network size + pub network_size: Option, +} + +impl QuotingMetrics { + /// construct an empty QuotingMetrics + pub fn new() -> Self { + Self { + close_records_stored: 0, + max_records: 0, + received_payment_count: 0, + live_time: 0, + network_density: None, + network_size: None, + } + } +} + +impl Default for QuotingMetrics { + fn default() -> Self { + Self::new() + } +} diff --git a/evmlib/src/transaction.rs b/evmlib/src/transaction.rs index 7e09e4495f..af23e4f026 100644 --- a/evmlib/src/transaction.rs +++ b/evmlib/src/transaction.rs @@ -106,7 +106,7 @@ pub async fn verify_data_payment( reward_addr: Address, amount: U256, quote_expiration_timestamp_in_secs: u64, -) -> Result<(), Error> { +) -> Result { debug!("Verifying data payment for tx_hash: {tx_hash:?}"); let transaction = get_transaction_receipt_by_hash(network, tx_hash) .await? @@ -148,7 +148,7 @@ pub async fn verify_data_payment( && event.rewards_address == reward_addr && event.amount >= amount { - return Ok(()); + return Ok(event.amount); } } } From fdf2c89989503579aab8486b5f06b08003c27416 Mon Sep 17 00:00:00 2001 From: grumbach Date: Thu, 5 Dec 2024 17:55:29 +0900 Subject: [PATCH 02/33] chore: revert client side attempt --- autonomi/src/client/data/public.rs | 8 +- autonomi/src/client/data_private.rs | 3 +- autonomi/src/client/mod.rs | 1 - autonomi/src/client/payment.rs | 10 +-- autonomi/src/client/registers.rs | 9 +-- autonomi/src/client/utils.rs | 109 +++++++++++++++++++++++----- autonomi/src/client/vault.rs | 17 +++-- 7 files changed, 111 insertions(+), 46 deletions(-) diff --git a/autonomi/src/client/data/public.rs b/autonomi/src/client/data/public.rs index 0a374f5c4b..2b018298a3 100644 --- a/autonomi/src/client/data/public.rs +++ b/autonomi/src/client/data/public.rs @@ -96,7 +96,8 @@ impl Client { if let Some(channel) = self.client_event_sender.as_ref() { let tokens_spent = receipt .values() - .fold(Amount::ZERO, |acc, (_, cost)| acc + cost.as_atto()); + .map(|proof| proof.quote.cost.as_atto()) + .sum::(); let summary = UploadSummary { record_count, @@ -169,7 +170,6 @@ impl Client { let total_cost = cost_map .values() .fold(Amount::ZERO, |acc, q| acc + q.total_cost.as_atto()); - debug!("Total cost calculated: {total_cost:?}"); Ok(AttoTokens::from_atto(total_cost)) } @@ -177,7 +177,7 @@ impl Client { pub(crate) async fn upload_chunks_with_retries<'a>( &self, mut chunks: Vec<&'a Chunk>, - receipt: &HashMap, AttoTokens)>, + receipt: &HashMap, ) -> Vec<(&'a Chunk, PutError)> { let mut current_attempt: usize = 1; @@ -194,7 +194,7 @@ impl Client { upload_tasks.push(async move { self_clone - .chunk_upload_with_payment(chunk, proof.0.clone()) + .chunk_upload_with_payment(chunk, proof.clone()) .await .inspect_err(|err| error!("Error uploading chunk {address:?} :{err:?}")) // Return chunk reference too, to re-use it next attempt/iteration diff --git a/autonomi/src/client/data_private.rs b/autonomi/src/client/data_private.rs index a04670b448..5f2dd1793c 100644 --- a/autonomi/src/client/data_private.rs +++ b/autonomi/src/client/data_private.rs @@ -100,7 +100,8 @@ impl Client { if let Some(channel) = self.client_event_sender.as_ref() { let tokens_spent = receipt .values() - .fold(Amount::ZERO, |acc, (_, cost)| acc + cost.as_atto()); + .map(|proof| proof.quote.cost.as_atto()) + .sum::(); let summary = UploadSummary { record_count, diff --git a/autonomi/src/client/mod.rs b/autonomi/src/client/mod.rs index 7ca25bd7a2..acc62981da 100644 --- a/autonomi/src/client/mod.rs +++ b/autonomi/src/client/mod.rs @@ -11,7 +11,6 @@ pub mod address; pub mod payment; -pub mod quote; pub mod data; #[cfg(feature = "external-signer")] diff --git a/autonomi/src/client/payment.rs b/autonomi/src/client/payment.rs index b8fc399c38..f9096f15cf 100644 --- a/autonomi/src/client/payment.rs +++ b/autonomi/src/client/payment.rs @@ -1,11 +1,11 @@ use crate::client::data::PayError; use crate::Client; -use ant_evm::{AttoTokens, EvmWallet, ProofOfPayment}; +use ant_evm::{EvmWallet, ProofOfPayment}; use std::collections::HashMap; use xor_name::XorName; -/// Contains the proof of payments for XOR addresses as well as the total cost. -pub type Receipt = HashMap, AttoTokens)>; +/// Contains the proof of payment for XOR addresses. +pub type Receipt = HashMap; /// Payment options for data payments. #[derive(Clone)] @@ -41,10 +41,6 @@ impl Client { match payment_option { PaymentOption::Wallet(wallet) => { let receipt = self.pay(content_addrs, &wallet).await?; - debug!( - "Paid for content addresses with wallet and the receipt is {:?}", - receipt - ); Ok(receipt) } PaymentOption::Receipt(receipt) => Ok(receipt), diff --git a/autonomi/src/client/registers.rs b/autonomi/src/client/registers.rs index 19447ce078..0d19fb27fe 100644 --- a/autonomi/src/client/registers.rs +++ b/autonomi/src/client/registers.rs @@ -302,7 +302,7 @@ impl Client { let reg_xor = address.xorname(); debug!("Paying for register at address: {address}"); - let payment_proofs = self + let (payment_proofs, _skipped) = self .pay(std::iter::once(reg_xor), wallet) .await .inspect_err(|err| { @@ -317,11 +317,6 @@ impl Client { }; let payee = proof - // NB TODO only pay the first one for now, but we should try all of them if first one fails - .0 - .first() - .expect("Missing proof of payment") - // TODO remove the tmp hack above and upload to all of them one by one until one succeeds .to_peer_id_payee() .ok_or(RegisterError::InvalidQuote) .inspect_err(|err| error!("Failed to get payee from payment proof: {err}"))?; @@ -364,7 +359,7 @@ impl Client { if let Some(channel) = self.client_event_sender.as_ref() { let summary = UploadSummary { record_count: 1, - tokens_spent: proof.1.as_atto(), + tokens_spent: proof.quote.cost.as_atto(), }; if let Err(err) = channel.send(ClientEvent::UploadComplete(summary)).await { error!("Failed to send client event: {err}"); diff --git a/autonomi/src/client/utils.rs b/autonomi/src/client/utils.rs index 886f007af0..9207b035c2 100644 --- a/autonomi/src/client/utils.rs +++ b/autonomi/src/client/utils.rs @@ -6,26 +6,27 @@ // KIND, either express or implied. Please review the Licences for the specific language governing // permissions and limitations relating to use of the SAFE Network Software. +use crate::client::payment::Receipt; +use crate::utils::receipt_from_cost_map_and_payments; use ant_evm::{EvmWallet, ProofOfPayment, QuotePayment}; use ant_networking::{ - GetRecordCfg, PutRecordCfg, VerificationKind, + GetRecordCfg, Network, NetworkError, PayeeQuote, PutRecordCfg, VerificationKind, }; use ant_protocol::{ messages::ChunkProof, - storage::{try_serialize_record, Chunk, RecordKind, RetryStrategy}, + storage::{try_serialize_record, Chunk, ChunkAddress, RecordKind, RetryStrategy}, + NetworkAddress, }; use bytes::Bytes; use futures::stream::{FuturesUnordered, StreamExt}; use libp2p::kad::{Quorum, Record}; use rand::{thread_rng, Rng}; use self_encryption::{decrypt_full_set, DataMap, EncryptedChunk}; -use std::{future::Future, num::NonZero}; +use std::{collections::HashMap, future::Future, num::NonZero}; use xor_name::XorName; use super::{ - quote::receipt_from_quotes_and_payments, - data::{GetError, PayError, PutError, CHUNK_DOWNLOAD_BATCH_SIZE}, - payment::Receipt, + data::{CostError, GetError, PayError, PutError, CHUNK_DOWNLOAD_BATCH_SIZE}, Client, }; use crate::self_encryption::DataMapLevel; @@ -101,11 +102,9 @@ impl Client { pub(crate) async fn chunk_upload_with_payment( &self, chunk: &Chunk, - payment: Vec, + payment: ProofOfPayment, ) -> Result<(), PutError> { - // NB TODO only pay the first one for now, but we should try all of them if first one fails - // NB TODO remove expects!! - let storing_node = payment.first().expect("Missing proof of payment").to_peer_id_payee().expect("Missing node Peer ID"); + let storing_node = payment.to_peer_id_payee().expect("Missing node Peer ID"); debug!("Storing chunk: {chunk:?} to {:?}", storing_node); @@ -165,9 +164,10 @@ impl Client { &self, content_addrs: impl Iterator, wallet: &EvmWallet, - ) -> Result { - let quotes = self.get_store_quotes(content_addrs).await?; - let quotes_to_pay: Vec = quotes.values().map(|q| q.nodes_to_pay.iter()).flatten().cloned().collect(); + ) -> Result<(Receipt, Vec), PayError> { + let cost_map = self.get_store_quotes(content_addrs).await?; + + let (quote_payments, skipped_chunks) = extract_quote_payments(&cost_map); // Make sure nobody else can use the wallet while we are paying debug!("Waiting for wallet lock"); @@ -178,7 +178,7 @@ impl Client { // TODO: retry when it fails? // Execute chunk payments let payments = wallet - .pay_for_quotes(quotes_to_pay.into_iter()) + .pay_for_quotes(quote_payments) .await .map_err(|err| PayError::from(err.0))?; @@ -186,18 +186,89 @@ impl Client { drop(lock_guard); debug!("Unlocked wallet"); - let proofs = receipt_from_quotes_and_payments(quotes, &payments); + let proofs = receipt_from_cost_map_and_payments(cost_map, &payments); - let already_paid_for = content_addrs.count() - quotes.len(); trace!( - "Chunk payments of {} chunks completed. {already_paid_for} chunks were free / already paid for", - proofs.len() + "Chunk payments of {} chunks completed. {} chunks were free / already paid for", + proofs.len(), + skipped_chunks.len() ); - Ok(proofs) + Ok((proofs, skipped_chunks)) + } + + pub(crate) async fn get_store_quotes( + &self, + content_addrs: impl Iterator, + ) -> Result, CostError> { + let futures: Vec<_> = content_addrs + .into_iter() + .map(|content_addr| fetch_store_quote_with_retries(&self.network, content_addr)) + .collect(); + + let quotes = futures::future::try_join_all(futures).await?; + + Ok(quotes.into_iter().collect::>()) } } +/// Fetch a store quote for a content address with a retry strategy. +async fn fetch_store_quote_with_retries( + network: &Network, + content_addr: XorName, +) -> Result<(XorName, PayeeQuote), CostError> { + let mut retries = 0; + + loop { + match fetch_store_quote(network, content_addr).await { + Ok(quote) => { + break Ok((content_addr, quote)); + } + Err(err) if retries < 2 => { + retries += 1; + error!("Error while fetching store quote: {err:?}, retry #{retries}"); + } + Err(err) => { + error!( + "Error while fetching store quote: {err:?}, stopping after {retries} retries" + ); + break Err(CostError::CouldNotGetStoreQuote(content_addr)); + } + } + } +} + +/// Fetch a store quote for a content address. +async fn fetch_store_quote( + network: &Network, + content_addr: XorName, +) -> Result { + network + .get_store_costs_from_network( + NetworkAddress::from_chunk_address(ChunkAddress::new(content_addr)), + vec![], + ) + .await +} + +/// Form to be executed payments and already executed payments from a cost map. +pub(crate) fn extract_quote_payments( + cost_map: &HashMap, +) -> (Vec, Vec) { + let mut to_be_paid = vec![]; + let mut already_paid = vec![]; + + for (chunk_address, (_, _, quote)) in cost_map.iter() { + if quote.cost.is_zero() { + already_paid.push(*chunk_address); + } else { + to_be_paid.push((quote.hash(), quote.rewards_address, quote.cost.as_atto())); + } + } + + (to_be_paid, already_paid) +} + pub(crate) async fn process_tasks_with_max_concurrency(tasks: I, batch_size: usize) -> Vec where I: IntoIterator, diff --git a/autonomi/src/client/vault.rs b/autonomi/src/client/vault.rs index 01b6dd66b7..83553e3e16 100644 --- a/autonomi/src/client/vault.rs +++ b/autonomi/src/client/vault.rs @@ -153,9 +153,14 @@ impl Client { // NB TODO: vault should be priced differently from other data let cost_map = self.get_store_quotes(std::iter::once(vault_xor)).await?; - let total_cost = cost_map.values().fold(Amount::ZERO, |acc, q| acc + q.total_cost.as_atto()); + let total_cost = AttoTokens::from_atto( + cost_map + .values() + .map(|quote| quote.2.cost.as_atto()) + .sum::(), + ); - Ok(AttoTokens::from_atto(total_cost)) + Ok(total_cost) } /// Put data into the client's VaultPacket @@ -193,14 +198,12 @@ impl Client { })?; let proof = match receipt.values().next() { - Some(proof) => { - // NB TODO only use the first one for now, but we should try the others if first one fails - total_cost = proof.1; - proof.0.first().expect("Missing proof of payment") - }, + Some(proof) => proof, None => return Err(PutError::PaymentUnexpectedlyInvalid(scratch_address)), }; + total_cost = proof.quote.cost; + Record { key: scratch_key, value: try_serialize_record(&(proof, scratch), RecordKind::ScratchpadWithPayment) From f096059783054bb3f37a1279abafd80ef4c6a8dd Mon Sep 17 00:00:00 2001 From: Warm Beer Date: Thu, 5 Dec 2024 15:15:35 +0100 Subject: [PATCH 03/33] feat: add payment vault smart contract interface --- Cargo.lock | 574 ++++++++++++++++++--- evmlib/Cargo.toml | 2 +- evmlib/abi/IPaymentVault.json | 191 +++++++ evmlib/src/contract/data_payments/error.rs | 24 - evmlib/src/contract/mod.rs | 1 + evmlib/src/contract/payment_vault/error.rs | 11 + evmlib/src/contract/payment_vault/mod.rs | 100 ++++ evmlib/tests/data_payments.rs | 137 ----- 8 files changed, 806 insertions(+), 234 deletions(-) create mode 100644 evmlib/abi/IPaymentVault.json delete mode 100644 evmlib/src/contract/data_payments/error.rs create mode 100644 evmlib/src/contract/payment_vault/error.rs create mode 100644 evmlib/src/contract/payment_vault/mod.rs delete mode 100644 evmlib/tests/data_payments.rs diff --git a/Cargo.lock b/Cargo.lock index 999850c2d5..d71bc86b4f 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -122,21 +122,45 @@ version = "0.5.4" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "ea8ebf106e84a1c37f86244df7da0c7587e697b71a0d565cce079449b85ac6f8" dependencies = [ - "alloy-consensus", - "alloy-contract", + "alloy-consensus 0.5.4", + "alloy-contract 0.5.4", "alloy-core", - "alloy-eips", - "alloy-genesis", - "alloy-network", - "alloy-node-bindings", - "alloy-provider", - "alloy-rpc-client", - "alloy-rpc-types", - "alloy-serde", - "alloy-signer", - "alloy-signer-local", - "alloy-transport", - "alloy-transport-http", + "alloy-eips 0.5.4", + "alloy-genesis 0.5.4", + "alloy-network 0.5.4", + "alloy-node-bindings 0.5.4", + "alloy-provider 0.5.4", + "alloy-rpc-client 0.5.4", + "alloy-rpc-types 0.5.4", + "alloy-serde 0.5.4", + "alloy-signer 0.5.4", + "alloy-signer-local 0.5.4", + "alloy-transport 0.5.4", + "alloy-transport-http 0.5.4", +] + +[[package]] +name = "alloy" +version = "0.7.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "02b0561294ccedc6181e5528b850b4579e3fbde696507baa00109bfd9054c5bb" +dependencies = [ + "alloy-consensus 0.7.3", + "alloy-contract 0.7.3", + "alloy-core", + "alloy-eips 0.7.3", + "alloy-genesis 0.7.3", + "alloy-json-rpc 0.7.3", + "alloy-network 0.7.3", + "alloy-node-bindings 0.7.3", + "alloy-provider 0.7.3", + "alloy-rpc-client 0.7.3", + "alloy-rpc-types 0.7.3", + "alloy-serde 0.7.3", + "alloy-signer 0.7.3", + "alloy-signer-local 0.7.3", + "alloy-transport 0.7.3", + "alloy-transport-http 0.7.3", ] [[package]] @@ -156,16 +180,47 @@ version = "0.5.4" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "41ed961a48297c732a5d97ee321aa8bb5009ecadbcb077d8bec90cb54e651629" dependencies = [ - "alloy-eips", + "alloy-eips 0.5.4", + "alloy-primitives", + "alloy-rlp", + "alloy-serde 0.5.4", + "auto_impl", + "c-kzg", + "derive_more", + "serde", +] + +[[package]] +name = "alloy-consensus" +version = "0.7.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a101d4d016f47f13890a74290fdd17b05dd175191d9337bc600791fb96e4dea8" +dependencies = [ + "alloy-eips 0.7.3", "alloy-primitives", "alloy-rlp", - "alloy-serde", + "alloy-serde 0.7.3", + "alloy-trie", "auto_impl", "c-kzg", "derive_more", "serde", ] +[[package]] +name = "alloy-consensus-any" +version = "0.7.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fa60357dda9a3d0f738f18844bd6d0f4a5924cc5cf00bfad2ff1369897966123" +dependencies = [ + "alloy-consensus 0.7.3", + "alloy-eips 0.7.3", + "alloy-primitives", + "alloy-rlp", + "alloy-serde 0.7.3", + "serde", +] + [[package]] name = "alloy-contract" version = "0.5.4" @@ -174,18 +229,38 @@ checksum = "460ab80ce4bda1c80bcf96fe7460520476f2c7b734581c6567fac2708e2a60ef" dependencies = [ "alloy-dyn-abi", "alloy-json-abi", - "alloy-network", - "alloy-network-primitives", + "alloy-network 0.5.4", + "alloy-network-primitives 0.5.4", "alloy-primitives", - "alloy-provider", - "alloy-rpc-types-eth", + "alloy-provider 0.5.4", + "alloy-rpc-types-eth 0.5.4", "alloy-sol-types", - "alloy-transport", + "alloy-transport 0.5.4", "futures", "futures-util", "thiserror 1.0.69", ] +[[package]] +name = "alloy-contract" +version = "0.7.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2869e4fb31331d3b8c58c7db567d1e4e4e94ef64640beda3b6dd9b7045690941" +dependencies = [ + "alloy-dyn-abi", + "alloy-json-abi", + "alloy-network 0.7.3", + "alloy-network-primitives 0.7.3", + "alloy-primitives", + "alloy-provider 0.7.3", + "alloy-rpc-types-eth 0.7.3", + "alloy-sol-types", + "alloy-transport 0.7.3", + "futures", + "futures-util", + "thiserror 2.0.3", +] + [[package]] name = "alloy-core" version = "0.8.14" @@ -239,6 +314,18 @@ dependencies = [ "serde", ] +[[package]] +name = "alloy-eip7702" +version = "0.4.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4c986539255fb839d1533c128e190e557e52ff652c9ef62939e233a81dd93f7e" +dependencies = [ + "alloy-primitives", + "alloy-rlp", + "derive_more", + "serde", +] + [[package]] name = "alloy-eips" version = "0.5.4" @@ -246,10 +333,28 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "b69e06cf9c37be824b9d26d6d101114fdde6af0c87de2828b414c05c4b3daa71" dependencies = [ "alloy-eip2930", - "alloy-eip7702", + "alloy-eip7702 0.3.2", + "alloy-primitives", + "alloy-rlp", + "alloy-serde 0.5.4", + "c-kzg", + "derive_more", + "once_cell", + "serde", + "sha2 0.10.8", +] + +[[package]] +name = "alloy-eips" +version = "0.7.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8b6755b093afef5925f25079dd5a7c8d096398b804ba60cb5275397b06b31689" +dependencies = [ + "alloy-eip2930", + "alloy-eip7702 0.4.2", "alloy-primitives", "alloy-rlp", - "alloy-serde", + "alloy-serde 0.7.3", "c-kzg", "derive_more", "once_cell", @@ -264,7 +369,19 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "dde15e14944a88bd6a57d325e9a49b75558746fe16aaccc79713ae50a6a9574c" dependencies = [ "alloy-primitives", - "alloy-serde", + "alloy-serde 0.5.4", + "serde", +] + +[[package]] +name = "alloy-genesis" +version = "0.7.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "aeec8e6eab6e52b7c9f918748c9b811e87dbef7312a2e3a2ca1729a92966a6af" +dependencies = [ + "alloy-primitives", + "alloy-serde 0.7.3", + "alloy-trie", "serde", ] @@ -294,20 +411,34 @@ dependencies = [ "tracing", ] +[[package]] +name = "alloy-json-rpc" +version = "0.7.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4fa077efe0b834bcd89ff4ba547f48fb081e4fdc3673dd7da1b295a2cf2bb7b7" +dependencies = [ + "alloy-primitives", + "alloy-sol-types", + "serde", + "serde_json", + "thiserror 2.0.3", + "tracing", +] + [[package]] name = "alloy-network" version = "0.5.4" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "204237129086ce5dc17a58025e93739b01b45313841f98fa339eb1d780511e57" dependencies = [ - "alloy-consensus", - "alloy-eips", - "alloy-json-rpc", - "alloy-network-primitives", + "alloy-consensus 0.5.4", + "alloy-eips 0.5.4", + "alloy-json-rpc 0.5.4", + "alloy-network-primitives 0.5.4", "alloy-primitives", - "alloy-rpc-types-eth", - "alloy-serde", - "alloy-signer", + "alloy-rpc-types-eth 0.5.4", + "alloy-serde 0.5.4", + "alloy-signer 0.5.4", "alloy-sol-types", "async-trait", "auto_impl", @@ -315,16 +446,54 @@ dependencies = [ "thiserror 1.0.69", ] +[[package]] +name = "alloy-network" +version = "0.7.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "209a1882a08e21aca4aac6e2a674dc6fcf614058ef8cb02947d63782b1899552" +dependencies = [ + "alloy-consensus 0.7.3", + "alloy-consensus-any", + "alloy-eips 0.7.3", + "alloy-json-rpc 0.7.3", + "alloy-network-primitives 0.7.3", + "alloy-primitives", + "alloy-rpc-types-any", + "alloy-rpc-types-eth 0.7.3", + "alloy-serde 0.7.3", + "alloy-signer 0.7.3", + "alloy-sol-types", + "async-trait", + "auto_impl", + "futures-utils-wasm", + "serde", + "serde_json", + "thiserror 2.0.3", +] + [[package]] name = "alloy-network-primitives" version = "0.5.4" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "514f70ee2a953db21631cd817b13a1571474ec77ddc03d47616d5e8203489fde" dependencies = [ - "alloy-consensus", - "alloy-eips", + "alloy-consensus 0.5.4", + "alloy-eips 0.5.4", + "alloy-primitives", + "alloy-serde 0.5.4", + "serde", +] + +[[package]] +name = "alloy-network-primitives" +version = "0.7.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c20219d1ad261da7a6331c16367214ee7ded41d001fabbbd656fbf71898b2773" +dependencies = [ + "alloy-consensus 0.7.3", + "alloy-eips 0.7.3", "alloy-primitives", - "alloy-serde", + "alloy-serde 0.7.3", "serde", ] @@ -334,7 +503,7 @@ version = "0.5.4" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "27444ea67d360508753022807cdd0b49a95c878924c9c5f8f32668b7d7768245" dependencies = [ - "alloy-genesis", + "alloy-genesis 0.5.4", "alloy-primitives", "k256", "rand 0.8.5", @@ -345,6 +514,23 @@ dependencies = [ "url", ] +[[package]] +name = "alloy-node-bindings" +version = "0.7.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "bffcf33dd319f21cd6f066d81cbdef0326d4bdaaf7cfe91110bc090707858e9f" +dependencies = [ + "alloy-genesis 0.7.3", + "alloy-primitives", + "k256", + "rand 0.8.5", + "serde_json", + "tempfile", + "thiserror 2.0.3", + "tracing", + "url", +] + [[package]] name = "alloy-primitives" version = "0.8.14" @@ -380,20 +566,20 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "4814d141ede360bb6cd1b4b064f1aab9de391e7c4d0d4d50ac89ea4bc1e25fbd" dependencies = [ "alloy-chains", - "alloy-consensus", - "alloy-eips", - "alloy-json-rpc", - "alloy-network", - "alloy-network-primitives", - "alloy-node-bindings", + "alloy-consensus 0.5.4", + "alloy-eips 0.5.4", + "alloy-json-rpc 0.5.4", + "alloy-network 0.5.4", + "alloy-network-primitives 0.5.4", + "alloy-node-bindings 0.5.4", "alloy-primitives", - "alloy-rpc-client", - "alloy-rpc-types-anvil", - "alloy-rpc-types-eth", - "alloy-signer", - "alloy-signer-local", - "alloy-transport", - "alloy-transport-http", + "alloy-rpc-client 0.5.4", + "alloy-rpc-types-anvil 0.5.4", + "alloy-rpc-types-eth 0.5.4", + "alloy-signer 0.5.4", + "alloy-signer-local 0.5.4", + "alloy-transport 0.5.4", + "alloy-transport-http 0.5.4", "async-stream", "async-trait", "auto_impl", @@ -411,7 +597,48 @@ dependencies = [ "tokio", "tracing", "url", - "wasmtimer", + "wasmtimer 0.2.1", +] + +[[package]] +name = "alloy-provider" +version = "0.7.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9eefa6f4c798ad01f9b4202d02cea75f5ec11fa180502f4701e2b47965a8c0bb" +dependencies = [ + "alloy-chains", + "alloy-consensus 0.7.3", + "alloy-eips 0.7.3", + "alloy-json-rpc 0.7.3", + "alloy-network 0.7.3", + "alloy-network-primitives 0.7.3", + "alloy-node-bindings 0.7.3", + "alloy-primitives", + "alloy-rpc-client 0.7.3", + "alloy-rpc-types-anvil 0.7.3", + "alloy-rpc-types-eth 0.7.3", + "alloy-signer 0.7.3", + "alloy-signer-local 0.7.3", + "alloy-transport 0.7.3", + "alloy-transport-http 0.7.3", + "async-stream", + "async-trait", + "auto_impl", + "dashmap", + "futures", + "futures-utils-wasm", + "lru", + "parking_lot", + "pin-project", + "reqwest 0.12.9", + "schnellru", + "serde", + "serde_json", + "thiserror 2.0.3", + "tokio", + "tracing", + "url", + "wasmtimer 0.4.1", ] [[package]] @@ -442,10 +669,10 @@ version = "0.5.4" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "7fc2bd1e7403463a5f2c61e955bcc9d3072b63aa177442b0f9aa6a6d22a941e3" dependencies = [ - "alloy-json-rpc", + "alloy-json-rpc 0.5.4", "alloy-primitives", - "alloy-transport", - "alloy-transport-http", + "alloy-transport 0.5.4", + "alloy-transport-http 0.5.4", "futures", "pin-project", "reqwest 0.12.9", @@ -456,7 +683,30 @@ dependencies = [ "tower 0.5.1", "tracing", "url", - "wasmtimer", + "wasmtimer 0.2.1", +] + +[[package]] +name = "alloy-rpc-client" +version = "0.7.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ed30bf1041e84cabc5900f52978ca345dd9969f2194a945e6fdec25b0620705c" +dependencies = [ + "alloy-json-rpc 0.7.3", + "alloy-primitives", + "alloy-transport 0.7.3", + "alloy-transport-http 0.7.3", + "futures", + "pin-project", + "reqwest 0.12.9", + "serde", + "serde_json", + "tokio", + "tokio-stream", + "tower 0.5.1", + "tracing", + "url", + "wasmtimer 0.4.1", ] [[package]] @@ -466,9 +716,21 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "eea9bf1abdd506f985a53533f5ac01296bcd6102c5e139bbc5d40bc468d2c916" dependencies = [ "alloy-primitives", - "alloy-rpc-types-anvil", - "alloy-rpc-types-eth", - "alloy-serde", + "alloy-rpc-types-anvil 0.5.4", + "alloy-rpc-types-eth 0.5.4", + "alloy-serde 0.5.4", + "serde", +] + +[[package]] +name = "alloy-rpc-types" +version = "0.7.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5ab686b0fa475d2a4f5916c5f07797734a691ec58e44f0f55d4746ea39cbcefb" +dependencies = [ + "alloy-primitives", + "alloy-rpc-types-eth 0.7.3", + "alloy-serde 0.7.3", "serde", ] @@ -479,22 +741,65 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "2382fc63fb0cf3e02818d547b80cb66cc49a31f8803d0c328402b2008bc13650" dependencies = [ "alloy-primitives", - "alloy-serde", + "alloy-serde 0.5.4", + "serde", +] + +[[package]] +name = "alloy-rpc-types-anvil" +version = "0.7.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d33bc190844626c08e21897736dbd7956ab323c09e6f141b118d1c8b7aff689e" +dependencies = [ + "alloy-primitives", + "alloy-rpc-types-eth 0.7.3", + "alloy-serde 0.7.3", "serde", ] +[[package]] +name = "alloy-rpc-types-any" +version = "0.7.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "200661999b6e235d9840be5d60a6e8ae2f0af9eb2a256dd378786744660e36ec" +dependencies = [ + "alloy-consensus-any", + "alloy-rpc-types-eth 0.7.3", + "alloy-serde 0.7.3", +] + [[package]] name = "alloy-rpc-types-eth" version = "0.5.4" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "00b034779a4850b4b03f5be5ea674a1cf7d746b2da762b34d1860ab45e48ca27" dependencies = [ - "alloy-consensus", - "alloy-eips", - "alloy-network-primitives", + "alloy-consensus 0.5.4", + "alloy-eips 0.5.4", + "alloy-network-primitives 0.5.4", "alloy-primitives", "alloy-rlp", - "alloy-serde", + "alloy-serde 0.5.4", + "alloy-sol-types", + "derive_more", + "itertools 0.13.0", + "serde", + "serde_json", +] + +[[package]] +name = "alloy-rpc-types-eth" +version = "0.7.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a0600b8b5e2dc0cab12cbf91b5a885c35871789fb7b3a57b434bd4fced5b7a8b" +dependencies = [ + "alloy-consensus 0.7.3", + "alloy-consensus-any", + "alloy-eips 0.7.3", + "alloy-network-primitives 0.7.3", + "alloy-primitives", + "alloy-rlp", + "alloy-serde 0.7.3", "alloy-sol-types", "derive_more", "itertools 0.13.0", @@ -513,6 +818,17 @@ dependencies = [ "serde_json", ] +[[package]] +name = "alloy-serde" +version = "0.7.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9afa753a97002a33b2ccb707d9f15f31c81b8c1b786c95b73cc62bb1d1fd0c3f" +dependencies = [ + "alloy-primitives", + "serde", + "serde_json", +] + [[package]] name = "alloy-signer" version = "0.5.4" @@ -527,22 +843,52 @@ dependencies = [ "thiserror 1.0.69", ] +[[package]] +name = "alloy-signer" +version = "0.7.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9b2cbff01a673936c2efd7e00d4c0e9a4dbbd6d600e2ce298078d33efbb19cd7" +dependencies = [ + "alloy-primitives", + "async-trait", + "auto_impl", + "elliptic-curve 0.13.8", + "k256", + "thiserror 2.0.3", +] + [[package]] name = "alloy-signer-local" version = "0.5.4" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "6614f02fc1d5b079b2a4a5320018317b506fd0a6d67c1fd5542a71201724986c" dependencies = [ - "alloy-consensus", - "alloy-network", + "alloy-consensus 0.5.4", + "alloy-network 0.5.4", "alloy-primitives", - "alloy-signer", + "alloy-signer 0.5.4", "async-trait", "k256", "rand 0.8.5", "thiserror 1.0.69", ] +[[package]] +name = "alloy-signer-local" +version = "0.7.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "bd6d988cb6cd7d2f428a74476515b1a6e901e08c796767f9f93311ab74005c8b" +dependencies = [ + "alloy-consensus 0.7.3", + "alloy-network 0.7.3", + "alloy-primitives", + "alloy-signer 0.7.3", + "async-trait", + "k256", + "rand 0.8.5", + "thiserror 2.0.3", +] + [[package]] name = "alloy-sol-macro" version = "0.8.14" @@ -622,7 +968,7 @@ version = "0.5.4" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "be77579633ebbc1266ae6fd7694f75c408beb1aeb6865d0b18f22893c265a061" dependencies = [ - "alloy-json-rpc", + "alloy-json-rpc 0.5.4", "base64 0.22.1", "futures-util", "futures-utils-wasm", @@ -633,8 +979,28 @@ dependencies = [ "tower 0.5.1", "tracing", "url", + "wasmtimer 0.2.1", +] + +[[package]] +name = "alloy-transport" +version = "0.7.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d69d36982b9e46075ae6b792b0f84208c6c2c15ad49f6c500304616ef67b70e0" +dependencies = [ + "alloy-json-rpc 0.7.3", + "base64 0.22.1", + "futures-util", + "futures-utils-wasm", + "serde", + "serde_json", + "thiserror 2.0.3", + "tokio", + "tower 0.5.1", + "tracing", + "url", "wasm-bindgen-futures", - "wasmtimer", + "wasmtimer 0.4.1", ] [[package]] @@ -643,8 +1009,23 @@ version = "0.5.4" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "91fd1a5d0827939847983b46f2f79510361f901dc82f8e3c38ac7397af142c6e" dependencies = [ - "alloy-json-rpc", - "alloy-transport", + "alloy-json-rpc 0.5.4", + "alloy-transport 0.5.4", + "reqwest 0.12.9", + "serde_json", + "tower 0.5.1", + "tracing", + "url", +] + +[[package]] +name = "alloy-transport-http" +version = "0.7.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2e02ffd5d93ffc51d72786e607c97de3b60736ca3e636ead0ec1f7dce68ea3fd" +dependencies = [ + "alloy-json-rpc 0.7.3", + "alloy-transport 0.7.3", "reqwest 0.12.9", "serde_json", "tower 0.5.1", @@ -652,6 +1033,22 @@ dependencies = [ "url", ] +[[package]] +name = "alloy-trie" +version = "0.7.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3a5fd8fea044cc9a8c8a50bb6f28e31f0385d820f116c5b98f6f4e55d6e5590b" +dependencies = [ + "alloy-primitives", + "alloy-rlp", + "arrayvec", + "derive_more", + "nybbles", + "serde", + "smallvec", + "tracing", +] + [[package]] name = "android-tzdata" version = "0.1.1" @@ -805,7 +1202,7 @@ dependencies = [ "tiny-keccak", "tokio", "tracing", - "wasmtimer", + "wasmtimer 0.2.1", "xor_name", ] @@ -892,7 +1289,7 @@ dependencies = [ "void", "walkdir", "wasm-bindgen-futures", - "wasmtimer", + "wasmtimer 0.2.1", "xor_name", ] @@ -1296,6 +1693,9 @@ name = "arrayvec" version = "0.7.6" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "7c02d123df017efcdfbd739ef81735b36c5ba83ec3c59c80a9d7ecc718f92e50" +dependencies = [ + "serde", +] [[package]] name = "ascii" @@ -1556,7 +1956,7 @@ checksum = "ace50bade8e6234aa140d9a2f552bbee1db4d353f69b8217bc503490fc1a9f26" name = "autonomi" version = "0.2.4" dependencies = [ - "alloy", + "alloy 0.5.4", "ant-bootstrap", "ant-evm", "ant-logging", @@ -3332,7 +3732,7 @@ dependencies = [ name = "evmlib" version = "0.1.4" dependencies = [ - "alloy", + "alloy 0.7.3", "dirs-next", "getrandom 0.2.15", "rand 0.8.5", @@ -6717,6 +7117,19 @@ version = "0.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "830b246a0e5f20af87141b25c173cd1b609bd7779a4617d6ec582abaf90870f3" +[[package]] +name = "nybbles" +version = "0.2.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "95f06be0417d97f81fe4e5c86d7d01b392655a9cac9c19a848aa033e18937b23" +dependencies = [ + "alloy-rlp", + "const-hex", + "proptest", + "serde", + "smallvec", +] + [[package]] name = "objc-sys" version = "0.3.5" @@ -9102,6 +9515,9 @@ name = "smallvec" version = "1.13.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "3c5e1a9a646d36c3599cd173a41282daf47c44583ad367b8e6837255952e5c67" +dependencies = [ + "serde", +] [[package]] name = "sn_bls_ckd" @@ -10559,6 +10975,20 @@ dependencies = [ "wasm-bindgen", ] +[[package]] +name = "wasmtimer" +version = "0.4.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0048ad49a55b9deb3953841fa1fc5858f0efbcb7a18868c899a360269fac1b23" +dependencies = [ + "futures", + "js-sys", + "parking_lot", + "pin-utils", + "slab", + "wasm-bindgen", +] + [[package]] name = "web-sys" version = "0.3.74" diff --git a/evmlib/Cargo.toml b/evmlib/Cargo.toml index cb567e24e3..5e4a5b805e 100644 --- a/evmlib/Cargo.toml +++ b/evmlib/Cargo.toml @@ -14,7 +14,7 @@ local = [] external-signer = [] [dependencies] -alloy = { version = "0.5.3", default-features = false, features = ["std", "reqwest-rustls-tls", "provider-anvil-node", "sol-types", "json", "signers", "contract", "signer-local", "network"] } +alloy = { version = "0.7.3", default-features = false, features = ["contract", "json-rpc", "network", "node-bindings", "provider-http", "reqwest-rustls-tls", "rpc-client", "rpc-types", "signer-local", "std"] } dirs-next = "~2.0.0" serde = "=1.0.210" serde_with = { version = "3.11.0", features = ["macros"] } diff --git a/evmlib/abi/IPaymentVault.json b/evmlib/abi/IPaymentVault.json new file mode 100644 index 0000000000..48f3303a77 --- /dev/null +++ b/evmlib/abi/IPaymentVault.json @@ -0,0 +1,191 @@ +[ + { + "anonymous": false, + "inputs": [ + { + "indexed": true, + "internalType": "address", + "name": "rewardsAddress", + "type": "address" + }, + { + "indexed": true, + "internalType": "uint256", + "name": "amount", + "type": "uint256" + }, + { + "indexed": true, + "internalType": "bytes32", + "name": "quoteHash", + "type": "bytes32" + } + ], + "name": "DataPaymentMade", + "type": "event" + }, + { + "inputs": [], + "name": "AntTokenNull", + "type": "error" + }, + { + "inputs": [], + "name": "BatchLimitExceeded", + "type": "error" + }, + { + "inputs": [ + { + "components": [ + { + "internalType": "uint256", + "name": "closeRecordsStored", + "type": "uint256" + }, + { + "internalType": "uint256", + "name": "maxRecords", + "type": "uint256" + }, + { + "internalType": "uint256", + "name": "receivedPaymentCount", + "type": "uint256" + }, + { + "internalType": "uint256", + "name": "liveTime", + "type": "uint256" + }, + { + "internalType": "uint256", + "name": "networkDensity", + "type": "uint256" + }, + { + "internalType": "uint256", + "name": "networkSize", + "type": "uint256" + } + ], + "internalType": "struct IPaymentVault.QuotingMetrics", + "name": "_metrics", + "type": "tuple" + } + ], + "name": "getQuote", + "outputs": [ + { + "internalType": "uint256", + "name": "price", + "type": "uint256" + } + ], + "stateMutability": "view", + "type": "function" + }, + { + "inputs": [ + { + "components": [ + { + "internalType": "address", + "name": "rewardsAddress", + "type": "address" + }, + { + "internalType": "uint256", + "name": "amount", + "type": "uint256" + }, + { + "internalType": "bytes32", + "name": "quoteHash", + "type": "bytes32" + } + ], + "internalType": "struct IPaymentVault.DataPayment[]", + "name": "_payments", + "type": "tuple[]" + } + ], + "name": "payForQuotes", + "outputs": [], + "stateMutability": "nonpayable", + "type": "function" + }, + { + "inputs": [ + { + "components": [ + { + "internalType": "uint256", + "name": "closeRecordsStored", + "type": "uint256" + }, + { + "internalType": "uint256", + "name": "maxRecords", + "type": "uint256" + }, + { + "internalType": "uint256", + "name": "receivedPaymentCount", + "type": "uint256" + }, + { + "internalType": "uint256", + "name": "liveTime", + "type": "uint256" + }, + { + "internalType": "uint256", + "name": "networkDensity", + "type": "uint256" + }, + { + "internalType": "uint256", + "name": "networkSize", + "type": "uint256" + } + ], + "internalType": "struct IPaymentVault.QuotingMetrics", + "name": "_metrics", + "type": "tuple" + }, + { + "components": [ + { + "internalType": "address", + "name": "rewardsAddress", + "type": "address" + }, + { + "internalType": "uint256", + "name": "amount", + "type": "uint256" + }, + { + "internalType": "bytes32", + "name": "quoteHash", + "type": "bytes32" + } + ], + "internalType": "struct IPaymentVault.DataPayment", + "name": "_payment", + "type": "tuple" + } + ], + "name": "verifyPayment", + "outputs": [ + { + "internalType": "bool", + "name": "isValid", + "type": "bool" + } + ], + "stateMutability": "view", + "type": "function" + } +] diff --git a/evmlib/src/contract/data_payments/error.rs b/evmlib/src/contract/data_payments/error.rs deleted file mode 100644 index 95ec1c1c27..0000000000 --- a/evmlib/src/contract/data_payments/error.rs +++ /dev/null @@ -1,24 +0,0 @@ -// Copyright 2024 MaidSafe.net limited. -// -// This SAFE Network Software is licensed to you under The General Public License (GPL), version 3. -// Unless required by applicable law or agreed to in writing, the SAFE Network Software distributed -// under the GPL Licence is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. Please review the Licences for the specific language governing -// permissions and limitations relating to use of the SAFE Network Software. - -use crate::contract::network_token; -use alloy::transports::{RpcError, TransportErrorKind}; - -#[derive(thiserror::Error, Debug)] -pub enum Error { - #[error(transparent)] - ContractError(#[from] alloy::contract::Error), - #[error(transparent)] - RpcError(#[from] RpcError), - #[error(transparent)] - NetworkTokenError(#[from] network_token::Error), - #[error(transparent)] - PendingTransactionError(#[from] alloy::providers::PendingTransactionError), - #[error("The transfer limit of 256 has been exceeded")] - TransferLimitExceeded, -} diff --git a/evmlib/src/contract/mod.rs b/evmlib/src/contract/mod.rs index d428880800..afec267527 100644 --- a/evmlib/src/contract/mod.rs +++ b/evmlib/src/contract/mod.rs @@ -8,3 +8,4 @@ pub mod data_payments; pub mod network_token; +pub mod payment_vault; diff --git a/evmlib/src/contract/payment_vault/error.rs b/evmlib/src/contract/payment_vault/error.rs new file mode 100644 index 0000000000..0441b5b1ea --- /dev/null +++ b/evmlib/src/contract/payment_vault/error.rs @@ -0,0 +1,11 @@ +use alloy::transports::{RpcError, TransportErrorKind}; + +#[derive(thiserror::Error, Debug)] +pub enum Error { + #[error(transparent)] + ContractError(#[from] alloy::contract::Error), + #[error(transparent)] + RpcError(#[from] RpcError), + #[error(transparent)] + PendingTransactionError(#[from] alloy::providers::PendingTransactionError), +} diff --git a/evmlib/src/contract/payment_vault/mod.rs b/evmlib/src/contract/payment_vault/mod.rs new file mode 100644 index 0000000000..63b16b1087 --- /dev/null +++ b/evmlib/src/contract/payment_vault/mod.rs @@ -0,0 +1,100 @@ +mod error; + +use crate::common::{Address, Amount, Calldata, TxHash}; +use crate::contract::payment_vault::error::Error; +use crate::contract::payment_vault::IPaymentVault::{IPaymentVaultInstance, QuotingMetrics}; +use alloy::network::{Network, TransactionBuilder}; +use alloy::providers::Provider; +use alloy::sol; +use alloy::transports::Transport; + +sol!( + #[allow(missing_docs)] + #[sol(rpc)] + IPaymentVault, + "abi/IPaymentVault.json" +); + +pub struct PaymentVaultHandler, N: Network> { + pub contract: IPaymentVaultInstance, +} + +impl PaymentVaultHandler +where + T: Transport + Clone, + P: Provider, + N: Network, +{ + /// Create a new PaymentVaultHandler instance from a deployed contract's address + pub fn new(contract_address: Address, provider: P) -> Self { + let contract = IPaymentVault::new(contract_address, provider); + Self { contract } + } + + /// Fetch a quote from the contract + pub async fn fetch_quote(&self, metrics: QuotingMetrics) -> Result { + let amount = self.contract.getQuote(metrics).call().await?.price; + Ok(amount) + } + + /// Pay for quotes. + /// Input: (quote_hash, reward_address, amount). + pub async fn pay_for_quotes>>( + &self, + data_payments: I, + ) -> Result { + let (calldata, to) = self.pay_for_quotes_calldata(data_payments)?; + + let transaction_request = self + .contract + .provider() + .transaction_request() + .with_to(to) + .with_input(calldata); + + let tx_hash = self + .contract + .provider() + .send_transaction(transaction_request) + .await? + .watch() + .await?; + + Ok(tx_hash) + } + + /// Pay for quotes. + /// Input: (quote_hash, reward_address, amount). + /// Returns the transaction calldata. + pub fn pay_for_quotes_calldata>>( + &self, + data_payments: I, + ) -> Result<(Calldata, Address), Error> { + let data_payments: Vec = + data_payments.into_iter().map(|item| item.into()).collect(); + + let calldata = self + .contract + .payForQuotes(data_payments) + .calldata() + .to_owned(); + + Ok((calldata, *self.contract.address())) + } + + /// Verify if a payment is valid + pub async fn validate_payment>( + &self, + metrics: QuotingMetrics, + payment: I, + ) -> Result { + let is_valid = self + .contract + .verifyPayment(metrics, payment.into()) + .call() + .await? + .isValid; + + Ok(is_valid) + } +} diff --git a/evmlib/tests/data_payments.rs b/evmlib/tests/data_payments.rs deleted file mode 100644 index 26223cfcc1..0000000000 --- a/evmlib/tests/data_payments.rs +++ /dev/null @@ -1,137 +0,0 @@ -mod common; - -use crate::common::quote::random_quote_payment; -use alloy::network::{Ethereum, EthereumWallet}; -use alloy::node_bindings::AnvilInstance; -use alloy::primitives::utils::parse_ether; -use alloy::providers::ext::AnvilApi; -use alloy::providers::fillers::{ - BlobGasFiller, ChainIdFiller, FillProvider, GasFiller, JoinFill, NonceFiller, WalletFiller, -}; -use alloy::providers::{Identity, ProviderBuilder, ReqwestProvider, WalletProvider}; -use alloy::signers::local::{LocalSigner, PrivateKeySigner}; -use alloy::transports::http::{Client, Http}; -use evmlib::common::U256; -use evmlib::contract::data_payments::{DataPaymentsHandler, MAX_TRANSFERS_PER_TRANSACTION}; -use evmlib::contract::network_token::NetworkToken; -use evmlib::testnet::{deploy_data_payments_contract, deploy_network_token_contract, start_node}; -use evmlib::wallet::wallet_address; - -async fn setup() -> ( - AnvilInstance, - NetworkToken< - Http, - FillProvider< - JoinFill< - JoinFill< - Identity, - JoinFill< - GasFiller, - JoinFill>, - >, - >, - WalletFiller, - >, - ReqwestProvider, - Http, - Ethereum, - >, - Ethereum, - >, - DataPaymentsHandler< - Http, - FillProvider< - JoinFill< - JoinFill< - Identity, - JoinFill< - GasFiller, - JoinFill>, - >, - >, - WalletFiller, - >, - ReqwestProvider, - Http, - Ethereum, - >, - Ethereum, - >, -) { - let (anvil, rpc_url) = start_node(); - - let network_token = deploy_network_token_contract(&rpc_url, &anvil).await; - - let data_payments = - deploy_data_payments_contract(&rpc_url, &anvil, *network_token.contract.address()).await; - - (anvil, network_token, data_payments) -} - -#[allow(clippy::unwrap_used)] -#[allow(clippy::type_complexity)] -#[allow(dead_code)] -async fn provider_with_gas_funded_wallet( - anvil: &AnvilInstance, -) -> FillProvider< - JoinFill< - JoinFill< - Identity, - JoinFill>>, - >, - WalletFiller, - >, - ReqwestProvider, - Http, - Ethereum, -> { - let signer: PrivateKeySigner = LocalSigner::random(); - let wallet = EthereumWallet::from(signer); - - let rpc_url = anvil.endpoint().parse().unwrap(); - - let provider = ProviderBuilder::new() - .with_recommended_fillers() - .wallet(wallet) - .on_http(rpc_url); - - let account = wallet_address(provider.wallet()); - - // Fund the wallet with plenty of gas tokens - provider - .anvil_set_balance(account, parse_ether("1000").expect("")) - .await - .unwrap(); - - provider -} - -#[tokio::test] -async fn test_deploy() { - setup().await; -} - -#[tokio::test] -async fn test_pay_for_quotes() { - let (_anvil, network_token, mut data_payments) = setup().await; - - let mut quote_payments = vec![]; - - for _ in 0..MAX_TRANSFERS_PER_TRANSACTION { - let quote_payment = random_quote_payment(); - quote_payments.push(quote_payment); - } - - let _ = network_token - .approve(*data_payments.contract.address(), U256::MAX) - .await - .unwrap(); - - // Contract provider has a different account coupled to it, - // so we set it to the same as the network token contract - data_payments.set_provider(network_token.contract.provider().clone()); - - let result = data_payments.pay_for_quotes(quote_payments).await; - - assert!(result.is_ok(), "Failed with error: {:?}", result.err()); -} From 7a14f0432d4f1a06c3260910576c907a390039ec Mon Sep 17 00:00:00 2001 From: Warm Beer Date: Thu, 5 Dec 2024 17:30:08 +0100 Subject: [PATCH 04/33] feat: added payment vault deploy fn --- evmlib/src/contract/mod.rs | 1 - evmlib/src/contract/payment_vault/handler.rs | 100 +++++++++++++ .../contract/payment_vault/implementation.rs | 30 ++++ .../src/contract/payment_vault/interface.rs | 19 +++ evmlib/src/contract/payment_vault/mod.rs | 104 +------------ evmlib/src/lib.rs | 2 +- evmlib/src/testnet.rs | 12 +- evmlib/src/transaction.rs | 6 +- evmlib/src/wallet.rs | 11 +- evmlib/tests/payment_vault.rs | 138 ++++++++++++++++++ 10 files changed, 313 insertions(+), 110 deletions(-) create mode 100644 evmlib/src/contract/payment_vault/handler.rs create mode 100644 evmlib/src/contract/payment_vault/implementation.rs create mode 100644 evmlib/src/contract/payment_vault/interface.rs create mode 100644 evmlib/tests/payment_vault.rs diff --git a/evmlib/src/contract/mod.rs b/evmlib/src/contract/mod.rs index afec267527..405f0c7fd5 100644 --- a/evmlib/src/contract/mod.rs +++ b/evmlib/src/contract/mod.rs @@ -6,6 +6,5 @@ // KIND, either express or implied. Please review the Licences for the specific language governing // permissions and limitations relating to use of the SAFE Network Software. -pub mod data_payments; pub mod network_token; pub mod payment_vault; diff --git a/evmlib/src/contract/payment_vault/handler.rs b/evmlib/src/contract/payment_vault/handler.rs new file mode 100644 index 0000000000..ad983e4d2b --- /dev/null +++ b/evmlib/src/contract/payment_vault/handler.rs @@ -0,0 +1,100 @@ +use crate::common::{Address, Amount, Calldata, TxHash}; +use crate::contract::payment_vault::error::Error; +use crate::contract::payment_vault::interface::IPaymentVault; +use crate::contract::payment_vault::interface::IPaymentVault::IPaymentVaultInstance; +use alloy::network::{Network, TransactionBuilder}; +use alloy::providers::Provider; +use alloy::transports::Transport; + +pub struct PaymentVaultHandler, N: Network> { + pub contract: IPaymentVaultInstance, +} + +impl PaymentVaultHandler +where + T: Transport + Clone, + P: Provider, + N: Network, +{ + /// Create a new PaymentVaultHandler instance from a deployed contract's address + pub fn new(contract_address: Address, provider: P) -> Self { + let contract = IPaymentVault::new(contract_address, provider); + Self { contract } + } + + /// Set the provider + pub fn set_provider(&mut self, provider: P) { + let address = *self.contract.address(); + self.contract = IPaymentVault::new(address, provider); + } + + /// Fetch a quote from the contract + pub async fn fetch_quote( + &self, + metrics: IPaymentVault::QuotingMetrics, + ) -> Result { + let amount = self.contract.getQuote(metrics).call().await?.price; + Ok(amount) + } + + /// Pay for quotes. + pub async fn pay_for_quotes>>( + &self, + data_payments: I, + ) -> Result { + let (calldata, to) = self.pay_for_quotes_calldata(data_payments)?; + + let transaction_request = self + .contract + .provider() + .transaction_request() + .with_to(to) + .with_input(calldata); + + let tx_hash = self + .contract + .provider() + .send_transaction(transaction_request) + .await? + .watch() + .await?; + + Ok(tx_hash) + } + + /// Returns the pay for quotes transaction calldata. + pub fn pay_for_quotes_calldata>>( + &self, + data_payments: I, + ) -> Result<(Calldata, Address), Error> { + let data_payments: Vec = + data_payments.into_iter().map(|item| item.into()).collect(); + + let calldata = self + .contract + .payForQuotes(data_payments) + .calldata() + .to_owned(); + + Ok((calldata, *self.contract.address())) + } + + /// Verify if a payment is valid + pub async fn verify_payment< + Q: Into, + I: Into, + >( + &self, + metrics: Q, + payment: I, + ) -> Result { + let is_valid = self + .contract + .verifyPayment(metrics.into(), payment.into()) + .call() + .await? + .isValid; + + Ok(is_valid) + } +} diff --git a/evmlib/src/contract/payment_vault/implementation.rs b/evmlib/src/contract/payment_vault/implementation.rs new file mode 100644 index 0000000000..78ae83117c --- /dev/null +++ b/evmlib/src/contract/payment_vault/implementation.rs @@ -0,0 +1,30 @@ +use crate::common::Address; +use alloy::hex; +use alloy::network::{Network, ReceiptResponse, TransactionBuilder}; +use alloy::providers::Provider; +use alloy::transports::Transport; + +const BYTE_CODE: &str = "0x60a060405230608052348015610013575f5ffd5b5061001c610021565b6100d3565b7ff0c57e16840df040f15088dc2f81fe391c3923bec73e23a9662efc9c229c6a00805468010000000000000000900460ff16156100715760405163f92ee8a960e01b815260040160405180910390fd5b80546001600160401b03908116146100d05780546001600160401b0319166001600160401b0390811782556040519081527fc7f505b2f371ae2175ee4913f4499e1f2633a7b5936321eed1cdaeb6115181d29060200160405180910390a15b50565b608051610ef76100f95f395f818161064d0152818161067601526107ba0152610ef75ff3fe6080604052600436106100bf575f3560e01c8063715018a61161007c578063ad3cb1cc11610057578063ad3cb1cc14610253578063b6c2141b14610290578063cd6dc687146102af578063f2fde38b146102ce575f5ffd5b8063715018a6146101d45780638da5cb5b146101e8578063a69bf4a314610224575f5ffd5b80630716326d146100c35780633c150bf214610132578063474740b1146101605780634ec42e8e146101745780634f1ef286146101ab57806352d1902d146101c0575b5f5ffd5b3480156100ce575f5ffd5b506101086100dd366004610bc4565b600260208190525f91825260409091208054600182015491909201546001600160a01b039092169183565b604080516001600160a01b0390941684526020840192909252908201526060015b60405180910390f35b34801561013d575f5ffd5b5061015261014c366004610bf1565b50600190565b604051908152602001610129565b34801561016b575f5ffd5b506101525f5481565b34801561017f575f5ffd5b50600154610193906001600160a01b031681565b6040516001600160a01b039091168152602001610129565b6101be6101b9366004610c33565b6102ed565b005b3480156101cb575f5ffd5b5061015261030c565b3480156101df575f5ffd5b506101be610327565b3480156101f3575f5ffd5b507f9016d09d72d40fdae2fd8ceac6b6234c7706214fd39c1cd1e609a0528c199300546001600160a01b0316610193565b34801561022f575f5ffd5b5061024361023e366004610cf9565b61033a565b6040519015158152602001610129565b34801561025e575f5ffd5b50610283604051806040016040528060058152602001640352e302e360dc1b81525081565b6040516101299190610d37565b34801561029b575f5ffd5b506101be6102aa366004610d6c565b6103b6565b3480156102ba575f5ffd5b506101be6102c9366004610ddd565b6104a3565b3480156102d9575f5ffd5b506101be6102e8366004610e07565b610600565b6102f5610642565b6102fe826106e6565b61030882826106ee565b5050565b5f6103156107af565b505f516020610ea25f395f51905f5290565b61032f6107f8565b6103385f610853565b565b6040808201355f90815260026020818152838320845160608101865281546001600160a01b031681526001820154818401819052919093015494830194909452919290918401351480156103ae57506103966020840184610e07565b6001600160a01b0316815f01516001600160a01b0316145b949350505050565b5f5481908111156103da57604051630d67f41160e21b815260040160405180910390fd5b5f5b8181101561049d57368484838181106103f7576103f7610e22565b60600291909101915061042b9050336104136020840184610e07565b6001546001600160a01b0316919060208501356108c3565b6040808201355f90815260026020522081906104478282610e36565b505060408101356020820180359061045f9084610e07565b6001600160a01b03167ff998960b1c6f0e0e89b7bbe6b6fbf3e03e6f08eee5b8430877d8adb8e149d58060405160405180910390a4506001016103dc565b50505050565b7ff0c57e16840df040f15088dc2f81fe391c3923bec73e23a9662efc9c229c6a008054600160401b810460ff16159067ffffffffffffffff165f811580156104e85750825b90505f8267ffffffffffffffff1660011480156105045750303b155b905081158015610512575080155b156105305760405163f92ee8a960e01b815260040160405180910390fd5b845467ffffffffffffffff19166001178555831561055a57845460ff60401b1916600160401b1785555b6001600160a01b03871661058157604051632d06160b60e21b815260040160405180910390fd5b600180546001600160a01b0319166001600160a01b0389161790555f8690556105a93361091d565b6105b161092e565b83156105f757845460ff60401b19168555604051600181527fc7f505b2f371ae2175ee4913f4499e1f2633a7b5936321eed1cdaeb6115181d29060200160405180910390a15b50505050505050565b6106086107f8565b6001600160a01b03811661063657604051631e4fbdf760e01b81525f60048201526024015b60405180910390fd5b61063f81610853565b50565b306001600160a01b037f00000000000000000000000000000000000000000000000000000000000000001614806106c857507f00000000000000000000000000000000000000000000000000000000000000006001600160a01b03166106bc5f516020610ea25f395f51905f52546001600160a01b031690565b6001600160a01b031614155b156103385760405163703e46dd60e11b815260040160405180910390fd5b61063f6107f8565b816001600160a01b03166352d1902d6040518163ffffffff1660e01b8152600401602060405180830381865afa925050508015610748575060408051601f3d908101601f1916820190925261074591810190610e74565b60015b61077057604051634c9c8ce360e01b81526001600160a01b038316600482015260240161062d565b5f516020610ea25f395f51905f5281146107a057604051632a87526960e21b81526004810182905260240161062d565b6107aa8383610936565b505050565b306001600160a01b037f000000000000000000000000000000000000000000000000000000000000000016146103385760405163703e46dd60e11b815260040160405180910390fd5b3361082a7f9016d09d72d40fdae2fd8ceac6b6234c7706214fd39c1cd1e609a0528c199300546001600160a01b031690565b6001600160a01b0316146103385760405163118cdaa760e01b815233600482015260240161062d565b7f9016d09d72d40fdae2fd8ceac6b6234c7706214fd39c1cd1e609a0528c19930080546001600160a01b031981166001600160a01b03848116918217845560405192169182907f8be0079c531659141344cd1fd0a4f28419497f9722a3daafe3b4186f6b6457e0905f90a3505050565b604080516001600160a01b0385811660248301528416604482015260648082018490528251808303909101815260849091019091526020810180516001600160e01b03166323b872dd60e01b17905261049d90859061098b565b6109256109f7565b61063f81610a40565b6103386109f7565b61093f82610a48565b6040516001600160a01b038316907fbc7cd75a20ee27fd9adebab32041f755214dbc6bffa90cc0225b39da2e5c2d3b905f90a2805115610983576107aa8282610aab565b610308610b1d565b5f5f60205f8451602086015f885af1806109aa576040513d5f823e3d81fd5b50505f513d915081156109c15780600114156109ce565b6001600160a01b0384163b155b1561049d57604051635274afe760e01b81526001600160a01b038516600482015260240161062d565b7ff0c57e16840df040f15088dc2f81fe391c3923bec73e23a9662efc9c229c6a0054600160401b900460ff1661033857604051631afcd79f60e31b815260040160405180910390fd5b6106086109f7565b806001600160a01b03163b5f03610a7d57604051634c9c8ce360e01b81526001600160a01b038216600482015260240161062d565b5f516020610ea25f395f51905f5280546001600160a01b0319166001600160a01b0392909216919091179055565b60605f5f846001600160a01b031684604051610ac79190610e8b565b5f60405180830381855af49150503d805f8114610aff576040519150601f19603f3d011682016040523d82523d5f602084013e610b04565b606091505b5091509150610b14858383610b3c565b95945050505050565b34156103385760405163b398979f60e01b815260040160405180910390fd5b606082610b5157610b4c82610b9b565b610b94565b8151158015610b6857506001600160a01b0384163b155b15610b9157604051639996b31560e01b81526001600160a01b038516600482015260240161062d565b50805b9392505050565b805115610bab5780518082602001fd5b60405163d6bda27560e01b815260040160405180910390fd5b5f60208284031215610bd4575f5ffd5b5035919050565b5f60c08284031215610beb575f5ffd5b50919050565b5f60c08284031215610c01575f5ffd5b610b948383610bdb565b6001600160a01b038116811461063f575f5ffd5b634e487b7160e01b5f52604160045260245ffd5b5f5f60408385031215610c44575f5ffd5b8235610c4f81610c0b565b9150602083013567ffffffffffffffff811115610c6a575f5ffd5b8301601f81018513610c7a575f5ffd5b803567ffffffffffffffff811115610c9457610c94610c1f565b604051601f8201601f19908116603f0116810167ffffffffffffffff81118282101715610cc357610cc3610c1f565b604052818152828201602001871015610cda575f5ffd5b816020840160208301375f602083830101528093505050509250929050565b5f5f828403610120811215610d0c575f5ffd5b610d168585610bdb565b9250606060bf1982011215610d29575f5ffd5b5060c0830190509250929050565b602081525f82518060208401528060208501604085015e5f604082850101526040601f19601f83011684010191505092915050565b5f5f60208385031215610d7d575f5ffd5b823567ffffffffffffffff811115610d93575f5ffd5b8301601f81018513610da3575f5ffd5b803567ffffffffffffffff811115610db9575f5ffd5b856020606083028401011115610dcd575f5ffd5b6020919091019590945092505050565b5f5f60408385031215610dee575f5ffd5b8235610df981610c0b565b946020939093013593505050565b5f60208284031215610e17575f5ffd5b8135610b9481610c0b565b634e487b7160e01b5f52603260045260245ffd5b8135610e4181610c0b565b81546001600160a01b0319166001600160a01b039190911617815560208201356001820155604090910135600290910155565b5f60208284031215610e84575f5ffd5b5051919050565b5f82518060208501845e5f92019182525091905056fe360894a13ba1a3210667c828492db98dca3e2076cc3735a920a3ca505d382bbca26469706673582212203894ca52be6e6323aa3d296efd566c7f21d1723d4c66c56aed8a5f75a96b579d64736f6c634300081c0033"; + +pub async fn deploy(provider: &P) -> Address +where + T: Transport + Clone, + P: Provider, + N: Network, +{ + let bytecode = hex::decode(BYTE_CODE).expect("Could not decode byte code"); + let tx = provider.transaction_request().with_deploy_code(bytecode); + + // Deploy the contract. + let receipt = provider + .send_transaction(tx) + .await + .expect("Could not send deployment transaction") + .get_receipt() + .await + .expect("Deployment transaction failed"); + + receipt + .contract_address() + .expect("Contract address missing") +} diff --git a/evmlib/src/contract/payment_vault/interface.rs b/evmlib/src/contract/payment_vault/interface.rs new file mode 100644 index 0000000000..bb43ac0927 --- /dev/null +++ b/evmlib/src/contract/payment_vault/interface.rs @@ -0,0 +1,19 @@ +use crate::common::{Address, Amount, QuoteHash}; +use alloy::sol; + +sol!( + #[allow(missing_docs)] + #[sol(rpc)] + IPaymentVault, + "abi/IPaymentVault.json" +); + +impl From<(QuoteHash, Address, Amount)> for IPaymentVault::DataPayment { + fn from(data: (QuoteHash, Address, Amount)) -> Self { + Self { + rewardsAddress: data.1, + amount: data.2, + quoteHash: data.0, + } + } +} diff --git a/evmlib/src/contract/payment_vault/mod.rs b/evmlib/src/contract/payment_vault/mod.rs index 63b16b1087..5cbc6f7718 100644 --- a/evmlib/src/contract/payment_vault/mod.rs +++ b/evmlib/src/contract/payment_vault/mod.rs @@ -1,100 +1,6 @@ -mod error; +pub mod error; +pub mod handler; +pub mod implementation; +pub mod interface; -use crate::common::{Address, Amount, Calldata, TxHash}; -use crate::contract::payment_vault::error::Error; -use crate::contract::payment_vault::IPaymentVault::{IPaymentVaultInstance, QuotingMetrics}; -use alloy::network::{Network, TransactionBuilder}; -use alloy::providers::Provider; -use alloy::sol; -use alloy::transports::Transport; - -sol!( - #[allow(missing_docs)] - #[sol(rpc)] - IPaymentVault, - "abi/IPaymentVault.json" -); - -pub struct PaymentVaultHandler, N: Network> { - pub contract: IPaymentVaultInstance, -} - -impl PaymentVaultHandler -where - T: Transport + Clone, - P: Provider, - N: Network, -{ - /// Create a new PaymentVaultHandler instance from a deployed contract's address - pub fn new(contract_address: Address, provider: P) -> Self { - let contract = IPaymentVault::new(contract_address, provider); - Self { contract } - } - - /// Fetch a quote from the contract - pub async fn fetch_quote(&self, metrics: QuotingMetrics) -> Result { - let amount = self.contract.getQuote(metrics).call().await?.price; - Ok(amount) - } - - /// Pay for quotes. - /// Input: (quote_hash, reward_address, amount). - pub async fn pay_for_quotes>>( - &self, - data_payments: I, - ) -> Result { - let (calldata, to) = self.pay_for_quotes_calldata(data_payments)?; - - let transaction_request = self - .contract - .provider() - .transaction_request() - .with_to(to) - .with_input(calldata); - - let tx_hash = self - .contract - .provider() - .send_transaction(transaction_request) - .await? - .watch() - .await?; - - Ok(tx_hash) - } - - /// Pay for quotes. - /// Input: (quote_hash, reward_address, amount). - /// Returns the transaction calldata. - pub fn pay_for_quotes_calldata>>( - &self, - data_payments: I, - ) -> Result<(Calldata, Address), Error> { - let data_payments: Vec = - data_payments.into_iter().map(|item| item.into()).collect(); - - let calldata = self - .contract - .payForQuotes(data_payments) - .calldata() - .to_owned(); - - Ok((calldata, *self.contract.address())) - } - - /// Verify if a payment is valid - pub async fn validate_payment>( - &self, - metrics: QuotingMetrics, - payment: I, - ) -> Result { - let is_valid = self - .contract - .verifyPayment(metrics, payment.into()) - .call() - .await? - .isValid; - - Ok(is_valid) - } -} +pub const MAX_TRANSFERS_PER_TRANSACTION: usize = 256; diff --git a/evmlib/src/lib.rs b/evmlib/src/lib.rs index fb3303fd47..331e1fbfa3 100644 --- a/evmlib/src/lib.rs +++ b/evmlib/src/lib.rs @@ -52,7 +52,7 @@ const ARBITRUM_SEPOLIA_PAYMENT_TOKEN_ADDRESS: Address = // Should be updated when the smart contract changes! const ARBITRUM_ONE_DATA_PAYMENTS_ADDRESS: Address = - address!("887930F30EDEb1B255Cd2273C3F4400919df2EFe"); + address!("607483B50C5F06c25cDC316b6d1E071084EeC9f5"); const ARBITRUM_SEPOLIA_DATA_PAYMENTS_ADDRESS: Address = address!("Dd56b03Dae2Ab8594D80269EC4518D13F1A110BD"); diff --git a/evmlib/src/testnet.rs b/evmlib/src/testnet.rs index e5f1f79708..f5b76fea5c 100644 --- a/evmlib/src/testnet.rs +++ b/evmlib/src/testnet.rs @@ -7,8 +7,9 @@ // permissions and limitations relating to use of the SAFE Network Software. use crate::common::Address; -use crate::contract::data_payments::DataPaymentsHandler; use crate::contract::network_token::NetworkToken; +use crate::contract::payment_vault; +use crate::contract::payment_vault::handler::PaymentVaultHandler; use crate::reqwest::Url; use crate::{CustomNetwork, Network}; use alloy::hex::ToHexExt; @@ -119,8 +120,8 @@ pub async fn deploy_network_token_contract( pub async fn deploy_data_payments_contract( rpc_url: &Url, anvil: &AnvilInstance, - token_address: Address, -) -> DataPaymentsHandler< + _token_address: Address, +) -> PaymentVaultHandler< Http, FillProvider< JoinFill< @@ -146,5 +147,8 @@ pub async fn deploy_data_payments_contract( .on_http(rpc_url.clone()); // Deploy the contract. - DataPaymentsHandler::deploy(provider, token_address).await + let payment_vault_contract_address = payment_vault::implementation::deploy(&provider).await; + + // Create a handler for the deployed contract + PaymentVaultHandler::new(payment_vault_contract_address, provider) } diff --git a/evmlib/src/transaction.rs b/evmlib/src/transaction.rs index af23e4f026..6ebd893a5d 100644 --- a/evmlib/src/transaction.rs +++ b/evmlib/src/transaction.rs @@ -10,6 +10,7 @@ use crate::common::{Address, QuoteHash, TxHash, U256}; use crate::event::{ChunkPaymentEvent, DATA_PAYMENT_EVENT_SIGNATURE}; use crate::Network; use alloy::eips::BlockNumberOrTag; +use alloy::network::primitives::BlockTransactionsKind; use alloy::primitives::FixedBytes; use alloy::providers::{Provider, ProviderBuilder}; use alloy::rpc::types::{Block, Filter, Log, TransactionReceipt}; @@ -55,7 +56,10 @@ async fn get_block_by_number(network: &Network, block_number: u64) -> Result>( } let provider = http_provider_with_wallet(network.rpc_url().clone(), wallet); - let data_payments = DataPaymentsHandler::new(*network.data_payments_address(), provider); + let data_payments = PaymentVaultHandler::new(*network.data_payments_address(), provider); // Divide transfers over multiple transactions if they exceed the max per transaction. let chunks = payments.chunks(MAX_TRANSFERS_PER_TRANSACTION); @@ -340,6 +341,7 @@ pub async fn pay_for_quotes>( for batch in chunks { let batch: Vec = batch.to_vec(); + debug!( "Paying for batch of quotes of len: {}, {batch:?}", batch.len() @@ -349,6 +351,7 @@ pub async fn pay_for_quotes>( .pay_for_quotes(batch.clone()) .await .map_err(|err| PayForQuotesError(Error::from(err), tx_hashes_by_quote.clone()))?; + info!("Paid for batch of quotes with final tx hash: {tx_hash}"); for (quote_hash, _, _) in batch { diff --git a/evmlib/tests/payment_vault.rs b/evmlib/tests/payment_vault.rs new file mode 100644 index 0000000000..b3d3ede55f --- /dev/null +++ b/evmlib/tests/payment_vault.rs @@ -0,0 +1,138 @@ +mod common; + +use crate::common::quote::random_quote_payment; +use alloy::network::{Ethereum, EthereumWallet}; +use alloy::node_bindings::AnvilInstance; +use alloy::primitives::utils::parse_ether; +use alloy::providers::ext::AnvilApi; +use alloy::providers::fillers::{ + BlobGasFiller, ChainIdFiller, FillProvider, GasFiller, JoinFill, NonceFiller, WalletFiller, +}; +use alloy::providers::{Identity, ProviderBuilder, ReqwestProvider, WalletProvider}; +use alloy::signers::local::{LocalSigner, PrivateKeySigner}; +use alloy::transports::http::{Client, Http}; +use evmlib::common::U256; +use evmlib::contract::network_token::NetworkToken; +use evmlib::contract::payment_vault::handler::PaymentVaultHandler; +use evmlib::contract::payment_vault::MAX_TRANSFERS_PER_TRANSACTION; +use evmlib::testnet::{deploy_data_payments_contract, deploy_network_token_contract, start_node}; +use evmlib::wallet::wallet_address; + +async fn setup() -> ( + AnvilInstance, + NetworkToken< + Http, + FillProvider< + JoinFill< + JoinFill< + Identity, + JoinFill< + GasFiller, + JoinFill>, + >, + >, + WalletFiller, + >, + ReqwestProvider, + Http, + Ethereum, + >, + Ethereum, + >, + PaymentVaultHandler< + Http, + FillProvider< + JoinFill< + JoinFill< + Identity, + JoinFill< + GasFiller, + JoinFill>, + >, + >, + WalletFiller, + >, + ReqwestProvider, + Http, + Ethereum, + >, + Ethereum, + >, +) { + let (anvil, rpc_url) = start_node(); + + let network_token = deploy_network_token_contract(&rpc_url, &anvil).await; + + let data_payments = + deploy_data_payments_contract(&rpc_url, &anvil, *network_token.contract.address()).await; + + (anvil, network_token, data_payments) +} + +#[allow(clippy::unwrap_used)] +#[allow(clippy::type_complexity)] +#[allow(dead_code)] +async fn provider_with_gas_funded_wallet( + anvil: &AnvilInstance, +) -> FillProvider< + JoinFill< + JoinFill< + Identity, + JoinFill>>, + >, + WalletFiller, + >, + ReqwestProvider, + Http, + Ethereum, +> { + let signer: PrivateKeySigner = LocalSigner::random(); + let wallet = EthereumWallet::from(signer); + + let rpc_url = anvil.endpoint().parse().unwrap(); + + let provider = ProviderBuilder::new() + .with_recommended_fillers() + .wallet(wallet) + .on_http(rpc_url); + + let account = wallet_address(provider.wallet()); + + // Fund the wallet with plenty of gas tokens + provider + .anvil_set_balance(account, parse_ether("1000").expect("")) + .await + .unwrap(); + + provider +} + +#[tokio::test] +async fn test_deploy() { + setup().await; +} + +#[tokio::test] +async fn test_pay_for_quotes() { + let (_anvil, network_token, mut data_payments) = setup().await; + + let mut quote_payments = vec![]; + + for _ in 0..MAX_TRANSFERS_PER_TRANSACTION { + let quote_payment = random_quote_payment(); + quote_payments.push(quote_payment); + } + + let _ = network_token + .approve(*data_payments.contract.address(), U256::MAX) + .await + .unwrap(); + + // Contract provider has a different account coupled to it, + // so we set it to the same as the network token contract + data_payments.set_provider(network_token.contract.provider().clone()); + + let result = data_payments.pay_for_quotes(quote_payments).await; + + assert!(result.is_ok(), "Failed with error: {:?}", result.err()); +} From 74d03a197b1a2c8c2cc0247fc1b4bced7586c933 Mon Sep 17 00:00:00 2001 From: Warm Beer Date: Thu, 5 Dec 2024 19:12:57 +0100 Subject: [PATCH 05/33] chore: update verify data payment logic --- Cargo.lock | 18 +-- autonomi/src/client/utils.rs | 2 +- autonomi/src/lib.rs | 1 + autonomi/src/utils.rs | 39 ++++++ .../src/contract/payment_vault/interface.rs | 26 +++- evmlib/src/event.rs | 71 ---------- evmlib/src/lib.rs | 21 +-- evmlib/src/transaction.rs | 124 ++++-------------- 8 files changed, 101 insertions(+), 201 deletions(-) create mode 100644 autonomi/src/utils.rs delete mode 100644 evmlib/src/event.rs diff --git a/Cargo.lock b/Cargo.lock index d71bc86b4f..598c271ac2 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -258,7 +258,7 @@ dependencies = [ "alloy-transport 0.7.3", "futures", "futures-util", - "thiserror 2.0.3", + "thiserror 2.0.4", ] [[package]] @@ -421,7 +421,7 @@ dependencies = [ "alloy-sol-types", "serde", "serde_json", - "thiserror 2.0.3", + "thiserror 2.0.4", "tracing", ] @@ -468,7 +468,7 @@ dependencies = [ "futures-utils-wasm", "serde", "serde_json", - "thiserror 2.0.3", + "thiserror 2.0.4", ] [[package]] @@ -526,7 +526,7 @@ dependencies = [ "rand 0.8.5", "serde_json", "tempfile", - "thiserror 2.0.3", + "thiserror 2.0.4", "tracing", "url", ] @@ -634,7 +634,7 @@ dependencies = [ "schnellru", "serde", "serde_json", - "thiserror 2.0.3", + "thiserror 2.0.4", "tokio", "tracing", "url", @@ -854,7 +854,7 @@ dependencies = [ "auto_impl", "elliptic-curve 0.13.8", "k256", - "thiserror 2.0.3", + "thiserror 2.0.4", ] [[package]] @@ -886,7 +886,7 @@ dependencies = [ "async-trait", "k256", "rand 0.8.5", - "thiserror 2.0.3", + "thiserror 2.0.4", ] [[package]] @@ -994,7 +994,7 @@ dependencies = [ "futures-utils-wasm", "serde", "serde_json", - "thiserror 2.0.3", + "thiserror 2.0.4", "tokio", "tower 0.5.1", "tracing", @@ -1140,7 +1140,7 @@ dependencies = [ "tracing", "tracing-subscriber", "url", - "wasmtimer", + "wasmtimer 0.2.1", "wiremock", ] diff --git a/autonomi/src/client/utils.rs b/autonomi/src/client/utils.rs index 9207b035c2..4c5f53b3a7 100644 --- a/autonomi/src/client/utils.rs +++ b/autonomi/src/client/utils.rs @@ -7,7 +7,6 @@ // permissions and limitations relating to use of the SAFE Network Software. use crate::client::payment::Receipt; -use crate::utils::receipt_from_cost_map_and_payments; use ant_evm::{EvmWallet, ProofOfPayment, QuotePayment}; use ant_networking::{ GetRecordCfg, Network, NetworkError, PayeeQuote, PutRecordCfg, VerificationKind, @@ -30,6 +29,7 @@ use super::{ Client, }; use crate::self_encryption::DataMapLevel; +use crate::utils::receipt_from_cost_map_and_payments; impl Client { /// Fetch and decrypt all chunks in the data map. diff --git a/autonomi/src/lib.rs b/autonomi/src/lib.rs index 7dd7aeb1a8..2564ee3b2e 100644 --- a/autonomi/src/lib.rs +++ b/autonomi/src/lib.rs @@ -82,3 +82,4 @@ pub use client::{files::archive::PrivateArchive, Client}; #[cfg(feature = "extension-module")] mod python; +mod utils; diff --git a/autonomi/src/utils.rs b/autonomi/src/utils.rs new file mode 100644 index 0000000000..1348c0c685 --- /dev/null +++ b/autonomi/src/utils.rs @@ -0,0 +1,39 @@ +use crate::client::payment::Receipt; +use ant_evm::{PaymentQuote, ProofOfPayment, QuoteHash, TxHash}; +use ant_networking::PayeeQuote; +use std::collections::{BTreeMap, HashMap}; +use xor_name::XorName; + +pub fn cost_map_to_quotes( + cost_map: HashMap, +) -> HashMap { + cost_map.into_iter().map(|(k, (_, _, v))| (k, v)).collect() +} + +pub fn receipt_from_cost_map_and_payments( + cost_map: HashMap, + payments: &BTreeMap, +) -> Receipt { + let quotes = cost_map_to_quotes(cost_map); + receipt_from_quotes_and_payments("es, payments) +} + +pub fn receipt_from_quotes_and_payments( + quotes: &HashMap, + payments: &BTreeMap, +) -> Receipt { + quotes + .iter() + .filter_map(|(xor_name, quote)| { + payments.get("e.hash()).map(|tx_hash| { + ( + *xor_name, + ProofOfPayment { + quote: quote.clone(), + tx_hash: *tx_hash, + }, + ) + }) + }) + .collect() +} diff --git a/evmlib/src/contract/payment_vault/interface.rs b/evmlib/src/contract/payment_vault/interface.rs index bb43ac0927..d99811e01a 100644 --- a/evmlib/src/contract/payment_vault/interface.rs +++ b/evmlib/src/contract/payment_vault/interface.rs @@ -1,4 +1,6 @@ -use crate::common::{Address, Amount, QuoteHash}; +use crate::common::{Address, Amount, QuoteHash, U256}; +use crate::quoting_metrics::QuotingMetrics; +use alloy::primitives::FixedBytes; use alloy::sol; sol!( @@ -9,11 +11,25 @@ sol!( ); impl From<(QuoteHash, Address, Amount)> for IPaymentVault::DataPayment { - fn from(data: (QuoteHash, Address, Amount)) -> Self { + fn from(value: (QuoteHash, Address, Amount)) -> Self { Self { - rewardsAddress: data.1, - amount: data.2, - quoteHash: data.0, + rewardsAddress: value.1, + amount: value.2, + quoteHash: value.0, + } + } +} + +impl From for IPaymentVault::QuotingMetrics { + fn from(value: QuotingMetrics) -> Self { + Self { + closeRecordsStored: U256::from(value.close_records_stored), + maxRecords: U256::from(value.max_records), + receivedPaymentCount: U256::from(value.received_payment_count), + liveTime: U256::from(value.live_time), + networkDensity: FixedBytes::<32>::from(value.network_density.unwrap_or_default()) + .into(), + networkSize: value.network_size.map(U256::from).unwrap_or_default(), } } } diff --git a/evmlib/src/event.rs b/evmlib/src/event.rs deleted file mode 100644 index 5cdda3d91e..0000000000 --- a/evmlib/src/event.rs +++ /dev/null @@ -1,71 +0,0 @@ -// Copyright 2024 MaidSafe.net limited. -// -// This SAFE Network Software is licensed to you under The General Public License (GPL), version 3. -// Unless required by applicable law or agreed to in writing, the SAFE Network Software distributed -// under the GPL Licence is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. Please review the Licences for the specific language governing -// permissions and limitations relating to use of the SAFE Network Software. - -use crate::common::{Address, Hash, U256}; -use alloy::primitives::{b256, FixedBytes}; -use alloy::rpc::types::Log; - -// Should be updated when the smart contract changes! -pub(crate) const DATA_PAYMENT_EVENT_SIGNATURE: FixedBytes<32> = - b256!("f998960b1c6f0e0e89b7bbe6b6fbf3e03e6f08eee5b8430877d8adb8e149d580"); // DevSkim: ignore DS173237 - -#[derive(thiserror::Error, Debug)] -pub enum Error { - #[error("Topics amount is unexpected. Was expecting 4")] - TopicsAmountUnexpected, - #[error("Event signature is missing")] - EventSignatureMissing, - #[error("Event signature does not match")] - EventSignatureDoesNotMatch, -} - -/// Struct for the ChunkPaymentEvent emitted by the ChunkPayments smart contract. -#[derive(Debug)] -pub(crate) struct ChunkPaymentEvent { - pub rewards_address: Address, - pub amount: U256, - pub quote_hash: Hash, -} - -impl TryFrom for ChunkPaymentEvent { - type Error = Error; - - fn try_from(log: Log) -> Result { - // Verify the amount of topics - if log.topics().len() != 4 { - error!("Topics amount is unexpected. Was expecting 4"); - return Err(Error::TopicsAmountUnexpected); - } - - let topic0 = log - .topics() - .first() - .ok_or(Error::EventSignatureMissing) - .inspect_err(|_| error!("Event signature is missing"))?; - - // Verify the event signature - if topic0 != &DATA_PAYMENT_EVENT_SIGNATURE { - error!( - "Event signature does not match. Expected: {:?}, got: {:?}", - DATA_PAYMENT_EVENT_SIGNATURE, topic0 - ); - return Err(Error::EventSignatureDoesNotMatch); - } - - // Extract the data - let rewards_address = Address::from_slice(&log.topics()[1][12..]); - let amount = U256::from_be_slice(&log.topics()[2][12..]); - let quote_hash = Hash::from_slice(log.topics()[3].as_slice()); - - Ok(Self { - rewards_address, - amount, - quote_hash, - }) - } -} diff --git a/evmlib/src/lib.rs b/evmlib/src/lib.rs index 331e1fbfa3..6c1054d600 100644 --- a/evmlib/src/lib.rs +++ b/evmlib/src/lib.rs @@ -6,11 +6,10 @@ // KIND, either express or implied. Please review the Licences for the specific language governing // permissions and limitations relating to use of the SAFE Network Software. -use crate::common::{Address, QuoteHash, TxHash}; +use crate::common::{Address, QuoteHash}; use crate::transaction::verify_data_payment; use alloy::primitives::address; use alloy::transports::http::reqwest; -use common::Amount; use quoting_metrics::QuotingMetrics; use serde::{Deserialize, Serialize}; use serde_with::{serde_as, DisplayFromStr}; @@ -23,7 +22,6 @@ extern crate tracing; pub mod common; pub mod contract; pub mod cryptography; -pub(crate) mod event; #[cfg(feature = "external-signer")] pub mod external_signer; pub mod quoting_metrics; @@ -139,21 +137,10 @@ impl Network { pub async fn verify_data_payment( &self, - tx_hash: TxHash, quote_hash: QuoteHash, - _quoting_metrics: QuotingMetrics, + quoting_metrics: QuotingMetrics, reward_addr: Address, - quote_expiration_timestamp_in_secs: u64, - ) -> Result { - verify_data_payment( - self, - tx_hash, - quote_hash, - // quoting_metrics, // NB TODO use them here @Mick - reward_addr, - Default::default(), // NB TODO remove amounts @Mick - quote_expiration_timestamp_in_secs, - ) - .await + ) -> Result<(), transaction::Error> { + verify_data_payment(self, quote_hash, reward_addr, quoting_metrics).await } } diff --git a/evmlib/src/transaction.rs b/evmlib/src/transaction.rs index 6ebd893a5d..6900664538 100644 --- a/evmlib/src/transaction.rs +++ b/evmlib/src/transaction.rs @@ -6,14 +6,11 @@ // KIND, either express or implied. Please review the Licences for the specific language governing // permissions and limitations relating to use of the SAFE Network Software. -use crate::common::{Address, QuoteHash, TxHash, U256}; -use crate::event::{ChunkPaymentEvent, DATA_PAYMENT_EVENT_SIGNATURE}; -use crate::Network; -use alloy::eips::BlockNumberOrTag; -use alloy::network::primitives::BlockTransactionsKind; -use alloy::primitives::FixedBytes; -use alloy::providers::{Provider, ProviderBuilder}; -use alloy::rpc::types::{Block, Filter, Log, TransactionReceipt}; +use crate::common::{Address, Amount, QuoteHash}; +use crate::contract::payment_vault::handler::PaymentVaultHandler; +use crate::quoting_metrics::QuotingMetrics; +use crate::utils::http_provider; +use crate::{contract, Network}; use alloy::transports::{RpcError, TransportErrorKind}; #[derive(thiserror::Error, Debug)] @@ -32,6 +29,10 @@ pub enum Error { EventProofNotFound, #[error("Payment was done after the quote expired")] QuoteExpired, + #[error(transparent)] + PaymentVaultError(#[from] contract::payment_vault::error::Error), + #[error("Payment missing")] + PaymentMissing, } /// Get a transaction receipt by its hash. @@ -105,118 +106,45 @@ async fn get_data_payment_event( /// Verify if a data payment is confirmed. pub async fn verify_data_payment( network: &Network, - tx_hash: TxHash, quote_hash: QuoteHash, reward_addr: Address, - amount: U256, - quote_expiration_timestamp_in_secs: u64, -) -> Result { - debug!("Verifying data payment for tx_hash: {tx_hash:?}"); - let transaction = get_transaction_receipt_by_hash(network, tx_hash) - .await? - .ok_or(Error::TransactionNotFound)?; - - // If the status is True, it means the tx is confirmed. - if !transaction.status() { - error!("Transaction {tx_hash:?} is not confirmed"); - return Err(Error::TransactionUnconfirmed); + quoting_metrics: QuotingMetrics, +) -> Result<(), Error> { + let provider = http_provider(network.rpc_url().clone()); + let payment_vault = PaymentVaultHandler::new(*network.data_payments_address(), provider); + + let is_paid = payment_vault + .verify_payment(quoting_metrics, (quote_hash, reward_addr, Amount::ZERO)) + .await?; + + if is_paid { + Ok(()) + } else { + Err(Error::PaymentMissing) } - - let block_number = transaction - .block_number - .ok_or(Error::TransactionNotInBlock) - .inspect_err(|_| error!("Transaction {tx_hash:?} has not been included in a block yet"))?; - - let block = get_block_by_number(network, block_number) - .await? - .ok_or(Error::BlockNotFound)?; - - // Check if payment was done within the quote expiration timeframe. - if quote_expiration_timestamp_in_secs < block.header.timestamp { - error!("Payment for tx_hash: {tx_hash:?} was done after the quote expired"); - return Err(Error::QuoteExpired); - } - - let logs = - get_data_payment_event(network, block_number, quote_hash, reward_addr, amount).await?; - - for log in logs { - if log.transaction_hash != Some(tx_hash) { - // Wrong transaction. - continue; - } - - if let Ok(event) = ChunkPaymentEvent::try_from(log) { - // Check if the event matches what we expect. - if event.quote_hash == quote_hash - && event.rewards_address == reward_addr - && event.amount >= amount - { - return Ok(event.amount); - } - } - } - - error!("No event proof found for tx_hash: {tx_hash:?}"); - - Err(Error::EventProofNotFound) } #[cfg(test)] mod tests { - use crate::common::{Address, U256}; - use crate::transaction::{ - get_data_payment_event, get_transaction_receipt_by_hash, verify_data_payment, - }; + use crate::common::Address; + use crate::quoting_metrics::QuotingMetrics; + use crate::transaction::verify_data_payment; use crate::Network; use alloy::hex::FromHex; use alloy::primitives::b256; - #[tokio::test] - async fn test_get_transaction_receipt_by_hash() { - let network = Network::ArbitrumOne; - - let tx_hash = b256!("3304465f38fa0bd9670a426108dd1ddd193e059dcb7c13982d31424646217a36"); // DevSkim: ignore DS173237 - - assert!(get_transaction_receipt_by_hash(&network, tx_hash) - .await - .unwrap() - .is_some()); - } - - #[tokio::test] - async fn test_get_data_payment_event() { - let network = Network::ArbitrumOne; - - let block_number: u64 = 260246302; - let reward_address = Address::from_hex("8AB15A43305854e4AE4E6FBEa0CD1CC0AB4ecB2A").unwrap(); // DevSkim: ignore DS173237 - let amount = U256::from(1); - let quote_hash = b256!("EBD943C38C0422901D4CF22E677DD95F2591CA8D6EBFEA8BAF1BFE9FF5506ECE"); // DevSkim: ignore DS173237 - - let logs = - get_data_payment_event(&network, block_number, quote_hash, reward_address, amount) - .await - .unwrap(); - - assert_eq!(logs.len(), 1); - } - #[tokio::test] async fn test_verify_data_payment() { let network = Network::ArbitrumOne; - let tx_hash = b256!("3304465f38fa0bd9670a426108dd1ddd193e059dcb7c13982d31424646217a36"); // DevSkim: ignore DS173237 let quote_hash = b256!("EBD943C38C0422901D4CF22E677DD95F2591CA8D6EBFEA8BAF1BFE9FF5506ECE"); // DevSkim: ignore DS173237 let reward_address = Address::from_hex("8AB15A43305854e4AE4E6FBEa0CD1CC0AB4ecB2A").unwrap(); // DevSkim: ignore DS173237 - let amount = U256::from(1); let result = verify_data_payment( &network, - tx_hash, quote_hash, reward_address, - amount, - 4102441200, + QuotingMetrics::default(), ) .await; From 5274730c143f23c5e4034f714e8983c561c4e91e Mon Sep 17 00:00:00 2001 From: grumbach Date: Fri, 6 Dec 2024 15:59:03 +0900 Subject: [PATCH 06/33] feat: require 1/3 of nodes to have the data to stop quoting --- ant-networking/src/lib.rs | 14 +++++++++++--- 1 file changed, 11 insertions(+), 3 deletions(-) diff --git a/ant-networking/src/lib.rs b/ant-networking/src/lib.rs index eb4c3dea2a..3cfe25a3f3 100644 --- a/ant-networking/src/lib.rs +++ b/ant-networking/src/lib.rs @@ -392,7 +392,7 @@ impl Network { close_nodes.retain(|peer_id| !ignore_peers.contains(peer_id)); if close_nodes.is_empty() { - error!("Cann't get store_cost of {record_address:?}, as all close_nodes are ignored"); + error!("Can't get store_cost of {record_address:?}, as all close_nodes are ignored"); return Err(NetworkError::NoStoreCostResponses); } @@ -406,6 +406,10 @@ impl Network { .send_and_get_responses(&close_nodes, &request, true) .await; + // consider data to be already paid for if 1/3 of the close nodes already have it + let mut peer_already_have_it = 0; + let enough_peers_already_have_it = close_nodes.len() / 3; + // loop over responses let mut all_quotes = vec![]; let mut quotes_to_pay = vec![]; @@ -438,8 +442,12 @@ impl Network { if !storage_proofs.is_empty() { debug!("Storage proofing during GetStoreQuote to be implemented."); } - info!("Address {record_address:?} was already paid for according to {peer_address:?}, ending quote request"); - return Ok(vec![]); + peer_already_have_it += 1; + info!("Address {record_address:?} was already paid for according to {peer_address:?} ({peer_already_have_it}/{enough_peers_already_have_it})"); + if peer_already_have_it >= enough_peers_already_have_it { + info!("Address {record_address:?} was already paid for according to {peer_already_have_it} peers, ending quote request"); + return Ok(vec![]); + } } Err(err) => { error!("Got an error while requesting quote from peer {peer:?}: {err}"); From 71de86b99ad3a025238c373a9b22effab4406c3a Mon Sep 17 00:00:00 2001 From: grumbach Date: Fri, 6 Dec 2024 17:04:39 +0900 Subject: [PATCH 07/33] chore: fix compile issues in evmlib, various fixes in node and networking side --- ant-networking/src/event/request_response.rs | 24 -------------------- ant-networking/src/lib.rs | 24 +++++++------------- ant-node/src/put_validation.rs | 14 ++++++++---- ant-protocol/src/messages/cmd.rs | 19 ---------------- ant-protocol/src/messages/response.rs | 5 ---- autonomi/src/client/quote.rs | 8 +++---- autonomi/src/client/utils.rs | 12 +++++----- autonomi/src/utils.rs | 8 +++---- evmlib/src/external_signer.rs | 12 ++++++---- evmlib/src/lib.rs | 3 ++- evmlib/src/transaction.rs | 5 ++-- evmlib/tests/wallet.rs | 19 ++++++---------- 12 files changed, 51 insertions(+), 102 deletions(-) diff --git a/ant-networking/src/event/request_response.rs b/ant-networking/src/event/request_response.rs index d7a210821b..ce6755e8dc 100644 --- a/ant-networking/src/event/request_response.rs +++ b/ant-networking/src/event/request_response.rs @@ -48,30 +48,6 @@ impl SwarmDriver { self.add_keys_to_replication_fetcher(holder, keys); } - Request::Cmd(ant_protocol::messages::Cmd::QuoteVerification { - quotes, - .. - }) => { - let response = Response::Cmd( - ant_protocol::messages::CmdResponse::QuoteVerification(Ok(())), - ); - self.queue_network_swarm_cmd(NetworkSwarmCmd::SendResponse { - resp: response, - channel: MsgResponder::FromPeer(channel), - }); - - // The keypair is required to verify the quotes, - // hence throw it up to Network layer for further actions. - let quotes = quotes - .iter() - .filter_map(|(peer_address, quote)| { - peer_address - .as_peer_id() - .map(|peer_id| (peer_id, quote.clone())) - }) - .collect(); - self.send_event(NetworkEvent::QuoteVerification { quotes }) - } Request::Cmd(ant_protocol::messages::Cmd::PeerConsideredAsBad { detected_by, bad_peer, diff --git a/ant-networking/src/lib.rs b/ant-networking/src/lib.rs index 3cfe25a3f3..1e7a46aed6 100644 --- a/ant-networking/src/lib.rs +++ b/ant-networking/src/lib.rs @@ -51,7 +51,7 @@ use self::{cmd::NetworkSwarmCmd, error::Result}; use ant_evm::{PaymentQuote, QuotingMetrics}; use ant_protocol::{ error::Error as ProtocolError, - messages::{ChunkProof, Cmd, Nonce, Query, QueryResponse, Request, Response}, + messages::{ChunkProof, Nonce, Query, QueryResponse, Request, Response}, storage::{RecordType, RetryStrategy, Scratchpad}, NetworkAddress, PrettyPrintKBucketKey, PrettyPrintRecordKey, CLOSE_GROUP_SIZE, }; @@ -83,8 +83,10 @@ use { std::collections::HashSet, }; -/// The type of quote for a selected payee. -pub type PayeeQuote = (PeerId, PaymentQuote); +/// Selected quotes to pay for a data address +pub struct SelectedQuotes { + pub quotes: Vec<(PeerId, PaymentQuote)>, +} /// Majority of a given group (i.e. > 1/2). #[inline] @@ -382,7 +384,7 @@ impl Network { &self, record_address: NetworkAddress, ignore_peers: Vec, - ) -> Result> { + ) -> Result { // The requirement of having at least CLOSE_GROUP_SIZE // close nodes will be checked internally automatically. let mut close_nodes = self @@ -446,7 +448,7 @@ impl Network { info!("Address {record_address:?} was already paid for according to {peer_address:?} ({peer_already_have_it}/{enough_peers_already_have_it})"); if peer_already_have_it >= enough_peers_already_have_it { info!("Address {record_address:?} was already paid for according to {peer_already_have_it} peers, ending quote request"); - return Ok(vec![]); + return Ok(SelectedQuotes { quotes: vec![] }); } } Err(err) => { @@ -458,17 +460,7 @@ impl Network { } } - // send the quotes to the other peers for verification - for peer_id in close_nodes.iter() { - let request = Request::Cmd(Cmd::QuoteVerification { - target: NetworkAddress::from_peer(*peer_id), - quotes: all_quotes.clone(), - }); - - self.send_req_ignore_reply(request, *peer_id); - } - - Ok(quotes_to_pay) + Ok(SelectedQuotes { quotes: quotes_to_pay }) } /// Get register from network. diff --git a/ant-node/src/put_validation.rs b/ant-node/src/put_validation.rs index ff9c5b3974..3fa1bb8fc2 100644 --- a/ant-node/src/put_validation.rs +++ b/ant-node/src/put_validation.rs @@ -19,7 +19,7 @@ use ant_protocol::{ }; use ant_registers::SignedRegister; use libp2p::kad::{Record, RecordKey}; -use std::time::{Duration, UNIX_EPOCH}; +use std::time::{Duration, SystemTime, UNIX_EPOCH}; use xor_name::XorName; impl Node { @@ -664,7 +664,7 @@ impl Node { // verify quote timestamp let quote_timestamp = payment.quote.timestamp; let quote_expiration_time = quote_timestamp + Duration::from_secs(QUOTE_EXPIRATION_SECS); - let quote_expiration_time_in_secs = quote_expiration_time + let _quote_expiration_time_in_secs = quote_expiration_time .duration_since(UNIX_EPOCH) .map_err(|e| { Error::InvalidRequest(format!( @@ -672,16 +672,22 @@ impl Node { )) })? .as_secs(); + // NB TODO @mick: can we check if the quote has expired with block time in evmlib? Or should nodes do it manually here? Else keep the block below + // manually check if the quote has expired + if quote_expiration_time < SystemTime::now() { + warn!("Payment quote has expired for record {pretty_key}"); + return Err(Error::InvalidRequest(format!( + "Payment quote has expired for record {pretty_key}" + ))); + } // check if payment is valid on chain debug!("Verifying payment for record {pretty_key}"); let reward_amount = self.evm_network() .verify_data_payment( - payment.tx_hash, payment.quote.hash(), payment.quote.quoting_metrics, *self.reward_address(), - quote_expiration_time_in_secs, ) .await .map_err(|e| Error::EvmNetwork(format!("Failed to verify chunk payment: {e}")))?; diff --git a/ant-protocol/src/messages/cmd.rs b/ant-protocol/src/messages/cmd.rs index cec0629259..1437c6540b 100644 --- a/ant-protocol/src/messages/cmd.rs +++ b/ant-protocol/src/messages/cmd.rs @@ -8,7 +8,6 @@ #![allow(clippy::mutable_key_type)] // for Bytes in NetworkAddress use crate::{storage::RecordType, NetworkAddress}; -pub use ant_evm::PaymentQuote; use serde::{Deserialize, Serialize}; /// Data and CashNote cmds - recording transactions or creating, updating, and removing data. @@ -28,11 +27,6 @@ pub enum Cmd { /// Keys of copy that shall be replicated. keys: Vec<(NetworkAddress, RecordType)>, }, - /// Write operation to notify nodes a list of PaymentQuote collected. - QuoteVerification { - target: NetworkAddress, - quotes: Vec<(NetworkAddress, PaymentQuote)>, - }, /// Notify the peer it is now being considered as BAD due to the included behaviour PeerConsideredAsBad { detected_by: NetworkAddress, @@ -52,11 +46,6 @@ impl std::fmt::Debug for Cmd { .field("first_ten_keys", &first_ten_keys) .finish() } - Cmd::QuoteVerification { target, quotes } => f - .debug_struct("Cmd::QuoteVerification") - .field("target", target) - .field("quotes_len", "es.len()) - .finish(), Cmd::PeerConsideredAsBad { detected_by, bad_peer, @@ -76,7 +65,6 @@ impl Cmd { pub fn dst(&self) -> NetworkAddress { match self { Cmd::Replicate { holder, .. } => holder.clone(), - Cmd::QuoteVerification { target, .. } => target.clone(), Cmd::PeerConsideredAsBad { bad_peer, .. } => bad_peer.clone(), } } @@ -93,13 +81,6 @@ impl std::fmt::Display for Cmd { keys.len() ) } - Cmd::QuoteVerification { target, quotes } => { - write!( - f, - "Cmd::QuoteVerification(sent to {target:?} has {} quotes)", - quotes.len() - ) - } Cmd::PeerConsideredAsBad { detected_by, bad_peer, diff --git a/ant-protocol/src/messages/response.rs b/ant-protocol/src/messages/response.rs index d3fc29ab31..48b332c60b 100644 --- a/ant-protocol/src/messages/response.rs +++ b/ant-protocol/src/messages/response.rs @@ -150,11 +150,6 @@ pub enum CmdResponse { /// Response to replication cmd Replicate(Result<()>), // - // ===== QuoteVerification ===== - // - /// Response to quote verification cmd - QuoteVerification(Result<()>), - // // ===== PeerConsideredAsBad ===== // /// Response to the considered as bad notification diff --git a/autonomi/src/client/quote.rs b/autonomi/src/client/quote.rs index 8b257f74d6..16cd377369 100644 --- a/autonomi/src/client/quote.rs +++ b/autonomi/src/client/quote.rs @@ -8,7 +8,7 @@ use ant_evm::{PaymentQuote, ProofOfPayment, QuoteHash, TxHash}; use ant_evm::{Amount, AttoTokens, QuotePayment}; -use ant_networking::{Network, NetworkError, PayeeQuote}; +use ant_networking::{Network, NetworkError, SelectedQuotes}; use ant_protocol::{ storage::ChunkAddress, NetworkAddress, @@ -21,7 +21,7 @@ use super::{data::CostError, Client}; pub struct QuotesToPay { pub nodes_to_pay: Vec, - pub nodes_to_upload_to: Vec, + pub nodes_to_upload_to: Vec, pub cost_per_node: AttoTokens, pub total_cost: AttoTokens, } @@ -66,7 +66,7 @@ impl Client { async fn fetch_store_quote( network: &Network, content_addr: XorName, -) -> Result, NetworkError> { +) -> Result, NetworkError> { network .get_store_quote_from_network( NetworkAddress::from_chunk_address(ChunkAddress::new(content_addr)), @@ -79,7 +79,7 @@ async fn fetch_store_quote( async fn fetch_store_quote_with_retries( network: &Network, content_addr: XorName, -) -> Result<(XorName, Vec), CostError> { +) -> Result<(XorName, Vec), CostError> { let mut retries = 0; loop { diff --git a/autonomi/src/client/utils.rs b/autonomi/src/client/utils.rs index 4c5f53b3a7..a6fc96c8ee 100644 --- a/autonomi/src/client/utils.rs +++ b/autonomi/src/client/utils.rs @@ -9,7 +9,7 @@ use crate::client::payment::Receipt; use ant_evm::{EvmWallet, ProofOfPayment, QuotePayment}; use ant_networking::{ - GetRecordCfg, Network, NetworkError, PayeeQuote, PutRecordCfg, VerificationKind, + GetRecordCfg, Network, NetworkError, SelectedQuotes, PutRecordCfg, VerificationKind, }; use ant_protocol::{ messages::ChunkProof, @@ -200,7 +200,7 @@ impl Client { pub(crate) async fn get_store_quotes( &self, content_addrs: impl Iterator, - ) -> Result, CostError> { + ) -> Result, CostError> { let futures: Vec<_> = content_addrs .into_iter() .map(|content_addr| fetch_store_quote_with_retries(&self.network, content_addr)) @@ -208,7 +208,7 @@ impl Client { let quotes = futures::future::try_join_all(futures).await?; - Ok(quotes.into_iter().collect::>()) + Ok(quotes.into_iter().collect::>()) } } @@ -216,7 +216,7 @@ impl Client { async fn fetch_store_quote_with_retries( network: &Network, content_addr: XorName, -) -> Result<(XorName, PayeeQuote), CostError> { +) -> Result<(XorName, SelectedQuotes), CostError> { let mut retries = 0; loop { @@ -242,7 +242,7 @@ async fn fetch_store_quote_with_retries( async fn fetch_store_quote( network: &Network, content_addr: XorName, -) -> Result { +) -> Result { network .get_store_costs_from_network( NetworkAddress::from_chunk_address(ChunkAddress::new(content_addr)), @@ -253,7 +253,7 @@ async fn fetch_store_quote( /// Form to be executed payments and already executed payments from a cost map. pub(crate) fn extract_quote_payments( - cost_map: &HashMap, + cost_map: &HashMap, ) -> (Vec, Vec) { let mut to_be_paid = vec![]; let mut already_paid = vec![]; diff --git a/autonomi/src/utils.rs b/autonomi/src/utils.rs index 1348c0c685..85d8f3f62c 100644 --- a/autonomi/src/utils.rs +++ b/autonomi/src/utils.rs @@ -1,17 +1,17 @@ use crate::client::payment::Receipt; use ant_evm::{PaymentQuote, ProofOfPayment, QuoteHash, TxHash}; -use ant_networking::PayeeQuote; +use ant_networking::SelectedQuotes; use std::collections::{BTreeMap, HashMap}; use xor_name::XorName; pub fn cost_map_to_quotes( - cost_map: HashMap, + cost_map: HashMap, ) -> HashMap { - cost_map.into_iter().map(|(k, (_, _, v))| (k, v)).collect() + cost_map.into_iter().map(|(k, (_, q))| (k, q)).collect() } pub fn receipt_from_cost_map_and_payments( - cost_map: HashMap, + cost_map: HashMap, payments: &BTreeMap, ) -> Receipt { let quotes = cost_map_to_quotes(cost_map); diff --git a/evmlib/src/external_signer.rs b/evmlib/src/external_signer.rs index 20c3aa95df..30186f031d 100644 --- a/evmlib/src/external_signer.rs +++ b/evmlib/src/external_signer.rs @@ -7,9 +7,8 @@ // permissions and limitations relating to use of the SAFE Network Software. use crate::common::{Address, Amount, Calldata, QuoteHash, QuotePayment, U256}; -use crate::contract::data_payments::{DataPaymentsHandler, MAX_TRANSFERS_PER_TRANSACTION}; -use crate::contract::network_token::NetworkToken; -use crate::contract::{data_payments, network_token}; +use crate::contract::network_token::{NetworkToken, self}; +use crate::contract::payment_vault::MAX_TRANSFERS_PER_TRANSACTION; use crate::utils::http_provider; use crate::Network; use serde::{Deserialize, Serialize}; @@ -20,7 +19,7 @@ pub enum Error { #[error("Network token contract error: {0}")] NetworkTokenContract(#[from] network_token::Error), #[error("Data payments contract error: {0}")] - DataPaymentsContract(#[from] data_payments::error::Error), + DataPaymentsContract(#[from] crate::contract::payment_vault::error::Error), } /// Approve an address / smart contract to spend this wallet's payment tokens. @@ -73,7 +72,10 @@ pub fn pay_for_quotes_calldata>( let approve_amount = total_amount; let provider = http_provider(network.rpc_url().clone()); - let data_payments = DataPaymentsHandler::new(*network.data_payments_address(), provider); + let data_payments = crate::contract::payment_vault::handler::PaymentVaultHandler::new( + *network.data_payments_address(), + provider, + ); // Divide transfers over multiple transactions if they exceed the max per transaction. let chunks = payments.chunks(MAX_TRANSFERS_PER_TRANSACTION); diff --git a/evmlib/src/lib.rs b/evmlib/src/lib.rs index 6c1054d600..a37ae2a16e 100644 --- a/evmlib/src/lib.rs +++ b/evmlib/src/lib.rs @@ -10,6 +10,7 @@ use crate::common::{Address, QuoteHash}; use crate::transaction::verify_data_payment; use alloy::primitives::address; use alloy::transports::http::reqwest; +use common::Amount; use quoting_metrics::QuotingMetrics; use serde::{Deserialize, Serialize}; use serde_with::{serde_as, DisplayFromStr}; @@ -140,7 +141,7 @@ impl Network { quote_hash: QuoteHash, quoting_metrics: QuotingMetrics, reward_addr: Address, - ) -> Result<(), transaction::Error> { + ) -> Result { verify_data_payment(self, quote_hash, reward_addr, quoting_metrics).await } } diff --git a/evmlib/src/transaction.rs b/evmlib/src/transaction.rs index 6900664538..993af3e074 100644 --- a/evmlib/src/transaction.rs +++ b/evmlib/src/transaction.rs @@ -109,16 +109,17 @@ pub async fn verify_data_payment( quote_hash: QuoteHash, reward_addr: Address, quoting_metrics: QuotingMetrics, -) -> Result<(), Error> { +) -> Result { let provider = http_provider(network.rpc_url().clone()); let payment_vault = PaymentVaultHandler::new(*network.data_payments_address(), provider); let is_paid = payment_vault .verify_payment(quoting_metrics, (quote_hash, reward_addr, Amount::ZERO)) .await?; + let amount_paid = Amount::ZERO; // NB TODO @mick we need to get the amount paid from the contract if is_paid { - Ok(()) + Ok(amount_paid) } else { Err(Error::PaymentMissing) } diff --git a/evmlib/tests/wallet.rs b/evmlib/tests/wallet.rs index 905f719fc3..c324f771fc 100644 --- a/evmlib/tests/wallet.rs +++ b/evmlib/tests/wallet.rs @@ -8,7 +8,8 @@ use alloy::providers::ext::AnvilApi; use alloy::providers::{ProviderBuilder, WalletProvider}; use alloy::signers::local::{LocalSigner, PrivateKeySigner}; use evmlib::common::{Amount, TxHash}; -use evmlib::contract::data_payments::MAX_TRANSFERS_PER_TRANSACTION; +use evmlib::contract::payment_vault::MAX_TRANSFERS_PER_TRANSACTION; +use evmlib::quoting_metrics::QuotingMetrics; use evmlib::testnet::{deploy_data_payments_contract, deploy_network_token_contract, start_node}; use evmlib::transaction::verify_data_payment; use evmlib::wallet::{transfer_tokens, wallet_address, Wallet}; @@ -67,7 +68,6 @@ async fn funded_wallet(network: &Network, genesis_wallet: EthereumWallet) -> Wal #[tokio::test] async fn test_pay_for_quotes_and_data_payment_verification() { const TRANSFERS: usize = 600; - const EXPIRATION_TIMESTAMP_IN_SECS: u64 = 4102441200; // The year 2100 let (_anvil, network, genesis_wallet) = local_testnet().await; let wallet = funded_wallet(&network, genesis_wallet).await; @@ -87,23 +87,18 @@ async fn test_pay_for_quotes_and_data_payment_verification() { unique_tx_hashes.len(), TRANSFERS.div_ceil(MAX_TRANSFERS_PER_TRANSACTION) ); - - for quote_payment in quote_payments.iter() { - let tx_hash = *tx_hashes.get("e_payment.0).unwrap(); - + for (quote_hash, reward_addr, _) in quote_payments.iter() { let result = verify_data_payment( &network, - tx_hash, - quote_payment.0, - quote_payment.1, - quote_payment.2, - EXPIRATION_TIMESTAMP_IN_SECS, + *quote_hash, + *reward_addr, + QuotingMetrics::default(), ) .await; assert!( result.is_ok(), - "Verification failed for: {quote_payment:?}. Error: {:?}", + "Verification failed for: {quote_hash:?}. Error: {:?}", result.err() ); } From 558bdc0fcb643a566ede991410fdf4033f938976 Mon Sep 17 00:00:00 2001 From: Warm Beer Date: Thu, 5 Dec 2024 20:53:37 +0100 Subject: [PATCH 08/33] test: add reach through proxy test --- evmlib/src/contract/payment_vault/handler.rs | 6 +++--- evmlib/src/utils.rs | 2 +- evmlib/tests/payment_vault.rs | 19 ++++++++++++++++++- 3 files changed, 22 insertions(+), 5 deletions(-) diff --git a/evmlib/src/contract/payment_vault/handler.rs b/evmlib/src/contract/payment_vault/handler.rs index ad983e4d2b..38d1dd2da8 100644 --- a/evmlib/src/contract/payment_vault/handler.rs +++ b/evmlib/src/contract/payment_vault/handler.rs @@ -29,11 +29,11 @@ where } /// Fetch a quote from the contract - pub async fn fetch_quote( + pub async fn get_quote>( &self, - metrics: IPaymentVault::QuotingMetrics, + metrics: I, ) -> Result { - let amount = self.contract.getQuote(metrics).call().await?.price; + let amount = self.contract.getQuote(metrics.into()).call().await?.price; Ok(amount) } diff --git a/evmlib/src/utils.rs b/evmlib/src/utils.rs index f212b466d5..4e3133713f 100644 --- a/evmlib/src/utils.rs +++ b/evmlib/src/utils.rs @@ -165,7 +165,7 @@ fn local_evm_network_from_csv() -> Result { } #[allow(clippy::type_complexity)] -pub(crate) fn http_provider( +pub fn http_provider( rpc_url: reqwest::Url, ) -> FillProvider< JoinFill< diff --git a/evmlib/tests/payment_vault.rs b/evmlib/tests/payment_vault.rs index b3d3ede55f..1e68e800c9 100644 --- a/evmlib/tests/payment_vault.rs +++ b/evmlib/tests/payment_vault.rs @@ -11,12 +11,15 @@ use alloy::providers::fillers::{ use alloy::providers::{Identity, ProviderBuilder, ReqwestProvider, WalletProvider}; use alloy::signers::local::{LocalSigner, PrivateKeySigner}; use alloy::transports::http::{Client, Http}; -use evmlib::common::U256; +use evmlib::common::{Amount, U256}; use evmlib::contract::network_token::NetworkToken; use evmlib::contract::payment_vault::handler::PaymentVaultHandler; use evmlib::contract::payment_vault::MAX_TRANSFERS_PER_TRANSACTION; +use evmlib::quoting_metrics::QuotingMetrics; use evmlib::testnet::{deploy_data_payments_contract, deploy_network_token_contract, start_node}; +use evmlib::utils::http_provider; use evmlib::wallet::wallet_address; +use evmlib::Network; async fn setup() -> ( AnvilInstance, @@ -112,6 +115,20 @@ async fn test_deploy() { setup().await; } +#[tokio::test] +async fn test_proxy_reachable() { + let network = Network::ArbitrumOne; + let provider = http_provider(network.rpc_url().clone()); + let payment_vault = PaymentVaultHandler::new(*network.data_payments_address(), provider); + + let amount = payment_vault + .get_quote(QuotingMetrics::default()) + .await + .unwrap(); + + assert_eq!(amount, Amount::from(1)); +} + #[tokio::test] async fn test_pay_for_quotes() { let (_anvil, network_token, mut data_payments) = setup().await; From 0becebab19bfa706648996a750a4e037c6c48dc1 Mon Sep 17 00:00:00 2001 From: Warm Beer Date: Sun, 8 Dec 2024 14:46:41 +0100 Subject: [PATCH 09/33] chore: tinkering with the quote flow --- ant-evm/src/lib.rs | 3 +- ant-node/src/put_validation.rs | 15 ++-- autonomi/src/client/mod.rs | 1 + autonomi/src/client/quote.rs | 60 ++++++++----- autonomi/src/client/utils.rs | 84 ++----------------- .../contract/payment_vault/implementation.rs | 1 + evmlib/src/contract/payment_vault/mod.rs | 16 ++++ evmlib/src/transaction.rs | 1 + 8 files changed, 73 insertions(+), 108 deletions(-) diff --git a/ant-evm/src/lib.rs b/ant-evm/src/lib.rs index d32ad1858f..30a42b34d4 100644 --- a/ant-evm/src/lib.rs +++ b/ant-evm/src/lib.rs @@ -13,6 +13,7 @@ pub use evmlib::common::Address as RewardsAddress; pub use evmlib::common::Address as EvmAddress; pub use evmlib::common::QuotePayment; pub use evmlib::common::{QuoteHash, TxHash}; +pub use evmlib::contract::payment_vault; pub use evmlib::cryptography; #[cfg(feature = "external-signer")] pub use evmlib::external_signer; @@ -28,8 +29,8 @@ mod amount; mod data_payments; mod error; -pub use evmlib::quoting_metrics::QuotingMetrics; pub use data_payments::{PaymentQuote, ProofOfPayment, QUOTE_EXPIRATION_SECS}; +pub use evmlib::quoting_metrics::QuotingMetrics; /// Types used in the public API pub use amount::{Amount, AttoTokens}; diff --git a/ant-node/src/put_validation.rs b/ant-node/src/put_validation.rs index 3fa1bb8fc2..6a38b4a37f 100644 --- a/ant-node/src/put_validation.rs +++ b/ant-node/src/put_validation.rs @@ -672,8 +672,7 @@ impl Node { )) })? .as_secs(); - // NB TODO @mick: can we check if the quote has expired with block time in evmlib? Or should nodes do it manually here? Else keep the block below - // manually check if the quote has expired + if quote_expiration_time < SystemTime::now() { warn!("Payment quote has expired for record {pretty_key}"); return Err(Error::InvalidRequest(format!( @@ -683,7 +682,8 @@ impl Node { // check if payment is valid on chain debug!("Verifying payment for record {pretty_key}"); - let reward_amount = self.evm_network() + let reward_amount = self + .evm_network() .verify_data_payment( payment.quote.hash(), payment.quote.quoting_metrics, @@ -707,7 +707,10 @@ impl Node { .set(new_value); } self.events_channel() - .broadcast(crate::NodeEvent::RewardReceived(AttoTokens::from(reward_amount), address.clone())); + .broadcast(crate::NodeEvent::RewardReceived( + AttoTokens::from(reward_amount), + address.clone(), + )); // vdash metric (if modified please notify at https://github.com/happybeing/vdash/issues): info!("Total payment of {reward_amount:?} atto tokens accepted for record {pretty_key}"); @@ -716,7 +719,9 @@ impl Node { #[cfg(feature = "loud")] { println!("🌟🌟🌟🌟🌟🌟🌟🌟🌟🌟 RECEIVED REWARD 🌟🌟🌟🌟🌟🌟🌟🌟🌟🌟"); - println!("Total payment of {reward_amount:?} atto tokens accepted for record {pretty_key}"); + println!( + "Total payment of {reward_amount:?} atto tokens accepted for record {pretty_key}" + ); println!("🌟🌟🌟🌟🌟🌟🌟🌟🌟🌟🌟🌟🌟🌟🌟🌟🌟🌟🌟🌟🌟🌟🌟🌟🌟🌟🌟🌟🌟🌟🌟🌟🌟"); } diff --git a/autonomi/src/client/mod.rs b/autonomi/src/client/mod.rs index acc62981da..7ca25bd7a2 100644 --- a/autonomi/src/client/mod.rs +++ b/autonomi/src/client/mod.rs @@ -11,6 +11,7 @@ pub mod address; pub mod payment; +pub mod quote; pub mod data; #[cfg(feature = "external-signer")] diff --git a/autonomi/src/client/quote.rs b/autonomi/src/client/quote.rs index 16cd377369..2f872363a2 100644 --- a/autonomi/src/client/quote.rs +++ b/autonomi/src/client/quote.rs @@ -6,18 +6,16 @@ // KIND, either express or implied. Please review the Licences for the specific language governing // permissions and limitations relating to use of the SAFE Network Software. -use ant_evm::{PaymentQuote, ProofOfPayment, QuoteHash, TxHash}; +use super::{data::CostError, Client}; +use crate::client::payment::Receipt; +use crate::EvmNetwork; +use ant_evm::payment_vault::get_quote; use ant_evm::{Amount, AttoTokens, QuotePayment}; +use ant_evm::{ProofOfPayment, QuoteHash, TxHash}; use ant_networking::{Network, NetworkError, SelectedQuotes}; -use ant_protocol::{ - storage::ChunkAddress, - NetworkAddress, -}; -use xor_name::XorName; +use ant_protocol::{storage::ChunkAddress, NetworkAddress}; use std::collections::{BTreeMap, HashMap}; - -use crate::client::payment::Receipt; -use super::{data::CostError, Client}; +use xor_name::XorName; pub struct QuotesToPay { pub nodes_to_pay: Vec, @@ -29,6 +27,7 @@ pub struct QuotesToPay { impl Client { pub(crate) async fn get_store_quotes( &self, + network: &EvmNetwork, content_addrs: impl Iterator, ) -> Result, CostError> { let futures: Vec<_> = content_addrs @@ -39,23 +38,38 @@ impl Client { let quotes = futures::future::try_join_all(futures).await?; let mut quotes_to_pay_per_addr = HashMap::new(); - for (content_addr, quotes) in quotes { - // NB TODO: get cost from smart contract for each quote and set this value to the median of all quotes! - let cost_per_node = Amount::from(1); + + for (content_addr, selected_quotes) in quotes { + let mut prices: Vec = vec![]; + + for quote in selected_quotes.quotes { + let price = get_quote(network, quote.1.quoting_metrics.clone()).await?; + prices.push(price); + } + + // TODO: set the cost per node by picking the median price of the prices above @anselme + let cost_per_node = Amount::from(1); // NB TODO: that's all the nodes except the invalid ones (rejected by smart contract) - let nodes_to_pay: Vec<_> = quotes.iter().map(|(_, q)| (q.hash(), q.rewards_address, cost_per_node)).collect(); - + let nodes_to_pay: Vec<_> = selected_quotes + .quotes + .iter() + .map(|(_, q)| (q.hash(), q.rewards_address, cost_per_node)) + .collect(); + // NB TODO: that's the lower half (quotes under or equal to the median price) - let nodes_to_upload_to = quotes.clone(); + let nodes_to_upload_to = quotes.clone(); let total_cost = cost_per_node * Amount::from(nodes_to_pay.len()); - quotes_to_pay_per_addr.insert(content_addr, QuotesToPay { - nodes_to_pay, - nodes_to_upload_to, - cost_per_node: AttoTokens::from_atto(cost_per_node), - total_cost: AttoTokens::from_atto(total_cost), - }); + quotes_to_pay_per_addr.insert( + content_addr, + QuotesToPay { + nodes_to_pay, + nodes_to_upload_to, + cost_per_node: AttoTokens::from_atto(cost_per_node), + total_cost: AttoTokens::from_atto(total_cost), + }, + ); } Ok(quotes_to_pay_per_addr) @@ -66,7 +80,7 @@ impl Client { async fn fetch_store_quote( network: &Network, content_addr: XorName, -) -> Result, NetworkError> { +) -> Result { network .get_store_quote_from_network( NetworkAddress::from_chunk_address(ChunkAddress::new(content_addr)), @@ -79,7 +93,7 @@ async fn fetch_store_quote( async fn fetch_store_quote_with_retries( network: &Network, content_addr: XorName, -) -> Result<(XorName, Vec), CostError> { +) -> Result<(XorName, SelectedQuotes), CostError> { let mut retries = 0; loop { diff --git a/autonomi/src/client/utils.rs b/autonomi/src/client/utils.rs index a6fc96c8ee..4b637c5f2d 100644 --- a/autonomi/src/client/utils.rs +++ b/autonomi/src/client/utils.rs @@ -7,25 +7,22 @@ // permissions and limitations relating to use of the SAFE Network Software. use crate::client::payment::Receipt; -use ant_evm::{EvmWallet, ProofOfPayment, QuotePayment}; -use ant_networking::{ - GetRecordCfg, Network, NetworkError, SelectedQuotes, PutRecordCfg, VerificationKind, -}; +use ant_evm::{EvmWallet, ProofOfPayment}; +use ant_networking::{GetRecordCfg, PutRecordCfg, VerificationKind}; use ant_protocol::{ messages::ChunkProof, - storage::{try_serialize_record, Chunk, ChunkAddress, RecordKind, RetryStrategy}, - NetworkAddress, + storage::{try_serialize_record, Chunk, RecordKind, RetryStrategy}, }; use bytes::Bytes; use futures::stream::{FuturesUnordered, StreamExt}; use libp2p::kad::{Quorum, Record}; use rand::{thread_rng, Rng}; use self_encryption::{decrypt_full_set, DataMap, EncryptedChunk}; -use std::{collections::HashMap, future::Future, num::NonZero}; +use std::{future::Future, num::NonZero}; use xor_name::XorName; use super::{ - data::{CostError, GetError, PayError, PutError, CHUNK_DOWNLOAD_BATCH_SIZE}, + data::{GetError, PayError, PutError, CHUNK_DOWNLOAD_BATCH_SIZE}, Client, }; use crate::self_encryption::DataMapLevel; @@ -196,77 +193,6 @@ impl Client { Ok((proofs, skipped_chunks)) } - - pub(crate) async fn get_store_quotes( - &self, - content_addrs: impl Iterator, - ) -> Result, CostError> { - let futures: Vec<_> = content_addrs - .into_iter() - .map(|content_addr| fetch_store_quote_with_retries(&self.network, content_addr)) - .collect(); - - let quotes = futures::future::try_join_all(futures).await?; - - Ok(quotes.into_iter().collect::>()) - } -} - -/// Fetch a store quote for a content address with a retry strategy. -async fn fetch_store_quote_with_retries( - network: &Network, - content_addr: XorName, -) -> Result<(XorName, SelectedQuotes), CostError> { - let mut retries = 0; - - loop { - match fetch_store_quote(network, content_addr).await { - Ok(quote) => { - break Ok((content_addr, quote)); - } - Err(err) if retries < 2 => { - retries += 1; - error!("Error while fetching store quote: {err:?}, retry #{retries}"); - } - Err(err) => { - error!( - "Error while fetching store quote: {err:?}, stopping after {retries} retries" - ); - break Err(CostError::CouldNotGetStoreQuote(content_addr)); - } - } - } -} - -/// Fetch a store quote for a content address. -async fn fetch_store_quote( - network: &Network, - content_addr: XorName, -) -> Result { - network - .get_store_costs_from_network( - NetworkAddress::from_chunk_address(ChunkAddress::new(content_addr)), - vec![], - ) - .await -} - -/// Form to be executed payments and already executed payments from a cost map. -pub(crate) fn extract_quote_payments( - cost_map: &HashMap, -) -> (Vec, Vec) { - let mut to_be_paid = vec![]; - let mut already_paid = vec![]; - - for (chunk_address, (_, _, quote)) in cost_map.iter() { - if quote.cost.is_zero() { - already_paid.push(*chunk_address); - } else { - to_be_paid.push((quote.hash(), quote.rewards_address, quote.cost.as_atto())); - } - } - - (to_be_paid, already_paid) } pub(crate) async fn process_tasks_with_max_concurrency(tasks: I, batch_size: usize) -> Vec diff --git a/evmlib/src/contract/payment_vault/implementation.rs b/evmlib/src/contract/payment_vault/implementation.rs index 78ae83117c..4cbc469248 100644 --- a/evmlib/src/contract/payment_vault/implementation.rs +++ b/evmlib/src/contract/payment_vault/implementation.rs @@ -4,6 +4,7 @@ use alloy::network::{Network, ReceiptResponse, TransactionBuilder}; use alloy::providers::Provider; use alloy::transports::Transport; +// Payment Vault contract byte code const BYTE_CODE: &str = "0x60a060405230608052348015610013575f5ffd5b5061001c610021565b6100d3565b7ff0c57e16840df040f15088dc2f81fe391c3923bec73e23a9662efc9c229c6a00805468010000000000000000900460ff16156100715760405163f92ee8a960e01b815260040160405180910390fd5b80546001600160401b03908116146100d05780546001600160401b0319166001600160401b0390811782556040519081527fc7f505b2f371ae2175ee4913f4499e1f2633a7b5936321eed1cdaeb6115181d29060200160405180910390a15b50565b608051610ef76100f95f395f818161064d0152818161067601526107ba0152610ef75ff3fe6080604052600436106100bf575f3560e01c8063715018a61161007c578063ad3cb1cc11610057578063ad3cb1cc14610253578063b6c2141b14610290578063cd6dc687146102af578063f2fde38b146102ce575f5ffd5b8063715018a6146101d45780638da5cb5b146101e8578063a69bf4a314610224575f5ffd5b80630716326d146100c35780633c150bf214610132578063474740b1146101605780634ec42e8e146101745780634f1ef286146101ab57806352d1902d146101c0575b5f5ffd5b3480156100ce575f5ffd5b506101086100dd366004610bc4565b600260208190525f91825260409091208054600182015491909201546001600160a01b039092169183565b604080516001600160a01b0390941684526020840192909252908201526060015b60405180910390f35b34801561013d575f5ffd5b5061015261014c366004610bf1565b50600190565b604051908152602001610129565b34801561016b575f5ffd5b506101525f5481565b34801561017f575f5ffd5b50600154610193906001600160a01b031681565b6040516001600160a01b039091168152602001610129565b6101be6101b9366004610c33565b6102ed565b005b3480156101cb575f5ffd5b5061015261030c565b3480156101df575f5ffd5b506101be610327565b3480156101f3575f5ffd5b507f9016d09d72d40fdae2fd8ceac6b6234c7706214fd39c1cd1e609a0528c199300546001600160a01b0316610193565b34801561022f575f5ffd5b5061024361023e366004610cf9565b61033a565b6040519015158152602001610129565b34801561025e575f5ffd5b50610283604051806040016040528060058152602001640352e302e360dc1b81525081565b6040516101299190610d37565b34801561029b575f5ffd5b506101be6102aa366004610d6c565b6103b6565b3480156102ba575f5ffd5b506101be6102c9366004610ddd565b6104a3565b3480156102d9575f5ffd5b506101be6102e8366004610e07565b610600565b6102f5610642565b6102fe826106e6565b61030882826106ee565b5050565b5f6103156107af565b505f516020610ea25f395f51905f5290565b61032f6107f8565b6103385f610853565b565b6040808201355f90815260026020818152838320845160608101865281546001600160a01b031681526001820154818401819052919093015494830194909452919290918401351480156103ae57506103966020840184610e07565b6001600160a01b0316815f01516001600160a01b0316145b949350505050565b5f5481908111156103da57604051630d67f41160e21b815260040160405180910390fd5b5f5b8181101561049d57368484838181106103f7576103f7610e22565b60600291909101915061042b9050336104136020840184610e07565b6001546001600160a01b0316919060208501356108c3565b6040808201355f90815260026020522081906104478282610e36565b505060408101356020820180359061045f9084610e07565b6001600160a01b03167ff998960b1c6f0e0e89b7bbe6b6fbf3e03e6f08eee5b8430877d8adb8e149d58060405160405180910390a4506001016103dc565b50505050565b7ff0c57e16840df040f15088dc2f81fe391c3923bec73e23a9662efc9c229c6a008054600160401b810460ff16159067ffffffffffffffff165f811580156104e85750825b90505f8267ffffffffffffffff1660011480156105045750303b155b905081158015610512575080155b156105305760405163f92ee8a960e01b815260040160405180910390fd5b845467ffffffffffffffff19166001178555831561055a57845460ff60401b1916600160401b1785555b6001600160a01b03871661058157604051632d06160b60e21b815260040160405180910390fd5b600180546001600160a01b0319166001600160a01b0389161790555f8690556105a93361091d565b6105b161092e565b83156105f757845460ff60401b19168555604051600181527fc7f505b2f371ae2175ee4913f4499e1f2633a7b5936321eed1cdaeb6115181d29060200160405180910390a15b50505050505050565b6106086107f8565b6001600160a01b03811661063657604051631e4fbdf760e01b81525f60048201526024015b60405180910390fd5b61063f81610853565b50565b306001600160a01b037f00000000000000000000000000000000000000000000000000000000000000001614806106c857507f00000000000000000000000000000000000000000000000000000000000000006001600160a01b03166106bc5f516020610ea25f395f51905f52546001600160a01b031690565b6001600160a01b031614155b156103385760405163703e46dd60e11b815260040160405180910390fd5b61063f6107f8565b816001600160a01b03166352d1902d6040518163ffffffff1660e01b8152600401602060405180830381865afa925050508015610748575060408051601f3d908101601f1916820190925261074591810190610e74565b60015b61077057604051634c9c8ce360e01b81526001600160a01b038316600482015260240161062d565b5f516020610ea25f395f51905f5281146107a057604051632a87526960e21b81526004810182905260240161062d565b6107aa8383610936565b505050565b306001600160a01b037f000000000000000000000000000000000000000000000000000000000000000016146103385760405163703e46dd60e11b815260040160405180910390fd5b3361082a7f9016d09d72d40fdae2fd8ceac6b6234c7706214fd39c1cd1e609a0528c199300546001600160a01b031690565b6001600160a01b0316146103385760405163118cdaa760e01b815233600482015260240161062d565b7f9016d09d72d40fdae2fd8ceac6b6234c7706214fd39c1cd1e609a0528c19930080546001600160a01b031981166001600160a01b03848116918217845560405192169182907f8be0079c531659141344cd1fd0a4f28419497f9722a3daafe3b4186f6b6457e0905f90a3505050565b604080516001600160a01b0385811660248301528416604482015260648082018490528251808303909101815260849091019091526020810180516001600160e01b03166323b872dd60e01b17905261049d90859061098b565b6109256109f7565b61063f81610a40565b6103386109f7565b61093f82610a48565b6040516001600160a01b038316907fbc7cd75a20ee27fd9adebab32041f755214dbc6bffa90cc0225b39da2e5c2d3b905f90a2805115610983576107aa8282610aab565b610308610b1d565b5f5f60205f8451602086015f885af1806109aa576040513d5f823e3d81fd5b50505f513d915081156109c15780600114156109ce565b6001600160a01b0384163b155b1561049d57604051635274afe760e01b81526001600160a01b038516600482015260240161062d565b7ff0c57e16840df040f15088dc2f81fe391c3923bec73e23a9662efc9c229c6a0054600160401b900460ff1661033857604051631afcd79f60e31b815260040160405180910390fd5b6106086109f7565b806001600160a01b03163b5f03610a7d57604051634c9c8ce360e01b81526001600160a01b038216600482015260240161062d565b5f516020610ea25f395f51905f5280546001600160a01b0319166001600160a01b0392909216919091179055565b60605f5f846001600160a01b031684604051610ac79190610e8b565b5f60405180830381855af49150503d805f8114610aff576040519150601f19603f3d011682016040523d82523d5f602084013e610b04565b606091505b5091509150610b14858383610b3c565b95945050505050565b34156103385760405163b398979f60e01b815260040160405180910390fd5b606082610b5157610b4c82610b9b565b610b94565b8151158015610b6857506001600160a01b0384163b155b15610b9157604051639996b31560e01b81526001600160a01b038516600482015260240161062d565b50805b9392505050565b805115610bab5780518082602001fd5b60405163d6bda27560e01b815260040160405180910390fd5b5f60208284031215610bd4575f5ffd5b5035919050565b5f60c08284031215610beb575f5ffd5b50919050565b5f60c08284031215610c01575f5ffd5b610b948383610bdb565b6001600160a01b038116811461063f575f5ffd5b634e487b7160e01b5f52604160045260245ffd5b5f5f60408385031215610c44575f5ffd5b8235610c4f81610c0b565b9150602083013567ffffffffffffffff811115610c6a575f5ffd5b8301601f81018513610c7a575f5ffd5b803567ffffffffffffffff811115610c9457610c94610c1f565b604051601f8201601f19908116603f0116810167ffffffffffffffff81118282101715610cc357610cc3610c1f565b604052818152828201602001871015610cda575f5ffd5b816020840160208301375f602083830101528093505050509250929050565b5f5f828403610120811215610d0c575f5ffd5b610d168585610bdb565b9250606060bf1982011215610d29575f5ffd5b5060c0830190509250929050565b602081525f82518060208401528060208501604085015e5f604082850101526040601f19601f83011684010191505092915050565b5f5f60208385031215610d7d575f5ffd5b823567ffffffffffffffff811115610d93575f5ffd5b8301601f81018513610da3575f5ffd5b803567ffffffffffffffff811115610db9575f5ffd5b856020606083028401011115610dcd575f5ffd5b6020919091019590945092505050565b5f5f60408385031215610dee575f5ffd5b8235610df981610c0b565b946020939093013593505050565b5f60208284031215610e17575f5ffd5b8135610b9481610c0b565b634e487b7160e01b5f52603260045260245ffd5b8135610e4181610c0b565b81546001600160a01b0319166001600160a01b039190911617815560208201356001820155604090910135600290910155565b5f60208284031215610e84575f5ffd5b5051919050565b5f82518060208501845e5f92019182525091905056fe360894a13ba1a3210667c828492db98dca3e2076cc3735a920a3ca505d382bbca26469706673582212203894ca52be6e6323aa3d296efd566c7f21d1723d4c66c56aed8a5f75a96b579d64736f6c634300081c0033"; pub async fn deploy(provider: &P) -> Address diff --git a/evmlib/src/contract/payment_vault/mod.rs b/evmlib/src/contract/payment_vault/mod.rs index 5cbc6f7718..9581eb183e 100644 --- a/evmlib/src/contract/payment_vault/mod.rs +++ b/evmlib/src/contract/payment_vault/mod.rs @@ -1,6 +1,22 @@ +use crate::common::Amount; +use crate::contract::payment_vault::handler::PaymentVaultHandler; +use crate::quoting_metrics::QuotingMetrics; +use crate::utils::http_provider; +use crate::Network; + pub mod error; pub mod handler; pub mod implementation; pub mod interface; pub const MAX_TRANSFERS_PER_TRANSACTION: usize = 256; + +/// Helper function to return a quote for the given quoting metrics +pub async fn get_quote( + network: &Network, + quoting_metrics: QuotingMetrics, +) -> Result { + let provider = http_provider(network.rpc_url().clone()); + let payment_vault = PaymentVaultHandler::new(*network.data_payments_address(), provider); + payment_vault.get_quote(quoting_metrics).await +} diff --git a/evmlib/src/transaction.rs b/evmlib/src/transaction.rs index 993af3e074..2a51faaf2d 100644 --- a/evmlib/src/transaction.rs +++ b/evmlib/src/transaction.rs @@ -116,6 +116,7 @@ pub async fn verify_data_payment( let is_paid = payment_vault .verify_payment(quoting_metrics, (quote_hash, reward_addr, Amount::ZERO)) .await?; + let amount_paid = Amount::ZERO; // NB TODO @mick we need to get the amount paid from the contract if is_paid { From b3068021aba92482df7f430ba5e71a30204a4bc2 Mon Sep 17 00:00:00 2001 From: grumbach Date: Mon, 9 Dec 2024 17:12:36 +0900 Subject: [PATCH 10/33] feat: wip new quoting payment integration --- ant-cli/src/commands/register.rs | 3 +- ant-evm/src/data_payments.rs | 60 +++++++-- ant-networking/src/lib.rs | 15 +-- ant-node/src/put_validation.rs | 30 ++--- autonomi/src/client/payment.rs | 6 +- autonomi/src/client/quote.rs | 147 +++++++++++------------ autonomi/src/client/registers.rs | 6 +- autonomi/src/client/utils.rs | 15 ++- autonomi/src/lib.rs | 1 - autonomi/src/utils.rs | 39 ------ evmlib/src/contract/payment_vault/mod.rs | 2 +- evmlib/src/lib.rs | 6 +- evmlib/src/transaction.rs | 18 +-- evmlib/src/wallet.rs | 27 ++--- evmlib/tests/wallet.rs | 4 +- 15 files changed, 174 insertions(+), 205 deletions(-) delete mode 100644 autonomi/src/utils.rs diff --git a/ant-cli/src/commands/register.rs b/ant-cli/src/commands/register.rs index 17c30b2559..20d7f6ea20 100644 --- a/ant-cli/src/commands/register.rs +++ b/ant-cli/src/commands/register.rs @@ -41,9 +41,10 @@ pub async fn cost(name: &str, peers: Vec) -> Result<()> { let register_key = crate::keys::get_register_signing_key() .wrap_err("The register key is required to perform this action")?; let client = crate::actions::connect_to_network(peers).await?; + let wallet = load_wallet()?; let cost = client - .register_cost(name.to_string(), register_key) + .register_cost(&wallet.network(), name.to_string(), register_key) .await .wrap_err("Failed to get cost for register")?; info!("Estimated cost to create a register with name {name}: {cost}"); diff --git a/ant-evm/src/data_payments.rs b/ant-evm/src/data_payments.rs index f091d65290..47476893aa 100644 --- a/ant-evm/src/data_payments.rs +++ b/ant-evm/src/data_payments.rs @@ -8,7 +8,7 @@ use crate::EvmError; use evmlib::{ - common::{Address as RewardsAddress, QuoteHash, TxHash}, quoting_metrics::QuotingMetrics, utils::dummy_address + common::{Address as RewardsAddress, QuoteHash}, quoting_metrics::QuotingMetrics, utils::dummy_address }; use libp2p::{identity::PublicKey, PeerId}; use serde::{Deserialize, Serialize}; @@ -24,19 +24,61 @@ pub const QUOTE_EXPIRATION_SECS: u64 = 3600; /// The margin allowed for live_time const LIVE_TIME_MARGIN: u64 = 10; +#[derive(Debug, Clone, PartialEq, Eq, Ord, PartialOrd, Serialize, Deserialize)] +pub struct EncodedPeerId(Vec); + +impl EncodedPeerId { + pub fn to_peer_id(&self) -> Result { + match PublicKey::try_decode_protobuf(&self.0) { + Ok(pub_key) => Ok(PeerId::from_public_key(&pub_key)), + Err(e) => Err(e) + } + } +} + /// The proof of payment for a data payment #[derive(Debug, Clone, PartialEq, Eq, Ord, PartialOrd, Serialize, Deserialize)] pub struct ProofOfPayment { - /// The Quote we're paying for - pub quote: PaymentQuote, - /// The transaction hash - pub tx_hash: TxHash, + peer_quotes: Vec<(EncodedPeerId, PaymentQuote)> } impl ProofOfPayment { - pub fn to_peer_id_payee(&self) -> Option { - let pub_key = PublicKey::try_decode_protobuf(&self.quote.pub_key).ok()?; - Some(PeerId::from_public_key(&pub_key)) + /// returns a short digest of the proof of payment to use for verification + pub fn digest(&self) -> Vec<(QuoteHash, QuotingMetrics, RewardsAddress)> { + self.peer_quotes.clone().into_iter().map(|(_, quote)| (quote.hash(), quote.quoting_metrics, quote.rewards_address)).collect() + } + + /// returns the list of payees + pub fn payees(&self) -> Vec { + self.peer_quotes.iter().filter_map(|(peer_id, _)| peer_id.to_peer_id().ok()).collect() + } + + /// has the quote expired + pub fn has_expired(&self) -> bool { + self.peer_quotes.iter().any(|(_, quote)| quote.has_expired()) + } + + /// verifies the proof of payment is valid for the given peer id + pub fn verify_for(&self, peer_id: PeerId) -> bool { + // make sure I am in the list of payees + if !self.payees().contains(&peer_id) { + return false; + } + + // verify all signatures + for (encoded_peer_id, quote) in self.peer_quotes.iter() { + let peer_id = match encoded_peer_id.to_peer_id() { + Ok(peer_id) => peer_id, + Err(e) => { + warn!("Invalid encoded peer id: {e}"); + return false; + }, + }; + if !quote.check_is_signed_by_claimed_peer(peer_id) { + return false; + } + } + true } } @@ -148,7 +190,7 @@ impl PaymentQuote { true } - /// Returns true) if the quote has not yet expired + /// Returns true if the quote has expired pub fn has_expired(&self) -> bool { let now = SystemTime::now(); diff --git a/ant-networking/src/lib.rs b/ant-networking/src/lib.rs index 1e7a46aed6..5973cb02c2 100644 --- a/ant-networking/src/lib.rs +++ b/ant-networking/src/lib.rs @@ -83,11 +83,6 @@ use { std::collections::HashSet, }; -/// Selected quotes to pay for a data address -pub struct SelectedQuotes { - pub quotes: Vec<(PeerId, PaymentQuote)>, -} - /// Majority of a given group (i.e. > 1/2). #[inline] pub const fn close_group_majority() -> usize { @@ -384,7 +379,7 @@ impl Network { &self, record_address: NetworkAddress, ignore_peers: Vec, - ) -> Result { + ) -> Result> { // The requirement of having at least CLOSE_GROUP_SIZE // close nodes will be checked internally automatically. let mut close_nodes = self @@ -408,9 +403,9 @@ impl Network { .send_and_get_responses(&close_nodes, &request, true) .await; - // consider data to be already paid for if 1/3 of the close nodes already have it + // consider data to be already paid for if 1/2 of the close nodes already have it let mut peer_already_have_it = 0; - let enough_peers_already_have_it = close_nodes.len() / 3; + let enough_peers_already_have_it = close_nodes.len() / 2; // loop over responses let mut all_quotes = vec![]; @@ -448,7 +443,7 @@ impl Network { info!("Address {record_address:?} was already paid for according to {peer_address:?} ({peer_already_have_it}/{enough_peers_already_have_it})"); if peer_already_have_it >= enough_peers_already_have_it { info!("Address {record_address:?} was already paid for according to {peer_already_have_it} peers, ending quote request"); - return Ok(SelectedQuotes { quotes: vec![] }); + return Ok(vec![]); } } Err(err) => { @@ -460,7 +455,7 @@ impl Network { } } - Ok(SelectedQuotes { quotes: quotes_to_pay }) + Ok(quotes_to_pay) } /// Get register from network. diff --git a/ant-node/src/put_validation.rs b/ant-node/src/put_validation.rs index 6a38b4a37f..95b37dcc5e 100644 --- a/ant-node/src/put_validation.rs +++ b/ant-node/src/put_validation.rs @@ -653,27 +653,16 @@ impl Node { // check if the quote is valid let self_peer_id = self.network().peer_id(); - if !payment.quote.check_is_signed_by_claimed_peer(self_peer_id) { - warn!("Payment quote signature is not valid for record {pretty_key}"); + if !payment.verify_for(self_peer_id) { + warn!("Payment is not valid for record {pretty_key}"); return Err(Error::InvalidRequest(format!( - "Payment quote signature is not valid for record {pretty_key}" + "Payment is not valid for record {pretty_key}" ))); } - debug!("Payment quote signature is valid for record {pretty_key}"); - - // verify quote timestamp - let quote_timestamp = payment.quote.timestamp; - let quote_expiration_time = quote_timestamp + Duration::from_secs(QUOTE_EXPIRATION_SECS); - let _quote_expiration_time_in_secs = quote_expiration_time - .duration_since(UNIX_EPOCH) - .map_err(|e| { - Error::InvalidRequest(format!( - "Payment quote timestamp is invalid for record {pretty_key}: {e}" - )) - })? - .as_secs(); + debug!("Payment is valid for record {pretty_key}"); - if quote_expiration_time < SystemTime::now() { + // verify quote expiration + if payment.has_expired() { warn!("Payment quote has expired for record {pretty_key}"); return Err(Error::InvalidRequest(format!( "Payment quote has expired for record {pretty_key}" @@ -681,14 +670,11 @@ impl Node { } // check if payment is valid on chain + let payments_to_verify = payment.digest(); debug!("Verifying payment for record {pretty_key}"); let reward_amount = self .evm_network() - .verify_data_payment( - payment.quote.hash(), - payment.quote.quoting_metrics, - *self.reward_address(), - ) + .verify_data_payment(payments_to_verify) .await .map_err(|e| Error::EvmNetwork(format!("Failed to verify chunk payment: {e}")))?; debug!("Payment of {reward_amount:?} is valid for record {pretty_key}"); diff --git a/autonomi/src/client/payment.rs b/autonomi/src/client/payment.rs index f9096f15cf..97416e3c09 100644 --- a/autonomi/src/client/payment.rs +++ b/autonomi/src/client/payment.rs @@ -1,11 +1,11 @@ use crate::client::data::PayError; use crate::Client; -use ant_evm::{EvmWallet, ProofOfPayment}; +use ant_evm::{AttoTokens, EvmWallet, ProofOfPayment}; use std::collections::HashMap; use xor_name::XorName; -/// Contains the proof of payment for XOR addresses. -pub type Receipt = HashMap; +/// Contains the proof of payments for each XOR address and the amount paid +pub type Receipt = HashMap>; /// Payment options for data payments. #[derive(Clone)] diff --git a/autonomi/src/client/quote.rs b/autonomi/src/client/quote.rs index 2f872363a2..1e5e6b80be 100644 --- a/autonomi/src/client/quote.rs +++ b/autonomi/src/client/quote.rs @@ -7,72 +7,95 @@ // permissions and limitations relating to use of the SAFE Network Software. use super::{data::CostError, Client}; -use crate::client::payment::Receipt; use crate::EvmNetwork; -use ant_evm::payment_vault::get_quote; -use ant_evm::{Amount, AttoTokens, QuotePayment}; -use ant_evm::{ProofOfPayment, QuoteHash, TxHash}; -use ant_networking::{Network, NetworkError, SelectedQuotes}; +use ant_evm::payment_vault::get_market_price; +use ant_evm::{Amount, PaymentQuote, QuotePayment}; +use ant_networking::{Network, NetworkError}; use ant_protocol::{storage::ChunkAddress, NetworkAddress}; -use std::collections::{BTreeMap, HashMap}; +use libp2p::PeerId; +use std::collections::HashMap; use xor_name::XorName; -pub struct QuotesToPay { - pub nodes_to_pay: Vec, - pub nodes_to_upload_to: Vec, - pub cost_per_node: AttoTokens, - pub total_cost: AttoTokens, +/// A quote for a single address +pub struct QuoteForAddress(Vec<(PeerId, PaymentQuote, Amount)>); + +impl QuoteForAddress { + pub fn price(&self) -> Amount { + self.0.iter().map(|(_, _, price)| price).sum() + } +} + +/// A quote for many addresses +pub struct StoreQuote(HashMap); + +impl StoreQuote { + pub fn price(&self) -> Amount { + self.0.iter().map(|(_, quote)| quote.price()).sum() + } + + pub fn len(&self) -> usize { + self.0.len() + } + + pub fn payments(&self) -> Vec { + let mut quote_payments = vec![]; + for (_address, quote) in self.0.iter() { + for (_peer, quote, price) in quote.0.iter() { + quote_payments.push((quote.hash(), quote.rewards_address, *price)); + } + } + quote_payments + } } impl Client { pub(crate) async fn get_store_quotes( &self, - network: &EvmNetwork, + evm_network: &EvmNetwork, content_addrs: impl Iterator, - ) -> Result, CostError> { + ) -> Result { + // get all quotes from nodes let futures: Vec<_> = content_addrs .into_iter() .map(|content_addr| fetch_store_quote_with_retries(&self.network, content_addr)) .collect(); + let raw_quotes_per_addr = futures::future::try_join_all(futures).await?; - let quotes = futures::future::try_join_all(futures).await?; - + // choose the quotes to pay for each address let mut quotes_to_pay_per_addr = HashMap::new(); - - for (content_addr, selected_quotes) in quotes { - let mut prices: Vec = vec![]; - - for quote in selected_quotes.quotes { - let price = get_quote(network, quote.1.quoting_metrics.clone()).await?; - prices.push(price); + for (content_addr, raw_quotes) in raw_quotes_per_addr { + // ask smart contract for the market price + let mut prices = vec![]; + for (peer, quote) in raw_quotes { + // NB TODO @mick we need to batch this smart contract call + let price = get_market_price(evm_network, quote.quoting_metrics.clone()).await?; + prices.push((peer, quote, price)); } - // TODO: set the cost per node by picking the median price of the prices above @anselme - let cost_per_node = Amount::from(1); - - // NB TODO: that's all the nodes except the invalid ones (rejected by smart contract) - let nodes_to_pay: Vec<_> = selected_quotes - .quotes - .iter() - .map(|(_, q)| (q.hash(), q.rewards_address, cost_per_node)) - .collect(); - - // NB TODO: that's the lower half (quotes under or equal to the median price) - let nodes_to_upload_to = quotes.clone(); - - let total_cost = cost_per_node * Amount::from(nodes_to_pay.len()); - quotes_to_pay_per_addr.insert( - content_addr, - QuotesToPay { - nodes_to_pay, - nodes_to_upload_to, - cost_per_node: AttoTokens::from_atto(cost_per_node), - total_cost: AttoTokens::from_atto(total_cost), - }, - ); + // sort by price + prices.sort_by(|(_, _, price_a), (_, _, price_b)| price_a.cmp(price_b)); + + // we need at least 5 valid quotes to pay for the data + const MINIMUM_QUOTES_TO_PAY: usize = 5; + match &prices[..] { + [first, second, third, fourth, fifth, ..] => { + let (p1, q1, _) = first; + let (p2, q2, _) = second; + + // don't pay for the cheapest 2 quotes but include them + let first = (*p1, q1.clone(), Amount::ZERO); + let second = (*p2, q2.clone(), Amount::ZERO); + + // pay for the rest + quotes_to_pay_per_addr.insert(content_addr, QuoteForAddress(vec![first, second, third.clone(), fourth.clone(), fifth.clone()])); + } + _ => { + return Err(CostError::NotEnoughNodeQuotes(content_addr, prices.len(), MINIMUM_QUOTES_TO_PAY)); + } + } } - Ok(quotes_to_pay_per_addr) + Ok(StoreQuote(quotes_to_pay_per_addr)) } } @@ -80,7 +103,7 @@ impl Client { async fn fetch_store_quote( network: &Network, content_addr: XorName, -) -> Result { +) -> Result, NetworkError> { network .get_store_quote_from_network( NetworkAddress::from_chunk_address(ChunkAddress::new(content_addr)), @@ -93,7 +116,7 @@ async fn fetch_store_quote( async fn fetch_store_quote_with_retries( network: &Network, content_addr: XorName, -) -> Result<(XorName, SelectedQuotes), CostError> { +) -> Result<(XorName, Vec<(PeerId, PaymentQuote)>), CostError> { let mut retries = 0; loop { @@ -114,31 +137,3 @@ async fn fetch_store_quote_with_retries( } } } - -pub fn receipt_from_quotes_and_payments( - quotes_map: HashMap, - payments: &BTreeMap, -) -> Receipt { - let quotes = cost_map_to_quotes(quotes_map); - receipt_from_quotes_and_payments("es, payments) -} - -pub fn receipt_from_quotes_and_payments( - quotes: &HashMap, - payments: &BTreeMap, -) -> Receipt { - quotes - .iter() - .filter_map(|(xor_name, quote)| { - payments.get("e.hash()).map(|tx_hash| { - ( - *xor_name, - ProofOfPayment { - quote: quote.clone(), - tx_hash: *tx_hash, - }, - ) - }) - }) - .collect() -} diff --git a/autonomi/src/client/registers.rs b/autonomi/src/client/registers.rs index 0d19fb27fe..9f30e8b565 100644 --- a/autonomi/src/client/registers.rs +++ b/autonomi/src/client/registers.rs @@ -11,6 +11,7 @@ use crate::client::Client; use crate::client::ClientEvent; use crate::client::UploadSummary; +use ant_evm::EvmNetwork; pub use ant_registers::{Permissions as RegisterPermissions, RegisterAddress}; pub use bls::SecretKey as RegisterSecretKey; @@ -234,6 +235,7 @@ impl Client { /// Get the cost to create a register pub async fn register_cost( &self, + evm_network: &EvmNetwork, name: String, owner: RegisterSecretKey, ) -> Result { @@ -247,7 +249,7 @@ impl Client { // get cost to store register // NB TODO: register should be priced differently from other data - let cost_map = self.get_store_quotes(std::iter::once(reg_xor)).await?; + let cost_map = self.get_store_quotes(evm_network, std::iter::once(reg_xor)).await?; let total_cost = AttoTokens::from_atto( cost_map .values() @@ -302,7 +304,7 @@ impl Client { let reg_xor = address.xorname(); debug!("Paying for register at address: {address}"); - let (payment_proofs, _skipped) = self + let payment_proofs = self .pay(std::iter::once(reg_xor), wallet) .await .inspect_err(|err| { diff --git a/autonomi/src/client/utils.rs b/autonomi/src/client/utils.rs index 4b637c5f2d..04e06fc4dc 100644 --- a/autonomi/src/client/utils.rs +++ b/autonomi/src/client/utils.rs @@ -7,7 +7,7 @@ // permissions and limitations relating to use of the SAFE Network Software. use crate::client::payment::Receipt; -use ant_evm::{EvmWallet, ProofOfPayment}; +use ant_evm::{EvmNetwork, EvmWallet, ProofOfPayment}; use ant_networking::{GetRecordCfg, PutRecordCfg, VerificationKind}; use ant_protocol::{ messages::ChunkProof, @@ -161,10 +161,8 @@ impl Client { &self, content_addrs: impl Iterator, wallet: &EvmWallet, - ) -> Result<(Receipt, Vec), PayError> { - let cost_map = self.get_store_quotes(content_addrs).await?; - - let (quote_payments, skipped_chunks) = extract_quote_payments(&cost_map); + ) -> Result { + let quotes = self.get_store_quotes(wallet.network(), content_addrs).await?; // Make sure nobody else can use the wallet while we are paying debug!("Waiting for wallet lock"); @@ -175,7 +173,7 @@ impl Client { // TODO: retry when it fails? // Execute chunk payments let payments = wallet - .pay_for_quotes(quote_payments) + .pay_for_quotes(quotes.payments()) .await .map_err(|err| PayError::from(err.0))?; @@ -185,13 +183,14 @@ impl Client { let proofs = receipt_from_cost_map_and_payments(cost_map, &payments); + let skipped_chunks = content_addrs.count() - quotes.len(); trace!( "Chunk payments of {} chunks completed. {} chunks were free / already paid for", proofs.len(), - skipped_chunks.len() + skipped_chunks ); - Ok((proofs, skipped_chunks)) + Ok(receipt) } } diff --git a/autonomi/src/lib.rs b/autonomi/src/lib.rs index 2564ee3b2e..7dd7aeb1a8 100644 --- a/autonomi/src/lib.rs +++ b/autonomi/src/lib.rs @@ -82,4 +82,3 @@ pub use client::{files::archive::PrivateArchive, Client}; #[cfg(feature = "extension-module")] mod python; -mod utils; diff --git a/autonomi/src/utils.rs b/autonomi/src/utils.rs deleted file mode 100644 index 85d8f3f62c..0000000000 --- a/autonomi/src/utils.rs +++ /dev/null @@ -1,39 +0,0 @@ -use crate::client::payment::Receipt; -use ant_evm::{PaymentQuote, ProofOfPayment, QuoteHash, TxHash}; -use ant_networking::SelectedQuotes; -use std::collections::{BTreeMap, HashMap}; -use xor_name::XorName; - -pub fn cost_map_to_quotes( - cost_map: HashMap, -) -> HashMap { - cost_map.into_iter().map(|(k, (_, q))| (k, q)).collect() -} - -pub fn receipt_from_cost_map_and_payments( - cost_map: HashMap, - payments: &BTreeMap, -) -> Receipt { - let quotes = cost_map_to_quotes(cost_map); - receipt_from_quotes_and_payments("es, payments) -} - -pub fn receipt_from_quotes_and_payments( - quotes: &HashMap, - payments: &BTreeMap, -) -> Receipt { - quotes - .iter() - .filter_map(|(xor_name, quote)| { - payments.get("e.hash()).map(|tx_hash| { - ( - *xor_name, - ProofOfPayment { - quote: quote.clone(), - tx_hash: *tx_hash, - }, - ) - }) - }) - .collect() -} diff --git a/evmlib/src/contract/payment_vault/mod.rs b/evmlib/src/contract/payment_vault/mod.rs index 9581eb183e..d6afbbd91a 100644 --- a/evmlib/src/contract/payment_vault/mod.rs +++ b/evmlib/src/contract/payment_vault/mod.rs @@ -12,7 +12,7 @@ pub mod interface; pub const MAX_TRANSFERS_PER_TRANSACTION: usize = 256; /// Helper function to return a quote for the given quoting metrics -pub async fn get_quote( +pub async fn get_market_price( network: &Network, quoting_metrics: QuotingMetrics, ) -> Result { diff --git a/evmlib/src/lib.rs b/evmlib/src/lib.rs index a37ae2a16e..1bc363925a 100644 --- a/evmlib/src/lib.rs +++ b/evmlib/src/lib.rs @@ -138,10 +138,8 @@ impl Network { pub async fn verify_data_payment( &self, - quote_hash: QuoteHash, - quoting_metrics: QuotingMetrics, - reward_addr: Address, + payment: Vec<(QuoteHash, QuotingMetrics, Address)> ) -> Result { - verify_data_payment(self, quote_hash, reward_addr, quoting_metrics).await + verify_data_payment(self, payment).await } } diff --git a/evmlib/src/transaction.rs b/evmlib/src/transaction.rs index 2a51faaf2d..4a54f8391e 100644 --- a/evmlib/src/transaction.rs +++ b/evmlib/src/transaction.rs @@ -106,16 +106,18 @@ async fn get_data_payment_event( /// Verify if a data payment is confirmed. pub async fn verify_data_payment( network: &Network, - quote_hash: QuoteHash, - reward_addr: Address, - quoting_metrics: QuotingMetrics, + payment: Vec<(QuoteHash, QuotingMetrics, Address)> ) -> Result { let provider = http_provider(network.rpc_url().clone()); let payment_vault = PaymentVaultHandler::new(*network.data_payments_address(), provider); - let is_paid = payment_vault - .verify_payment(quoting_metrics, (quote_hash, reward_addr, Amount::ZERO)) - .await?; + // NB TODO @mick remove tmp loop and support verification of the whole payment at once + let mut is_paid = true; + for (quote_hash, quoting_metrics, reward_addr) in payment { + is_paid = payment_vault + .verify_payment(quoting_metrics, (quote_hash, reward_addr, Amount::ZERO)) + .await?; + } let amount_paid = Amount::ZERO; // NB TODO @mick we need to get the amount paid from the contract @@ -144,9 +146,7 @@ mod tests { let result = verify_data_payment( &network, - quote_hash, - reward_address, - QuotingMetrics::default(), + vec![(quote_hash, QuotingMetrics::default(), reward_address)] ) .await; diff --git a/evmlib/src/wallet.rs b/evmlib/src/wallet.rs index 53cfe2673f..327c0faf40 100644 --- a/evmlib/src/wallet.rs +++ b/evmlib/src/wallet.rs @@ -6,7 +6,7 @@ // KIND, either express or implied. Please review the Licences for the specific language governing // permissions and limitations relating to use of the SAFE Network Software. -use crate::common::{Address, Amount, QuoteHash, QuotePayment, TxHash, U256}; +use crate::common::{Address, Amount, QuotePayment, QuoteHash, TxHash, U256}; use crate::contract::network_token::NetworkToken; use crate::contract::payment_vault::handler::PaymentVaultHandler; use crate::contract::payment_vault::MAX_TRANSFERS_PER_TRANSACTION; @@ -120,26 +120,13 @@ impl Wallet { approve_to_spend_tokens(self.wallet.clone(), &self.network, spender, amount).await } - /// Pays for a single quote. Returns transaction hash of the payment. - pub async fn pay_for_quote( - &self, - quote_hash: QuoteHash, - rewards_addr: Address, - amount: U256, - ) -> Result { - self.pay_for_quotes([(quote_hash, rewards_addr, amount)]) - .await - .map(|v| v.values().last().cloned().expect("Infallible")) - .map_err(|err| err.0) - } - /// Function for batch payments of quotes. It accepts an iterator of QuotePayment and returns /// transaction hashes of the payments by quotes. pub async fn pay_for_quotes>( &self, - data_payments: I, + quote_payments: I, ) -> Result, PayForQuotesError> { - pay_for_quotes(self.wallet.clone(), &self.network, data_payments).await + pay_for_quotes(self.wallet.clone(), &self.network, quote_payments).await } /// Build a provider using this wallet. @@ -334,8 +321,14 @@ pub async fn pay_for_quotes>( let provider = http_provider_with_wallet(network.rpc_url().clone(), wallet); let data_payments = PaymentVaultHandler::new(*network.data_payments_address(), provider); + // remove payments with 0 amount as they don't need to be paid for + let payment_for_batch: Vec = payments + .into_iter() + .filter(|(_, _, amount)| *amount > Amount::ZERO) + .collect(); + // Divide transfers over multiple transactions if they exceed the max per transaction. - let chunks = payments.chunks(MAX_TRANSFERS_PER_TRANSACTION); + let chunks = payment_for_batch.chunks(MAX_TRANSFERS_PER_TRANSACTION); let mut tx_hashes_by_quote = BTreeMap::new(); diff --git a/evmlib/tests/wallet.rs b/evmlib/tests/wallet.rs index c324f771fc..cab48d254b 100644 --- a/evmlib/tests/wallet.rs +++ b/evmlib/tests/wallet.rs @@ -90,9 +90,7 @@ async fn test_pay_for_quotes_and_data_payment_verification() { for (quote_hash, reward_addr, _) in quote_payments.iter() { let result = verify_data_payment( &network, - *quote_hash, - *reward_addr, - QuotingMetrics::default(), + vec![(*quote_hash, QuotingMetrics::default(), *reward_addr)] ) .await; From ef02c0234999db96734398746c9e4e3ac2dce2fa Mon Sep 17 00:00:00 2001 From: grumbach Date: Mon, 9 Dec 2024 17:39:21 +0900 Subject: [PATCH 11/33] chore: notes for takeover --- autonomi/src/client/utils.rs | 6 ++---- evmlib/src/transaction.rs | 1 + 2 files changed, 3 insertions(+), 4 deletions(-) diff --git a/autonomi/src/client/utils.rs b/autonomi/src/client/utils.rs index 04e06fc4dc..da5c51509f 100644 --- a/autonomi/src/client/utils.rs +++ b/autonomi/src/client/utils.rs @@ -26,7 +26,6 @@ use super::{ Client, }; use crate::self_encryption::DataMapLevel; -use crate::utils::receipt_from_cost_map_and_payments; impl Client { /// Fetch and decrypt all chunks in the data map. @@ -172,6 +171,7 @@ impl Client { // TODO: the error might contain some succeeded quote payments as well. These should be returned on err, so that they can be skipped when retrying. // TODO: retry when it fails? // Execute chunk payments + // NB TODO: make this return a Receipt or something that can turn into a Receipt @mick let payments = wallet .pay_for_quotes(quotes.payments()) .await @@ -181,12 +181,10 @@ impl Client { drop(lock_guard); debug!("Unlocked wallet"); - let proofs = receipt_from_cost_map_and_payments(cost_map, &payments); - let skipped_chunks = content_addrs.count() - quotes.len(); trace!( "Chunk payments of {} chunks completed. {} chunks were free / already paid for", - proofs.len(), + quotes.len(), skipped_chunks ); diff --git a/evmlib/src/transaction.rs b/evmlib/src/transaction.rs index 4a54f8391e..48df355638 100644 --- a/evmlib/src/transaction.rs +++ b/evmlib/src/transaction.rs @@ -106,6 +106,7 @@ async fn get_data_payment_event( /// Verify if a data payment is confirmed. pub async fn verify_data_payment( network: &Network, + my_quote_hashes: Vec, // TODO @mick hashes the node owns so it knows how much it received from them payment: Vec<(QuoteHash, QuotingMetrics, Address)> ) -> Result { let provider = http_provider(network.rpc_url().clone()); From c950dec4c4977ae32778aa46eec880ba932e633b Mon Sep 17 00:00:00 2001 From: Warm Beer Date: Mon, 9 Dec 2024 14:28:14 +0100 Subject: [PATCH 12/33] feat: pay returns a receipt --- ant-evm/src/data_payments.rs | 33 ++++++++++++++++++++------ ant-evm/src/lib.rs | 2 +- autonomi/src/client/payment.rs | 42 +++++++++++++++++++++++++++++++--- autonomi/src/client/quote.rs | 21 +++++++++++++---- autonomi/src/client/utils.rs | 9 +++++--- 5 files changed, 89 insertions(+), 18 deletions(-) diff --git a/ant-evm/src/data_payments.rs b/ant-evm/src/data_payments.rs index 47476893aa..9f959a93fa 100644 --- a/ant-evm/src/data_payments.rs +++ b/ant-evm/src/data_payments.rs @@ -8,7 +8,9 @@ use crate::EvmError; use evmlib::{ - common::{Address as RewardsAddress, QuoteHash}, quoting_metrics::QuotingMetrics, utils::dummy_address + common::{Address as RewardsAddress, QuoteHash}, + quoting_metrics::QuotingMetrics, + utils::dummy_address, }; use libp2p::{identity::PublicKey, PeerId}; use serde::{Deserialize, Serialize}; @@ -31,31 +33,48 @@ impl EncodedPeerId { pub fn to_peer_id(&self) -> Result { match PublicKey::try_decode_protobuf(&self.0) { Ok(pub_key) => Ok(PeerId::from_public_key(&pub_key)), - Err(e) => Err(e) + Err(e) => Err(e), } } } +// TODO: @anselme is this conversion right? +impl From for EncodedPeerId { + fn from(peer_id: PeerId) -> Self { + let bytes = peer_id.to_bytes(); + EncodedPeerId(bytes) + } +} + /// The proof of payment for a data payment #[derive(Debug, Clone, PartialEq, Eq, Ord, PartialOrd, Serialize, Deserialize)] pub struct ProofOfPayment { - peer_quotes: Vec<(EncodedPeerId, PaymentQuote)> + pub peer_quotes: Vec<(EncodedPeerId, PaymentQuote)>, } impl ProofOfPayment { /// returns a short digest of the proof of payment to use for verification pub fn digest(&self) -> Vec<(QuoteHash, QuotingMetrics, RewardsAddress)> { - self.peer_quotes.clone().into_iter().map(|(_, quote)| (quote.hash(), quote.quoting_metrics, quote.rewards_address)).collect() + self.peer_quotes + .clone() + .into_iter() + .map(|(_, quote)| (quote.hash(), quote.quoting_metrics, quote.rewards_address)) + .collect() } /// returns the list of payees pub fn payees(&self) -> Vec { - self.peer_quotes.iter().filter_map(|(peer_id, _)| peer_id.to_peer_id().ok()).collect() + self.peer_quotes + .iter() + .filter_map(|(peer_id, _)| peer_id.to_peer_id().ok()) + .collect() } /// has the quote expired pub fn has_expired(&self) -> bool { - self.peer_quotes.iter().any(|(_, quote)| quote.has_expired()) + self.peer_quotes + .iter() + .any(|(_, quote)| quote.has_expired()) } /// verifies the proof of payment is valid for the given peer id @@ -72,7 +91,7 @@ impl ProofOfPayment { Err(e) => { warn!("Invalid encoded peer id: {e}"); return false; - }, + } }; if !quote.check_is_signed_by_claimed_peer(peer_id) { return false; diff --git a/ant-evm/src/lib.rs b/ant-evm/src/lib.rs index 30a42b34d4..10f557e395 100644 --- a/ant-evm/src/lib.rs +++ b/ant-evm/src/lib.rs @@ -29,7 +29,7 @@ mod amount; mod data_payments; mod error; -pub use data_payments::{PaymentQuote, ProofOfPayment, QUOTE_EXPIRATION_SECS}; +pub use data_payments::{EncodedPeerId, PaymentQuote, ProofOfPayment, QUOTE_EXPIRATION_SECS}; pub use evmlib::quoting_metrics::QuotingMetrics; /// Types used in the public API diff --git a/autonomi/src/client/payment.rs b/autonomi/src/client/payment.rs index 97416e3c09..48c199c4a6 100644 --- a/autonomi/src/client/payment.rs +++ b/autonomi/src/client/payment.rs @@ -1,11 +1,47 @@ use crate::client::data::PayError; +use crate::client::quote::StoreQuote; use crate::Client; -use ant_evm::{AttoTokens, EvmWallet, ProofOfPayment}; -use std::collections::HashMap; +use ant_evm::{AttoTokens, EncodedPeerId, EvmWallet, ProofOfPayment, QuoteHash, TxHash}; +use std::collections::{BTreeMap, HashMap}; use xor_name::XorName; /// Contains the proof of payments for each XOR address and the amount paid -pub type Receipt = HashMap>; +pub type Receipt = HashMap; + +pub fn receipt_from_store_quotes_and_payments( + quotes: StoreQuote, + payments: BTreeMap, +) -> Receipt { + let mut receipt = Receipt::new(); + + for (content_addr, quote_for_address) in quotes.0 { + let price = AttoTokens::from_atto(quote_for_address.price()); + + let mut proof_of_payment = ProofOfPayment { + peer_quotes: vec![], + }; + + for (peer_id, quote, _amount) in quote_for_address.0 { + // skip quotes that haven't been paid + if !payments.contains_key("e.hash()) { + continue; + } + + proof_of_payment + .peer_quotes + .push((EncodedPeerId::from(peer_id), quote)); + } + + // skip empty proofs + if proof_of_payment.peer_quotes.is_empty() { + continue; + } + + receipt.insert(content_addr, (proof_of_payment, price)); + } + + receipt +} /// Payment options for data payments. #[derive(Clone)] diff --git a/autonomi/src/client/quote.rs b/autonomi/src/client/quote.rs index 1e5e6b80be..f1bc67e61a 100644 --- a/autonomi/src/client/quote.rs +++ b/autonomi/src/client/quote.rs @@ -17,7 +17,7 @@ use std::collections::HashMap; use xor_name::XorName; /// A quote for a single address -pub struct QuoteForAddress(Vec<(PeerId, PaymentQuote, Amount)>); +pub struct QuoteForAddress(pub(crate) Vec<(PeerId, PaymentQuote, Amount)>); impl QuoteForAddress { pub fn price(&self) -> Amount { @@ -26,7 +26,7 @@ impl QuoteForAddress { } /// A quote for many addresses -pub struct StoreQuote(HashMap); +pub struct StoreQuote(pub(crate) HashMap); impl StoreQuote { pub fn price(&self) -> Amount { @@ -87,10 +87,23 @@ impl Client { let second = (*p2, q2.clone(), Amount::ZERO); // pay for the rest - quotes_to_pay_per_addr.insert(content_addr, QuoteForAddress(vec![first, second, third.clone(), fourth.clone(), fifth.clone()])); + quotes_to_pay_per_addr.insert( + content_addr, + QuoteForAddress(vec![ + first, + second, + third.clone(), + fourth.clone(), + fifth.clone(), + ]), + ); } _ => { - return Err(CostError::NotEnoughNodeQuotes(content_addr, prices.len(), MINIMUM_QUOTES_TO_PAY)); + return Err(CostError::NotEnoughNodeQuotes( + content_addr, + prices.len(), + MINIMUM_QUOTES_TO_PAY, + )); } } } diff --git a/autonomi/src/client/utils.rs b/autonomi/src/client/utils.rs index da5c51509f..61bf4dadd1 100644 --- a/autonomi/src/client/utils.rs +++ b/autonomi/src/client/utils.rs @@ -6,7 +6,7 @@ // KIND, either express or implied. Please review the Licences for the specific language governing // permissions and limitations relating to use of the SAFE Network Software. -use crate::client::payment::Receipt; +use crate::client::payment::{receipt_from_store_quotes_and_payments, Receipt}; use ant_evm::{EvmNetwork, EvmWallet, ProofOfPayment}; use ant_networking::{GetRecordCfg, PutRecordCfg, VerificationKind}; use ant_protocol::{ @@ -161,7 +161,9 @@ impl Client { content_addrs: impl Iterator, wallet: &EvmWallet, ) -> Result { - let quotes = self.get_store_quotes(wallet.network(), content_addrs).await?; + let quotes = self + .get_store_quotes(wallet.network(), content_addrs.clone()) + .await?; // Make sure nobody else can use the wallet while we are paying debug!("Waiting for wallet lock"); @@ -171,7 +173,6 @@ impl Client { // TODO: the error might contain some succeeded quote payments as well. These should be returned on err, so that they can be skipped when retrying. // TODO: retry when it fails? // Execute chunk payments - // NB TODO: make this return a Receipt or something that can turn into a Receipt @mick let payments = wallet .pay_for_quotes(quotes.payments()) .await @@ -181,6 +182,8 @@ impl Client { drop(lock_guard); debug!("Unlocked wallet"); + let receipt = receipt_from_store_quotes_and_payments(quotes, payments); + let skipped_chunks = content_addrs.count() - quotes.len(); trace!( "Chunk payments of {} chunks completed. {} chunks were free / already paid for", From 222daf45003dcd66a4a6aca65b6572c4442f5d80 Mon Sep 17 00:00:00 2001 From: Warm Beer Date: Mon, 9 Dec 2024 18:52:27 +0100 Subject: [PATCH 13/33] chore: autonomi compiles! --- autonomi/src/client/data/mod.rs | 6 +++ autonomi/src/client/data/public.rs | 27 +++++++------ autonomi/src/client/data_private.rs | 2 +- autonomi/src/client/mod.rs | 7 ++++ autonomi/src/client/payment.rs | 2 +- autonomi/src/client/quote.rs | 5 +-- autonomi/src/client/registers.rs | 35 ++++++++++------- autonomi/src/client/utils.rs | 25 ++++++------ autonomi/src/client/vault.rs | 12 +++--- evmlib/src/contract/payment_vault/error.rs | 2 + .../src/contract/payment_vault/interface.rs | 6 +++ evmlib/src/contract/payment_vault/mod.rs | 38 ++++++++++++++++++- evmlib/src/lib.rs | 13 +------ 13 files changed, 122 insertions(+), 58 deletions(-) diff --git a/autonomi/src/client/data/mod.rs b/autonomi/src/client/data/mod.rs index b85f54a68e..bf5ba191e8 100644 --- a/autonomi/src/client/data/mod.rs +++ b/autonomi/src/client/data/mod.rs @@ -84,6 +84,8 @@ pub enum PutError { VaultBadOwner, #[error("Payment unexpectedly invalid for {0:?}")] PaymentUnexpectedlyInvalid(NetworkAddress), + #[error("The payment proof contains no payees.")] + PayeesMissing, } /// Errors that can occur during the pay operation. @@ -121,8 +123,12 @@ pub enum CostError { CouldNotGetStoreQuote(XorName), #[error("Could not get store costs: {0:?}")] CouldNotGetStoreCosts(NetworkError), + #[error("Not enough node quotes for {0:?}, got: {1:?} and need at least {2:?}")] + NotEnoughNodeQuotes(XorName, usize, usize), #[error("Failed to serialize {0}")] Serialization(String), + #[error("Market price error: {0:?}")] + MarketPriceError(#[from] ant_evm::payment_vault::error::Error), } /// Private data on the network can be accessed with this diff --git a/autonomi/src/client/data/public.rs b/autonomi/src/client/data/public.rs index 2b018298a3..28bad9dc4f 100644 --- a/autonomi/src/client/data/public.rs +++ b/autonomi/src/client/data/public.rs @@ -8,15 +8,14 @@ use bytes::Bytes; use libp2p::kad::Quorum; +use std::collections::HashSet; -use std::collections::{HashMap, HashSet}; use xor_name::XorName; -use crate::client::payment::PaymentOption; +use crate::client::payment::{PaymentOption, Receipt}; use crate::client::utils::process_tasks_with_max_concurrency; use crate::client::{ClientEvent, UploadSummary}; use crate::{self_encryption::encrypt, Client}; -use ant_evm::ProofOfPayment; use ant_evm::{Amount, AttoTokens}; use ant_networking::{GetRecordCfg, NetworkError}; use ant_protocol::{ @@ -96,7 +95,7 @@ impl Client { if let Some(channel) = self.client_event_sender.as_ref() { let tokens_spent = receipt .values() - .map(|proof| proof.quote.cost.as_atto()) + .map(|(_proof, price)| price.as_atto()) .sum::(); let summary = UploadSummary { @@ -163,21 +162,27 @@ impl Client { content_addrs.len() ); - let cost_map = self + let store_quote = self .get_store_quotes(content_addrs.into_iter()) .await .inspect_err(|err| error!("Error getting store quotes: {err:?}"))?; - let total_cost = cost_map - .values() - .fold(Amount::ZERO, |acc, q| acc + q.total_cost.as_atto()); - Ok(AttoTokens::from_atto(total_cost)) + + let total_cost = AttoTokens::from_atto( + store_quote + .0 + .values() + .map(|quote| quote.price()) + .sum::(), + ); + + Ok(total_cost) } // Upload chunks and retry failed uploads up to `RETRY_ATTEMPTS` times. pub(crate) async fn upload_chunks_with_retries<'a>( &self, mut chunks: Vec<&'a Chunk>, - receipt: &HashMap, + receipt: &Receipt, ) -> Vec<(&'a Chunk, PutError)> { let mut current_attempt: usize = 1; @@ -187,7 +192,7 @@ impl Client { let self_clone = self.clone(); let address = *chunk.address(); - let Some(proof) = receipt.get(chunk.name()) else { + let Some((proof, _)) = receipt.get(chunk.name()) else { debug!("Chunk at {address:?} was already paid for so skipping"); continue; }; diff --git a/autonomi/src/client/data_private.rs b/autonomi/src/client/data_private.rs index 5f2dd1793c..d1288bb193 100644 --- a/autonomi/src/client/data_private.rs +++ b/autonomi/src/client/data_private.rs @@ -100,7 +100,7 @@ impl Client { if let Some(channel) = self.client_event_sender.as_ref() { let tokens_spent = receipt .values() - .map(|proof| proof.quote.cost.as_atto()) + .map(|(_proof, price)| price.as_atto()) .sum::(); let summary = UploadSummary { diff --git a/autonomi/src/client/mod.rs b/autonomi/src/client/mod.rs index 7ca25bd7a2..b01a6a9b2d 100644 --- a/autonomi/src/client/mod.rs +++ b/autonomi/src/client/mod.rs @@ -34,6 +34,7 @@ mod utils; use ant_bootstrap::{BootstrapCacheConfig, BootstrapCacheStore}; pub use ant_evm::Amount; +use crate::EvmNetwork; use ant_networking::{interval, multiaddr_is_global, Network, NetworkBuilder, NetworkEvent}; use ant_protocol::{version::IDENTIFY_PROTOCOL_STR, CLOSE_GROUP_SIZE}; use libp2p::{identity::Keypair, Multiaddr}; @@ -64,6 +65,7 @@ const CLIENT_EVENT_CHANNEL_SIZE: usize = 100; pub struct Client { pub(crate) network: Network, pub(crate) client_event_sender: Arc>>, + pub(crate) evm_network: EvmNetwork, } /// Error returned by [`Client::connect`]. @@ -118,6 +120,7 @@ impl Client { Ok(Self { network, client_event_sender: Arc::new(None), + evm_network: Default::default(), }) } @@ -130,6 +133,10 @@ impl Client { client_event_receiver } + + pub fn set_evm_network(&mut self, evm_network: EvmNetwork) { + self.evm_network = evm_network; + } } fn build_client_and_run_swarm(local: bool) -> (Network, mpsc::Receiver) { diff --git a/autonomi/src/client/payment.rs b/autonomi/src/client/payment.rs index 48c199c4a6..509615fb20 100644 --- a/autonomi/src/client/payment.rs +++ b/autonomi/src/client/payment.rs @@ -71,7 +71,7 @@ impl From for PaymentOption { impl Client { pub(crate) async fn pay_for_content_addrs( &self, - content_addrs: impl Iterator, + content_addrs: impl Iterator + Clone, payment_option: PaymentOption, ) -> Result { match payment_option { diff --git a/autonomi/src/client/quote.rs b/autonomi/src/client/quote.rs index f1bc67e61a..514cf9c4d5 100644 --- a/autonomi/src/client/quote.rs +++ b/autonomi/src/client/quote.rs @@ -7,7 +7,6 @@ // permissions and limitations relating to use of the SAFE Network Software. use super::{data::CostError, Client}; -use crate::EvmNetwork; use ant_evm::payment_vault::get_market_price; use ant_evm::{Amount, PaymentQuote, QuotePayment}; use ant_networking::{Network, NetworkError}; @@ -51,7 +50,6 @@ impl StoreQuote { impl Client { pub(crate) async fn get_store_quotes( &self, - evm_network: &EvmNetwork, content_addrs: impl Iterator, ) -> Result { // get all quotes from nodes @@ -68,7 +66,8 @@ impl Client { let mut prices = vec![]; for (peer, quote) in raw_quotes { // NB TODO @mick we need to batch this smart contract call - let price = get_market_price(evm_network, quote.quoting_metrics.clone()).await?; + let price = + get_market_price(&self.evm_network, quote.quoting_metrics.clone()).await?; prices.push((peer, quote, price)); } diff --git a/autonomi/src/client/registers.rs b/autonomi/src/client/registers.rs index 9f30e8b565..0fc502426b 100644 --- a/autonomi/src/client/registers.rs +++ b/autonomi/src/client/registers.rs @@ -11,7 +11,6 @@ use crate::client::Client; use crate::client::ClientEvent; use crate::client::UploadSummary; -use ant_evm::EvmNetwork; pub use ant_registers::{Permissions as RegisterPermissions, RegisterAddress}; pub use bls::SecretKey as RegisterSecretKey; @@ -50,6 +49,8 @@ pub enum RegisterError { CouldNotSign(#[source] ant_registers::Error), #[error("Received invalid quote from node, this node is possibly malfunctioning, try another node by trying another register name")] InvalidQuote, + #[error("The payment proof contains no payees.")] + PayeesMissing, } #[derive(Clone, Debug)] @@ -235,7 +236,6 @@ impl Client { /// Get the cost to create a register pub async fn register_cost( &self, - evm_network: &EvmNetwork, name: String, owner: RegisterSecretKey, ) -> Result { @@ -249,11 +249,13 @@ impl Client { // get cost to store register // NB TODO: register should be priced differently from other data - let cost_map = self.get_store_quotes(evm_network, std::iter::once(reg_xor)).await?; + let store_quote = self.get_store_quotes(std::iter::once(reg_xor)).await?; + let total_cost = AttoTokens::from_atto( - cost_map + store_quote + .0 .values() - .map(|quote| quote.2.cost.as_atto()) + .map(|quote| quote.price()) .sum::(), ); debug!("Calculated the cost to create register with name: {name} is {total_cost}"); @@ -310,18 +312,24 @@ impl Client { .inspect_err(|err| { error!("Failed to pay for register at address: {address} : {err}") })?; - let proof = if let Some(proof) = payment_proofs.get(®_xor) { - proof + let (proof, price) = if let Some((proof, price)) = payment_proofs.get(®_xor) { + (proof, price) } else { // register was skipped, meaning it was already paid for error!("Register at address: {address} was already paid for"); return Err(RegisterError::Network(NetworkError::RegisterAlreadyExists)); }; - let payee = proof - .to_peer_id_payee() - .ok_or(RegisterError::InvalidQuote) - .inspect_err(|err| error!("Failed to get payee from payment proof: {err}"))?; + let payees = proof.payees(); + + if payees.is_empty() { + error!( + "Failed to get payees from payment proof: {:?}", + RegisterError::PayeesMissing + ); + return Err(RegisterError::PayeesMissing); + } + let signed_register = register.signed_reg.clone(); let record = Record { @@ -343,10 +351,11 @@ impl Client { expected_holders: Default::default(), is_register: true, }; + let put_cfg = PutRecordCfg { put_quorum: Quorum::All, retry_strategy: None, - use_put_record_to: Some(vec![payee]), + use_put_record_to: Some(payees), // CODE REVIEW: should we put to all or just one here? verification: Some((VerificationKind::Network, get_cfg)), }; @@ -361,7 +370,7 @@ impl Client { if let Some(channel) = self.client_event_sender.as_ref() { let summary = UploadSummary { record_count: 1, - tokens_spent: proof.quote.cost.as_atto(), + tokens_spent: price.as_atto(), }; if let Err(err) = channel.send(ClientEvent::UploadComplete(summary)).await { error!("Failed to send client event: {err}"); diff --git a/autonomi/src/client/utils.rs b/autonomi/src/client/utils.rs index 61bf4dadd1..0b7540af62 100644 --- a/autonomi/src/client/utils.rs +++ b/autonomi/src/client/utils.rs @@ -7,7 +7,7 @@ // permissions and limitations relating to use of the SAFE Network Software. use crate::client::payment::{receipt_from_store_quotes_and_payments, Receipt}; -use ant_evm::{EvmNetwork, EvmWallet, ProofOfPayment}; +use ant_evm::{EvmWallet, ProofOfPayment}; use ant_networking::{GetRecordCfg, PutRecordCfg, VerificationKind}; use ant_protocol::{ messages::ChunkProof, @@ -100,9 +100,13 @@ impl Client { chunk: &Chunk, payment: ProofOfPayment, ) -> Result<(), PutError> { - let storing_node = payment.to_peer_id_payee().expect("Missing node Peer ID"); + let storing_nodes = payment.payees(); - debug!("Storing chunk: {chunk:?} to {:?}", storing_node); + if storing_nodes.is_empty() { + return Err(PutError::PayeesMissing); + } + + debug!("Storing chunk: {chunk:?} to {:?}", storing_nodes); let key = chunk.network_address().to_record_key(); @@ -147,7 +151,7 @@ impl Client { let put_cfg = PutRecordCfg { put_quorum: Quorum::One, retry_strategy: Some(RetryStrategy::Balanced), - use_put_record_to: Some(vec![storing_node]), + use_put_record_to: Some(storing_nodes), // CODE REVIEW: do we put to all payees or just one? verification, }; let payment_upload = Ok(self.network.put_record(record, &put_cfg).await?); @@ -158,12 +162,11 @@ impl Client { /// Pay for the chunks and get the proof of payment. pub(crate) async fn pay( &self, - content_addrs: impl Iterator, + content_addrs: impl Iterator + Clone, wallet: &EvmWallet, ) -> Result { - let quotes = self - .get_store_quotes(wallet.network(), content_addrs.clone()) - .await?; + let number_of_content_addrs = content_addrs.clone().count(); + let quotes = self.get_store_quotes(content_addrs).await?; // Make sure nobody else can use the wallet while we are paying debug!("Waiting for wallet lock"); @@ -182,15 +185,15 @@ impl Client { drop(lock_guard); debug!("Unlocked wallet"); - let receipt = receipt_from_store_quotes_and_payments(quotes, payments); - - let skipped_chunks = content_addrs.count() - quotes.len(); + let skipped_chunks = number_of_content_addrs - quotes.len(); trace!( "Chunk payments of {} chunks completed. {} chunks were free / already paid for", quotes.len(), skipped_chunks ); + let receipt = receipt_from_store_quotes_and_payments(quotes, payments); + Ok(receipt) } } diff --git a/autonomi/src/client/vault.rs b/autonomi/src/client/vault.rs index 83553e3e16..4633004aa8 100644 --- a/autonomi/src/client/vault.rs +++ b/autonomi/src/client/vault.rs @@ -152,11 +152,13 @@ impl Client { let vault_xor = scratch.network_address().as_xorname().unwrap_or_default(); // NB TODO: vault should be priced differently from other data - let cost_map = self.get_store_quotes(std::iter::once(vault_xor)).await?; + let store_quote = self.get_store_quotes(std::iter::once(vault_xor)).await?; + let total_cost = AttoTokens::from_atto( - cost_map + store_quote + .0 .values() - .map(|quote| quote.2.cost.as_atto()) + .map(|quote| quote.price()) .sum::(), ); @@ -197,12 +199,12 @@ impl Client { error!("Failed to pay for new vault at addr: {scratch_address:?} : {err}"); })?; - let proof = match receipt.values().next() { + let (proof, price) = match receipt.values().next() { Some(proof) => proof, None => return Err(PutError::PaymentUnexpectedlyInvalid(scratch_address)), }; - total_cost = proof.quote.cost; + total_cost = price.clone(); Record { key: scratch_key, diff --git a/evmlib/src/contract/payment_vault/error.rs b/evmlib/src/contract/payment_vault/error.rs index 0441b5b1ea..6c94c680f1 100644 --- a/evmlib/src/contract/payment_vault/error.rs +++ b/evmlib/src/contract/payment_vault/error.rs @@ -8,4 +8,6 @@ pub enum Error { RpcError(#[from] RpcError), #[error(transparent)] PendingTransactionError(#[from] alloy::providers::PendingTransactionError), + #[error("Payment is invalid.")] + PaymentInvalid, } diff --git a/evmlib/src/contract/payment_vault/interface.rs b/evmlib/src/contract/payment_vault/interface.rs index d99811e01a..9f2d6f3490 100644 --- a/evmlib/src/contract/payment_vault/interface.rs +++ b/evmlib/src/contract/payment_vault/interface.rs @@ -10,6 +10,12 @@ sol!( "abi/IPaymentVault.json" ); +pub struct PaymentVerification { + pub quote_hash: FixedBytes<32>, + pub amount_paid: Amount, + pub is_valid: bool, +} + impl From<(QuoteHash, Address, Amount)> for IPaymentVault::DataPayment { fn from(value: (QuoteHash, Address, Amount)) -> Self { Self { diff --git a/evmlib/src/contract/payment_vault/mod.rs b/evmlib/src/contract/payment_vault/mod.rs index d6afbbd91a..8ed1a9a92b 100644 --- a/evmlib/src/contract/payment_vault/mod.rs +++ b/evmlib/src/contract/payment_vault/mod.rs @@ -1,5 +1,6 @@ -use crate::common::Amount; +use crate::common::{Address, Amount, QuoteHash}; use crate::contract::payment_vault::handler::PaymentVaultHandler; +use crate::contract::payment_vault::interface::PaymentVerification; use crate::quoting_metrics::QuotingMetrics; use crate::utils::http_provider; use crate::Network; @@ -20,3 +21,38 @@ pub async fn get_market_price( let payment_vault = PaymentVaultHandler::new(*network.data_payments_address(), provider); payment_vault.get_quote(quoting_metrics).await } + +/// Helper function to verify whether a data payment is valid +pub async fn verify_data_payment( + network: &Network, + owned_quote_hashes: Vec, + payment: Vec<(QuoteHash, QuotingMetrics, Address)>, +) -> Result { + let provider = http_provider(network.rpc_url().clone()); + let payment_vault = PaymentVaultHandler::new(*network.data_payments_address(), provider); + + let mut amount = Amount::ZERO; + + // TODO: @mick change this for loop to a batch when the smart contract changes + for (quote_hash, quoting_metrics, rewards_address) in payment { + let payment_verification: PaymentVerification = payment_vault + .verify_payment(quoting_metrics, (quote_hash, rewards_address, Amount::ZERO)) + .await + .map(|is_valid| PaymentVerification { + quote_hash, + amount_paid: Amount::from(1), // TODO: update placeholder amount when the smart contract changes + is_valid, + })?; + + // CODE REVIEW: should we fail on a single invalid payment? + if !payment_verification.is_valid { + return Err(error::Error::PaymentInvalid); + } + + if owned_quote_hashes.contains("e_hash) { + amount += payment_verification.amount_paid; + } + } + + Ok(amount) +} diff --git a/evmlib/src/lib.rs b/evmlib/src/lib.rs index 1bc363925a..6de2343462 100644 --- a/evmlib/src/lib.rs +++ b/evmlib/src/lib.rs @@ -6,12 +6,9 @@ // KIND, either express or implied. Please review the Licences for the specific language governing // permissions and limitations relating to use of the SAFE Network Software. -use crate::common::{Address, QuoteHash}; -use crate::transaction::verify_data_payment; +use crate::common::Address; use alloy::primitives::address; use alloy::transports::http::reqwest; -use common::Amount; -use quoting_metrics::QuotingMetrics; use serde::{Deserialize, Serialize}; use serde_with::{serde_as, DisplayFromStr}; use std::str::FromStr; @@ -27,7 +24,6 @@ pub mod cryptography; pub mod external_signer; pub mod quoting_metrics; pub mod testnet; -pub mod transaction; pub mod utils; pub mod wallet; @@ -135,11 +131,4 @@ impl Network { Network::Custom(custom) => &custom.data_payments_address, } } - - pub async fn verify_data_payment( - &self, - payment: Vec<(QuoteHash, QuotingMetrics, Address)> - ) -> Result { - verify_data_payment(self, payment).await - } } From f692b2ed293c4a5ceaef2044b68f019a67688d72 Mon Sep 17 00:00:00 2001 From: Warm Beer Date: Mon, 9 Dec 2024 20:48:11 +0100 Subject: [PATCH 14/33] fix: put validation verify payment import and input --- ant-evm/src/data_payments.rs | 15 +++++++++++++++ ant-node/src/put_validation.rs | 19 ++++++++++++------- 2 files changed, 27 insertions(+), 7 deletions(-) diff --git a/ant-evm/src/data_payments.rs b/ant-evm/src/data_payments.rs index 9f959a93fa..f11486dd0e 100644 --- a/ant-evm/src/data_payments.rs +++ b/ant-evm/src/data_payments.rs @@ -77,6 +77,21 @@ impl ProofOfPayment { .any(|(_, quote)| quote.has_expired()) } + /// Returns all quotes by given peer id + pub fn quotes_by_peer(&self, peer_id: &PeerId) -> Vec<&PaymentQuote> { + self.peer_quotes + .iter() + .filter_map(|(id, quote)| { + if let Ok(id) = id.to_peer_id() { + if id == *peer_id { + return Some(quote); + } + } + None + }) + .collect() + } + /// verifies the proof of payment is valid for the given peer id pub fn verify_for(&self, peer_id: PeerId) -> bool { // make sure I am in the list of payees diff --git a/ant-node/src/put_validation.rs b/ant-node/src/put_validation.rs index 95b37dcc5e..05ca698e96 100644 --- a/ant-node/src/put_validation.rs +++ b/ant-node/src/put_validation.rs @@ -7,7 +7,8 @@ // permissions and limitations relating to use of the SAFE Network Software. use crate::{node::Node, Error, Marker, Result}; -use ant_evm::{AttoTokens, ProofOfPayment, QUOTE_EXPIRATION_SECS}; +use ant_evm::payment_vault::verify_data_payment; +use ant_evm::{AttoTokens, ProofOfPayment}; use ant_networking::NetworkError; use ant_protocol::storage::Transaction; use ant_protocol::{ @@ -19,7 +20,6 @@ use ant_protocol::{ }; use ant_registers::SignedRegister; use libp2p::kad::{Record, RecordKey}; -use std::time::{Duration, SystemTime, UNIX_EPOCH}; use xor_name::XorName; impl Node { @@ -669,14 +669,19 @@ impl Node { ))); } + let owned_payment_quotes = payment + .quotes_by_peer(&self_peer_id) + .iter() + .map(|quote| quote.hash()) + .collect(); + // check if payment is valid on chain let payments_to_verify = payment.digest(); debug!("Verifying payment for record {pretty_key}"); - let reward_amount = self - .evm_network() - .verify_data_payment(payments_to_verify) - .await - .map_err(|e| Error::EvmNetwork(format!("Failed to verify chunk payment: {e}")))?; + let reward_amount = + verify_data_payment(self.evm_network(), owned_payment_quotes, payments_to_verify) + .await + .map_err(|e| Error::EvmNetwork(format!("Failed to verify chunk payment: {e}")))?; debug!("Payment of {reward_amount:?} is valid for record {pretty_key}"); // Notify `record_store` that the node received a payment. From d3cdc3ee8c65274c0adc0a43c12b066dc136fff8 Mon Sep 17 00:00:00 2001 From: grumbach Date: Tue, 10 Dec 2024 17:09:02 +0900 Subject: [PATCH 15/33] feat: compiling CLI along with various fixes --- Cargo.lock | 207 +++++++++++++++---------- ant-cli/src/commands/register.rs | 3 +- ant-evm/src/data_payments.rs | 16 +- autonomi/src/client/external_signer.rs | 21 +-- autonomi/src/client/quote.rs | 6 +- autonomi/src/client/vault.rs | 2 +- autonomi/src/lib.rs | 2 - autonomi/tests/external_signer.rs | 6 +- evmlib/tests/wallet.rs | 10 +- 9 files changed, 159 insertions(+), 114 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 598c271ac2..b15c7fb01c 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -258,14 +258,14 @@ dependencies = [ "alloy-transport 0.7.3", "futures", "futures-util", - "thiserror 2.0.4", + "thiserror 2.0.6", ] [[package]] name = "alloy-core" -version = "0.8.14" +version = "0.8.15" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c3d14d531c99995de71558e8e2206c27d709559ee8e5a0452b965ea82405a013" +checksum = "c618bd382f0bc2ac26a7e4bfae01c9b015ca8f21b37ca40059ae35a7e62b3dc6" dependencies = [ "alloy-dyn-abi", "alloy-json-abi", @@ -276,9 +276,9 @@ dependencies = [ [[package]] name = "alloy-dyn-abi" -version = "0.8.14" +version = "0.8.15" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "80759b3f57b3b20fa7cd8fef6479930fc95461b58ff8adea6e87e618449c8a1d" +checksum = "41056bde53ae10ffbbf11618efbe1e0290859e5eab0fe9ef82ebdb62f12a866f" dependencies = [ "alloy-json-abi", "alloy-primitives", @@ -387,9 +387,9 @@ dependencies = [ [[package]] name = "alloy-json-abi" -version = "0.8.14" +version = "0.8.15" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ac4b22b3e51cac09fd2adfcc73b55f447b4df669f983c13f7894ec82b607c63f" +checksum = "c357da577dfb56998d01f574d81ad7a1958d248740a7981b205d69d65a7da404" dependencies = [ "alloy-primitives", "alloy-sol-type-parser", @@ -421,7 +421,7 @@ dependencies = [ "alloy-sol-types", "serde", "serde_json", - "thiserror 2.0.4", + "thiserror 2.0.6", "tracing", ] @@ -468,7 +468,7 @@ dependencies = [ "futures-utils-wasm", "serde", "serde_json", - "thiserror 2.0.4", + "thiserror 2.0.6", ] [[package]] @@ -526,16 +526,16 @@ dependencies = [ "rand 0.8.5", "serde_json", "tempfile", - "thiserror 2.0.4", + "thiserror 2.0.6", "tracing", "url", ] [[package]] name = "alloy-primitives" -version = "0.8.14" +version = "0.8.15" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9db948902dfbae96a73c2fbf1f7abec62af034ab883e4c777c3fd29702bd6e2c" +checksum = "6259a506ab13e1d658796c31e6e39d2e2ee89243bcc505ddc613b35732e0a430" dependencies = [ "alloy-rlp", "bytes", @@ -634,7 +634,7 @@ dependencies = [ "schnellru", "serde", "serde_json", - "thiserror 2.0.4", + "thiserror 2.0.6", "tokio", "tracing", "url", @@ -854,7 +854,7 @@ dependencies = [ "auto_impl", "elliptic-curve 0.13.8", "k256", - "thiserror 2.0.4", + "thiserror 2.0.6", ] [[package]] @@ -886,14 +886,14 @@ dependencies = [ "async-trait", "k256", "rand 0.8.5", - "thiserror 2.0.4", + "thiserror 2.0.6", ] [[package]] name = "alloy-sol-macro" -version = "0.8.14" +version = "0.8.15" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3bfd7853b65a2b4f49629ec975fee274faf6dff15ab8894c620943398ef283c0" +checksum = "d9d64f851d95619233f74b310f12bcf16e0cbc27ee3762b6115c14a84809280a" dependencies = [ "alloy-sol-macro-expander", "alloy-sol-macro-input", @@ -905,9 +905,9 @@ dependencies = [ [[package]] name = "alloy-sol-macro-expander" -version = "0.8.14" +version = "0.8.15" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "82ec42f342d9a9261699f8078e57a7a4fda8aaa73c1a212ed3987080e6a9cd13" +checksum = "6bf7ed1574b699f48bf17caab4e6e54c6d12bc3c006ab33d58b1e227c1c3559f" dependencies = [ "alloy-json-abi", "alloy-sol-macro-input", @@ -924,9 +924,9 @@ dependencies = [ [[package]] name = "alloy-sol-macro-input" -version = "0.8.14" +version = "0.8.15" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ed2c50e6a62ee2b4f7ab3c6d0366e5770a21cad426e109c2f40335a1b3aff3df" +checksum = "8c02997ccef5f34f9c099277d4145f183b422938ed5322dc57a089fe9b9ad9ee" dependencies = [ "alloy-json-abi", "const-hex", @@ -941,9 +941,9 @@ dependencies = [ [[package]] name = "alloy-sol-type-parser" -version = "0.8.14" +version = "0.8.15" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ac17c6e89a50fb4a758012e4b409d9a0ba575228e69b539fe37d7a1bd507ca4a" +checksum = "ce13ff37285b0870d0a0746992a4ae48efaf34b766ae4c2640fa15e5305f8e73" dependencies = [ "serde", "winnow", @@ -951,9 +951,9 @@ dependencies = [ [[package]] name = "alloy-sol-types" -version = "0.8.14" +version = "0.8.15" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c9dc0fffe397aa17628160e16b89f704098bf3c9d74d5d369ebc239575936de5" +checksum = "1174cafd6c6d810711b4e00383037bdb458efc4fe3dbafafa16567e0320c54d8" dependencies = [ "alloy-json-abi", "alloy-primitives", @@ -994,7 +994,7 @@ dependencies = [ "futures-utils-wasm", "serde", "serde_json", - "thiserror 2.0.4", + "thiserror 2.0.6", "tokio", "tower 0.5.1", "tracing", @@ -2523,9 +2523,9 @@ dependencies = [ [[package]] name = "cc" -version = "1.2.2" +version = "1.2.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f34d93e62b03caf570cccc334cbc6c2fceca82f39211051345108adcba3eebdc" +checksum = "27f657647bcff5394bf56c7317665bbf790a137a50eaaa5c6bfbb9e27a518f2d" dependencies = [ "jobserver", "libc", @@ -2570,9 +2570,9 @@ dependencies = [ [[package]] name = "chrono" -version = "0.4.38" +version = "0.4.39" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a21f936df1771bf62b77f047b726c4625ff2e8aa607c01ec06e5a05bd8463401" +checksum = "7e36cc9d416881d2e24f9a963be5fb1cd90966419ac844274161d10488b3e825" dependencies = [ "android-tzdata", "iana-time-zone", @@ -3787,18 +3787,9 @@ checksum = "a2a2b11eda1d40935b26cf18f6833c526845ae8c41e58d09af6adeb6f0269183" [[package]] name = "fastrand" -version = "1.9.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e51093e27b0797c359783294ca4f0a911c270184cb10f85783b118614a1501be" -dependencies = [ - "instant", -] - -[[package]] -name = "fastrand" -version = "2.2.0" +version = "2.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "486f806e73c5707928240ddc295403b1b93c96a02038563881c4a2fd84b81ac4" +checksum = "37909eebbb50d72f9059c3b6d82c0463f2ff062c9e95845c43a6c9c0355411be" [[package]] name = "fastrlp" @@ -4300,7 +4291,11 @@ version = "0.2.13" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "d48b897b4bbc881aea994b4a5bbb340a04979d7be9089791304e04a9fbc66b53" dependencies = [ +<<<<<<< HEAD "thiserror 2.0.4", +======= + "thiserror 2.0.6", +>>>>>>> e8bb86956 (feat: compiling CLI along with various fixes) ] [[package]] @@ -4309,7 +4304,11 @@ version = "0.4.10" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "c6ffbeb3a5c0b8b84c3fe4133a6f8c82fa962f4caefe8d0762eced025d3eb4f7" dependencies = [ +<<<<<<< HEAD "thiserror 2.0.4", +======= + "thiserror 2.0.6", +>>>>>>> e8bb86956 (feat: compiling CLI along with various fixes) ] [[package]] @@ -4357,7 +4356,11 @@ dependencies = [ "bstr", "gix-path", "libc", +<<<<<<< HEAD "thiserror 2.0.4", +======= + "thiserror 2.0.6", +>>>>>>> e8bb86956 (feat: compiling CLI along with various fixes) ] [[package]] @@ -4580,7 +4583,11 @@ dependencies = [ "gix-trace", "home", "once_cell", +<<<<<<< HEAD "thiserror 2.0.4", +======= + "thiserror 2.0.6", +>>>>>>> e8bb86956 (feat: compiling CLI along with various fixes) ] [[package]] @@ -4591,7 +4598,11 @@ checksum = "64a1e282216ec2ab2816cd57e6ed88f8009e634aec47562883c05ac8a7009a63" dependencies = [ "bstr", "gix-utils", +<<<<<<< HEAD "thiserror 2.0.4", +======= + "thiserror 2.0.6", +>>>>>>> e8bb86956 (feat: compiling CLI along with various fixes) ] [[package]] @@ -4829,6 +4840,7 @@ dependencies = [ "slab", "tokio", "tokio-util 0.7.13", +<<<<<<< HEAD "tracing", ] @@ -4848,6 +4860,8 @@ dependencies = [ "slab", "tokio", "tokio-util 0.7.13", +======= +>>>>>>> e8bb86956 (feat: compiling CLI along with various fixes) "tracing", ] @@ -5235,7 +5249,10 @@ dependencies = [ "bytes", "futures-channel", "futures-util", +<<<<<<< HEAD "h2 0.4.7", +======= +>>>>>>> e8bb86956 (feat: compiling CLI along with various fixes) "http 1.2.0", "http-body 1.0.1", "httparse", @@ -5765,9 +5782,9 @@ checksum = "f5d4a7da358eff58addd2877a45865158f0d78c911d43a5784ceb7bbf52833b0" [[package]] name = "js-sys" -version = "0.3.74" +version = "0.3.76" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a865e038f7f6ed956f788f0d7d60c541fff74c7bd74272c5d4cf15c63743e705" +checksum = "6717b6b5b077764fb5966237269cb3c64edddde4b14ce42647430a78ced9e7b7" dependencies = [ "once_cell", "wasm-bindgen", @@ -5824,9 +5841,9 @@ checksum = "e2abad23fbc42b3700f2f279844dc832adb2b2eb069b2df918f455c4e18cc646" [[package]] name = "libc" -version = "0.2.167" +version = "0.2.168" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "09d6582e104315a817dff97f75133544b2e094ee22447d2acf4a74e189ba06fc" +checksum = "5aaeb2981e0606ca11d79718f8bb01164f1d6ed75080182d3abf017e6d244b6d" [[package]] name = "libm" @@ -6772,9 +6789,9 @@ dependencies = [ [[package]] name = "multihash" -version = "0.19.2" +version = "0.19.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cc41f430805af9d1cf4adae4ed2149c759b877b01d909a1f40256188d09345d2" +checksum = "6b430e7953c29dd6a09afc29ff0bb69c6e306329ee6794700aee27b76a1aea8d" dependencies = [ "core2", "serde", @@ -7560,20 +7577,20 @@ checksum = "e3148f5046208a5d56bcfc03053e3ca6334e51da8dfb19b6cdc8b306fae3283e" [[package]] name = "pest" -version = "2.7.14" +version = "2.7.15" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "879952a81a83930934cbf1786752d6dedc3b1f29e8f8fb2ad1d0a36f377cf442" +checksum = "8b7cafe60d6cf8e62e1b9b2ea516a089c008945bb5a275416789e7db0bc199dc" dependencies = [ "memchr", - "thiserror 1.0.69", + "thiserror 2.0.6", "ucd-trie", ] [[package]] name = "pest_derive" -version = "2.7.14" +version = "2.7.15" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d214365f632b123a47fd913301e14c946c61d1c183ee245fa76eb752e59a02dd" +checksum = "816518421cfc6887a0d62bf441b6ffb4536fcc926395a69e1a85852d4363f57e" dependencies = [ "pest", "pest_generator", @@ -7581,9 +7598,9 @@ dependencies = [ [[package]] name = "pest_generator" -version = "2.7.14" +version = "2.7.15" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "eb55586734301717aea2ac313f50b2eb8f60d2fc3dc01d190eefa2e625f60c4e" +checksum = "7d1396fd3a870fc7838768d171b4616d5c91f6cc25e377b673d714567d99377b" dependencies = [ "pest", "pest_meta", @@ -7594,9 +7611,9 @@ dependencies = [ [[package]] name = "pest_meta" -version = "2.7.14" +version = "2.7.15" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b75da2a70cf4d9cb76833c990ac9cd3923c9a8905a8929789ce347c84564d03d" +checksum = "e1e58089ea25d717bfd31fb534e4f3afcc2cc569c70de3e239778991ea3b7dea" dependencies = [ "once_cell", "pest", @@ -7714,9 +7731,9 @@ dependencies = [ [[package]] name = "png" -version = "0.17.14" +version = "0.17.15" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "52f9d46a34a05a6a57566bc2bfae066ef07585a6e3fa30fbbdff5936380623f0" +checksum = "b67582bd5b65bdff614270e2ea89a1cf15bef71245cc1e5f7ea126977144211d" dependencies = [ "bitflags 1.3.2", "crc32fast", @@ -8177,7 +8194,11 @@ dependencies = [ "rustc-hash", "rustls 0.23.19", "socket2", +<<<<<<< HEAD "thiserror 2.0.4", +======= + "thiserror 2.0.6", +>>>>>>> e8bb86956 (feat: compiling CLI along with various fixes) "tokio", "tracing", ] @@ -8196,7 +8217,11 @@ dependencies = [ "rustls 0.23.19", "rustls-pki-types", "slab", +<<<<<<< HEAD "thiserror 2.0.4", +======= + "thiserror 2.0.6", +>>>>>>> e8bb86956 (feat: compiling CLI along with various fixes) "tinyvec", "tracing", "web-time", @@ -8204,9 +8229,9 @@ dependencies = [ [[package]] name = "quinn-udp" -version = "0.5.7" +version = "0.5.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7d5a626c6807713b15cac82a6acaccd6043c9a5408c24baae07611fec3f243da" +checksum = "52cd4b1eff68bf27940dd39811292c49e007f4d0b4c357358dc9b0197be6b527" dependencies = [ "cfg_aliases", "libc", @@ -8887,15 +8912,15 @@ dependencies = [ [[package]] name = "rustix" -version = "0.38.41" +version = "0.38.42" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d7f649912bc1495e167a6edee79151c84b1bad49748cb4f1f1167f459f6224f6" +checksum = "f93dc38ecbab2eb790ff964bb77fa94faf256fd3e73285fd7ba0903b76bedb85" dependencies = [ "bitflags 2.6.0", "errno", "libc", "linux-raw-sys", - "windows-sys 0.52.0", + "windows-sys 0.59.0", ] [[package]] @@ -9717,9 +9742,9 @@ dependencies = [ [[package]] name = "syn-solidity" -version = "0.8.14" +version = "0.8.15" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "da0523f59468a2696391f2a772edc089342aacd53c3caa2ac3264e598edf119b" +checksum = "219389c1ebe89f8333df8bdfb871f6631c552ff399c23cac02480b6088aad8f0" dependencies = [ "paste", "proc-macro2", @@ -9898,11 +9923,19 @@ dependencies = [ [[package]] name = "thiserror" +<<<<<<< HEAD version = "2.0.4" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "2f49a1853cf82743e3b7950f77e0f4d622ca36cf4317cba00c767838bac8d490" dependencies = [ "thiserror-impl 2.0.4", +======= +version = "2.0.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8fec2a1820ebd077e2b90c4df007bebf344cd394098a13c563957d0afc83ea47" +dependencies = [ + "thiserror-impl 2.0.6", +>>>>>>> e8bb86956 (feat: compiling CLI along with various fixes) ] [[package]] @@ -9918,9 +9951,15 @@ dependencies = [ [[package]] name = "thiserror-impl" +<<<<<<< HEAD version = "2.0.4" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "8381894bb3efe0c4acac3ded651301ceee58a15d47c2e34885ed1908ad667061" +======= +version = "2.0.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d65750cab40f4ff1929fb1ba509e9914eb756131cef4210da8d5d700d26f6312" +>>>>>>> e8bb86956 (feat: compiling CLI along with various fixes) dependencies = [ "proc-macro2", "quote", @@ -10868,9 +10907,9 @@ checksum = "9c8d87e72b64a3b4db28d11ce29237c246188f4f51057d65a7eab63b7987e423" [[package]] name = "wasm-bindgen" -version = "0.2.97" +version = "0.2.99" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d15e63b4482863c109d70a7b8706c1e364eb6ea449b201a76c5b89cedcec2d5c" +checksum = "a474f6281d1d70c17ae7aa6a613c87fce69a127e2624002df63dcb39d6cf6396" dependencies = [ "cfg-if", "once_cell", @@ -10879,13 +10918,12 @@ dependencies = [ [[package]] name = "wasm-bindgen-backend" -version = "0.2.97" +version = "0.2.99" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8d36ef12e3aaca16ddd3f67922bc63e48e953f126de60bd33ccc0101ef9998cd" +checksum = "5f89bb38646b4f81674e8f5c3fb81b562be1fd936d84320f3264486418519c79" dependencies = [ "bumpalo", "log", - "once_cell", "proc-macro2", "quote", "syn 2.0.90", @@ -10894,9 +10932,9 @@ dependencies = [ [[package]] name = "wasm-bindgen-futures" -version = "0.4.47" +version = "0.4.49" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9dfaf8f50e5f293737ee323940c7d8b08a66a95a419223d9f41610ca08b0833d" +checksum = "38176d9b44ea84e9184eff0bc34cc167ed044f816accfe5922e54d84cf48eca2" dependencies = [ "cfg-if", "js-sys", @@ -10907,9 +10945,9 @@ dependencies = [ [[package]] name = "wasm-bindgen-macro" -version = "0.2.97" +version = "0.2.99" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "705440e08b42d3e4b36de7d66c944be628d579796b8090bfa3471478a2260051" +checksum = "2cc6181fd9a7492eef6fef1f33961e3695e4579b9872a6f7c83aee556666d4fe" dependencies = [ "quote", "wasm-bindgen-macro-support", @@ -10917,9 +10955,9 @@ dependencies = [ [[package]] name = "wasm-bindgen-macro-support" -version = "0.2.97" +version = "0.2.99" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "98c9ae5a76e46f4deecd0f0255cc223cfa18dc9b261213b8aa0c7b36f61b3f1d" +checksum = "30d7a95b763d3c45903ed6c81f156801839e5ee968bb07e534c44df0fcd330c2" dependencies = [ "proc-macro2", "quote", @@ -10930,19 +10968,18 @@ dependencies = [ [[package]] name = "wasm-bindgen-shared" -version = "0.2.97" +version = "0.2.99" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6ee99da9c5ba11bd675621338ef6fa52296b76b83305e9b6e5c77d4c286d6d49" +checksum = "943aab3fdaaa029a6e0271b35ea10b72b943135afe9bffca82384098ad0e06a6" [[package]] name = "wasm-bindgen-test" -version = "0.3.47" +version = "0.3.49" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3d919bb60ebcecb9160afee6c71b43a58a4f0517a2de0054cd050d02cec08201" +checksum = "c61d44563646eb934577f2772656c7ad5e9c90fac78aa8013d776fcdaf24625d" dependencies = [ "js-sys", "minicov", - "once_cell", "scoped-tls", "wasm-bindgen", "wasm-bindgen-futures", @@ -10951,9 +10988,9 @@ dependencies = [ [[package]] name = "wasm-bindgen-test-macro" -version = "0.3.47" +version = "0.3.49" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "222ebde6ea87fbfa6bdd2e9f1fd8a91d60aee5db68792632176c4e16a74fc7d8" +checksum = "54171416ce73aa0b9c377b51cc3cb542becee1cd678204812e8392e5b0e4a031" dependencies = [ "proc-macro2", "quote", @@ -10991,9 +11028,9 @@ dependencies = [ [[package]] name = "web-sys" -version = "0.3.74" +version = "0.3.76" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a98bc3c33f0fe7e59ad7cd041b89034fa82a7c2d4365ca538dda6cdaf513863c" +checksum = "04dd7223427d52553d3702c004d3b2fe07c148165faa56313cb00211e31c12bc" dependencies = [ "js-sys", "wasm-bindgen", diff --git a/ant-cli/src/commands/register.rs b/ant-cli/src/commands/register.rs index 20d7f6ea20..17c30b2559 100644 --- a/ant-cli/src/commands/register.rs +++ b/ant-cli/src/commands/register.rs @@ -41,10 +41,9 @@ pub async fn cost(name: &str, peers: Vec) -> Result<()> { let register_key = crate::keys::get_register_signing_key() .wrap_err("The register key is required to perform this action")?; let client = crate::actions::connect_to_network(peers).await?; - let wallet = load_wallet()?; let cost = client - .register_cost(&wallet.network(), name.to_string(), register_key) + .register_cost(name.to_string(), register_key) .await .wrap_err("Failed to get cost for register")?; info!("Estimated cost to create a register with name {name}: {cost}"); diff --git a/ant-evm/src/data_payments.rs b/ant-evm/src/data_payments.rs index f11486dd0e..ddc1840998 100644 --- a/ant-evm/src/data_payments.rs +++ b/ant-evm/src/data_payments.rs @@ -30,15 +30,11 @@ const LIVE_TIME_MARGIN: u64 = 10; pub struct EncodedPeerId(Vec); impl EncodedPeerId { - pub fn to_peer_id(&self) -> Result { - match PublicKey::try_decode_protobuf(&self.0) { - Ok(pub_key) => Ok(PeerId::from_public_key(&pub_key)), - Err(e) => Err(e), - } + pub fn to_peer_id(&self) -> Result { + PeerId::from_bytes(&self.0) } } -// TODO: @anselme is this conversion right? impl From for EncodedPeerId { fn from(peer_id: PeerId) -> Self { let bytes = peer_id.to_bytes(); @@ -322,6 +318,14 @@ mod tests { use libp2p::identity::Keypair; use std::{thread::sleep, time::Duration}; + #[test] + fn test_encode_decode_peer_id() { + let id = PeerId::random(); + let encoded = EncodedPeerId::from(id); + let decoded = encoded.to_peer_id().expect("decode to work"); + assert_eq!(id, decoded); + } + #[test] fn test_is_newer_than() { let old_quote = PaymentQuote::zero(); diff --git a/autonomi/src/client/external_signer.rs b/autonomi/src/client/external_signer.rs index d3b7ede67d..8c3d6969f6 100644 --- a/autonomi/src/client/external_signer.rs +++ b/autonomi/src/client/external_signer.rs @@ -1,40 +1,41 @@ use crate::client::data::PutError; -use crate::client::utils::extract_quote_payments; use crate::self_encryption::encrypt; use crate::Client; -use ant_evm::{PaymentQuote, QuotePayment}; +use ant_evm::QuotePayment; use ant_protocol::storage::Chunk; use bytes::Bytes; use std::collections::HashMap; use xor_name::XorName; -use crate::utils::cost_map_to_quotes; #[allow(unused_imports)] pub use ant_evm::external_signer::*; +use super::quote::QuoteForAddress; + impl Client { /// Get quotes for data. /// Returns a cost map, data payments to be executed and a list of free (already paid for) chunks. pub async fn get_quotes_for_content_addresses( &self, - content_addrs: impl Iterator, + content_addrs: impl Iterator + Clone, ) -> Result< ( - HashMap, + HashMap, Vec, Vec, ), PutError, > { - let cost_map = self.get_store_quotes(content_addrs).await?; - let (quote_payments, free_chunks) = extract_quote_payments(&cost_map); - let quotes = cost_map_to_quotes(cost_map); + let quote = self.get_store_quotes(content_addrs.clone()).await?; + let payments = quote.payments(); + let free_chunks = content_addrs.filter(|addr| !quote.0.contains_key(addr)).collect(); + let quotes_per_addr = quote.0.into_iter().collect(); debug!( "Got the quotes , quote_payments and freechunks from the network {:?}", - (quotes.clone(), quote_payments.clone(), free_chunks.clone()) + (quotes_per_addr.clone(), payments.clone(), free_chunks.clone()) ); - Ok((quotes, quote_payments, free_chunks)) + Ok((quotes_per_addr, payments, free_chunks)) } } diff --git a/autonomi/src/client/quote.rs b/autonomi/src/client/quote.rs index 514cf9c4d5..3f48587552 100644 --- a/autonomi/src/client/quote.rs +++ b/autonomi/src/client/quote.rs @@ -29,13 +29,17 @@ pub struct StoreQuote(pub(crate) HashMap); impl StoreQuote { pub fn price(&self) -> Amount { - self.0.iter().map(|(_, quote)| quote.price()).sum() + self.0.values().map(|quote| quote.price()).sum() } pub fn len(&self) -> usize { self.0.len() } + pub fn is_empty(&self) -> bool { + self.0.is_empty() + } + pub fn payments(&self) -> Vec { let mut quote_payments = vec![]; for (_address, quote) in self.0.iter() { diff --git a/autonomi/src/client/vault.rs b/autonomi/src/client/vault.rs index 4633004aa8..dd69f8f9d7 100644 --- a/autonomi/src/client/vault.rs +++ b/autonomi/src/client/vault.rs @@ -204,7 +204,7 @@ impl Client { None => return Err(PutError::PaymentUnexpectedlyInvalid(scratch_address)), }; - total_cost = price.clone(); + total_cost = *price; Record { key: scratch_key, diff --git a/autonomi/src/lib.rs b/autonomi/src/lib.rs index 7dd7aeb1a8..f612146f1d 100644 --- a/autonomi/src/lib.rs +++ b/autonomi/src/lib.rs @@ -69,8 +69,6 @@ pub use ant_evm::get_evm_network_from_env; pub use ant_evm::EvmNetwork as Network; pub use ant_evm::EvmWallet as Wallet; pub use ant_evm::RewardsAddress; -#[cfg(feature = "external-signer")] -pub use utils::receipt_from_quotes_and_payments; #[doc(no_inline)] // Place this under 'Re-exports' in the docs. pub use bytes::Bytes; diff --git a/autonomi/tests/external_signer.rs b/autonomi/tests/external_signer.rs index 58722c5d45..ff709a3bcc 100644 --- a/autonomi/tests/external_signer.rs +++ b/autonomi/tests/external_signer.rs @@ -1,7 +1,5 @@ #![cfg(feature = "external-signer")] -use alloy::network::TransactionBuilder; -use alloy::providers::Provider; use ant_evm::{QuoteHash, TxHash}; use ant_logging::LogBuilder; use autonomi::client::external_signer::encrypt_data; @@ -9,7 +7,7 @@ use autonomi::client::files::archive::{Metadata, PrivateArchive}; use autonomi::client::payment::Receipt; use autonomi::client::vault::user_data::USER_DATA_VAULT_CONTENT_IDENTIFIER; use autonomi::client::vault::VaultSecretKey; -use autonomi::{receipt_from_quotes_and_payments, Client, Wallet}; +use autonomi::{Client, Wallet}; use bytes::Bytes; use std::collections::BTreeMap; use std::time::Duration; @@ -34,7 +32,7 @@ async fn pay_for_data(client: &Client, wallet: &Wallet, data: Bytes) -> eyre::Re async fn pay_for_content_addresses( client: &Client, wallet: &Wallet, - content_addrs: impl Iterator, + content_addrs: impl Iterator + Clone, ) -> eyre::Result { let (quotes, quote_payments, _free_chunks) = client .get_quotes_for_content_addresses(content_addrs) diff --git a/evmlib/tests/wallet.rs b/evmlib/tests/wallet.rs index cab48d254b..f6ac01e3a4 100644 --- a/evmlib/tests/wallet.rs +++ b/evmlib/tests/wallet.rs @@ -8,10 +8,9 @@ use alloy::providers::ext::AnvilApi; use alloy::providers::{ProviderBuilder, WalletProvider}; use alloy::signers::local::{LocalSigner, PrivateKeySigner}; use evmlib::common::{Amount, TxHash}; -use evmlib::contract::payment_vault::MAX_TRANSFERS_PER_TRANSACTION; +use evmlib::contract::payment_vault::{verify_data_payment, MAX_TRANSFERS_PER_TRANSACTION}; use evmlib::quoting_metrics::QuotingMetrics; use evmlib::testnet::{deploy_data_payments_contract, deploy_network_token_contract, start_node}; -use evmlib::transaction::verify_data_payment; use evmlib::wallet::{transfer_tokens, wallet_address, Wallet}; use evmlib::{CustomNetwork, Network}; use std::collections::HashSet; @@ -90,7 +89,12 @@ async fn test_pay_for_quotes_and_data_payment_verification() { for (quote_hash, reward_addr, _) in quote_payments.iter() { let result = verify_data_payment( &network, - vec![(*quote_hash, QuotingMetrics::default(), *reward_addr)] + vec![*quote_hash], + vec![( + *quote_hash, + QuotingMetrics::default(), + *reward_addr, + )], ) .await; From d1fa2a2580fa01f7acca8f38836f92956d26dc06 Mon Sep 17 00:00:00 2001 From: Warm Beer Date: Tue, 10 Dec 2024 15:00:43 +0100 Subject: [PATCH 16/33] chore: update payment vault interface --- autonomi/Cargo.toml | 2 +- autonomi/tests/external_signer.rs | 6 +- evmlib/abi/IPaymentVault.json | 147 +++++--- evmlib/artifacts/DataPayments.json | 90 ----- evmlib/artifacts/PaymentVaultNoProxy.json | 351 ++++++++++++++++++ evmlib/src/contract/payment_vault/error.rs | 2 + evmlib/src/contract/payment_vault/handler.rs | 35 +- .../contract/payment_vault/implementation.rs | 37 +- .../src/contract/payment_vault/interface.rs | 14 +- evmlib/src/contract/payment_vault/mod.rs | 29 +- evmlib/src/testnet.rs | 10 +- evmlib/tests/common/quote.rs | 2 +- evmlib/tests/payment_vault.rs | 58 ++- 13 files changed, 569 insertions(+), 214 deletions(-) delete mode 100644 evmlib/artifacts/DataPayments.json create mode 100644 evmlib/artifacts/PaymentVaultNoProxy.json diff --git a/autonomi/Cargo.toml b/autonomi/Cargo.toml index 941cc9748e..f4db2fa04d 100644 --- a/autonomi/Cargo.toml +++ b/autonomi/Cargo.toml @@ -59,7 +59,7 @@ wasm-bindgen-futures = "0.4.43" xor_name = "5.0.0" [dev-dependencies] -alloy = { version = "0.5.3", default-features = false, features = ["std", "reqwest-rustls-tls", "provider-anvil-node", "sol-types", "json", "signers", "contract", "signer-local", "network"] } +alloy = { version = "0.7.3", default-features = false, features = ["contract", "json-rpc", "network", "node-bindings", "provider-http", "reqwest-rustls-tls", "rpc-client", "rpc-types", "signer-local", "std"] } ant-logging = { path = "../ant-logging", version = "0.2.40" } eyre = "0.6.5" sha2 = "0.10.6" diff --git a/autonomi/tests/external_signer.rs b/autonomi/tests/external_signer.rs index ff709a3bcc..fa648e8461 100644 --- a/autonomi/tests/external_signer.rs +++ b/autonomi/tests/external_signer.rs @@ -1,10 +1,12 @@ #![cfg(feature = "external-signer")] +use alloy::network::TransactionBuilder; +use alloy::providers::Provider; use ant_evm::{QuoteHash, TxHash}; use ant_logging::LogBuilder; use autonomi::client::external_signer::encrypt_data; use autonomi::client::files::archive::{Metadata, PrivateArchive}; -use autonomi::client::payment::Receipt; +use autonomi::client::payment::{receipt_from_store_quotes_and_payments, Receipt}; use autonomi::client::vault::user_data::USER_DATA_VAULT_CONTENT_IDENTIFIER; use autonomi::client::vault::VaultSecretKey; use autonomi::{Client, Wallet}; @@ -91,7 +93,7 @@ async fn pay_for_content_addresses( } // Payment proofs - Ok(receipt_from_quotes_and_payments("es, &payments)) + Ok(receipt_from_store_quotes_and_payments("es, payments)) } // Example of how put would be done using external signers. diff --git a/evmlib/abi/IPaymentVault.json b/evmlib/abi/IPaymentVault.json index 48f3303a77..d2bc495a5f 100644 --- a/evmlib/abi/IPaymentVault.json +++ b/evmlib/abi/IPaymentVault.json @@ -1,4 +1,19 @@ [ + { + "inputs": [], + "name": "AntTokenNull", + "type": "error" + }, + { + "inputs": [], + "name": "BatchLimitExceeded", + "type": "error" + }, + { + "inputs": [], + "name": "InvalidInputLength", + "type": "error" + }, { "anonymous": false, "inputs": [ @@ -24,16 +39,6 @@ "name": "DataPaymentMade", "type": "event" }, - { - "inputs": [], - "name": "AntTokenNull", - "type": "error" - }, - { - "inputs": [], - "name": "BatchLimitExceeded", - "type": "error" - }, { "inputs": [ { @@ -120,72 +125,96 @@ { "components": [ { - "internalType": "uint256", - "name": "closeRecordsStored", - "type": "uint256" - }, - { - "internalType": "uint256", - "name": "maxRecords", - "type": "uint256" - }, - { - "internalType": "uint256", - "name": "receivedPaymentCount", - "type": "uint256" - }, - { - "internalType": "uint256", - "name": "liveTime", - "type": "uint256" - }, - { - "internalType": "uint256", - "name": "networkDensity", - "type": "uint256" + "components": [ + { + "internalType": "uint256", + "name": "closeRecordsStored", + "type": "uint256" + }, + { + "internalType": "uint256", + "name": "maxRecords", + "type": "uint256" + }, + { + "internalType": "uint256", + "name": "receivedPaymentCount", + "type": "uint256" + }, + { + "internalType": "uint256", + "name": "liveTime", + "type": "uint256" + }, + { + "internalType": "uint256", + "name": "networkDensity", + "type": "uint256" + }, + { + "internalType": "uint256", + "name": "networkSize", + "type": "uint256" + } + ], + "internalType": "struct IPaymentVault.QuotingMetrics", + "name": "metrics", + "type": "tuple" }, { - "internalType": "uint256", - "name": "networkSize", - "type": "uint256" + "components": [ + { + "internalType": "address", + "name": "rewardsAddress", + "type": "address" + }, + { + "internalType": "uint256", + "name": "amount", + "type": "uint256" + }, + { + "internalType": "bytes32", + "name": "quoteHash", + "type": "bytes32" + } + ], + "internalType": "struct IPaymentVault.DataPayment", + "name": "dataPayment", + "type": "tuple" } ], - "internalType": "struct IPaymentVault.QuotingMetrics", - "name": "_metrics", - "type": "tuple" - }, + "internalType": "struct IPaymentVault.PaymentVerification[]", + "name": "_payments", + "type": "tuple[]" + } + ], + "name": "verifyPayment", + "outputs": [ { "components": [ { - "internalType": "address", - "name": "rewardsAddress", - "type": "address" + "internalType": "bytes32", + "name": "quoteHash", + "type": "bytes32" }, { "internalType": "uint256", - "name": "amount", + "name": "amountPaid", "type": "uint256" }, { - "internalType": "bytes32", - "name": "quoteHash", - "type": "bytes32" + "internalType": "bool", + "name": "isValid", + "type": "bool" } ], - "internalType": "struct IPaymentVault.DataPayment", - "name": "_payment", - "type": "tuple" - } - ], - "name": "verifyPayment", - "outputs": [ - { - "internalType": "bool", - "name": "isValid", - "type": "bool" + "internalType": "struct IPaymentVault.PaymentVerificationResult[3]", + "name": "verificationResults", + "type": "tuple[3]" } ], "stateMutability": "view", "type": "function" } -] +] \ No newline at end of file diff --git a/evmlib/artifacts/DataPayments.json b/evmlib/artifacts/DataPayments.json deleted file mode 100644 index a72afa0b8b..0000000000 --- a/evmlib/artifacts/DataPayments.json +++ /dev/null @@ -1,90 +0,0 @@ -{ - "_format": "hh-sol-artifact-1", - "contractName": "DataPayments", - "sourceName": "contracts/DataPayments.sol", - "abi": [ - { - "inputs": [ - { - "internalType": "address", - "name": "_tokenAddress", - "type": "address" - } - ], - "stateMutability": "nonpayable", - "type": "constructor" - }, - { - "anonymous": false, - "inputs": [ - { - "indexed": true, - "internalType": "address", - "name": "rewardsAddress", - "type": "address" - }, - { - "indexed": true, - "internalType": "uint256", - "name": "amount", - "type": "uint256" - }, - { - "indexed": true, - "internalType": "bytes32", - "name": "quoteHash", - "type": "bytes32" - } - ], - "name": "DataPaymentMade", - "type": "event" - }, - { - "inputs": [], - "name": "PAYMENT_TOKEN_ADDRESS", - "outputs": [ - { - "internalType": "address", - "name": "", - "type": "address" - } - ], - "stateMutability": "view", - "type": "function" - }, - { - "inputs": [ - { - "components": [ - { - "internalType": "address", - "name": "rewardsAddress", - "type": "address" - }, - { - "internalType": "uint256", - "name": "amount", - "type": "uint256" - }, - { - "internalType": "bytes32", - "name": "quoteHash", - "type": "bytes32" - } - ], - "internalType": "struct DataPayments.DataPayment[]", - "name": "dataPayments", - "type": "tuple[]" - } - ], - "name": "submitDataPayments", - "outputs": [], - "stateMutability": "nonpayable", - "type": "function" - } - ], - "bytecode": "0x60a060405234801561001057600080fd5b506040516105f73803806105f783398101604081905261002f916100a6565b6001600160a01b0381166100955760405162461bcd60e51b8152602060048201526024808201527f546f6b656e20616464726573732063616e6e6f74206265207a65726f206164646044820152637265737360e01b606482015260840160405180910390fd5b6001600160a01b03166080526100d6565b6000602082840312156100b857600080fd5b81516001600160a01b03811681146100cf57600080fd5b9392505050565b6080516104f26101056000396000818160400152818161015101528181610253015261035301526104f26000f3fe608060405234801561001057600080fd5b50600436106100365760003560e01c80635c0d32861461003b578063dee1dfa01461007e575b600080fd5b6100627f000000000000000000000000000000000000000000000000000000000000000081565b6040516001600160a01b03909116815260200160405180910390f35b61009161008c3660046103c6565b610093565b005b60005b8181101561012b57368383838181106100b1576100b161043b565b6060029190910191506100d79050336100cd6020840184610451565b8360200135610130565b6040810135602082018035906100ed9084610451565b6001600160a01b03167ff998960b1c6f0e0e89b7bbe6b6fbf3e03e6f08eee5b8430877d8adb8e149d58060405160405180910390a450600101610096565b505050565b6040516370a0823160e01b81526001600160a01b03848116600483015282917f0000000000000000000000000000000000000000000000000000000000000000909116906370a0823190602401602060405180830381865afa15801561019a573d6000803e3d6000fd5b505050506040513d601f19601f820116820180604052508101906101be9190610481565b101561021c5760405162461bcd60e51b815260206004820152602260248201527f57616c6c657420646f6573206e6f74206861766520656e6f75676820746f6b656044820152616e7360f01b60648201526084015b60405180910390fd5b6001600160a01b038316301461032557604051636eb1769f60e11b81526001600160a01b03848116600483015230602483015282917f00000000000000000000000000000000000000000000000000000000000000009091169063dd62ed3e90604401602060405180830381865afa15801561029c573d6000803e3d6000fd5b505050506040513d601f19601f820116820180604052508101906102c09190610481565b10156103255760405162461bcd60e51b815260206004820152602e60248201527f436f6e7472616374206973206e6f7420616c6c6f77656420746f207370656e6460448201526d20656e6f75676820746f6b656e7360901b6064820152608401610213565b6040516323b872dd60e01b81526001600160a01b0384811660048301528381166024830152604482018390527f000000000000000000000000000000000000000000000000000000000000000016906323b872dd906064016020604051808303816000875af115801561039c573d6000803e3d6000fd5b505050506040513d601f19601f820116820180604052508101906103c0919061049a565b50505050565b600080602083850312156103d957600080fd5b823567ffffffffffffffff808211156103f157600080fd5b818501915085601f83011261040557600080fd5b81358181111561041457600080fd5b86602060608302850101111561042957600080fd5b60209290920196919550909350505050565b634e487b7160e01b600052603260045260246000fd5b60006020828403121561046357600080fd5b81356001600160a01b038116811461047a57600080fd5b9392505050565b60006020828403121561049357600080fd5b5051919050565b6000602082840312156104ac57600080fd5b8151801515811461047a57600080fdfea26469706673582212206f3a305284dc687832455d7d49b202dcf22b32d76aff5ccd14c3c8539596bcf464736f6c63430008180033", - "deployedBytecode": "0x608060405234801561001057600080fd5b50600436106100365760003560e01c80635c0d32861461003b578063dee1dfa01461007e575b600080fd5b6100627f000000000000000000000000000000000000000000000000000000000000000081565b6040516001600160a01b03909116815260200160405180910390f35b61009161008c3660046103c6565b610093565b005b60005b8181101561012b57368383838181106100b1576100b161043b565b6060029190910191506100d79050336100cd6020840184610451565b8360200135610130565b6040810135602082018035906100ed9084610451565b6001600160a01b03167ff998960b1c6f0e0e89b7bbe6b6fbf3e03e6f08eee5b8430877d8adb8e149d58060405160405180910390a450600101610096565b505050565b6040516370a0823160e01b81526001600160a01b03848116600483015282917f0000000000000000000000000000000000000000000000000000000000000000909116906370a0823190602401602060405180830381865afa15801561019a573d6000803e3d6000fd5b505050506040513d601f19601f820116820180604052508101906101be9190610481565b101561021c5760405162461bcd60e51b815260206004820152602260248201527f57616c6c657420646f6573206e6f74206861766520656e6f75676820746f6b656044820152616e7360f01b60648201526084015b60405180910390fd5b6001600160a01b038316301461032557604051636eb1769f60e11b81526001600160a01b03848116600483015230602483015282917f00000000000000000000000000000000000000000000000000000000000000009091169063dd62ed3e90604401602060405180830381865afa15801561029c573d6000803e3d6000fd5b505050506040513d601f19601f820116820180604052508101906102c09190610481565b10156103255760405162461bcd60e51b815260206004820152602e60248201527f436f6e7472616374206973206e6f7420616c6c6f77656420746f207370656e6460448201526d20656e6f75676820746f6b656e7360901b6064820152608401610213565b6040516323b872dd60e01b81526001600160a01b0384811660048301528381166024830152604482018390527f000000000000000000000000000000000000000000000000000000000000000016906323b872dd906064016020604051808303816000875af115801561039c573d6000803e3d6000fd5b505050506040513d601f19601f820116820180604052508101906103c0919061049a565b50505050565b600080602083850312156103d957600080fd5b823567ffffffffffffffff808211156103f157600080fd5b818501915085601f83011261040557600080fd5b81358181111561041457600080fd5b86602060608302850101111561042957600080fd5b60209290920196919550909350505050565b634e487b7160e01b600052603260045260246000fd5b60006020828403121561046357600080fd5b81356001600160a01b038116811461047a57600080fd5b9392505050565b60006020828403121561049357600080fd5b5051919050565b6000602082840312156104ac57600080fd5b8151801515811461047a57600080fdfea26469706673582212206f3a305284dc687832455d7d49b202dcf22b32d76aff5ccd14c3c8539596bcf464736f6c63430008180033", - "linkReferences": {}, - "deployedLinkReferences": {} -} \ No newline at end of file diff --git a/evmlib/artifacts/PaymentVaultNoProxy.json b/evmlib/artifacts/PaymentVaultNoProxy.json new file mode 100644 index 0000000000..5514cc77f7 --- /dev/null +++ b/evmlib/artifacts/PaymentVaultNoProxy.json @@ -0,0 +1,351 @@ +{ + "_format": "hh-sol-artifact-1", + "contractName": "PaymentVaultNoProxy", + "sourceName": "contracts/PaymentVaultNoProxy.sol", + "abi": [ + { + "inputs": [ + { + "internalType": "contract IERC20", + "name": "_antToken", + "type": "address" + }, + { + "internalType": "uint256", + "name": "_batchLimit", + "type": "uint256" + } + ], + "stateMutability": "nonpayable", + "type": "constructor" + }, + { + "inputs": [ + { + "internalType": "address", + "name": "target", + "type": "address" + } + ], + "name": "AddressEmptyCode", + "type": "error" + }, + { + "inputs": [ + { + "internalType": "address", + "name": "account", + "type": "address" + } + ], + "name": "AddressInsufficientBalance", + "type": "error" + }, + { + "inputs": [], + "name": "AntTokenNull", + "type": "error" + }, + { + "inputs": [], + "name": "BatchLimitExceeded", + "type": "error" + }, + { + "inputs": [], + "name": "FailedInnerCall", + "type": "error" + }, + { + "inputs": [], + "name": "InvalidInputLength", + "type": "error" + }, + { + "inputs": [ + { + "internalType": "address", + "name": "token", + "type": "address" + } + ], + "name": "SafeERC20FailedOperation", + "type": "error" + }, + { + "anonymous": false, + "inputs": [ + { + "indexed": true, + "internalType": "address", + "name": "rewardsAddress", + "type": "address" + }, + { + "indexed": true, + "internalType": "uint256", + "name": "amount", + "type": "uint256" + }, + { + "indexed": true, + "internalType": "bytes32", + "name": "quoteHash", + "type": "bytes32" + } + ], + "name": "DataPaymentMade", + "type": "event" + }, + { + "inputs": [], + "name": "antToken", + "outputs": [ + { + "internalType": "contract IERC20", + "name": "", + "type": "address" + } + ], + "stateMutability": "view", + "type": "function" + }, + { + "inputs": [], + "name": "batchLimit", + "outputs": [ + { + "internalType": "uint256", + "name": "", + "type": "uint256" + } + ], + "stateMutability": "view", + "type": "function" + }, + { + "inputs": [ + { + "components": [ + { + "internalType": "uint256", + "name": "closeRecordsStored", + "type": "uint256" + }, + { + "internalType": "uint256", + "name": "maxRecords", + "type": "uint256" + }, + { + "internalType": "uint256", + "name": "receivedPaymentCount", + "type": "uint256" + }, + { + "internalType": "uint256", + "name": "liveTime", + "type": "uint256" + }, + { + "internalType": "uint256", + "name": "networkDensity", + "type": "uint256" + }, + { + "internalType": "uint256", + "name": "networkSize", + "type": "uint256" + } + ], + "internalType": "struct IPaymentVault.QuotingMetrics", + "name": "", + "type": "tuple" + } + ], + "name": "getQuote", + "outputs": [ + { + "internalType": "uint256", + "name": "price", + "type": "uint256" + } + ], + "stateMutability": "pure", + "type": "function" + }, + { + "inputs": [ + { + "components": [ + { + "internalType": "address", + "name": "rewardsAddress", + "type": "address" + }, + { + "internalType": "uint256", + "name": "amount", + "type": "uint256" + }, + { + "internalType": "bytes32", + "name": "quoteHash", + "type": "bytes32" + } + ], + "internalType": "struct IPaymentVault.DataPayment[]", + "name": "_payments", + "type": "tuple[]" + } + ], + "name": "payForQuotes", + "outputs": [], + "stateMutability": "nonpayable", + "type": "function" + }, + { + "inputs": [ + { + "internalType": "bytes32", + "name": "", + "type": "bytes32" + } + ], + "name": "payments", + "outputs": [ + { + "internalType": "address", + "name": "rewardsAddress", + "type": "address" + }, + { + "internalType": "uint256", + "name": "amount", + "type": "uint256" + }, + { + "internalType": "bytes32", + "name": "quoteHash", + "type": "bytes32" + } + ], + "stateMutability": "view", + "type": "function" + }, + { + "inputs": [], + "name": "requiredPaymentVerificationLength", + "outputs": [ + { + "internalType": "uint256", + "name": "", + "type": "uint256" + } + ], + "stateMutability": "view", + "type": "function" + }, + { + "inputs": [ + { + "components": [ + { + "components": [ + { + "internalType": "uint256", + "name": "closeRecordsStored", + "type": "uint256" + }, + { + "internalType": "uint256", + "name": "maxRecords", + "type": "uint256" + }, + { + "internalType": "uint256", + "name": "receivedPaymentCount", + "type": "uint256" + }, + { + "internalType": "uint256", + "name": "liveTime", + "type": "uint256" + }, + { + "internalType": "uint256", + "name": "networkDensity", + "type": "uint256" + }, + { + "internalType": "uint256", + "name": "networkSize", + "type": "uint256" + } + ], + "internalType": "struct IPaymentVault.QuotingMetrics", + "name": "metrics", + "type": "tuple" + }, + { + "components": [ + { + "internalType": "address", + "name": "rewardsAddress", + "type": "address" + }, + { + "internalType": "uint256", + "name": "amount", + "type": "uint256" + }, + { + "internalType": "bytes32", + "name": "quoteHash", + "type": "bytes32" + } + ], + "internalType": "struct IPaymentVault.DataPayment", + "name": "dataPayment", + "type": "tuple" + } + ], + "internalType": "struct IPaymentVault.PaymentVerification[]", + "name": "_payments", + "type": "tuple[]" + } + ], + "name": "verifyPayment", + "outputs": [ + { + "components": [ + { + "internalType": "bytes32", + "name": "quoteHash", + "type": "bytes32" + }, + { + "internalType": "uint256", + "name": "amountPaid", + "type": "uint256" + }, + { + "internalType": "bool", + "name": "isValid", + "type": "bool" + } + ], + "internalType": "struct IPaymentVault.PaymentVerificationResult[3]", + "name": "verificationResults", + "type": "tuple[3]" + } + ], + "stateMutability": "view", + "type": "function" + } + ], + "bytecode": "0x6080604052348015600f57600080fd5b50604051610cce380380610cce833981016040819052602c91607f565b6001600160a01b038216605257604051632d06160b60e21b815260040160405180910390fd5b600180546001600160a01b0319166001600160a01b039390931692909217909155600055600560035560b7565b60008060408385031215609157600080fd5b82516001600160a01b038116811460a757600080fd5b6020939093015192949293505050565b610c08806100c66000396000f3fe608060405234801561001057600080fd5b506004361061007d5760003560e01c80634ec42e8e1161005b5780634ec42e8e14610111578063b6c2141b1461013c578063bcb2c1da14610151578063c7170bb61461017157600080fd5b80630716326d146100825780633c150bf2146100e6578063474740b114610108575b600080fd5b6100bc61009036600461082f565b60026020819052600091825260409091208054600182015491909201546001600160a01b039092169183565b604080516001600160a01b0390941684526020840192909252908201526060015b60405180910390f35b6100fa6100f4366004610848565b50600190565b6040519081526020016100dd565b6100fa60005481565b600154610124906001600160a01b031681565b6040516001600160a01b0390911681526020016100dd565b61014f61014a366004610863565b61017a565b005b61016461015f3660046108da565b61026a565b6040516100dd9190610942565b6100fa60035481565b600054819081111561019f57604051630d67f41160e21b815260040160405180910390fd5b60005b8181101561026457368484838181106101bd576101bd610992565b6060029190910191506101f19050336101d960208401846109bd565b6001546001600160a01b0316919060208501356103f5565b6040808201356000908152600260205220819061020e82826109da565b505060408101356020820180359061022690846109bd565b6001600160a01b03167ff998960b1c6f0e0e89b7bbe6b6fbf3e03e6f08eee5b8430877d8adb8e149d58060405160405180910390a4506001016101a2565b50505050565b61027261075c565b600354821461029457604051637db491eb60e01b815260040160405180910390fd5b60006102a0848461044f565b905060005b60038110156103ed576000600260008484600381106102c6576102c6610992565b602090810291909101518101516040908101518352828201939093529082016000908120835160608101855281546001600160a01b031681526001820154938101939093526002015492820192909252915083836003811061032a5761032a610992565b602002015160200151602001518260200151149050600084846003811061035357610353610992565b602002015160200151600001516001600160a01b031683600001516001600160a01b03161490506000604051806060016040528087876003811061039957610399610992565b602002015160200151604001518152602001856020015181526020018480156103bf5750835b151590529050808786600381106103d8576103d8610992565b60200201525050600190920191506102a59050565b505092915050565b604080516001600160a01b0385811660248301528416604482015260648082018490528251808303909101815260849091019091526020810180516001600160e01b03166323b872dd60e01b1790526102649085906105b5565b61045761079b565b60005b828110156105ae578151602090810151015184848381811061047e5761047e610992565b9050610120020160c0016020013511156104d8576020820180516040840152825190528383828181106104b3576104b3610992565b905061012002018036038101906104ca9190610af1565b8260005b60200201526105a6565b60208083015181015101518484838181106104f5576104f5610992565b9050610120020160c001602001351115610544576020820151604083015283838281811061052557610525610992565b9050610120020180360381019061053c9190610af1565b8260016104ce565b6040820151602090810151015184848381811061056357610563610992565b9050610120020160c0016020013511156105a65783838281811061058957610589610992565b905061012002018036038101906105a09190610af1565b60408301525b60010161045a565b5092915050565b60006105ca6001600160a01b03841683610622565b905080516000141580156105ef5750808060200190518101906105ed9190610b81565b155b1561061d57604051635274afe760e01b81526001600160a01b03841660048201526024015b60405180910390fd5b505050565b606061063083836000610637565b9392505050565b60608147101561065c5760405163cd78605960e01b8152306004820152602401610614565b600080856001600160a01b031684866040516106789190610ba3565b60006040518083038185875af1925050503d80600081146106b5576040519150601f19603f3d011682016040523d82523d6000602084013e6106ba565b606091505b50915091506106ca8683836106d4565b9695505050505050565b6060826106e9576106e482610730565b610630565b815115801561070057506001600160a01b0384163b155b1561072957604051639996b31560e01b81526001600160a01b0385166004820152602401610614565b5080610630565b8051156107405780518082602001fd5b604051630a12f52160e11b815260040160405180910390fd5b50565b60405180606001604052806003905b604080516060810182526000808252602080830182905292820152825260001990920191018161076b5790505090565b60405180606001604052806003905b6107b26107c8565b8152602001906001900390816107aa5790505090565b604051806040016040528061080c6040518060c001604052806000815260200160008152602001600081526020016000815260200160008152602001600081525090565b815260408051606081018252600080825260208281018290529282015291015290565b60006020828403121561084157600080fd5b5035919050565b600060c082840312801561085b57600080fd5b509092915050565b6000806020838503121561087657600080fd5b823567ffffffffffffffff81111561088d57600080fd5b8301601f8101851361089e57600080fd5b803567ffffffffffffffff8111156108b557600080fd5b8560206060830284010111156108ca57600080fd5b6020919091019590945092505050565b600080602083850312156108ed57600080fd5b823567ffffffffffffffff81111561090457600080fd5b8301601f8101851361091557600080fd5b803567ffffffffffffffff81111561092c57600080fd5b856020610120830284010111156108ca57600080fd5b6101208101818360005b600381101561098957815180518452602081015160208501526040810151151560408501525060608301925060208201915060018101905061094c565b50505092915050565b634e487b7160e01b600052603260045260246000fd5b6001600160a01b038116811461075957600080fd5b6000602082840312156109cf57600080fd5b8135610630816109a8565b81356109e5816109a8565b81546001600160a01b0319166001600160a01b039190911617815560208201356001820155604090910135600290910155565b6040805190810167ffffffffffffffff81118282101715610a4957634e487b7160e01b600052604160045260246000fd5b60405290565b60405160c0810167ffffffffffffffff81118282101715610a4957634e487b7160e01b600052604160045260246000fd5b600060608284031215610a9257600080fd5b6040516060810167ffffffffffffffff81118282101715610ac357634e487b7160e01b600052604160045260246000fd5b6040529050808235610ad4816109a8565b815260208381013590820152604092830135920191909152919050565b600081830361012081128015610b0657600080fd5b506000610b11610a18565b60c0831215610b1e578182fd5b610b26610a4f565b853581526020808701359082015260408087013590820152606080870135908201526080808701359082015260a0808701359082018190528183529093509150610b738660c08701610a80565b602082015295945050505050565b600060208284031215610b9357600080fd5b8151801515811461063057600080fd5b6000825160005b81811015610bc45760208186018101518583015201610baa565b50600092019182525091905056fea2646970667358221220fd6ef361aaba52d0f9503b51aea1d0b7a8363a9a66c9502aa7b931f1f44c507f64736f6c634300081c0033", + "deployedBytecode": "0x608060405234801561001057600080fd5b506004361061007d5760003560e01c80634ec42e8e1161005b5780634ec42e8e14610111578063b6c2141b1461013c578063bcb2c1da14610151578063c7170bb61461017157600080fd5b80630716326d146100825780633c150bf2146100e6578063474740b114610108575b600080fd5b6100bc61009036600461082f565b60026020819052600091825260409091208054600182015491909201546001600160a01b039092169183565b604080516001600160a01b0390941684526020840192909252908201526060015b60405180910390f35b6100fa6100f4366004610848565b50600190565b6040519081526020016100dd565b6100fa60005481565b600154610124906001600160a01b031681565b6040516001600160a01b0390911681526020016100dd565b61014f61014a366004610863565b61017a565b005b61016461015f3660046108da565b61026a565b6040516100dd9190610942565b6100fa60035481565b600054819081111561019f57604051630d67f41160e21b815260040160405180910390fd5b60005b8181101561026457368484838181106101bd576101bd610992565b6060029190910191506101f19050336101d960208401846109bd565b6001546001600160a01b0316919060208501356103f5565b6040808201356000908152600260205220819061020e82826109da565b505060408101356020820180359061022690846109bd565b6001600160a01b03167ff998960b1c6f0e0e89b7bbe6b6fbf3e03e6f08eee5b8430877d8adb8e149d58060405160405180910390a4506001016101a2565b50505050565b61027261075c565b600354821461029457604051637db491eb60e01b815260040160405180910390fd5b60006102a0848461044f565b905060005b60038110156103ed576000600260008484600381106102c6576102c6610992565b602090810291909101518101516040908101518352828201939093529082016000908120835160608101855281546001600160a01b031681526001820154938101939093526002015492820192909252915083836003811061032a5761032a610992565b602002015160200151602001518260200151149050600084846003811061035357610353610992565b602002015160200151600001516001600160a01b031683600001516001600160a01b03161490506000604051806060016040528087876003811061039957610399610992565b602002015160200151604001518152602001856020015181526020018480156103bf5750835b151590529050808786600381106103d8576103d8610992565b60200201525050600190920191506102a59050565b505092915050565b604080516001600160a01b0385811660248301528416604482015260648082018490528251808303909101815260849091019091526020810180516001600160e01b03166323b872dd60e01b1790526102649085906105b5565b61045761079b565b60005b828110156105ae578151602090810151015184848381811061047e5761047e610992565b9050610120020160c0016020013511156104d8576020820180516040840152825190528383828181106104b3576104b3610992565b905061012002018036038101906104ca9190610af1565b8260005b60200201526105a6565b60208083015181015101518484838181106104f5576104f5610992565b9050610120020160c001602001351115610544576020820151604083015283838281811061052557610525610992565b9050610120020180360381019061053c9190610af1565b8260016104ce565b6040820151602090810151015184848381811061056357610563610992565b9050610120020160c0016020013511156105a65783838281811061058957610589610992565b905061012002018036038101906105a09190610af1565b60408301525b60010161045a565b5092915050565b60006105ca6001600160a01b03841683610622565b905080516000141580156105ef5750808060200190518101906105ed9190610b81565b155b1561061d57604051635274afe760e01b81526001600160a01b03841660048201526024015b60405180910390fd5b505050565b606061063083836000610637565b9392505050565b60608147101561065c5760405163cd78605960e01b8152306004820152602401610614565b600080856001600160a01b031684866040516106789190610ba3565b60006040518083038185875af1925050503d80600081146106b5576040519150601f19603f3d011682016040523d82523d6000602084013e6106ba565b606091505b50915091506106ca8683836106d4565b9695505050505050565b6060826106e9576106e482610730565b610630565b815115801561070057506001600160a01b0384163b155b1561072957604051639996b31560e01b81526001600160a01b0385166004820152602401610614565b5080610630565b8051156107405780518082602001fd5b604051630a12f52160e11b815260040160405180910390fd5b50565b60405180606001604052806003905b604080516060810182526000808252602080830182905292820152825260001990920191018161076b5790505090565b60405180606001604052806003905b6107b26107c8565b8152602001906001900390816107aa5790505090565b604051806040016040528061080c6040518060c001604052806000815260200160008152602001600081526020016000815260200160008152602001600081525090565b815260408051606081018252600080825260208281018290529282015291015290565b60006020828403121561084157600080fd5b5035919050565b600060c082840312801561085b57600080fd5b509092915050565b6000806020838503121561087657600080fd5b823567ffffffffffffffff81111561088d57600080fd5b8301601f8101851361089e57600080fd5b803567ffffffffffffffff8111156108b557600080fd5b8560206060830284010111156108ca57600080fd5b6020919091019590945092505050565b600080602083850312156108ed57600080fd5b823567ffffffffffffffff81111561090457600080fd5b8301601f8101851361091557600080fd5b803567ffffffffffffffff81111561092c57600080fd5b856020610120830284010111156108ca57600080fd5b6101208101818360005b600381101561098957815180518452602081015160208501526040810151151560408501525060608301925060208201915060018101905061094c565b50505092915050565b634e487b7160e01b600052603260045260246000fd5b6001600160a01b038116811461075957600080fd5b6000602082840312156109cf57600080fd5b8135610630816109a8565b81356109e5816109a8565b81546001600160a01b0319166001600160a01b039190911617815560208201356001820155604090910135600290910155565b6040805190810167ffffffffffffffff81118282101715610a4957634e487b7160e01b600052604160045260246000fd5b60405290565b60405160c0810167ffffffffffffffff81118282101715610a4957634e487b7160e01b600052604160045260246000fd5b600060608284031215610a9257600080fd5b6040516060810167ffffffffffffffff81118282101715610ac357634e487b7160e01b600052604160045260246000fd5b6040529050808235610ad4816109a8565b815260208381013590820152604092830135920191909152919050565b600081830361012081128015610b0657600080fd5b506000610b11610a18565b60c0831215610b1e578182fd5b610b26610a4f565b853581526020808701359082015260408087013590820152606080870135908201526080808701359082015260a0808701359082018190528183529093509150610b738660c08701610a80565b602082015295945050505050565b600060208284031215610b9357600080fd5b8151801515811461063057600080fd5b6000825160005b81811015610bc45760208186018101518583015201610baa565b50600092019182525091905056fea2646970667358221220fd6ef361aaba52d0f9503b51aea1d0b7a8363a9a66c9502aa7b931f1f44c507f64736f6c634300081c0033", + "linkReferences": {}, + "deployedLinkReferences": {} +} diff --git a/evmlib/src/contract/payment_vault/error.rs b/evmlib/src/contract/payment_vault/error.rs index 6c94c680f1..f4a5b76cce 100644 --- a/evmlib/src/contract/payment_vault/error.rs +++ b/evmlib/src/contract/payment_vault/error.rs @@ -10,4 +10,6 @@ pub enum Error { PendingTransactionError(#[from] alloy::providers::PendingTransactionError), #[error("Payment is invalid.")] PaymentInvalid, + #[error("Payment verification length must be 3.")] + PaymentVerificationLengthInvalid, } diff --git a/evmlib/src/contract/payment_vault/handler.rs b/evmlib/src/contract/payment_vault/handler.rs index 38d1dd2da8..5f21e5574e 100644 --- a/evmlib/src/contract/payment_vault/handler.rs +++ b/evmlib/src/contract/payment_vault/handler.rs @@ -1,7 +1,9 @@ use crate::common::{Address, Amount, Calldata, TxHash}; use crate::contract::payment_vault::error::Error; -use crate::contract::payment_vault::interface::IPaymentVault; use crate::contract::payment_vault::interface::IPaymentVault::IPaymentVaultInstance; +use crate::contract::payment_vault::interface::{ + IPaymentVault, REQUIRED_PAYMENT_VERIFICATION_LENGTH, +}; use alloy::network::{Network, TransactionBuilder}; use alloy::providers::Provider; use alloy::transports::Transport; @@ -16,7 +18,7 @@ where P: Provider, N: Network, { - /// Create a new PaymentVaultHandler instance from a deployed contract's address + /// Create a new PaymentVaultHandler instance from a (proxy) contract's address pub fn new(contract_address: Address, provider: P) -> Self { let contract = IPaymentVault::new(contract_address, provider); Self { contract } @@ -79,22 +81,27 @@ where Ok((calldata, *self.contract.address())) } - /// Verify if a payment is valid - pub async fn verify_payment< - Q: Into, - I: Into, - >( + /// Verify if payments are valid + pub async fn verify_payment>>( &self, - metrics: Q, - payment: I, - ) -> Result { - let is_valid = self + payment_verifications: I, + ) -> Result<[IPaymentVault::PaymentVerificationResult; 3], Error> { + let payment_verifications: Vec = payment_verifications + .into_iter() + .map(|v| v.into()) + .collect(); + + if payment_verifications.len() != REQUIRED_PAYMENT_VERIFICATION_LENGTH { + return Err(Error::PaymentVerificationLengthInvalid); + } + + let results = self .contract - .verifyPayment(metrics.into(), payment.into()) + .verifyPayment(payment_verifications) .call() .await? - .isValid; + .verificationResults; - Ok(is_valid) + Ok(results) } } diff --git a/evmlib/src/contract/payment_vault/implementation.rs b/evmlib/src/contract/payment_vault/implementation.rs index 4cbc469248..64fd9da1f9 100644 --- a/evmlib/src/contract/payment_vault/implementation.rs +++ b/evmlib/src/contract/payment_vault/implementation.rs @@ -1,31 +1,30 @@ -use crate::common::Address; -use alloy::hex; -use alloy::network::{Network, ReceiptResponse, TransactionBuilder}; +use crate::common::{Address, U256}; +use alloy::network::Network; use alloy::providers::Provider; +use alloy::sol; use alloy::transports::Transport; -// Payment Vault contract byte code -const BYTE_CODE: &str = "0x60a060405230608052348015610013575f5ffd5b5061001c610021565b6100d3565b7ff0c57e16840df040f15088dc2f81fe391c3923bec73e23a9662efc9c229c6a00805468010000000000000000900460ff16156100715760405163f92ee8a960e01b815260040160405180910390fd5b80546001600160401b03908116146100d05780546001600160401b0319166001600160401b0390811782556040519081527fc7f505b2f371ae2175ee4913f4499e1f2633a7b5936321eed1cdaeb6115181d29060200160405180910390a15b50565b608051610ef76100f95f395f818161064d0152818161067601526107ba0152610ef75ff3fe6080604052600436106100bf575f3560e01c8063715018a61161007c578063ad3cb1cc11610057578063ad3cb1cc14610253578063b6c2141b14610290578063cd6dc687146102af578063f2fde38b146102ce575f5ffd5b8063715018a6146101d45780638da5cb5b146101e8578063a69bf4a314610224575f5ffd5b80630716326d146100c35780633c150bf214610132578063474740b1146101605780634ec42e8e146101745780634f1ef286146101ab57806352d1902d146101c0575b5f5ffd5b3480156100ce575f5ffd5b506101086100dd366004610bc4565b600260208190525f91825260409091208054600182015491909201546001600160a01b039092169183565b604080516001600160a01b0390941684526020840192909252908201526060015b60405180910390f35b34801561013d575f5ffd5b5061015261014c366004610bf1565b50600190565b604051908152602001610129565b34801561016b575f5ffd5b506101525f5481565b34801561017f575f5ffd5b50600154610193906001600160a01b031681565b6040516001600160a01b039091168152602001610129565b6101be6101b9366004610c33565b6102ed565b005b3480156101cb575f5ffd5b5061015261030c565b3480156101df575f5ffd5b506101be610327565b3480156101f3575f5ffd5b507f9016d09d72d40fdae2fd8ceac6b6234c7706214fd39c1cd1e609a0528c199300546001600160a01b0316610193565b34801561022f575f5ffd5b5061024361023e366004610cf9565b61033a565b6040519015158152602001610129565b34801561025e575f5ffd5b50610283604051806040016040528060058152602001640352e302e360dc1b81525081565b6040516101299190610d37565b34801561029b575f5ffd5b506101be6102aa366004610d6c565b6103b6565b3480156102ba575f5ffd5b506101be6102c9366004610ddd565b6104a3565b3480156102d9575f5ffd5b506101be6102e8366004610e07565b610600565b6102f5610642565b6102fe826106e6565b61030882826106ee565b5050565b5f6103156107af565b505f516020610ea25f395f51905f5290565b61032f6107f8565b6103385f610853565b565b6040808201355f90815260026020818152838320845160608101865281546001600160a01b031681526001820154818401819052919093015494830194909452919290918401351480156103ae57506103966020840184610e07565b6001600160a01b0316815f01516001600160a01b0316145b949350505050565b5f5481908111156103da57604051630d67f41160e21b815260040160405180910390fd5b5f5b8181101561049d57368484838181106103f7576103f7610e22565b60600291909101915061042b9050336104136020840184610e07565b6001546001600160a01b0316919060208501356108c3565b6040808201355f90815260026020522081906104478282610e36565b505060408101356020820180359061045f9084610e07565b6001600160a01b03167ff998960b1c6f0e0e89b7bbe6b6fbf3e03e6f08eee5b8430877d8adb8e149d58060405160405180910390a4506001016103dc565b50505050565b7ff0c57e16840df040f15088dc2f81fe391c3923bec73e23a9662efc9c229c6a008054600160401b810460ff16159067ffffffffffffffff165f811580156104e85750825b90505f8267ffffffffffffffff1660011480156105045750303b155b905081158015610512575080155b156105305760405163f92ee8a960e01b815260040160405180910390fd5b845467ffffffffffffffff19166001178555831561055a57845460ff60401b1916600160401b1785555b6001600160a01b03871661058157604051632d06160b60e21b815260040160405180910390fd5b600180546001600160a01b0319166001600160a01b0389161790555f8690556105a93361091d565b6105b161092e565b83156105f757845460ff60401b19168555604051600181527fc7f505b2f371ae2175ee4913f4499e1f2633a7b5936321eed1cdaeb6115181d29060200160405180910390a15b50505050505050565b6106086107f8565b6001600160a01b03811661063657604051631e4fbdf760e01b81525f60048201526024015b60405180910390fd5b61063f81610853565b50565b306001600160a01b037f00000000000000000000000000000000000000000000000000000000000000001614806106c857507f00000000000000000000000000000000000000000000000000000000000000006001600160a01b03166106bc5f516020610ea25f395f51905f52546001600160a01b031690565b6001600160a01b031614155b156103385760405163703e46dd60e11b815260040160405180910390fd5b61063f6107f8565b816001600160a01b03166352d1902d6040518163ffffffff1660e01b8152600401602060405180830381865afa925050508015610748575060408051601f3d908101601f1916820190925261074591810190610e74565b60015b61077057604051634c9c8ce360e01b81526001600160a01b038316600482015260240161062d565b5f516020610ea25f395f51905f5281146107a057604051632a87526960e21b81526004810182905260240161062d565b6107aa8383610936565b505050565b306001600160a01b037f000000000000000000000000000000000000000000000000000000000000000016146103385760405163703e46dd60e11b815260040160405180910390fd5b3361082a7f9016d09d72d40fdae2fd8ceac6b6234c7706214fd39c1cd1e609a0528c199300546001600160a01b031690565b6001600160a01b0316146103385760405163118cdaa760e01b815233600482015260240161062d565b7f9016d09d72d40fdae2fd8ceac6b6234c7706214fd39c1cd1e609a0528c19930080546001600160a01b031981166001600160a01b03848116918217845560405192169182907f8be0079c531659141344cd1fd0a4f28419497f9722a3daafe3b4186f6b6457e0905f90a3505050565b604080516001600160a01b0385811660248301528416604482015260648082018490528251808303909101815260849091019091526020810180516001600160e01b03166323b872dd60e01b17905261049d90859061098b565b6109256109f7565b61063f81610a40565b6103386109f7565b61093f82610a48565b6040516001600160a01b038316907fbc7cd75a20ee27fd9adebab32041f755214dbc6bffa90cc0225b39da2e5c2d3b905f90a2805115610983576107aa8282610aab565b610308610b1d565b5f5f60205f8451602086015f885af1806109aa576040513d5f823e3d81fd5b50505f513d915081156109c15780600114156109ce565b6001600160a01b0384163b155b1561049d57604051635274afe760e01b81526001600160a01b038516600482015260240161062d565b7ff0c57e16840df040f15088dc2f81fe391c3923bec73e23a9662efc9c229c6a0054600160401b900460ff1661033857604051631afcd79f60e31b815260040160405180910390fd5b6106086109f7565b806001600160a01b03163b5f03610a7d57604051634c9c8ce360e01b81526001600160a01b038216600482015260240161062d565b5f516020610ea25f395f51905f5280546001600160a01b0319166001600160a01b0392909216919091179055565b60605f5f846001600160a01b031684604051610ac79190610e8b565b5f60405180830381855af49150503d805f8114610aff576040519150601f19603f3d011682016040523d82523d5f602084013e610b04565b606091505b5091509150610b14858383610b3c565b95945050505050565b34156103385760405163b398979f60e01b815260040160405180910390fd5b606082610b5157610b4c82610b9b565b610b94565b8151158015610b6857506001600160a01b0384163b155b15610b9157604051639996b31560e01b81526001600160a01b038516600482015260240161062d565b50805b9392505050565b805115610bab5780518082602001fd5b60405163d6bda27560e01b815260040160405180910390fd5b5f60208284031215610bd4575f5ffd5b5035919050565b5f60c08284031215610beb575f5ffd5b50919050565b5f60c08284031215610c01575f5ffd5b610b948383610bdb565b6001600160a01b038116811461063f575f5ffd5b634e487b7160e01b5f52604160045260245ffd5b5f5f60408385031215610c44575f5ffd5b8235610c4f81610c0b565b9150602083013567ffffffffffffffff811115610c6a575f5ffd5b8301601f81018513610c7a575f5ffd5b803567ffffffffffffffff811115610c9457610c94610c1f565b604051601f8201601f19908116603f0116810167ffffffffffffffff81118282101715610cc357610cc3610c1f565b604052818152828201602001871015610cda575f5ffd5b816020840160208301375f602083830101528093505050509250929050565b5f5f828403610120811215610d0c575f5ffd5b610d168585610bdb565b9250606060bf1982011215610d29575f5ffd5b5060c0830190509250929050565b602081525f82518060208401528060208501604085015e5f604082850101526040601f19601f83011684010191505092915050565b5f5f60208385031215610d7d575f5ffd5b823567ffffffffffffffff811115610d93575f5ffd5b8301601f81018513610da3575f5ffd5b803567ffffffffffffffff811115610db9575f5ffd5b856020606083028401011115610dcd575f5ffd5b6020919091019590945092505050565b5f5f60408385031215610dee575f5ffd5b8235610df981610c0b565b946020939093013593505050565b5f60208284031215610e17575f5ffd5b8135610b9481610c0b565b634e487b7160e01b5f52603260045260245ffd5b8135610e4181610c0b565b81546001600160a01b0319166001600160a01b039190911617815560208201356001820155604090910135600290910155565b5f60208284031215610e84575f5ffd5b5051919050565b5f82518060208501845e5f92019182525091905056fe360894a13ba1a3210667c828492db98dca3e2076cc3735a920a3ca505d382bbca26469706673582212203894ca52be6e6323aa3d296efd566c7f21d1723d4c66c56aed8a5f75a96b579d64736f6c634300081c0033"; +sol!( + #[allow(missing_docs)] + #[sol(rpc)] + PaymentVaultImplementation, + "artifacts/PaymentVaultNoProxy.json" +); -pub async fn deploy(provider: &P) -> Address +/// Deploys the payment vault contract and returns the contract address +pub async fn deploy( + provider: &P, + network_token_address: Address, + batch_limit: U256, +) -> Address where T: Transport + Clone, P: Provider, N: Network, { - let bytecode = hex::decode(BYTE_CODE).expect("Could not decode byte code"); - let tx = provider.transaction_request().with_deploy_code(bytecode); - - // Deploy the contract. - let receipt = provider - .send_transaction(tx) - .await - .expect("Could not send deployment transaction") - .get_receipt() + let contract = PaymentVaultImplementation::deploy(provider, network_token_address, batch_limit) .await - .expect("Deployment transaction failed"); + .expect("Could not deploy payment vault implementation contract"); - receipt - .contract_address() - .expect("Contract address missing") + *contract.address() } diff --git a/evmlib/src/contract/payment_vault/interface.rs b/evmlib/src/contract/payment_vault/interface.rs index 9f2d6f3490..36ec3ee6b8 100644 --- a/evmlib/src/contract/payment_vault/interface.rs +++ b/evmlib/src/contract/payment_vault/interface.rs @@ -3,17 +3,23 @@ use crate::quoting_metrics::QuotingMetrics; use alloy::primitives::FixedBytes; use alloy::sol; +pub const REQUIRED_PAYMENT_VERIFICATION_LENGTH: usize = 5; + sol!( #[allow(missing_docs)] + #[derive(Debug)] #[sol(rpc)] IPaymentVault, "abi/IPaymentVault.json" ); -pub struct PaymentVerification { - pub quote_hash: FixedBytes<32>, - pub amount_paid: Amount, - pub is_valid: bool, +impl From<(QuoteHash, QuotingMetrics, Address)> for IPaymentVault::PaymentVerification { + fn from(value: (QuoteHash, QuotingMetrics, Address)) -> Self { + Self { + metrics: value.1.into(), + dataPayment: (value.0, value.2, Amount::ZERO).into(), + } + } } impl From<(QuoteHash, Address, Amount)> for IPaymentVault::DataPayment { diff --git a/evmlib/src/contract/payment_vault/mod.rs b/evmlib/src/contract/payment_vault/mod.rs index 8ed1a9a92b..efa6f4fbd0 100644 --- a/evmlib/src/contract/payment_vault/mod.rs +++ b/evmlib/src/contract/payment_vault/mod.rs @@ -1,6 +1,5 @@ use crate::common::{Address, Amount, QuoteHash}; use crate::contract::payment_vault::handler::PaymentVaultHandler; -use crate::contract::payment_vault::interface::PaymentVerification; use crate::quoting_metrics::QuotingMetrics; use crate::utils::http_provider; use crate::Network; @@ -12,7 +11,7 @@ pub mod interface; pub const MAX_TRANSFERS_PER_TRANSACTION: usize = 256; -/// Helper function to return a quote for the given quoting metrics +/// Helper function to return a quote for the given quoting metrics. pub async fn get_market_price( network: &Network, quoting_metrics: QuotingMetrics, @@ -22,7 +21,8 @@ pub async fn get_market_price( payment_vault.get_quote(quoting_metrics).await } -/// Helper function to verify whether a data payment is valid +/// Helper function to verify whether a data payment is valid. +/// Returns the amount paid to the owned quote hashes. pub async fn verify_data_payment( network: &Network, owned_quote_hashes: Vec, @@ -33,24 +33,21 @@ pub async fn verify_data_payment( let mut amount = Amount::ZERO; - // TODO: @mick change this for loop to a batch when the smart contract changes - for (quote_hash, quoting_metrics, rewards_address) in payment { - let payment_verification: PaymentVerification = payment_vault - .verify_payment(quoting_metrics, (quote_hash, rewards_address, Amount::ZERO)) - .await - .map(|is_valid| PaymentVerification { - quote_hash, - amount_paid: Amount::from(1), // TODO: update placeholder amount when the smart contract changes - is_valid, - })?; + let payment_verifications: Vec<_> = payment + .into_iter() + .map(interface::IPaymentVault::PaymentVerification::from) + .collect(); + let payment_verification_results = payment_vault.verify_payment(payment_verifications).await?; + + for payment_verification_result in payment_verification_results { // CODE REVIEW: should we fail on a single invalid payment? - if !payment_verification.is_valid { + if !payment_verification_result.isValid { return Err(error::Error::PaymentInvalid); } - if owned_quote_hashes.contains("e_hash) { - amount += payment_verification.amount_paid; + if owned_quote_hashes.contains(&payment_verification_result.quoteHash) { + amount += payment_verification_result.amountPaid; } } diff --git a/evmlib/src/testnet.rs b/evmlib/src/testnet.rs index f5b76fea5c..d9c25bcffd 100644 --- a/evmlib/src/testnet.rs +++ b/evmlib/src/testnet.rs @@ -6,7 +6,7 @@ // KIND, either express or implied. Please review the Licences for the specific language governing // permissions and limitations relating to use of the SAFE Network Software. -use crate::common::Address; +use crate::common::{Address, Amount}; use crate::contract::network_token::NetworkToken; use crate::contract::payment_vault; use crate::contract::payment_vault::handler::PaymentVaultHandler; @@ -22,6 +22,8 @@ use alloy::providers::{Identity, ProviderBuilder, ReqwestProvider}; use alloy::signers::local::PrivateKeySigner; use alloy::transports::http::{Client, Http}; +const BATCH_LIMIT: u16 = 256; + pub struct Testnet { anvil: AnvilInstance, rpc_url: Url, @@ -120,7 +122,7 @@ pub async fn deploy_network_token_contract( pub async fn deploy_data_payments_contract( rpc_url: &Url, anvil: &AnvilInstance, - _token_address: Address, + token_address: Address, ) -> PaymentVaultHandler< Http, FillProvider< @@ -147,7 +149,9 @@ pub async fn deploy_data_payments_contract( .on_http(rpc_url.clone()); // Deploy the contract. - let payment_vault_contract_address = payment_vault::implementation::deploy(&provider).await; + let payment_vault_contract_address = + payment_vault::implementation::deploy(&provider, token_address, Amount::from(BATCH_LIMIT)) + .await; // Create a handler for the deployed contract PaymentVaultHandler::new(payment_vault_contract_address, provider) diff --git a/evmlib/tests/common/quote.rs b/evmlib/tests/common/quote.rs index 21d05cf189..28f8cbd3a8 100644 --- a/evmlib/tests/common/quote.rs +++ b/evmlib/tests/common/quote.rs @@ -5,6 +5,6 @@ use evmlib::utils::{dummy_address, dummy_hash}; pub fn random_quote_payment() -> QuotePayment { let quote_hash = dummy_hash(); let reward_address = dummy_address(); - let amount = Amount::from(200); + let amount = Amount::from(1); (quote_hash, reward_address, amount) } diff --git a/evmlib/tests/payment_vault.rs b/evmlib/tests/payment_vault.rs index 1e68e800c9..fe2df5905f 100644 --- a/evmlib/tests/payment_vault.rs +++ b/evmlib/tests/payment_vault.rs @@ -14,7 +14,9 @@ use alloy::transports::http::{Client, Http}; use evmlib::common::{Amount, U256}; use evmlib::contract::network_token::NetworkToken; use evmlib::contract::payment_vault::handler::PaymentVaultHandler; -use evmlib::contract::payment_vault::MAX_TRANSFERS_PER_TRANSACTION; +use evmlib::contract::payment_vault::interface::IPaymentVault::DataPayment; +use evmlib::contract::payment_vault::interface::REQUIRED_PAYMENT_VERIFICATION_LENGTH; +use evmlib::contract::payment_vault::{interface, MAX_TRANSFERS_PER_TRANSACTION}; use evmlib::quoting_metrics::QuotingMetrics; use evmlib::testnet::{deploy_data_payments_contract, deploy_network_token_contract, start_node}; use evmlib::utils::http_provider; @@ -129,9 +131,55 @@ async fn test_proxy_reachable() { assert_eq!(amount, Amount::from(1)); } +#[tokio::test] +async fn test_verify_payment() { + let (_anvil, network_token, mut payment_vault) = setup().await; + + let mut quote_payments = vec![]; + + for _ in 0..REQUIRED_PAYMENT_VERIFICATION_LENGTH { + let quote_payment = random_quote_payment(); + quote_payments.push(quote_payment); + } + + let _ = network_token + .approve(*payment_vault.contract.address(), U256::MAX) + .await + .unwrap(); + + // Contract provider has a different account coupled to it, + // so we set it to the same as the network token contract + payment_vault.set_provider(network_token.contract.provider().clone()); + + let result = payment_vault.pay_for_quotes(quote_payments.clone()).await; + + assert!(result.is_ok(), "Failed with error: {:?}", result.err()); + + let payment_verifications: Vec<_> = quote_payments + .into_iter() + .map(|v| interface::IPaymentVault::PaymentVerification { + metrics: QuotingMetrics::default().into(), + dataPayment: DataPayment { + rewardsAddress: v.1, + amount: v.2, + quoteHash: v.0, + }, + }) + .collect(); + + let results = payment_vault + .verify_payment(payment_verifications) + .await + .expect("Verify payment failed"); + + for result in results { + assert!(result.isValid); + } +} + #[tokio::test] async fn test_pay_for_quotes() { - let (_anvil, network_token, mut data_payments) = setup().await; + let (_anvil, network_token, mut payment_vault) = setup().await; let mut quote_payments = vec![]; @@ -141,15 +189,15 @@ async fn test_pay_for_quotes() { } let _ = network_token - .approve(*data_payments.contract.address(), U256::MAX) + .approve(*payment_vault.contract.address(), U256::MAX) .await .unwrap(); // Contract provider has a different account coupled to it, // so we set it to the same as the network token contract - data_payments.set_provider(network_token.contract.provider().clone()); + payment_vault.set_provider(network_token.contract.provider().clone()); - let result = data_payments.pay_for_quotes(quote_payments).await; + let result = payment_vault.pay_for_quotes(quote_payments).await; assert!(result.is_ok(), "Failed with error: {:?}", result.err()); } From e3bef52ad8d0cf7d17cc8e2f8737308706f0f7f6 Mon Sep 17 00:00:00 2001 From: Warm Beer Date: Tue, 10 Dec 2024 16:42:48 +0100 Subject: [PATCH 17/33] fix: include unpaid store quotes in receipt --- autonomi/src/client/payment.rs | 14 +++----------- autonomi/src/client/utils.rs | 6 +++--- 2 files changed, 6 insertions(+), 14 deletions(-) diff --git a/autonomi/src/client/payment.rs b/autonomi/src/client/payment.rs index 509615fb20..29a8f11576 100644 --- a/autonomi/src/client/payment.rs +++ b/autonomi/src/client/payment.rs @@ -1,17 +1,14 @@ use crate::client::data::PayError; use crate::client::quote::StoreQuote; use crate::Client; -use ant_evm::{AttoTokens, EncodedPeerId, EvmWallet, ProofOfPayment, QuoteHash, TxHash}; -use std::collections::{BTreeMap, HashMap}; +use ant_evm::{AttoTokens, EncodedPeerId, EvmWallet, ProofOfPayment}; +use std::collections::HashMap; use xor_name::XorName; /// Contains the proof of payments for each XOR address and the amount paid pub type Receipt = HashMap; -pub fn receipt_from_store_quotes_and_payments( - quotes: StoreQuote, - payments: BTreeMap, -) -> Receipt { +pub fn receipt_from_store_quotes(quotes: StoreQuote) -> Receipt { let mut receipt = Receipt::new(); for (content_addr, quote_for_address) in quotes.0 { @@ -22,11 +19,6 @@ pub fn receipt_from_store_quotes_and_payments( }; for (peer_id, quote, _amount) in quote_for_address.0 { - // skip quotes that haven't been paid - if !payments.contains_key("e.hash()) { - continue; - } - proof_of_payment .peer_quotes .push((EncodedPeerId::from(peer_id), quote)); diff --git a/autonomi/src/client/utils.rs b/autonomi/src/client/utils.rs index 0b7540af62..915d8c8d41 100644 --- a/autonomi/src/client/utils.rs +++ b/autonomi/src/client/utils.rs @@ -6,7 +6,7 @@ // KIND, either express or implied. Please review the Licences for the specific language governing // permissions and limitations relating to use of the SAFE Network Software. -use crate::client::payment::{receipt_from_store_quotes_and_payments, Receipt}; +use crate::client::payment::{receipt_from_store_quotes, Receipt}; use ant_evm::{EvmWallet, ProofOfPayment}; use ant_networking::{GetRecordCfg, PutRecordCfg, VerificationKind}; use ant_protocol::{ @@ -176,7 +176,7 @@ impl Client { // TODO: the error might contain some succeeded quote payments as well. These should be returned on err, so that they can be skipped when retrying. // TODO: retry when it fails? // Execute chunk payments - let payments = wallet + let _payments = wallet .pay_for_quotes(quotes.payments()) .await .map_err(|err| PayError::from(err.0))?; @@ -192,7 +192,7 @@ impl Client { skipped_chunks ); - let receipt = receipt_from_store_quotes_and_payments(quotes, payments); + let receipt = receipt_from_store_quotes(quotes); Ok(receipt) } From 83644f8f481b1ab81c6c2d715f3d2c834e7edeb5 Mon Sep 17 00:00:00 2001 From: Warm Beer Date: Tue, 10 Dec 2024 16:43:05 +0100 Subject: [PATCH 18/33] fix: add rate limit to get market price RPC calls --- autonomi/src/client/quote.rs | 23 +++++++++++++++++++++++ 1 file changed, 23 insertions(+) diff --git a/autonomi/src/client/quote.rs b/autonomi/src/client/quote.rs index 3f48587552..2c527fafd2 100644 --- a/autonomi/src/client/quote.rs +++ b/autonomi/src/client/quote.rs @@ -9,12 +9,16 @@ use super::{data::CostError, Client}; use ant_evm::payment_vault::get_market_price; use ant_evm::{Amount, PaymentQuote, QuotePayment}; +use ant_networking::target_arch::{sleep, Duration, Instant}; use ant_networking::{Network, NetworkError}; use ant_protocol::{storage::ChunkAddress, NetworkAddress}; use libp2p::PeerId; use std::collections::HashMap; use xor_name::XorName; +// set rate limit to 2 req/s +const TIME_BETWEEN_RPC_CALLS_IN_MS: u64 = 500; + /// A quote for a single address pub struct QuoteForAddress(pub(crate) Vec<(PeerId, PaymentQuote, Amount)>); @@ -63,15 +67,34 @@ impl Client { .collect(); let raw_quotes_per_addr = futures::future::try_join_all(futures).await?; + debug!("Fetched store quotes: {raw_quotes_per_addr:?}"); + // choose the quotes to pay for each address let mut quotes_to_pay_per_addr = HashMap::new(); for (content_addr, raw_quotes) in raw_quotes_per_addr { // ask smart contract for the market price let mut prices = vec![]; + + // rate limit + let mut maybe_last_call: Option = None; + for (peer, quote) in raw_quotes { // NB TODO @mick we need to batch this smart contract call + // check if we have to wait for the rate limit + if let Some(last_call) = maybe_last_call { + let elapsed = Instant::now() - last_call; + let time_to_sleep_ms = + TIME_BETWEEN_RPC_CALLS_IN_MS as u128 - elapsed.as_millis(); + if time_to_sleep_ms > 0 { + sleep(Duration::from_millis(time_to_sleep_ms as u64)).await; + } + } + let price = get_market_price(&self.evm_network, quote.quoting_metrics.clone()).await?; + + maybe_last_call = Some(Instant::now()); + prices.push((peer, quote, price)); } From 7bfce8393bb7925767eeb900ef0e53e7c590a84b Mon Sep 17 00:00:00 2001 From: qima Date: Tue, 10 Dec 2024 02:53:02 +0800 Subject: [PATCH 19/33] feat(node): carry out quote's payee neighbourhood check --- ant-networking/src/lib.rs | 3 +-- ant-networking/src/log_markers.rs | 4 +--- ant-networking/src/metrics/mod.rs | 4 +--- ant-node/src/node.rs | 5 +++-- ant-node/src/put_validation.rs | 11 ++++++++++- ant-node/src/quote.rs | 8 ++------ autonomi/src/client/external_signer.rs | 4 +++- evmlib/src/external_signer.rs | 2 +- evmlib/src/wallet.rs | 2 +- evmlib/tests/wallet.rs | 6 +----- 10 files changed, 24 insertions(+), 25 deletions(-) diff --git a/ant-networking/src/lib.rs b/ant-networking/src/lib.rs index 5973cb02c2..a02767594c 100644 --- a/ant-networking/src/lib.rs +++ b/ant-networking/src/lib.rs @@ -411,8 +411,7 @@ impl Network { let mut all_quotes = vec![]; let mut quotes_to_pay = vec![]; for (peer, response) in responses { - info!( - "StoreCostReq for {record_address:?} received response: {response:?}"); + info!("StoreCostReq for {record_address:?} received response: {response:?}"); match response { Ok(Response::Query(QueryResponse::GetStoreQuote { quote: Ok(quote), diff --git a/ant-networking/src/log_markers.rs b/ant-networking/src/log_markers.rs index c8ce2ce744..71787c0a65 100644 --- a/ant-networking/src/log_markers.rs +++ b/ant-networking/src/log_markers.rs @@ -20,9 +20,7 @@ pub enum Marker<'a> { /// Close records held (Used in VDash) CloseRecordsLen(usize), /// Quoting metrics - QuotingMetrics { - quoting_metrics: &'a QuotingMetrics, - }, + QuotingMetrics { quoting_metrics: &'a QuotingMetrics }, /// The peer has been considered as bad PeerConsideredAsBad { bad_peer: &'a PeerId }, /// We have been flagged as a bad node by a peer. diff --git a/ant-networking/src/metrics/mod.rs b/ant-networking/src/metrics/mod.rs index cb90d9b28e..ef9f636bcb 100644 --- a/ant-networking/src/metrics/mod.rs +++ b/ant-networking/src/metrics/mod.rs @@ -284,9 +284,7 @@ impl NetworkMetricsRecorder { } }); } - Marker::QuotingMetrics { - quoting_metrics, - } => { + Marker::QuotingMetrics { quoting_metrics } => { let _ = self.relevant_records.set( quoting_metrics .close_records_stored diff --git a/ant-node/src/node.rs b/ant-node/src/node.rs index 7c34c0cfa9..b2b08232ef 100644 --- a/ant-node/src/node.rs +++ b/ant-node/src/node.rs @@ -580,7 +580,8 @@ impl Node { let record_key = key.to_record_key(); let self_id = network.peer_id(); - let maybe_quoting_metrics = network.get_local_quoting_metrics(record_key.clone()).await; + let maybe_quoting_metrics = + network.get_local_quoting_metrics(record_key.clone()).await; let storage_proofs = if let Some(nonce) = nonce { Self::respond_x_closest_record_proof( @@ -607,7 +608,7 @@ impl Node { } } else { QueryResponse::GetStoreQuote { - quote: Self::create_quote_for_storecost( + quote: Self::create_quote_for_storecost( network, &key, "ing_metrics, diff --git a/ant-node/src/put_validation.rs b/ant-node/src/put_validation.rs index 05ca698e96..9beec8b740 100644 --- a/ant-node/src/put_validation.rs +++ b/ant-node/src/put_validation.rs @@ -669,12 +669,21 @@ impl Node { ))); } + // verify the claimed payees are all known to us within the certain range. + let closest_k_peers = self.network().get_closest_k_value_local_peers().await?; + let mut payees = payment.payees(); + payees.retain(|peer_id| !closest_k_peers.contains(peer_id)); + if !payees.is_empty() { + return Err(Error::InvalidRequest(format!( + "Payment quote has out-of-range payees {payees:?}" + ))); + } + let owned_payment_quotes = payment .quotes_by_peer(&self_peer_id) .iter() .map(|quote| quote.hash()) .collect(); - // check if payment is valid on chain let payments_to_verify = payment.digest(); debug!("Verifying payment for record {pretty_key}"); diff --git a/ant-node/src/quote.rs b/ant-node/src/quote.rs index 4a11fd2ef7..f7c61b2af8 100644 --- a/ant-node/src/quote.rs +++ b/ant-node/src/quote.rs @@ -22,12 +22,8 @@ impl Node { ) -> Result { let content = address.as_xorname().unwrap_or_default(); let timestamp = std::time::SystemTime::now(); - let bytes = PaymentQuote::bytes_for_signing( - content, - timestamp, - quoting_metrics, - payment_address, - ); + let bytes = + PaymentQuote::bytes_for_signing(content, timestamp, quoting_metrics, payment_address); let Ok(signature) = network.sign(&bytes) else { return Err(ProtocolError::QuoteGenerationFailed); diff --git a/autonomi/src/client/external_signer.rs b/autonomi/src/client/external_signer.rs index 8c3d6969f6..0037ba7e25 100644 --- a/autonomi/src/client/external_signer.rs +++ b/autonomi/src/client/external_signer.rs @@ -28,7 +28,9 @@ impl Client { > { let quote = self.get_store_quotes(content_addrs.clone()).await?; let payments = quote.payments(); - let free_chunks = content_addrs.filter(|addr| !quote.0.contains_key(addr)).collect(); + let free_chunks = content_addrs + .filter(|addr| !quote.0.contains_key(addr)) + .collect(); let quotes_per_addr = quote.0.into_iter().collect(); debug!( diff --git a/evmlib/src/external_signer.rs b/evmlib/src/external_signer.rs index 30186f031d..b7f7ce9b6d 100644 --- a/evmlib/src/external_signer.rs +++ b/evmlib/src/external_signer.rs @@ -7,7 +7,7 @@ // permissions and limitations relating to use of the SAFE Network Software. use crate::common::{Address, Amount, Calldata, QuoteHash, QuotePayment, U256}; -use crate::contract::network_token::{NetworkToken, self}; +use crate::contract::network_token::{self, NetworkToken}; use crate::contract::payment_vault::MAX_TRANSFERS_PER_TRANSACTION; use crate::utils::http_provider; use crate::Network; diff --git a/evmlib/src/wallet.rs b/evmlib/src/wallet.rs index 327c0faf40..0f6ba3acea 100644 --- a/evmlib/src/wallet.rs +++ b/evmlib/src/wallet.rs @@ -6,7 +6,7 @@ // KIND, either express or implied. Please review the Licences for the specific language governing // permissions and limitations relating to use of the SAFE Network Software. -use crate::common::{Address, Amount, QuotePayment, QuoteHash, TxHash, U256}; +use crate::common::{Address, Amount, QuoteHash, QuotePayment, TxHash, U256}; use crate::contract::network_token::NetworkToken; use crate::contract::payment_vault::handler::PaymentVaultHandler; use crate::contract::payment_vault::MAX_TRANSFERS_PER_TRANSACTION; diff --git a/evmlib/tests/wallet.rs b/evmlib/tests/wallet.rs index f6ac01e3a4..e9e5f0a077 100644 --- a/evmlib/tests/wallet.rs +++ b/evmlib/tests/wallet.rs @@ -90,11 +90,7 @@ async fn test_pay_for_quotes_and_data_payment_verification() { let result = verify_data_payment( &network, vec![*quote_hash], - vec![( - *quote_hash, - QuotingMetrics::default(), - *reward_addr, - )], + vec![(*quote_hash, QuotingMetrics::default(), *reward_addr)], ) .await; From 9bb49c7295c1ac01a0b9645401bf677d6369a88f Mon Sep 17 00:00:00 2001 From: grumbach Date: Wed, 11 Dec 2024 15:24:20 +0900 Subject: [PATCH 20/33] chore: compile and fixes after rebase --- Cargo.lock | 619 ++++------------------- ant-node/src/node.rs | 2 +- autonomi/src/client/data/mod.rs | 2 +- autonomi/src/client/data/public.rs | 2 - autonomi/src/client/external_signer.rs | 6 +- autonomi/src/client/mod.rs | 2 +- autonomi/src/client/registers.rs | 2 +- autonomi/src/client/utils.rs | 4 +- autonomi/tests/external_signer.rs | 2 +- evmlib/src/contract/payment_vault/mod.rs | 2 +- 10 files changed, 117 insertions(+), 526 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index b15c7fb01c..831e66f05e 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1,6 +1,6 @@ # This file is automatically @generated by Cargo. # It is not intended for manual editing. -version = 4 +version = 3 [[package]] name = "addr2line" @@ -116,51 +116,28 @@ version = "0.2.21" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "683d7910e743518b0e34f1186f92494becacb047c7b6bf616c96772180fef923" -[[package]] -name = "alloy" -version = "0.5.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ea8ebf106e84a1c37f86244df7da0c7587e697b71a0d565cce079449b85ac6f8" -dependencies = [ - "alloy-consensus 0.5.4", - "alloy-contract 0.5.4", - "alloy-core", - "alloy-eips 0.5.4", - "alloy-genesis 0.5.4", - "alloy-network 0.5.4", - "alloy-node-bindings 0.5.4", - "alloy-provider 0.5.4", - "alloy-rpc-client 0.5.4", - "alloy-rpc-types 0.5.4", - "alloy-serde 0.5.4", - "alloy-signer 0.5.4", - "alloy-signer-local 0.5.4", - "alloy-transport 0.5.4", - "alloy-transport-http 0.5.4", -] - [[package]] name = "alloy" version = "0.7.3" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "02b0561294ccedc6181e5528b850b4579e3fbde696507baa00109bfd9054c5bb" dependencies = [ - "alloy-consensus 0.7.3", - "alloy-contract 0.7.3", + "alloy-consensus", + "alloy-contract", "alloy-core", - "alloy-eips 0.7.3", - "alloy-genesis 0.7.3", - "alloy-json-rpc 0.7.3", - "alloy-network 0.7.3", - "alloy-node-bindings 0.7.3", - "alloy-provider 0.7.3", - "alloy-rpc-client 0.7.3", - "alloy-rpc-types 0.7.3", - "alloy-serde 0.7.3", - "alloy-signer 0.7.3", - "alloy-signer-local 0.7.3", - "alloy-transport 0.7.3", - "alloy-transport-http 0.7.3", + "alloy-eips", + "alloy-genesis", + "alloy-json-rpc", + "alloy-network", + "alloy-node-bindings", + "alloy-provider", + "alloy-rpc-client", + "alloy-rpc-types", + "alloy-serde", + "alloy-signer", + "alloy-signer-local", + "alloy-transport", + "alloy-transport-http", ] [[package]] @@ -174,32 +151,16 @@ dependencies = [ "strum", ] -[[package]] -name = "alloy-consensus" -version = "0.5.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "41ed961a48297c732a5d97ee321aa8bb5009ecadbcb077d8bec90cb54e651629" -dependencies = [ - "alloy-eips 0.5.4", - "alloy-primitives", - "alloy-rlp", - "alloy-serde 0.5.4", - "auto_impl", - "c-kzg", - "derive_more", - "serde", -] - [[package]] name = "alloy-consensus" version = "0.7.3" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "a101d4d016f47f13890a74290fdd17b05dd175191d9337bc600791fb96e4dea8" dependencies = [ - "alloy-eips 0.7.3", + "alloy-eips", "alloy-primitives", "alloy-rlp", - "alloy-serde 0.7.3", + "alloy-serde", "alloy-trie", "auto_impl", "c-kzg", @@ -213,34 +174,14 @@ version = "0.7.3" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "fa60357dda9a3d0f738f18844bd6d0f4a5924cc5cf00bfad2ff1369897966123" dependencies = [ - "alloy-consensus 0.7.3", - "alloy-eips 0.7.3", + "alloy-consensus", + "alloy-eips", "alloy-primitives", "alloy-rlp", - "alloy-serde 0.7.3", + "alloy-serde", "serde", ] -[[package]] -name = "alloy-contract" -version = "0.5.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "460ab80ce4bda1c80bcf96fe7460520476f2c7b734581c6567fac2708e2a60ef" -dependencies = [ - "alloy-dyn-abi", - "alloy-json-abi", - "alloy-network 0.5.4", - "alloy-network-primitives 0.5.4", - "alloy-primitives", - "alloy-provider 0.5.4", - "alloy-rpc-types-eth 0.5.4", - "alloy-sol-types", - "alloy-transport 0.5.4", - "futures", - "futures-util", - "thiserror 1.0.69", -] - [[package]] name = "alloy-contract" version = "0.7.3" @@ -249,13 +190,13 @@ checksum = "2869e4fb31331d3b8c58c7db567d1e4e4e94ef64640beda3b6dd9b7045690941" dependencies = [ "alloy-dyn-abi", "alloy-json-abi", - "alloy-network 0.7.3", - "alloy-network-primitives 0.7.3", + "alloy-network", + "alloy-network-primitives", "alloy-primitives", - "alloy-provider 0.7.3", - "alloy-rpc-types-eth 0.7.3", + "alloy-provider", + "alloy-rpc-types-eth", "alloy-sol-types", - "alloy-transport 0.7.3", + "alloy-transport", "futures", "futures-util", "thiserror 2.0.6", @@ -302,18 +243,6 @@ dependencies = [ "serde", ] -[[package]] -name = "alloy-eip7702" -version = "0.3.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "64ffc577390ce50234e02d841214b3dc0bea6aaaae8e04bbf3cb82e9a45da9eb" -dependencies = [ - "alloy-primitives", - "alloy-rlp", - "derive_more", - "serde", -] - [[package]] name = "alloy-eip7702" version = "0.4.2" @@ -326,24 +255,6 @@ dependencies = [ "serde", ] -[[package]] -name = "alloy-eips" -version = "0.5.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b69e06cf9c37be824b9d26d6d101114fdde6af0c87de2828b414c05c4b3daa71" -dependencies = [ - "alloy-eip2930", - "alloy-eip7702 0.3.2", - "alloy-primitives", - "alloy-rlp", - "alloy-serde 0.5.4", - "c-kzg", - "derive_more", - "once_cell", - "serde", - "sha2 0.10.8", -] - [[package]] name = "alloy-eips" version = "0.7.3" @@ -351,10 +262,10 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "8b6755b093afef5925f25079dd5a7c8d096398b804ba60cb5275397b06b31689" dependencies = [ "alloy-eip2930", - "alloy-eip7702 0.4.2", + "alloy-eip7702", "alloy-primitives", "alloy-rlp", - "alloy-serde 0.7.3", + "alloy-serde", "c-kzg", "derive_more", "once_cell", @@ -362,17 +273,6 @@ dependencies = [ "sha2 0.10.8", ] -[[package]] -name = "alloy-genesis" -version = "0.5.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "dde15e14944a88bd6a57d325e9a49b75558746fe16aaccc79713ae50a6a9574c" -dependencies = [ - "alloy-primitives", - "alloy-serde 0.5.4", - "serde", -] - [[package]] name = "alloy-genesis" version = "0.7.3" @@ -380,7 +280,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "aeec8e6eab6e52b7c9f918748c9b811e87dbef7312a2e3a2ca1729a92966a6af" dependencies = [ "alloy-primitives", - "alloy-serde 0.7.3", + "alloy-serde", "alloy-trie", "serde", ] @@ -397,20 +297,6 @@ dependencies = [ "serde_json", ] -[[package]] -name = "alloy-json-rpc" -version = "0.5.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "af5979e0d5a7bf9c7eb79749121e8256e59021af611322aee56e77e20776b4b3" -dependencies = [ - "alloy-primitives", - "alloy-sol-types", - "serde", - "serde_json", - "thiserror 1.0.69", - "tracing", -] - [[package]] name = "alloy-json-rpc" version = "0.7.3" @@ -425,43 +311,22 @@ dependencies = [ "tracing", ] -[[package]] -name = "alloy-network" -version = "0.5.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "204237129086ce5dc17a58025e93739b01b45313841f98fa339eb1d780511e57" -dependencies = [ - "alloy-consensus 0.5.4", - "alloy-eips 0.5.4", - "alloy-json-rpc 0.5.4", - "alloy-network-primitives 0.5.4", - "alloy-primitives", - "alloy-rpc-types-eth 0.5.4", - "alloy-serde 0.5.4", - "alloy-signer 0.5.4", - "alloy-sol-types", - "async-trait", - "auto_impl", - "futures-utils-wasm", - "thiserror 1.0.69", -] - [[package]] name = "alloy-network" version = "0.7.3" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "209a1882a08e21aca4aac6e2a674dc6fcf614058ef8cb02947d63782b1899552" dependencies = [ - "alloy-consensus 0.7.3", + "alloy-consensus", "alloy-consensus-any", - "alloy-eips 0.7.3", - "alloy-json-rpc 0.7.3", - "alloy-network-primitives 0.7.3", + "alloy-eips", + "alloy-json-rpc", + "alloy-network-primitives", "alloy-primitives", "alloy-rpc-types-any", - "alloy-rpc-types-eth 0.7.3", - "alloy-serde 0.7.3", - "alloy-signer 0.7.3", + "alloy-rpc-types-eth", + "alloy-serde", + "alloy-signer", "alloy-sol-types", "async-trait", "auto_impl", @@ -471,56 +336,26 @@ dependencies = [ "thiserror 2.0.6", ] -[[package]] -name = "alloy-network-primitives" -version = "0.5.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "514f70ee2a953db21631cd817b13a1571474ec77ddc03d47616d5e8203489fde" -dependencies = [ - "alloy-consensus 0.5.4", - "alloy-eips 0.5.4", - "alloy-primitives", - "alloy-serde 0.5.4", - "serde", -] - [[package]] name = "alloy-network-primitives" version = "0.7.3" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "c20219d1ad261da7a6331c16367214ee7ded41d001fabbbd656fbf71898b2773" dependencies = [ - "alloy-consensus 0.7.3", - "alloy-eips 0.7.3", + "alloy-consensus", + "alloy-eips", "alloy-primitives", - "alloy-serde 0.7.3", + "alloy-serde", "serde", ] -[[package]] -name = "alloy-node-bindings" -version = "0.5.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "27444ea67d360508753022807cdd0b49a95c878924c9c5f8f32668b7d7768245" -dependencies = [ - "alloy-genesis 0.5.4", - "alloy-primitives", - "k256", - "rand 0.8.5", - "serde_json", - "tempfile", - "thiserror 1.0.69", - "tracing", - "url", -] - [[package]] name = "alloy-node-bindings" version = "0.7.3" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "bffcf33dd319f21cd6f066d81cbdef0326d4bdaaf7cfe91110bc090707858e9f" dependencies = [ - "alloy-genesis 0.7.3", + "alloy-genesis", "alloy-primitives", "k256", "rand 0.8.5", @@ -559,47 +394,6 @@ dependencies = [ "tiny-keccak", ] -[[package]] -name = "alloy-provider" -version = "0.5.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4814d141ede360bb6cd1b4b064f1aab9de391e7c4d0d4d50ac89ea4bc1e25fbd" -dependencies = [ - "alloy-chains", - "alloy-consensus 0.5.4", - "alloy-eips 0.5.4", - "alloy-json-rpc 0.5.4", - "alloy-network 0.5.4", - "alloy-network-primitives 0.5.4", - "alloy-node-bindings 0.5.4", - "alloy-primitives", - "alloy-rpc-client 0.5.4", - "alloy-rpc-types-anvil 0.5.4", - "alloy-rpc-types-eth 0.5.4", - "alloy-signer 0.5.4", - "alloy-signer-local 0.5.4", - "alloy-transport 0.5.4", - "alloy-transport-http 0.5.4", - "async-stream", - "async-trait", - "auto_impl", - "dashmap", - "futures", - "futures-utils-wasm", - "lru", - "parking_lot", - "pin-project", - "reqwest 0.12.9", - "schnellru", - "serde", - "serde_json", - "thiserror 1.0.69", - "tokio", - "tracing", - "url", - "wasmtimer 0.2.1", -] - [[package]] name = "alloy-provider" version = "0.7.3" @@ -607,20 +401,20 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "9eefa6f4c798ad01f9b4202d02cea75f5ec11fa180502f4701e2b47965a8c0bb" dependencies = [ "alloy-chains", - "alloy-consensus 0.7.3", - "alloy-eips 0.7.3", - "alloy-json-rpc 0.7.3", - "alloy-network 0.7.3", - "alloy-network-primitives 0.7.3", - "alloy-node-bindings 0.7.3", + "alloy-consensus", + "alloy-eips", + "alloy-json-rpc", + "alloy-network", + "alloy-network-primitives", + "alloy-node-bindings", "alloy-primitives", - "alloy-rpc-client 0.7.3", - "alloy-rpc-types-anvil 0.7.3", - "alloy-rpc-types-eth 0.7.3", - "alloy-signer 0.7.3", - "alloy-signer-local 0.7.3", - "alloy-transport 0.7.3", - "alloy-transport-http 0.7.3", + "alloy-rpc-client", + "alloy-rpc-types-anvil", + "alloy-rpc-types-eth", + "alloy-signer", + "alloy-signer-local", + "alloy-transport", + "alloy-transport-http", "async-stream", "async-trait", "auto_impl", @@ -663,39 +457,16 @@ dependencies = [ "syn 2.0.90", ] -[[package]] -name = "alloy-rpc-client" -version = "0.5.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7fc2bd1e7403463a5f2c61e955bcc9d3072b63aa177442b0f9aa6a6d22a941e3" -dependencies = [ - "alloy-json-rpc 0.5.4", - "alloy-primitives", - "alloy-transport 0.5.4", - "alloy-transport-http 0.5.4", - "futures", - "pin-project", - "reqwest 0.12.9", - "serde", - "serde_json", - "tokio", - "tokio-stream", - "tower 0.5.1", - "tracing", - "url", - "wasmtimer 0.2.1", -] - [[package]] name = "alloy-rpc-client" version = "0.7.3" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "ed30bf1041e84cabc5900f52978ca345dd9969f2194a945e6fdec25b0620705c" dependencies = [ - "alloy-json-rpc 0.7.3", + "alloy-json-rpc", "alloy-primitives", - "alloy-transport 0.7.3", - "alloy-transport-http 0.7.3", + "alloy-transport", + "alloy-transport-http", "futures", "pin-project", "reqwest 0.12.9", @@ -709,19 +480,6 @@ dependencies = [ "wasmtimer 0.4.1", ] -[[package]] -name = "alloy-rpc-types" -version = "0.5.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "eea9bf1abdd506f985a53533f5ac01296bcd6102c5e139bbc5d40bc468d2c916" -dependencies = [ - "alloy-primitives", - "alloy-rpc-types-anvil 0.5.4", - "alloy-rpc-types-eth 0.5.4", - "alloy-serde 0.5.4", - "serde", -] - [[package]] name = "alloy-rpc-types" version = "0.7.3" @@ -729,19 +487,8 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "5ab686b0fa475d2a4f5916c5f07797734a691ec58e44f0f55d4746ea39cbcefb" dependencies = [ "alloy-primitives", - "alloy-rpc-types-eth 0.7.3", - "alloy-serde 0.7.3", - "serde", -] - -[[package]] -name = "alloy-rpc-types-anvil" -version = "0.5.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2382fc63fb0cf3e02818d547b80cb66cc49a31f8803d0c328402b2008bc13650" -dependencies = [ - "alloy-primitives", - "alloy-serde 0.5.4", + "alloy-rpc-types-eth", + "alloy-serde", "serde", ] @@ -752,8 +499,8 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "d33bc190844626c08e21897736dbd7956ab323c09e6f141b118d1c8b7aff689e" dependencies = [ "alloy-primitives", - "alloy-rpc-types-eth 0.7.3", - "alloy-serde 0.7.3", + "alloy-rpc-types-eth", + "alloy-serde", "serde", ] @@ -764,27 +511,8 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "200661999b6e235d9840be5d60a6e8ae2f0af9eb2a256dd378786744660e36ec" dependencies = [ "alloy-consensus-any", - "alloy-rpc-types-eth 0.7.3", - "alloy-serde 0.7.3", -] - -[[package]] -name = "alloy-rpc-types-eth" -version = "0.5.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "00b034779a4850b4b03f5be5ea674a1cf7d746b2da762b34d1860ab45e48ca27" -dependencies = [ - "alloy-consensus 0.5.4", - "alloy-eips 0.5.4", - "alloy-network-primitives 0.5.4", - "alloy-primitives", - "alloy-rlp", - "alloy-serde 0.5.4", - "alloy-sol-types", - "derive_more", - "itertools 0.13.0", - "serde", - "serde_json", + "alloy-rpc-types-eth", + "alloy-serde", ] [[package]] @@ -793,13 +521,13 @@ version = "0.7.3" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "a0600b8b5e2dc0cab12cbf91b5a885c35871789fb7b3a57b434bd4fced5b7a8b" dependencies = [ - "alloy-consensus 0.7.3", + "alloy-consensus", "alloy-consensus-any", - "alloy-eips 0.7.3", - "alloy-network-primitives 0.7.3", + "alloy-eips", + "alloy-network-primitives", "alloy-primitives", "alloy-rlp", - "alloy-serde 0.7.3", + "alloy-serde", "alloy-sol-types", "derive_more", "itertools 0.13.0", @@ -807,17 +535,6 @@ dependencies = [ "serde_json", ] -[[package]] -name = "alloy-serde" -version = "0.5.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "028e72eaa9703e4882344983cfe7636ce06d8cce104a78ea62fd19b46659efc4" -dependencies = [ - "alloy-primitives", - "serde", - "serde_json", -] - [[package]] name = "alloy-serde" version = "0.7.3" @@ -829,20 +546,6 @@ dependencies = [ "serde_json", ] -[[package]] -name = "alloy-signer" -version = "0.5.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "592c185d7100258c041afac51877660c7bf6213447999787197db4842f0e938e" -dependencies = [ - "alloy-primitives", - "async-trait", - "auto_impl", - "elliptic-curve 0.13.8", - "k256", - "thiserror 1.0.69", -] - [[package]] name = "alloy-signer" version = "0.7.3" @@ -857,32 +560,16 @@ dependencies = [ "thiserror 2.0.6", ] -[[package]] -name = "alloy-signer-local" -version = "0.5.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6614f02fc1d5b079b2a4a5320018317b506fd0a6d67c1fd5542a71201724986c" -dependencies = [ - "alloy-consensus 0.5.4", - "alloy-network 0.5.4", - "alloy-primitives", - "alloy-signer 0.5.4", - "async-trait", - "k256", - "rand 0.8.5", - "thiserror 1.0.69", -] - [[package]] name = "alloy-signer-local" version = "0.7.3" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "bd6d988cb6cd7d2f428a74476515b1a6e901e08c796767f9f93311ab74005c8b" dependencies = [ - "alloy-consensus 0.7.3", - "alloy-network 0.7.3", + "alloy-consensus", + "alloy-network", "alloy-primitives", - "alloy-signer 0.7.3", + "alloy-signer", "async-trait", "k256", "rand 0.8.5", @@ -962,33 +649,13 @@ dependencies = [ "serde", ] -[[package]] -name = "alloy-transport" -version = "0.5.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "be77579633ebbc1266ae6fd7694f75c408beb1aeb6865d0b18f22893c265a061" -dependencies = [ - "alloy-json-rpc 0.5.4", - "base64 0.22.1", - "futures-util", - "futures-utils-wasm", - "serde", - "serde_json", - "thiserror 1.0.69", - "tokio", - "tower 0.5.1", - "tracing", - "url", - "wasmtimer 0.2.1", -] - [[package]] name = "alloy-transport" version = "0.7.3" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "d69d36982b9e46075ae6b792b0f84208c6c2c15ad49f6c500304616ef67b70e0" dependencies = [ - "alloy-json-rpc 0.7.3", + "alloy-json-rpc", "base64 0.22.1", "futures-util", "futures-utils-wasm", @@ -1003,29 +670,14 @@ dependencies = [ "wasmtimer 0.4.1", ] -[[package]] -name = "alloy-transport-http" -version = "0.5.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "91fd1a5d0827939847983b46f2f79510361f901dc82f8e3c38ac7397af142c6e" -dependencies = [ - "alloy-json-rpc 0.5.4", - "alloy-transport 0.5.4", - "reqwest 0.12.9", - "serde_json", - "tower 0.5.1", - "tracing", - "url", -] - [[package]] name = "alloy-transport-http" version = "0.7.3" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "2e02ffd5d93ffc51d72786e607c97de3b60736ca3e636ead0ec1f7dce68ea3fd" dependencies = [ - "alloy-json-rpc 0.7.3", - "alloy-transport 0.7.3", + "alloy-json-rpc", + "alloy-transport", "reqwest 0.12.9", "serde_json", "tower 0.5.1", @@ -1956,7 +1608,7 @@ checksum = "ace50bade8e6234aa140d9a2f552bbee1db4d353f69b8217bc503490fc1a9f26" name = "autonomi" version = "0.2.4" dependencies = [ - "alloy 0.5.4", + "alloy", "ant-bootstrap", "ant-evm", "ant-logging", @@ -3732,7 +3384,7 @@ dependencies = [ name = "evmlib" version = "0.1.4" dependencies = [ - "alloy 0.7.3", + "alloy", "dirs-next", "getrandom 0.2.15", "rand 0.8.5", @@ -3749,7 +3401,7 @@ version = "2.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "6ffb309d235a642598183aeda8925e871e85dd5a433c2c877e69ff0a960f4c02" dependencies = [ - "fastrand 2.2.0", + "fastrand 2.3.0", ] [[package]] @@ -3785,6 +3437,15 @@ version = "0.9.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "a2a2b11eda1d40935b26cf18f6833c526845ae8c41e58d09af6adeb6f0269183" +[[package]] +name = "fastrand" +version = "1.9.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e51093e27b0797c359783294ca4f0a911c270184cb10f85783b118614a1501be" +dependencies = [ + "instant", +] + [[package]] name = "fastrand" version = "2.3.0" @@ -4291,11 +3952,7 @@ version = "0.2.13" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "d48b897b4bbc881aea994b4a5bbb340a04979d7be9089791304e04a9fbc66b53" dependencies = [ -<<<<<<< HEAD - "thiserror 2.0.4", -======= "thiserror 2.0.6", ->>>>>>> e8bb86956 (feat: compiling CLI along with various fixes) ] [[package]] @@ -4304,11 +3961,7 @@ version = "0.4.10" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "c6ffbeb3a5c0b8b84c3fe4133a6f8c82fa962f4caefe8d0762eced025d3eb4f7" dependencies = [ -<<<<<<< HEAD - "thiserror 2.0.4", -======= "thiserror 2.0.6", ->>>>>>> e8bb86956 (feat: compiling CLI along with various fixes) ] [[package]] @@ -4356,11 +4009,7 @@ dependencies = [ "bstr", "gix-path", "libc", -<<<<<<< HEAD - "thiserror 2.0.4", -======= "thiserror 2.0.6", ->>>>>>> e8bb86956 (feat: compiling CLI along with various fixes) ] [[package]] @@ -4428,7 +4077,7 @@ version = "0.11.3" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "f2bfe6249cfea6d0c0e0990d5226a4cb36f030444ba9e35e0639275db8f98575" dependencies = [ - "fastrand 2.2.0", + "fastrand 2.3.0", "gix-features", "gix-utils", ] @@ -4583,11 +4232,7 @@ dependencies = [ "gix-trace", "home", "once_cell", -<<<<<<< HEAD - "thiserror 2.0.4", -======= "thiserror 2.0.6", ->>>>>>> e8bb86956 (feat: compiling CLI along with various fixes) ] [[package]] @@ -4598,11 +4243,7 @@ checksum = "64a1e282216ec2ab2816cd57e6ed88f8009e634aec47562883c05ac8a7009a63" dependencies = [ "bstr", "gix-utils", -<<<<<<< HEAD - "thiserror 2.0.4", -======= "thiserror 2.0.6", ->>>>>>> e8bb86956 (feat: compiling CLI along with various fixes) ] [[package]] @@ -4742,7 +4383,7 @@ version = "0.1.13" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "ba427e3e9599508ed98a6ddf8ed05493db114564e338e41f6a996d2e4790335f" dependencies = [ - "fastrand 2.2.0", + "fastrand 2.3.0", "unicode-normalization", ] @@ -4840,7 +4481,6 @@ dependencies = [ "slab", "tokio", "tokio-util 0.7.13", -<<<<<<< HEAD "tracing", ] @@ -4860,8 +4500,6 @@ dependencies = [ "slab", "tokio", "tokio-util 0.7.13", -======= ->>>>>>> e8bb86956 (feat: compiling CLI along with various fixes) "tracing", ] @@ -5019,9 +4657,9 @@ checksum = "b07f60793ff0a4d9cef0f18e63b5357e06209987153a64648c972c1e5aff336f" [[package]] name = "hickory-proto" -version = "0.24.1" +version = "0.24.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "07698b8420e2f0d6447a436ba999ec85d8fbf2a398bbd737b82cac4a2e96e512" +checksum = "447afdcdb8afb9d0a852af6dc65d9b285ce720ed7a59e42a8bf2e931c67bc1b5" dependencies = [ "async-trait", "cfg-if", @@ -5030,7 +4668,7 @@ dependencies = [ "futures-channel", "futures-io", "futures-util", - "idna 0.4.0", + "idna", "ipnet", "once_cell", "rand 0.8.5", @@ -5044,9 +4682,9 @@ dependencies = [ [[package]] name = "hickory-resolver" -version = "0.24.1" +version = "0.24.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "28757f23aa75c98f254cf0405e6d8c25b831b32921b050a66692427679b1f243" +checksum = "0a2e2aba9c389ce5267d31cf1e4dace82390ae276b0b364ea55630b1fa1b44b4" dependencies = [ "cfg-if", "futures-util", @@ -5249,10 +4887,7 @@ dependencies = [ "bytes", "futures-channel", "futures-util", -<<<<<<< HEAD "h2 0.4.7", -======= ->>>>>>> e8bb86956 (feat: compiling CLI along with various fixes) "http 1.2.0", "http-body 1.0.1", "httparse", @@ -5473,16 +5108,6 @@ version = "1.0.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "b9e0384b61958566e926dc50660321d12159025e767c18e043daf26b70104c39" -[[package]] -name = "idna" -version = "0.4.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7d20d6b07bfbc108882d88ed8e37d39636dcc260e15e30c45e6ba089610b917c" -dependencies = [ - "unicode-bidi", - "unicode-normalization", -] - [[package]] name = "idna" version = "1.0.3" @@ -5909,7 +5534,7 @@ dependencies = [ "multiaddr", "pin-project", "rw-stream-sink 0.4.0 (git+https://github.com/maqi/rust-libp2p.git?branch=master)", - "thiserror 2.0.4", + "thiserror 2.0.6", ] [[package]] @@ -5954,7 +5579,7 @@ dependencies = [ "quick-protobuf-codec 0.3.1 (git+https://github.com/maqi/rust-libp2p.git?branch=master)", "rand 0.8.5", "rand_core 0.6.4", - "thiserror 2.0.4", + "thiserror 2.0.6", "tracing", "web-time", ] @@ -6030,7 +5655,7 @@ dependencies = [ "rand 0.8.5", "rw-stream-sink 0.4.0 (git+https://github.com/maqi/rust-libp2p.git?branch=master)", "smallvec", - "thiserror 2.0.4", + "thiserror 2.0.6", "tracing", "unsigned-varint 0.8.0", "web-time", @@ -6130,7 +5755,7 @@ dependencies = [ "quick-protobuf", "quick-protobuf-codec 0.3.1 (git+https://github.com/maqi/rust-libp2p.git?branch=master)", "smallvec", - "thiserror 2.0.4", + "thiserror 2.0.6", "tracing", ] @@ -6204,7 +5829,7 @@ dependencies = [ "rand 0.8.5", "sha2 0.10.8", "smallvec", - "thiserror 2.0.4", + "thiserror 2.0.6", "tracing", "uint", "web-time", @@ -6265,7 +5890,7 @@ dependencies = [ "sha2 0.10.8", "snow", "static_assertions", - "thiserror 2.0.4", + "thiserror 2.0.6", "tracing", "x25519-dalek", "zeroize", @@ -6289,7 +5914,7 @@ dependencies = [ "ring 0.17.8", "rustls 0.23.19", "socket2", - "thiserror 2.0.4", + "thiserror 2.0.6", "tokio", "tracing", ] @@ -6312,7 +5937,7 @@ dependencies = [ "quick-protobuf-codec 0.3.1 (git+https://github.com/maqi/rust-libp2p.git?branch=master)", "rand 0.8.5", "static_assertions", - "thiserror 2.0.4", + "thiserror 2.0.6", "tracing", "web-time", ] @@ -6423,7 +6048,7 @@ dependencies = [ "ring 0.17.8", "rustls 0.23.19", "rustls-webpki 0.101.7", - "thiserror 2.0.4", + "thiserror 2.0.6", "x509-parser", "yasna", ] @@ -6456,7 +6081,7 @@ dependencies = [ "pin-project-lite", "rw-stream-sink 0.4.0 (git+https://github.com/maqi/rust-libp2p.git?branch=master)", "soketto", - "thiserror 2.0.4", + "thiserror 2.0.6", "tracing", "url", "webpki-roots 0.25.4", @@ -6473,7 +6098,7 @@ dependencies = [ "libp2p-core 0.42.1", "parking_lot", "send_wrapper 0.6.0", - "thiserror 2.0.4", + "thiserror 2.0.6", "tracing", "wasm-bindgen", "web-sys", @@ -6487,7 +6112,7 @@ dependencies = [ "either", "futures", "libp2p-core 0.42.1", - "thiserror 2.0.4", + "thiserror 2.0.6", "tracing", "yamux 0.12.1", "yamux 0.13.4", @@ -6903,9 +6528,9 @@ dependencies = [ [[package]] name = "netlink-sys" -version = "0.8.6" +version = "0.8.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "416060d346fbaf1f23f9512963e3e878f1a78e707cb699ba9215761754244307" +checksum = "16c903aa70590cb93691bf97a767c8d1d6122d2cc9070433deb3bbf36ce8bd23" dependencies = [ "bytes", "futures", @@ -8156,7 +7781,7 @@ dependencies = [ "asynchronous-codec", "bytes", "quick-protobuf", - "thiserror 2.0.4", + "thiserror 2.0.6", "unsigned-varint 0.8.0", ] @@ -8194,11 +7819,7 @@ dependencies = [ "rustc-hash", "rustls 0.23.19", "socket2", -<<<<<<< HEAD - "thiserror 2.0.4", -======= "thiserror 2.0.6", ->>>>>>> e8bb86956 (feat: compiling CLI along with various fixes) "tokio", "tracing", ] @@ -8217,11 +7838,7 @@ dependencies = [ "rustls 0.23.19", "rustls-pki-types", "slab", -<<<<<<< HEAD - "thiserror 2.0.4", -======= "thiserror 2.0.6", ->>>>>>> e8bb86956 (feat: compiling CLI along with various fixes) "tinyvec", "tracing", "web-time", @@ -9865,7 +9482,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "28cce251fcbc87fac86a866eeb0d6c2d536fc16d06f184bb61aeae11aa4cee0c" dependencies = [ "cfg-if", - "fastrand 2.2.0", + "fastrand 2.3.0", "once_cell", "rustix", "windows-sys 0.59.0", @@ -9923,19 +9540,11 @@ dependencies = [ [[package]] name = "thiserror" -<<<<<<< HEAD -version = "2.0.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2f49a1853cf82743e3b7950f77e0f4d622ca36cf4317cba00c767838bac8d490" -dependencies = [ - "thiserror-impl 2.0.4", -======= version = "2.0.6" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "8fec2a1820ebd077e2b90c4df007bebf344cd394098a13c563957d0afc83ea47" dependencies = [ "thiserror-impl 2.0.6", ->>>>>>> e8bb86956 (feat: compiling CLI along with various fixes) ] [[package]] @@ -9951,15 +9560,9 @@ dependencies = [ [[package]] name = "thiserror-impl" -<<<<<<< HEAD -version = "2.0.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8381894bb3efe0c4acac3ded651301ceee58a15d47c2e34885ed1908ad667061" -======= version = "2.0.6" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "d65750cab40f4ff1929fb1ba509e9914eb756131cef4210da8d5d700d26f6312" ->>>>>>> e8bb86956 (feat: compiling CLI along with various fixes) dependencies = [ "proc-macro2", "quote", @@ -10608,12 +10211,6 @@ version = "2.8.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "7e51b68083f157f853b6379db119d1c1be0e6e4dec98101079dec41f6f5cf6df" -[[package]] -name = "unicode-bidi" -version = "0.3.17" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5ab17db44d7388991a428b2ee655ce0c212e862eff1768a455c58f9aad6e7893" - [[package]] name = "unicode-bom" version = "2.0.3" @@ -10723,7 +10320,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "32f8b686cadd1473f4bd0117a5d28d36b1ade384ea9b5069a1c40aefed7fda60" dependencies = [ "form_urlencoded", - "idna 1.0.3", + "idna", "percent-encoding", "serde", ] diff --git a/ant-node/src/node.rs b/ant-node/src/node.rs index b2b08232ef..7c028dac5f 100644 --- a/ant-node/src/node.rs +++ b/ant-node/src/node.rs @@ -13,7 +13,7 @@ use super::{ use crate::metrics::NodeMetricsRecorder; use crate::RunningNode; use ant_bootstrap::BootstrapCacheStore; -use ant_evm::{AttoTokens, RewardsAddress}; +use ant_evm::RewardsAddress; #[cfg(feature = "open-metrics")] use ant_networking::MetricsRegistries; use ant_networking::{Instant, Network, NetworkBuilder, NetworkEvent, NodeIssue, SwarmDriver}; diff --git a/autonomi/src/client/data/mod.rs b/autonomi/src/client/data/mod.rs index bf5ba191e8..e1967f0c95 100644 --- a/autonomi/src/client/data/mod.rs +++ b/autonomi/src/client/data/mod.rs @@ -215,7 +215,7 @@ impl Client { if let Some(channel) = self.client_event_sender.as_ref() { let tokens_spent = receipt .values() - .map(|proof| proof.quote.cost.as_atto()) + .map(|(_, cost)| cost.as_atto()) .sum::(); let summary = UploadSummary { diff --git a/autonomi/src/client/data/public.rs b/autonomi/src/client/data/public.rs index 28bad9dc4f..9f758edde8 100644 --- a/autonomi/src/client/data/public.rs +++ b/autonomi/src/client/data/public.rs @@ -10,8 +10,6 @@ use bytes::Bytes; use libp2p::kad::Quorum; use std::collections::HashSet; -use xor_name::XorName; - use crate::client::payment::{PaymentOption, Receipt}; use crate::client::utils::process_tasks_with_max_concurrency; use crate::client::{ClientEvent, UploadSummary}; diff --git a/autonomi/src/client/external_signer.rs b/autonomi/src/client/external_signer.rs index 0037ba7e25..30114712f3 100644 --- a/autonomi/src/client/external_signer.rs +++ b/autonomi/src/client/external_signer.rs @@ -31,12 +31,8 @@ impl Client { let free_chunks = content_addrs .filter(|addr| !quote.0.contains_key(addr)) .collect(); - let quotes_per_addr = quote.0.into_iter().collect(); + let quotes_per_addr: HashMap<_, _> = quote.0.into_iter().collect(); - debug!( - "Got the quotes , quote_payments and freechunks from the network {:?}", - (quotes_per_addr.clone(), payments.clone(), free_chunks.clone()) - ); Ok((quotes_per_addr, payments, free_chunks)) } } diff --git a/autonomi/src/client/mod.rs b/autonomi/src/client/mod.rs index b01a6a9b2d..15e1c83ae1 100644 --- a/autonomi/src/client/mod.rs +++ b/autonomi/src/client/mod.rs @@ -34,7 +34,7 @@ mod utils; use ant_bootstrap::{BootstrapCacheConfig, BootstrapCacheStore}; pub use ant_evm::Amount; -use crate::EvmNetwork; +use ant_evm::EvmNetwork; use ant_networking::{interval, multiaddr_is_global, Network, NetworkBuilder, NetworkEvent}; use ant_protocol::{version::IDENTIFY_PROTOCOL_STR, CLOSE_GROUP_SIZE}; use libp2p::{identity::Keypair, Multiaddr}; diff --git a/autonomi/src/client/registers.rs b/autonomi/src/client/registers.rs index 0fc502426b..fa353d4873 100644 --- a/autonomi/src/client/registers.rs +++ b/autonomi/src/client/registers.rs @@ -355,7 +355,7 @@ impl Client { let put_cfg = PutRecordCfg { put_quorum: Quorum::All, retry_strategy: None, - use_put_record_to: Some(payees), // CODE REVIEW: should we put to all or just one here? + use_put_record_to: Some(payees), verification: Some((VerificationKind::Network, get_cfg)), }; diff --git a/autonomi/src/client/utils.rs b/autonomi/src/client/utils.rs index 915d8c8d41..ad2aeececb 100644 --- a/autonomi/src/client/utils.rs +++ b/autonomi/src/client/utils.rs @@ -151,11 +151,11 @@ impl Client { let put_cfg = PutRecordCfg { put_quorum: Quorum::One, retry_strategy: Some(RetryStrategy::Balanced), - use_put_record_to: Some(storing_nodes), // CODE REVIEW: do we put to all payees or just one? + use_put_record_to: Some(storing_nodes.clone()), verification, }; let payment_upload = Ok(self.network.put_record(record, &put_cfg).await?); - debug!("Successfully stored chunk: {chunk:?} to {storing_node:?}"); + debug!("Successfully stored chunk: {chunk:?} to {storing_nodes:?}"); payment_upload } diff --git a/autonomi/tests/external_signer.rs b/autonomi/tests/external_signer.rs index fa648e8461..39cb49cb64 100644 --- a/autonomi/tests/external_signer.rs +++ b/autonomi/tests/external_signer.rs @@ -6,7 +6,7 @@ use ant_evm::{QuoteHash, TxHash}; use ant_logging::LogBuilder; use autonomi::client::external_signer::encrypt_data; use autonomi::client::files::archive::{Metadata, PrivateArchive}; -use autonomi::client::payment::{receipt_from_store_quotes_and_payments, Receipt}; +use autonomi::client::payment::Receipt; use autonomi::client::vault::user_data::USER_DATA_VAULT_CONTENT_IDENTIFIER; use autonomi::client::vault::VaultSecretKey; use autonomi::{Client, Wallet}; diff --git a/evmlib/src/contract/payment_vault/mod.rs b/evmlib/src/contract/payment_vault/mod.rs index efa6f4fbd0..7658bac9d7 100644 --- a/evmlib/src/contract/payment_vault/mod.rs +++ b/evmlib/src/contract/payment_vault/mod.rs @@ -41,7 +41,7 @@ pub async fn verify_data_payment( let payment_verification_results = payment_vault.verify_payment(payment_verifications).await?; for payment_verification_result in payment_verification_results { - // CODE REVIEW: should we fail on a single invalid payment? + // TODO we currently fail on a single invalid payment, maybe we should deal with this in a different way if !payment_verification_result.isValid { return Err(error::Error::PaymentInvalid); } From 7766ff094ed248b0d5d763e33c6b0bc5e42018ee Mon Sep 17 00:00:00 2001 From: grumbach Date: Wed, 11 Dec 2024 15:35:01 +0900 Subject: [PATCH 21/33] chore: clippy --- autonomi/src/client/quote.rs | 2 +- autonomi/tests/external_signer.rs | 5 +++-- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/autonomi/src/client/quote.rs b/autonomi/src/client/quote.rs index 2c527fafd2..789db9613a 100644 --- a/autonomi/src/client/quote.rs +++ b/autonomi/src/client/quote.rs @@ -29,7 +29,7 @@ impl QuoteForAddress { } /// A quote for many addresses -pub struct StoreQuote(pub(crate) HashMap); +pub struct StoreQuote(pub HashMap); impl StoreQuote { pub fn price(&self) -> Amount { diff --git a/autonomi/tests/external_signer.rs b/autonomi/tests/external_signer.rs index 39cb49cb64..6b918f9370 100644 --- a/autonomi/tests/external_signer.rs +++ b/autonomi/tests/external_signer.rs @@ -6,7 +6,8 @@ use ant_evm::{QuoteHash, TxHash}; use ant_logging::LogBuilder; use autonomi::client::external_signer::encrypt_data; use autonomi::client::files::archive::{Metadata, PrivateArchive}; -use autonomi::client::payment::Receipt; +use autonomi::client::payment::{receipt_from_store_quotes, Receipt}; +use autonomi::client::quote::StoreQuote; use autonomi::client::vault::user_data::USER_DATA_VAULT_CONTENT_IDENTIFIER; use autonomi::client::vault::VaultSecretKey; use autonomi::{Client, Wallet}; @@ -93,7 +94,7 @@ async fn pay_for_content_addresses( } // Payment proofs - Ok(receipt_from_store_quotes_and_payments("es, payments)) + Ok(receipt_from_store_quotes(StoreQuote(quotes))) } // Example of how put would be done using external signers. From 9592fa404b077b30a0fee761e0f7e9d86317982a Mon Sep 17 00:00:00 2001 From: grumbach Date: Wed, 11 Dec 2024 15:52:16 +0900 Subject: [PATCH 22/33] ci: increase upload timeout --- .github/workflows/memcheck.yml | 2 +- .github/workflows/merge.yml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/.github/workflows/memcheck.yml b/.github/workflows/memcheck.yml index 3eca5f494d..e6556b9f57 100644 --- a/.github/workflows/memcheck.yml +++ b/.github/workflows/memcheck.yml @@ -73,7 +73,7 @@ jobs: run: ./target/release/ant --log-output-dest=data-dir file upload --public "./the-test-data.zip" > ./upload_output 2>&1 env: ANT_LOG: "v" - timeout-minutes: 5 + timeout-minutes: 15 - name: showing the upload terminal output run: cat upload_output diff --git a/.github/workflows/merge.yml b/.github/workflows/merge.yml index 564de2707e..9ccbef19d4 100644 --- a/.github/workflows/merge.yml +++ b/.github/workflows/merge.yml @@ -1333,7 +1333,7 @@ jobs: run: ./target/release/ant --log-output-dest data-dir file upload "./test_data_1.tar.gz" > ./upload_output 2>&1 env: ANT_LOG: "v" - timeout-minutes: 5 + timeout-minutes: 15 - name: showing the upload terminal output run: cat upload_output From d4d6e8c794b16a20c6fd32f33f5ba99d1364bd45 Mon Sep 17 00:00:00 2001 From: grumbach Date: Wed, 11 Dec 2024 17:04:11 +0900 Subject: [PATCH 23/33] fix: prepare for smart contract quote batching --- autonomi/src/client/quote.rs | 41 ++++++-------------- evmlib/src/contract/payment_vault/handler.rs | 29 ++++++++++++-- evmlib/src/contract/payment_vault/mod.rs | 4 +- evmlib/tests/payment_vault.rs | 4 +- 4 files changed, 41 insertions(+), 37 deletions(-) diff --git a/autonomi/src/client/quote.rs b/autonomi/src/client/quote.rs index 789db9613a..0aa2850af4 100644 --- a/autonomi/src/client/quote.rs +++ b/autonomi/src/client/quote.rs @@ -8,17 +8,13 @@ use super::{data::CostError, Client}; use ant_evm::payment_vault::get_market_price; -use ant_evm::{Amount, PaymentQuote, QuotePayment}; -use ant_networking::target_arch::{sleep, Duration, Instant}; +use ant_evm::{Amount, PaymentQuote, QuotePayment, QuotingMetrics}; use ant_networking::{Network, NetworkError}; use ant_protocol::{storage::ChunkAddress, NetworkAddress}; use libp2p::PeerId; use std::collections::HashMap; use xor_name::XorName; -// set rate limit to 2 req/s -const TIME_BETWEEN_RPC_CALLS_IN_MS: u64 = 500; - /// A quote for a single address pub struct QuoteForAddress(pub(crate) Vec<(PeerId, PaymentQuote, Amount)>); @@ -73,30 +69,17 @@ impl Client { let mut quotes_to_pay_per_addr = HashMap::new(); for (content_addr, raw_quotes) in raw_quotes_per_addr { // ask smart contract for the market price - let mut prices = vec![]; - - // rate limit - let mut maybe_last_call: Option = None; - - for (peer, quote) in raw_quotes { - // NB TODO @mick we need to batch this smart contract call - // check if we have to wait for the rate limit - if let Some(last_call) = maybe_last_call { - let elapsed = Instant::now() - last_call; - let time_to_sleep_ms = - TIME_BETWEEN_RPC_CALLS_IN_MS as u128 - elapsed.as_millis(); - if time_to_sleep_ms > 0 { - sleep(Duration::from_millis(time_to_sleep_ms as u64)).await; - } - } - - let price = - get_market_price(&self.evm_network, quote.quoting_metrics.clone()).await?; - - maybe_last_call = Some(Instant::now()); - - prices.push((peer, quote, price)); - } + let quoting_metrics: Vec = raw_quotes + .clone() + .iter() + .map(|(_, q)| q.quoting_metrics.clone()) + .collect(); + let all_prices = get_market_price(&self.evm_network, quoting_metrics).await?; + let mut prices: Vec<(PeerId, PaymentQuote, Amount)> = all_prices + .into_iter() + .zip(raw_quotes.into_iter()) + .map(|(price, (peer, quote))| (peer, quote, price)) + .collect(); // sort by price prices.sort_by(|(_, _, price_a), (_, _, price_b)| price_a.cmp(price_b)); diff --git a/evmlib/src/contract/payment_vault/handler.rs b/evmlib/src/contract/payment_vault/handler.rs index 5f21e5574e..29ef362b51 100644 --- a/evmlib/src/contract/payment_vault/handler.rs +++ b/evmlib/src/contract/payment_vault/handler.rs @@ -31,12 +31,33 @@ where } /// Fetch a quote from the contract - pub async fn get_quote>( + pub async fn get_quote>>( &self, metrics: I, - ) -> Result { - let amount = self.contract.getQuote(metrics.into()).call().await?.price; - Ok(amount) + ) -> Result, Error> { + // NB TODO @mick we need to batch this smart contract call + let mut amounts = vec![]; + + // set rate limit to 2 req/s + const TIME_BETWEEN_RPC_CALLS_IN_MS: u64 = 700; + let mut maybe_last_call: Option = None; + for metric in metrics { + // check if we have to wait for the rate limit + if let Some(last_call) = maybe_last_call { + let elapsed = std::time::Instant::now() - last_call; + let time_to_sleep_ms = TIME_BETWEEN_RPC_CALLS_IN_MS as u128 - elapsed.as_millis(); + if time_to_sleep_ms > 0 { + tokio::time::sleep(std::time::Duration::from_millis(time_to_sleep_ms as u64)) + .await; + } + } + + let amount = self.contract.getQuote(metric.into()).call().await?.price; + amounts.push(amount); + maybe_last_call = Some(std::time::Instant::now()); + } + + Ok(amounts) } /// Pay for quotes. diff --git a/evmlib/src/contract/payment_vault/mod.rs b/evmlib/src/contract/payment_vault/mod.rs index 7658bac9d7..9b7d1eed3b 100644 --- a/evmlib/src/contract/payment_vault/mod.rs +++ b/evmlib/src/contract/payment_vault/mod.rs @@ -14,8 +14,8 @@ pub const MAX_TRANSFERS_PER_TRANSACTION: usize = 256; /// Helper function to return a quote for the given quoting metrics. pub async fn get_market_price( network: &Network, - quoting_metrics: QuotingMetrics, -) -> Result { + quoting_metrics: Vec, +) -> Result, error::Error> { let provider = http_provider(network.rpc_url().clone()); let payment_vault = PaymentVaultHandler::new(*network.data_payments_address(), provider); payment_vault.get_quote(quoting_metrics).await diff --git a/evmlib/tests/payment_vault.rs b/evmlib/tests/payment_vault.rs index fe2df5905f..7578786c11 100644 --- a/evmlib/tests/payment_vault.rs +++ b/evmlib/tests/payment_vault.rs @@ -124,11 +124,11 @@ async fn test_proxy_reachable() { let payment_vault = PaymentVaultHandler::new(*network.data_payments_address(), provider); let amount = payment_vault - .get_quote(QuotingMetrics::default()) + .get_quote(vec![QuotingMetrics::default()]) .await .unwrap(); - assert_eq!(amount, Amount::from(1)); + assert_eq!(amount, vec![Amount::from(1)]); } #[tokio::test] From 06d93e73321adc942e486f578e5f360d66fd8bab Mon Sep 17 00:00:00 2001 From: Warm Beer Date: Wed, 11 Dec 2024 11:09:09 +0100 Subject: [PATCH 24/33] chore: remove excessive debug log --- autonomi/src/client/quote.rs | 2 -- 1 file changed, 2 deletions(-) diff --git a/autonomi/src/client/quote.rs b/autonomi/src/client/quote.rs index 0aa2850af4..c8cc8058d7 100644 --- a/autonomi/src/client/quote.rs +++ b/autonomi/src/client/quote.rs @@ -63,8 +63,6 @@ impl Client { .collect(); let raw_quotes_per_addr = futures::future::try_join_all(futures).await?; - debug!("Fetched store quotes: {raw_quotes_per_addr:?}"); - // choose the quotes to pay for each address let mut quotes_to_pay_per_addr = HashMap::new(); for (content_addr, raw_quotes) in raw_quotes_per_addr { From e85d4e46030fa09b4935614b0578191804c7b6e6 Mon Sep 17 00:00:00 2001 From: Warm Beer Date: Wed, 11 Dec 2024 11:10:00 +0100 Subject: [PATCH 25/33] chore: update payment vault interface and implementation --- evmlib/abi/IPaymentVault.json | 10 +- evmlib/artifacts/PaymentVaultNoProxy.json | 14 +- evmlib/src/contract/payment_vault/handler.rs | 27 +--- evmlib/src/transaction.rs | 156 ------------------- 4 files changed, 19 insertions(+), 188 deletions(-) delete mode 100644 evmlib/src/transaction.rs diff --git a/evmlib/abi/IPaymentVault.json b/evmlib/abi/IPaymentVault.json index d2bc495a5f..5f34d178f7 100644 --- a/evmlib/abi/IPaymentVault.json +++ b/evmlib/abi/IPaymentVault.json @@ -74,17 +74,17 @@ "type": "uint256" } ], - "internalType": "struct IPaymentVault.QuotingMetrics", + "internalType": "struct IPaymentVault.QuotingMetrics[]", "name": "_metrics", - "type": "tuple" + "type": "tuple[]" } ], "name": "getQuote", "outputs": [ { - "internalType": "uint256", - "name": "price", - "type": "uint256" + "internalType": "uint256[]", + "name": "prices", + "type": "uint256[]" } ], "stateMutability": "view", diff --git a/evmlib/artifacts/PaymentVaultNoProxy.json b/evmlib/artifacts/PaymentVaultNoProxy.json index 5514cc77f7..914e28d0f3 100644 --- a/evmlib/artifacts/PaymentVaultNoProxy.json +++ b/evmlib/artifacts/PaymentVaultNoProxy.json @@ -158,17 +158,17 @@ "type": "uint256" } ], - "internalType": "struct IPaymentVault.QuotingMetrics", + "internalType": "struct IPaymentVault.QuotingMetrics[]", "name": "", - "type": "tuple" + "type": "tuple[]" } ], "name": "getQuote", "outputs": [ { - "internalType": "uint256", - "name": "price", - "type": "uint256" + "internalType": "uint256[]", + "name": "prices", + "type": "uint256[]" } ], "stateMutability": "pure", @@ -344,8 +344,8 @@ "type": "function" } ], - "bytecode": "0x6080604052348015600f57600080fd5b50604051610cce380380610cce833981016040819052602c91607f565b6001600160a01b038216605257604051632d06160b60e21b815260040160405180910390fd5b600180546001600160a01b0319166001600160a01b039390931692909217909155600055600560035560b7565b60008060408385031215609157600080fd5b82516001600160a01b038116811460a757600080fd5b6020939093015192949293505050565b610c08806100c66000396000f3fe608060405234801561001057600080fd5b506004361061007d5760003560e01c80634ec42e8e1161005b5780634ec42e8e14610111578063b6c2141b1461013c578063bcb2c1da14610151578063c7170bb61461017157600080fd5b80630716326d146100825780633c150bf2146100e6578063474740b114610108575b600080fd5b6100bc61009036600461082f565b60026020819052600091825260409091208054600182015491909201546001600160a01b039092169183565b604080516001600160a01b0390941684526020840192909252908201526060015b60405180910390f35b6100fa6100f4366004610848565b50600190565b6040519081526020016100dd565b6100fa60005481565b600154610124906001600160a01b031681565b6040516001600160a01b0390911681526020016100dd565b61014f61014a366004610863565b61017a565b005b61016461015f3660046108da565b61026a565b6040516100dd9190610942565b6100fa60035481565b600054819081111561019f57604051630d67f41160e21b815260040160405180910390fd5b60005b8181101561026457368484838181106101bd576101bd610992565b6060029190910191506101f19050336101d960208401846109bd565b6001546001600160a01b0316919060208501356103f5565b6040808201356000908152600260205220819061020e82826109da565b505060408101356020820180359061022690846109bd565b6001600160a01b03167ff998960b1c6f0e0e89b7bbe6b6fbf3e03e6f08eee5b8430877d8adb8e149d58060405160405180910390a4506001016101a2565b50505050565b61027261075c565b600354821461029457604051637db491eb60e01b815260040160405180910390fd5b60006102a0848461044f565b905060005b60038110156103ed576000600260008484600381106102c6576102c6610992565b602090810291909101518101516040908101518352828201939093529082016000908120835160608101855281546001600160a01b031681526001820154938101939093526002015492820192909252915083836003811061032a5761032a610992565b602002015160200151602001518260200151149050600084846003811061035357610353610992565b602002015160200151600001516001600160a01b031683600001516001600160a01b03161490506000604051806060016040528087876003811061039957610399610992565b602002015160200151604001518152602001856020015181526020018480156103bf5750835b151590529050808786600381106103d8576103d8610992565b60200201525050600190920191506102a59050565b505092915050565b604080516001600160a01b0385811660248301528416604482015260648082018490528251808303909101815260849091019091526020810180516001600160e01b03166323b872dd60e01b1790526102649085906105b5565b61045761079b565b60005b828110156105ae578151602090810151015184848381811061047e5761047e610992565b9050610120020160c0016020013511156104d8576020820180516040840152825190528383828181106104b3576104b3610992565b905061012002018036038101906104ca9190610af1565b8260005b60200201526105a6565b60208083015181015101518484838181106104f5576104f5610992565b9050610120020160c001602001351115610544576020820151604083015283838281811061052557610525610992565b9050610120020180360381019061053c9190610af1565b8260016104ce565b6040820151602090810151015184848381811061056357610563610992565b9050610120020160c0016020013511156105a65783838281811061058957610589610992565b905061012002018036038101906105a09190610af1565b60408301525b60010161045a565b5092915050565b60006105ca6001600160a01b03841683610622565b905080516000141580156105ef5750808060200190518101906105ed9190610b81565b155b1561061d57604051635274afe760e01b81526001600160a01b03841660048201526024015b60405180910390fd5b505050565b606061063083836000610637565b9392505050565b60608147101561065c5760405163cd78605960e01b8152306004820152602401610614565b600080856001600160a01b031684866040516106789190610ba3565b60006040518083038185875af1925050503d80600081146106b5576040519150601f19603f3d011682016040523d82523d6000602084013e6106ba565b606091505b50915091506106ca8683836106d4565b9695505050505050565b6060826106e9576106e482610730565b610630565b815115801561070057506001600160a01b0384163b155b1561072957604051639996b31560e01b81526001600160a01b0385166004820152602401610614565b5080610630565b8051156107405780518082602001fd5b604051630a12f52160e11b815260040160405180910390fd5b50565b60405180606001604052806003905b604080516060810182526000808252602080830182905292820152825260001990920191018161076b5790505090565b60405180606001604052806003905b6107b26107c8565b8152602001906001900390816107aa5790505090565b604051806040016040528061080c6040518060c001604052806000815260200160008152602001600081526020016000815260200160008152602001600081525090565b815260408051606081018252600080825260208281018290529282015291015290565b60006020828403121561084157600080fd5b5035919050565b600060c082840312801561085b57600080fd5b509092915050565b6000806020838503121561087657600080fd5b823567ffffffffffffffff81111561088d57600080fd5b8301601f8101851361089e57600080fd5b803567ffffffffffffffff8111156108b557600080fd5b8560206060830284010111156108ca57600080fd5b6020919091019590945092505050565b600080602083850312156108ed57600080fd5b823567ffffffffffffffff81111561090457600080fd5b8301601f8101851361091557600080fd5b803567ffffffffffffffff81111561092c57600080fd5b856020610120830284010111156108ca57600080fd5b6101208101818360005b600381101561098957815180518452602081015160208501526040810151151560408501525060608301925060208201915060018101905061094c565b50505092915050565b634e487b7160e01b600052603260045260246000fd5b6001600160a01b038116811461075957600080fd5b6000602082840312156109cf57600080fd5b8135610630816109a8565b81356109e5816109a8565b81546001600160a01b0319166001600160a01b039190911617815560208201356001820155604090910135600290910155565b6040805190810167ffffffffffffffff81118282101715610a4957634e487b7160e01b600052604160045260246000fd5b60405290565b60405160c0810167ffffffffffffffff81118282101715610a4957634e487b7160e01b600052604160045260246000fd5b600060608284031215610a9257600080fd5b6040516060810167ffffffffffffffff81118282101715610ac357634e487b7160e01b600052604160045260246000fd5b6040529050808235610ad4816109a8565b815260208381013590820152604092830135920191909152919050565b600081830361012081128015610b0657600080fd5b506000610b11610a18565b60c0831215610b1e578182fd5b610b26610a4f565b853581526020808701359082015260408087013590820152606080870135908201526080808701359082015260a0808701359082018190528183529093509150610b738660c08701610a80565b602082015295945050505050565b600060208284031215610b9357600080fd5b8151801515811461063057600080fd5b6000825160005b81811015610bc45760208186018101518583015201610baa565b50600092019182525091905056fea2646970667358221220fd6ef361aaba52d0f9503b51aea1d0b7a8363a9a66c9502aa7b931f1f44c507f64736f6c634300081c0033", - "deployedBytecode": "0x608060405234801561001057600080fd5b506004361061007d5760003560e01c80634ec42e8e1161005b5780634ec42e8e14610111578063b6c2141b1461013c578063bcb2c1da14610151578063c7170bb61461017157600080fd5b80630716326d146100825780633c150bf2146100e6578063474740b114610108575b600080fd5b6100bc61009036600461082f565b60026020819052600091825260409091208054600182015491909201546001600160a01b039092169183565b604080516001600160a01b0390941684526020840192909252908201526060015b60405180910390f35b6100fa6100f4366004610848565b50600190565b6040519081526020016100dd565b6100fa60005481565b600154610124906001600160a01b031681565b6040516001600160a01b0390911681526020016100dd565b61014f61014a366004610863565b61017a565b005b61016461015f3660046108da565b61026a565b6040516100dd9190610942565b6100fa60035481565b600054819081111561019f57604051630d67f41160e21b815260040160405180910390fd5b60005b8181101561026457368484838181106101bd576101bd610992565b6060029190910191506101f19050336101d960208401846109bd565b6001546001600160a01b0316919060208501356103f5565b6040808201356000908152600260205220819061020e82826109da565b505060408101356020820180359061022690846109bd565b6001600160a01b03167ff998960b1c6f0e0e89b7bbe6b6fbf3e03e6f08eee5b8430877d8adb8e149d58060405160405180910390a4506001016101a2565b50505050565b61027261075c565b600354821461029457604051637db491eb60e01b815260040160405180910390fd5b60006102a0848461044f565b905060005b60038110156103ed576000600260008484600381106102c6576102c6610992565b602090810291909101518101516040908101518352828201939093529082016000908120835160608101855281546001600160a01b031681526001820154938101939093526002015492820192909252915083836003811061032a5761032a610992565b602002015160200151602001518260200151149050600084846003811061035357610353610992565b602002015160200151600001516001600160a01b031683600001516001600160a01b03161490506000604051806060016040528087876003811061039957610399610992565b602002015160200151604001518152602001856020015181526020018480156103bf5750835b151590529050808786600381106103d8576103d8610992565b60200201525050600190920191506102a59050565b505092915050565b604080516001600160a01b0385811660248301528416604482015260648082018490528251808303909101815260849091019091526020810180516001600160e01b03166323b872dd60e01b1790526102649085906105b5565b61045761079b565b60005b828110156105ae578151602090810151015184848381811061047e5761047e610992565b9050610120020160c0016020013511156104d8576020820180516040840152825190528383828181106104b3576104b3610992565b905061012002018036038101906104ca9190610af1565b8260005b60200201526105a6565b60208083015181015101518484838181106104f5576104f5610992565b9050610120020160c001602001351115610544576020820151604083015283838281811061052557610525610992565b9050610120020180360381019061053c9190610af1565b8260016104ce565b6040820151602090810151015184848381811061056357610563610992565b9050610120020160c0016020013511156105a65783838281811061058957610589610992565b905061012002018036038101906105a09190610af1565b60408301525b60010161045a565b5092915050565b60006105ca6001600160a01b03841683610622565b905080516000141580156105ef5750808060200190518101906105ed9190610b81565b155b1561061d57604051635274afe760e01b81526001600160a01b03841660048201526024015b60405180910390fd5b505050565b606061063083836000610637565b9392505050565b60608147101561065c5760405163cd78605960e01b8152306004820152602401610614565b600080856001600160a01b031684866040516106789190610ba3565b60006040518083038185875af1925050503d80600081146106b5576040519150601f19603f3d011682016040523d82523d6000602084013e6106ba565b606091505b50915091506106ca8683836106d4565b9695505050505050565b6060826106e9576106e482610730565b610630565b815115801561070057506001600160a01b0384163b155b1561072957604051639996b31560e01b81526001600160a01b0385166004820152602401610614565b5080610630565b8051156107405780518082602001fd5b604051630a12f52160e11b815260040160405180910390fd5b50565b60405180606001604052806003905b604080516060810182526000808252602080830182905292820152825260001990920191018161076b5790505090565b60405180606001604052806003905b6107b26107c8565b8152602001906001900390816107aa5790505090565b604051806040016040528061080c6040518060c001604052806000815260200160008152602001600081526020016000815260200160008152602001600081525090565b815260408051606081018252600080825260208281018290529282015291015290565b60006020828403121561084157600080fd5b5035919050565b600060c082840312801561085b57600080fd5b509092915050565b6000806020838503121561087657600080fd5b823567ffffffffffffffff81111561088d57600080fd5b8301601f8101851361089e57600080fd5b803567ffffffffffffffff8111156108b557600080fd5b8560206060830284010111156108ca57600080fd5b6020919091019590945092505050565b600080602083850312156108ed57600080fd5b823567ffffffffffffffff81111561090457600080fd5b8301601f8101851361091557600080fd5b803567ffffffffffffffff81111561092c57600080fd5b856020610120830284010111156108ca57600080fd5b6101208101818360005b600381101561098957815180518452602081015160208501526040810151151560408501525060608301925060208201915060018101905061094c565b50505092915050565b634e487b7160e01b600052603260045260246000fd5b6001600160a01b038116811461075957600080fd5b6000602082840312156109cf57600080fd5b8135610630816109a8565b81356109e5816109a8565b81546001600160a01b0319166001600160a01b039190911617815560208201356001820155604090910135600290910155565b6040805190810167ffffffffffffffff81118282101715610a4957634e487b7160e01b600052604160045260246000fd5b60405290565b60405160c0810167ffffffffffffffff81118282101715610a4957634e487b7160e01b600052604160045260246000fd5b600060608284031215610a9257600080fd5b6040516060810167ffffffffffffffff81118282101715610ac357634e487b7160e01b600052604160045260246000fd5b6040529050808235610ad4816109a8565b815260208381013590820152604092830135920191909152919050565b600081830361012081128015610b0657600080fd5b506000610b11610a18565b60c0831215610b1e578182fd5b610b26610a4f565b853581526020808701359082015260408087013590820152606080870135908201526080808701359082015260a0808701359082018190528183529093509150610b738660c08701610a80565b602082015295945050505050565b600060208284031215610b9357600080fd5b8151801515811461063057600080fd5b6000825160005b81811015610bc45760208186018101518583015201610baa565b50600092019182525091905056fea2646970667358221220fd6ef361aaba52d0f9503b51aea1d0b7a8363a9a66c9502aa7b931f1f44c507f64736f6c634300081c0033", + "bytecode": "0x6080604052348015600f57600080fd5b50604051610db6380380610db6833981016040819052602c91607f565b6001600160a01b038216605257604051632d06160b60e21b815260040160405180910390fd5b600180546001600160a01b0319166001600160a01b039390931692909217909155600055600560035560b7565b60008060408385031215609157600080fd5b82516001600160a01b038116811460a757600080fd5b6020939093015192949293505050565b610cf0806100c66000396000f3fe608060405234801561001057600080fd5b506004361061007d5760003560e01c806380a38d971161005b57806380a38d9714610128578063b6c2141b14610148578063bcb2c1da1461015d578063c7170bb61461017d57600080fd5b80630716326d14610082578063474740b1146100e65780634ec42e8e146100fd575b600080fd5b6100bc610090366004610888565b60026020819052600091825260409091208054600182015491909201546001600160a01b039092169183565b604080516001600160a01b0390941684526020840192909252908201526060015b60405180910390f35b6100ef60005481565b6040519081526020016100dd565b600154610110906001600160a01b031681565b6040516001600160a01b0390911681526020016100dd565b61013b6101363660046108a1565b610186565b6040516100dd9190610918565b61015b61015636600461095b565b6101d3565b005b61017061016b3660046109c2565b6102c3565b6040516100dd9190610a2a565b6100ef60035481565b60408051600180825281830190925260609160009190602080830190803683370190505090506001816000815181106101c1576101c1610a7a565b60209081029190910101529392505050565b60005481908111156101f857604051630d67f41160e21b815260040160405180910390fd5b60005b818110156102bd573684848381811061021657610216610a7a565b60600291909101915061024a9050336102326020840184610aa5565b6001546001600160a01b03169190602085013561044e565b604080820135600090815260026020522081906102678282610ac2565b505060408101356020820180359061027f9084610aa5565b6001600160a01b03167ff998960b1c6f0e0e89b7bbe6b6fbf3e03e6f08eee5b8430877d8adb8e149d58060405160405180910390a4506001016101fb565b50505050565b6102cb6107b5565b60035482146102ed57604051637db491eb60e01b815260040160405180910390fd5b60006102f984846104a8565b905060005b60038110156104465760006002600084846003811061031f5761031f610a7a565b602090810291909101518101516040908101518352828201939093529082016000908120835160608101855281546001600160a01b031681526001820154938101939093526002015492820192909252915083836003811061038357610383610a7a565b60200201516020015160200151826020015114905060008484600381106103ac576103ac610a7a565b602002015160200151600001516001600160a01b031683600001516001600160a01b0316149050600060405180606001604052808787600381106103f2576103f2610a7a565b602002015160200151604001518152602001856020015181526020018480156104185750835b1515905290508087866003811061043157610431610a7a565b60200201525050600190920191506102fe9050565b505092915050565b604080516001600160a01b0385811660248301528416604482015260648082018490528251808303909101815260849091019091526020810180516001600160e01b03166323b872dd60e01b1790526102bd90859061060e565b6104b06107f4565b60005b8281101561060757815160209081015101518484838181106104d7576104d7610a7a565b9050610120020160c0016020013511156105315760208201805160408401528251905283838281811061050c5761050c610a7a565b905061012002018036038101906105239190610bd9565b8260005b60200201526105ff565b602080830151810151015184848381811061054e5761054e610a7a565b9050610120020160c00160200135111561059d576020820151604083015283838281811061057e5761057e610a7a565b905061012002018036038101906105959190610bd9565b826001610527565b604082015160209081015101518484838181106105bc576105bc610a7a565b9050610120020160c0016020013511156105ff578383828181106105e2576105e2610a7a565b905061012002018036038101906105f99190610bd9565b60408301525b6001016104b3565b5092915050565b60006106236001600160a01b0384168361067b565b905080516000141580156106485750808060200190518101906106469190610c69565b155b1561067657604051635274afe760e01b81526001600160a01b03841660048201526024015b60405180910390fd5b505050565b606061068983836000610690565b9392505050565b6060814710156106b55760405163cd78605960e01b815230600482015260240161066d565b600080856001600160a01b031684866040516106d19190610c8b565b60006040518083038185875af1925050503d806000811461070e576040519150601f19603f3d011682016040523d82523d6000602084013e610713565b606091505b509150915061072386838361072d565b9695505050505050565b6060826107425761073d82610789565b610689565b815115801561075957506001600160a01b0384163b155b1561078257604051639996b31560e01b81526001600160a01b038516600482015260240161066d565b5080610689565b8051156107995780518082602001fd5b604051630a12f52160e11b815260040160405180910390fd5b50565b60405180606001604052806003905b60408051606081018252600080825260208083018290529282015282526000199092019101816107c45790505090565b60405180606001604052806003905b61080b610821565b8152602001906001900390816108035790505090565b60405180604001604052806108656040518060c001604052806000815260200160008152602001600081526020016000815260200160008152602001600081525090565b815260408051606081018252600080825260208281018290529282015291015290565b60006020828403121561089a57600080fd5b5035919050565b600080602083850312156108b457600080fd5b823567ffffffffffffffff8111156108cb57600080fd5b8301601f810185136108dc57600080fd5b803567ffffffffffffffff8111156108f357600080fd5b85602060c08302840101111561090857600080fd5b6020919091019590945092505050565b602080825282518282018190526000918401906040840190835b81811015610950578351835260209384019390920191600101610932565b509095945050505050565b6000806020838503121561096e57600080fd5b823567ffffffffffffffff81111561098557600080fd5b8301601f8101851361099657600080fd5b803567ffffffffffffffff8111156109ad57600080fd5b85602060608302840101111561090857600080fd5b600080602083850312156109d557600080fd5b823567ffffffffffffffff8111156109ec57600080fd5b8301601f810185136109fd57600080fd5b803567ffffffffffffffff811115610a1457600080fd5b8560206101208302840101111561090857600080fd5b6101208101818360005b6003811015610a71578151805184526020810151602085015260408101511515604085015250606083019250602082019150600181019050610a34565b50505092915050565b634e487b7160e01b600052603260045260246000fd5b6001600160a01b03811681146107b257600080fd5b600060208284031215610ab757600080fd5b813561068981610a90565b8135610acd81610a90565b81546001600160a01b0319166001600160a01b039190911617815560208201356001820155604090910135600290910155565b6040805190810167ffffffffffffffff81118282101715610b3157634e487b7160e01b600052604160045260246000fd5b60405290565b60405160c0810167ffffffffffffffff81118282101715610b3157634e487b7160e01b600052604160045260246000fd5b600060608284031215610b7a57600080fd5b6040516060810167ffffffffffffffff81118282101715610bab57634e487b7160e01b600052604160045260246000fd5b6040529050808235610bbc81610a90565b815260208381013590820152604092830135920191909152919050565b600081830361012081128015610bee57600080fd5b506000610bf9610b00565b60c0831215610c06578182fd5b610c0e610b37565b853581526020808701359082015260408087013590820152606080870135908201526080808701359082015260a0808701359082018190528183529093509150610c5b8660c08701610b68565b602082015295945050505050565b600060208284031215610c7b57600080fd5b8151801515811461068957600080fd5b6000825160005b81811015610cac5760208186018101518583015201610c92565b50600092019182525091905056fea26469706673582212201a41add79cb171abb895d9581179301bd58160abb58ca4394c6b7d771da054a464736f6c634300081c0033", + "deployedBytecode": "0x608060405234801561001057600080fd5b506004361061007d5760003560e01c806380a38d971161005b57806380a38d9714610128578063b6c2141b14610148578063bcb2c1da1461015d578063c7170bb61461017d57600080fd5b80630716326d14610082578063474740b1146100e65780634ec42e8e146100fd575b600080fd5b6100bc610090366004610888565b60026020819052600091825260409091208054600182015491909201546001600160a01b039092169183565b604080516001600160a01b0390941684526020840192909252908201526060015b60405180910390f35b6100ef60005481565b6040519081526020016100dd565b600154610110906001600160a01b031681565b6040516001600160a01b0390911681526020016100dd565b61013b6101363660046108a1565b610186565b6040516100dd9190610918565b61015b61015636600461095b565b6101d3565b005b61017061016b3660046109c2565b6102c3565b6040516100dd9190610a2a565b6100ef60035481565b60408051600180825281830190925260609160009190602080830190803683370190505090506001816000815181106101c1576101c1610a7a565b60209081029190910101529392505050565b60005481908111156101f857604051630d67f41160e21b815260040160405180910390fd5b60005b818110156102bd573684848381811061021657610216610a7a565b60600291909101915061024a9050336102326020840184610aa5565b6001546001600160a01b03169190602085013561044e565b604080820135600090815260026020522081906102678282610ac2565b505060408101356020820180359061027f9084610aa5565b6001600160a01b03167ff998960b1c6f0e0e89b7bbe6b6fbf3e03e6f08eee5b8430877d8adb8e149d58060405160405180910390a4506001016101fb565b50505050565b6102cb6107b5565b60035482146102ed57604051637db491eb60e01b815260040160405180910390fd5b60006102f984846104a8565b905060005b60038110156104465760006002600084846003811061031f5761031f610a7a565b602090810291909101518101516040908101518352828201939093529082016000908120835160608101855281546001600160a01b031681526001820154938101939093526002015492820192909252915083836003811061038357610383610a7a565b60200201516020015160200151826020015114905060008484600381106103ac576103ac610a7a565b602002015160200151600001516001600160a01b031683600001516001600160a01b0316149050600060405180606001604052808787600381106103f2576103f2610a7a565b602002015160200151604001518152602001856020015181526020018480156104185750835b1515905290508087866003811061043157610431610a7a565b60200201525050600190920191506102fe9050565b505092915050565b604080516001600160a01b0385811660248301528416604482015260648082018490528251808303909101815260849091019091526020810180516001600160e01b03166323b872dd60e01b1790526102bd90859061060e565b6104b06107f4565b60005b8281101561060757815160209081015101518484838181106104d7576104d7610a7a565b9050610120020160c0016020013511156105315760208201805160408401528251905283838281811061050c5761050c610a7a565b905061012002018036038101906105239190610bd9565b8260005b60200201526105ff565b602080830151810151015184848381811061054e5761054e610a7a565b9050610120020160c00160200135111561059d576020820151604083015283838281811061057e5761057e610a7a565b905061012002018036038101906105959190610bd9565b826001610527565b604082015160209081015101518484838181106105bc576105bc610a7a565b9050610120020160c0016020013511156105ff578383828181106105e2576105e2610a7a565b905061012002018036038101906105f99190610bd9565b60408301525b6001016104b3565b5092915050565b60006106236001600160a01b0384168361067b565b905080516000141580156106485750808060200190518101906106469190610c69565b155b1561067657604051635274afe760e01b81526001600160a01b03841660048201526024015b60405180910390fd5b505050565b606061068983836000610690565b9392505050565b6060814710156106b55760405163cd78605960e01b815230600482015260240161066d565b600080856001600160a01b031684866040516106d19190610c8b565b60006040518083038185875af1925050503d806000811461070e576040519150601f19603f3d011682016040523d82523d6000602084013e610713565b606091505b509150915061072386838361072d565b9695505050505050565b6060826107425761073d82610789565b610689565b815115801561075957506001600160a01b0384163b155b1561078257604051639996b31560e01b81526001600160a01b038516600482015260240161066d565b5080610689565b8051156107995780518082602001fd5b604051630a12f52160e11b815260040160405180910390fd5b50565b60405180606001604052806003905b60408051606081018252600080825260208083018290529282015282526000199092019101816107c45790505090565b60405180606001604052806003905b61080b610821565b8152602001906001900390816108035790505090565b60405180604001604052806108656040518060c001604052806000815260200160008152602001600081526020016000815260200160008152602001600081525090565b815260408051606081018252600080825260208281018290529282015291015290565b60006020828403121561089a57600080fd5b5035919050565b600080602083850312156108b457600080fd5b823567ffffffffffffffff8111156108cb57600080fd5b8301601f810185136108dc57600080fd5b803567ffffffffffffffff8111156108f357600080fd5b85602060c08302840101111561090857600080fd5b6020919091019590945092505050565b602080825282518282018190526000918401906040840190835b81811015610950578351835260209384019390920191600101610932565b509095945050505050565b6000806020838503121561096e57600080fd5b823567ffffffffffffffff81111561098557600080fd5b8301601f8101851361099657600080fd5b803567ffffffffffffffff8111156109ad57600080fd5b85602060608302840101111561090857600080fd5b600080602083850312156109d557600080fd5b823567ffffffffffffffff8111156109ec57600080fd5b8301601f810185136109fd57600080fd5b803567ffffffffffffffff811115610a1457600080fd5b8560206101208302840101111561090857600080fd5b6101208101818360005b6003811015610a71578151805184526020810151602085015260408101511515604085015250606083019250602082019150600181019050610a34565b50505092915050565b634e487b7160e01b600052603260045260246000fd5b6001600160a01b03811681146107b257600080fd5b600060208284031215610ab757600080fd5b813561068981610a90565b8135610acd81610a90565b81546001600160a01b0319166001600160a01b039190911617815560208201356001820155604090910135600290910155565b6040805190810167ffffffffffffffff81118282101715610b3157634e487b7160e01b600052604160045260246000fd5b60405290565b60405160c0810167ffffffffffffffff81118282101715610b3157634e487b7160e01b600052604160045260246000fd5b600060608284031215610b7a57600080fd5b6040516060810167ffffffffffffffff81118282101715610bab57634e487b7160e01b600052604160045260246000fd5b6040529050808235610bbc81610a90565b815260208381013590820152604092830135920191909152919050565b600081830361012081128015610bee57600080fd5b506000610bf9610b00565b60c0831215610c06578182fd5b610c0e610b37565b853581526020808701359082015260408087013590820152606080870135908201526080808701359082015260a0808701359082018190528183529093509150610c5b8660c08701610b68565b602082015295945050505050565b600060208284031215610c7b57600080fd5b8151801515811461068957600080fd5b6000825160005b81811015610cac5760208186018101518583015201610c92565b50600092019182525091905056fea26469706673582212201a41add79cb171abb895d9581179301bd58160abb58ca4394c6b7d771da054a464736f6c634300081c0033", "linkReferences": {}, "deployedLinkReferences": {} } diff --git a/evmlib/src/contract/payment_vault/handler.rs b/evmlib/src/contract/payment_vault/handler.rs index 29ef362b51..e1bdaec50f 100644 --- a/evmlib/src/contract/payment_vault/handler.rs +++ b/evmlib/src/contract/payment_vault/handler.rs @@ -35,26 +35,13 @@ where &self, metrics: I, ) -> Result, Error> { - // NB TODO @mick we need to batch this smart contract call - let mut amounts = vec![]; - - // set rate limit to 2 req/s - const TIME_BETWEEN_RPC_CALLS_IN_MS: u64 = 700; - let mut maybe_last_call: Option = None; - for metric in metrics { - // check if we have to wait for the rate limit - if let Some(last_call) = maybe_last_call { - let elapsed = std::time::Instant::now() - last_call; - let time_to_sleep_ms = TIME_BETWEEN_RPC_CALLS_IN_MS as u128 - elapsed.as_millis(); - if time_to_sleep_ms > 0 { - tokio::time::sleep(std::time::Duration::from_millis(time_to_sleep_ms as u64)) - .await; - } - } - - let amount = self.contract.getQuote(metric.into()).call().await?.price; - amounts.push(amount); - maybe_last_call = Some(std::time::Instant::now()); + let metrics: Vec<_> = metrics.into_iter().map(|v| v.into()).collect(); + let mut amounts = self.contract.getQuote(metrics.clone()).call().await?.prices; + + // FIXME: temporary logic until the smart contract gets updated + if amounts.len() == 1 { + let value = amounts[0]; + amounts.resize(metrics.len(), value); } Ok(amounts) diff --git a/evmlib/src/transaction.rs b/evmlib/src/transaction.rs deleted file mode 100644 index 48df355638..0000000000 --- a/evmlib/src/transaction.rs +++ /dev/null @@ -1,156 +0,0 @@ -// Copyright 2024 MaidSafe.net limited. -// -// This SAFE Network Software is licensed to you under The General Public License (GPL), version 3. -// Unless required by applicable law or agreed to in writing, the SAFE Network Software distributed -// under the GPL Licence is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. Please review the Licences for the specific language governing -// permissions and limitations relating to use of the SAFE Network Software. - -use crate::common::{Address, Amount, QuoteHash}; -use crate::contract::payment_vault::handler::PaymentVaultHandler; -use crate::quoting_metrics::QuotingMetrics; -use crate::utils::http_provider; -use crate::{contract, Network}; -use alloy::transports::{RpcError, TransportErrorKind}; - -#[derive(thiserror::Error, Debug)] -pub enum Error { - #[error(transparent)] - RpcError(#[from] RpcError), - #[error("Transaction is not confirmed")] - TransactionUnconfirmed, - #[error("Transaction was not found")] - TransactionNotFound, - #[error("Transaction has not been included in a block yet")] - TransactionNotInBlock, - #[error("Block was not found")] - BlockNotFound, - #[error("No event proof found")] - EventProofNotFound, - #[error("Payment was done after the quote expired")] - QuoteExpired, - #[error(transparent)] - PaymentVaultError(#[from] contract::payment_vault::error::Error), - #[error("Payment missing")] - PaymentMissing, -} - -/// Get a transaction receipt by its hash. -pub async fn get_transaction_receipt_by_hash( - network: &Network, - transaction_hash: TxHash, -) -> Result, Error> { - let provider = ProviderBuilder::new() - .with_recommended_fillers() - .on_http(network.rpc_url().clone()); - let maybe_receipt = provider - .get_transaction_receipt(transaction_hash) - .await - .inspect_err(|err| error!("Error getting transaction receipt for transaction_hash: {transaction_hash:?} : {err:?}", ))?; - debug!("Transaction receipt for {transaction_hash:?}: {maybe_receipt:?}"); - Ok(maybe_receipt) -} - -/// Get a block by its block number. -async fn get_block_by_number(network: &Network, block_number: u64) -> Result, Error> { - let provider = ProviderBuilder::new() - .with_recommended_fillers() - .on_http(network.rpc_url().clone()); - let block = provider - .get_block_by_number( - BlockNumberOrTag::Number(block_number), - BlockTransactionsKind::Full, - ) - .await - .inspect_err(|err| error!("Error getting block by number for {block_number} : {err:?}",))?; - Ok(block) -} - -/// Get transaction logs using a filter. -async fn get_transaction_logs(network: &Network, filter: Filter) -> Result, Error> { - let provider = ProviderBuilder::new() - .with_recommended_fillers() - .on_http(network.rpc_url().clone()); - let logs = provider - .get_logs(&filter) - .await - .inspect_err(|err| error!("Error getting logs for filter: {filter:?} : {err:?}"))?; - Ok(logs) -} - -/// Get a DataPaymentMade event, filtered by a hashed chunk address and a node address. -/// Useful for a node if it wants to check if payment for a certain chunk has been made. -async fn get_data_payment_event( - network: &Network, - block_number: u64, - quote_hash: QuoteHash, - reward_addr: Address, - amount: U256, -) -> Result, Error> { - debug!( - "Getting data payment event for quote_hash: {quote_hash:?}, reward_addr: {reward_addr:?}" - ); - let topic1: FixedBytes<32> = FixedBytes::left_padding_from(reward_addr.as_slice()); - - let filter = Filter::new() - .event_signature(DATA_PAYMENT_EVENT_SIGNATURE) - .topic1(topic1) - .topic2(amount) - .topic3(quote_hash) - .from_block(block_number) - .to_block(block_number); - - get_transaction_logs(network, filter).await -} - -/// Verify if a data payment is confirmed. -pub async fn verify_data_payment( - network: &Network, - my_quote_hashes: Vec, // TODO @mick hashes the node owns so it knows how much it received from them - payment: Vec<(QuoteHash, QuotingMetrics, Address)> -) -> Result { - let provider = http_provider(network.rpc_url().clone()); - let payment_vault = PaymentVaultHandler::new(*network.data_payments_address(), provider); - - // NB TODO @mick remove tmp loop and support verification of the whole payment at once - let mut is_paid = true; - for (quote_hash, quoting_metrics, reward_addr) in payment { - is_paid = payment_vault - .verify_payment(quoting_metrics, (quote_hash, reward_addr, Amount::ZERO)) - .await?; - } - - let amount_paid = Amount::ZERO; // NB TODO @mick we need to get the amount paid from the contract - - if is_paid { - Ok(amount_paid) - } else { - Err(Error::PaymentMissing) - } -} - -#[cfg(test)] -mod tests { - use crate::common::Address; - use crate::quoting_metrics::QuotingMetrics; - use crate::transaction::verify_data_payment; - use crate::Network; - use alloy::hex::FromHex; - use alloy::primitives::b256; - - #[tokio::test] - async fn test_verify_data_payment() { - let network = Network::ArbitrumOne; - - let quote_hash = b256!("EBD943C38C0422901D4CF22E677DD95F2591CA8D6EBFEA8BAF1BFE9FF5506ECE"); // DevSkim: ignore DS173237 - let reward_address = Address::from_hex("8AB15A43305854e4AE4E6FBEa0CD1CC0AB4ecB2A").unwrap(); // DevSkim: ignore DS173237 - - let result = verify_data_payment( - &network, - vec![(quote_hash, QuotingMetrics::default(), reward_address)] - ) - .await; - - assert!(result.is_ok(), "Error: {:?}", result.err()); - } -} From ef9822d2d04d963639e2350df44176171a6bfcf0 Mon Sep 17 00:00:00 2001 From: Warm Beer Date: Wed, 11 Dec 2024 11:49:02 +0100 Subject: [PATCH 26/33] chore: introduce rate limiter and retry strategy for `get_market_price` --- autonomi/src/client/mod.rs | 1 + autonomi/src/client/quote.rs | 48 +++++++++++++++++++++++++++-- autonomi/src/client/rate_limiter.rs | 28 +++++++++++++++++ 3 files changed, 75 insertions(+), 2 deletions(-) create mode 100644 autonomi/src/client/rate_limiter.rs diff --git a/autonomi/src/client/mod.rs b/autonomi/src/client/mod.rs index 15e1c83ae1..8ebfc004fa 100644 --- a/autonomi/src/client/mod.rs +++ b/autonomi/src/client/mod.rs @@ -29,6 +29,7 @@ pub mod vault; pub mod wasm; // private module with utility functions +mod rate_limiter; mod utils; use ant_bootstrap::{BootstrapCacheConfig, BootstrapCacheStore}; diff --git a/autonomi/src/client/quote.rs b/autonomi/src/client/quote.rs index c8cc8058d7..b720ea5b3b 100644 --- a/autonomi/src/client/quote.rs +++ b/autonomi/src/client/quote.rs @@ -7,8 +7,9 @@ // permissions and limitations relating to use of the SAFE Network Software. use super::{data::CostError, Client}; +use crate::client::rate_limiter::RateLimiter; use ant_evm::payment_vault::get_market_price; -use ant_evm::{Amount, PaymentQuote, QuotePayment, QuotingMetrics}; +use ant_evm::{Amount, EvmNetwork, PaymentQuote, QuotePayment, QuotingMetrics}; use ant_networking::{Network, NetworkError}; use ant_protocol::{storage::ChunkAddress, NetworkAddress}; use libp2p::PeerId; @@ -65,6 +66,9 @@ impl Client { // choose the quotes to pay for each address let mut quotes_to_pay_per_addr = HashMap::new(); + + let mut rate_limiter = RateLimiter::new(); + for (content_addr, raw_quotes) in raw_quotes_per_addr { // ask smart contract for the market price let quoting_metrics: Vec = raw_quotes @@ -72,7 +76,14 @@ impl Client { .iter() .map(|(_, q)| q.quoting_metrics.clone()) .collect(); - let all_prices = get_market_price(&self.evm_network, quoting_metrics).await?; + + let all_prices = get_market_price_with_rate_limiter_and_retries( + &self.evm_network, + &mut rate_limiter, + quoting_metrics.clone(), + ) + .await?; + let mut prices: Vec<(PeerId, PaymentQuote, Amount)> = all_prices .into_iter() .zip(raw_quotes.into_iter()) @@ -157,3 +168,36 @@ async fn fetch_store_quote_with_retries( } } } + +async fn get_market_price_with_rate_limiter_and_retries( + evm_network: &EvmNetwork, + rate_limiter: &mut RateLimiter, + quoting_metrics: Vec, +) -> Result, ant_evm::payment_vault::error::Error> { + const MAX_RETRIES: u64 = 2; + let mut retries: u64 = 0; + let mut interval_in_ms: u64 = 1000; + + loop { + rate_limiter + .wait_interval_since_last_request(interval_in_ms) + .await; + + match get_market_price(evm_network, quoting_metrics.clone()).await { + Ok(amounts) => { + break Ok(amounts); + } + Err(err) => { + if err.to_string().contains("429") && retries < MAX_RETRIES { + retries += 1; + interval_in_ms *= retries * 2; + error!("Error while fetching quote market price: {err:?}, retry #{retries}"); + continue; + } else { + error!("Error while fetching quote market price: {err:?}, stopping after {retries} retries"); + break Err(err); + } + } + } + } +} diff --git a/autonomi/src/client/rate_limiter.rs b/autonomi/src/client/rate_limiter.rs new file mode 100644 index 0000000000..3cef0e0434 --- /dev/null +++ b/autonomi/src/client/rate_limiter.rs @@ -0,0 +1,28 @@ +use ant_networking::target_arch::{sleep, Duration, Instant}; + +pub struct RateLimiter { + last_request_time: Option, +} + +impl RateLimiter { + pub fn new() -> Self { + Self { + last_request_time: None, + } + } + + pub async fn wait_interval_since_last_request(&mut self, interval_in_ms: u64) { + if let Some(last_request_time) = self.last_request_time { + let elapsed_time = last_request_time.elapsed(); + + let interval = Duration::from_millis(interval_in_ms); + + if elapsed_time < interval { + println!("Waiting for: {:?}", interval - elapsed_time); + sleep(interval - elapsed_time).await; + } + } + + self.last_request_time = Some(Instant::now()); + } +} From 72fa1a913e372d24c570b011926742afb3b78014 Mon Sep 17 00:00:00 2001 From: Warm Beer Date: Wed, 11 Dec 2024 12:17:48 +0100 Subject: [PATCH 27/33] chore: delete print --- autonomi/src/client/rate_limiter.rs | 1 - 1 file changed, 1 deletion(-) diff --git a/autonomi/src/client/rate_limiter.rs b/autonomi/src/client/rate_limiter.rs index 3cef0e0434..b1935f6e83 100644 --- a/autonomi/src/client/rate_limiter.rs +++ b/autonomi/src/client/rate_limiter.rs @@ -18,7 +18,6 @@ impl RateLimiter { let interval = Duration::from_millis(interval_in_ms); if elapsed_time < interval { - println!("Waiting for: {:?}", interval - elapsed_time); sleep(interval - elapsed_time).await; } } From d2df2cc7d2e65b75945a37e93d84425dfd1b2a74 Mon Sep 17 00:00:00 2001 From: Warm Beer Date: Wed, 11 Dec 2024 12:48:07 +0100 Subject: [PATCH 28/33] fix: assume that content_addrs with no quotes are already uploaded --- autonomi/src/client/quote.rs | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/autonomi/src/client/quote.rs b/autonomi/src/client/quote.rs index b720ea5b3b..9794f165d7 100644 --- a/autonomi/src/client/quote.rs +++ b/autonomi/src/client/quote.rs @@ -70,6 +70,12 @@ impl Client { let mut rate_limiter = RateLimiter::new(); for (content_addr, raw_quotes) in raw_quotes_per_addr { + // FIXME: find better way to deal with paid content addrs and feedback to the user + // assume that content addr is already paid for and uploaded + if raw_quotes.is_empty() { + continue; + } + // ask smart contract for the market price let quoting_metrics: Vec = raw_quotes .clone() From 7ce323615d6149b7833321b1a1db5dd562ea6885 Mon Sep 17 00:00:00 2001 From: Warm Beer Date: Wed, 11 Dec 2024 13:35:19 +0100 Subject: [PATCH 29/33] chore: set arbitrum sepolia data payments address --- evmlib/src/lib.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/evmlib/src/lib.rs b/evmlib/src/lib.rs index 6de2343462..abd5d3309a 100644 --- a/evmlib/src/lib.rs +++ b/evmlib/src/lib.rs @@ -50,7 +50,7 @@ const ARBITRUM_ONE_DATA_PAYMENTS_ADDRESS: Address = address!("607483B50C5F06c25cDC316b6d1E071084EeC9f5"); const ARBITRUM_SEPOLIA_DATA_PAYMENTS_ADDRESS: Address = - address!("Dd56b03Dae2Ab8594D80269EC4518D13F1A110BD"); + address!("993C7739f50899A997fEF20860554b8a28113634"); #[serde_as] #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] From b6b230fdd4391b1440cf407a529c85d4fd7e12d4 Mon Sep 17 00:00:00 2001 From: Warm Beer Date: Wed, 11 Dec 2024 16:13:52 +0100 Subject: [PATCH 30/33] fix: clippy error --- ant-networking/src/record_store.rs | 2 -- 1 file changed, 2 deletions(-) diff --git a/ant-networking/src/record_store.rs b/ant-networking/src/record_store.rs index 8d6e078e40..b4ab4ff6b3 100644 --- a/ant-networking/src/record_store.rs +++ b/ant-networking/src/record_store.rs @@ -1003,8 +1003,6 @@ mod tests { use bls::SecretKey; use xor_name::XorName; - use ant_evm::utils::dummy_address; - use ant_evm::{PaymentQuote, RewardsAddress}; use ant_protocol::convert_distance_to_u256; use ant_protocol::storage::{ try_deserialize_record, try_serialize_record, Chunk, ChunkAddress, Scratchpad, From 00898645c1dae1ab623542f1f2cd927f2294c3cb Mon Sep 17 00:00:00 2001 From: Warm Beer Date: Thu, 12 Dec 2024 16:46:09 +0100 Subject: [PATCH 31/33] chore: update contract interface --- evmlib/abi/IPaymentVault.json | 28 +++++---------- evmlib/artifacts/PaymentVaultNoProxy.json | 34 ++++++------------- evmlib/src/contract/payment_vault/handler.rs | 8 +---- .../src/contract/payment_vault/interface.rs | 5 ++- evmlib/tests/payment_vault.rs | 3 +- 5 files changed, 23 insertions(+), 55 deletions(-) diff --git a/evmlib/abi/IPaymentVault.json b/evmlib/abi/IPaymentVault.json index 5f34d178f7..d1ca0a9f67 100644 --- a/evmlib/abi/IPaymentVault.json +++ b/evmlib/abi/IPaymentVault.json @@ -162,26 +162,14 @@ "type": "tuple" }, { - "components": [ - { - "internalType": "address", - "name": "rewardsAddress", - "type": "address" - }, - { - "internalType": "uint256", - "name": "amount", - "type": "uint256" - }, - { - "internalType": "bytes32", - "name": "quoteHash", - "type": "bytes32" - } - ], - "internalType": "struct IPaymentVault.DataPayment", - "name": "dataPayment", - "type": "tuple" + "internalType": "address", + "name": "rewardsAddress", + "type": "address" + }, + { + "internalType": "bytes32", + "name": "quoteHash", + "type": "bytes32" } ], "internalType": "struct IPaymentVault.PaymentVerification[]", diff --git a/evmlib/artifacts/PaymentVaultNoProxy.json b/evmlib/artifacts/PaymentVaultNoProxy.json index 914e28d0f3..9b006d274e 100644 --- a/evmlib/artifacts/PaymentVaultNoProxy.json +++ b/evmlib/artifacts/PaymentVaultNoProxy.json @@ -1,6 +1,6 @@ { "_format": "hh-sol-artifact-1", - "contractName": "PaymentVaultNoProxy", + "contractName": "PaymentVault", "sourceName": "contracts/PaymentVaultNoProxy.sol", "abi": [ { @@ -288,26 +288,14 @@ "type": "tuple" }, { - "components": [ - { - "internalType": "address", - "name": "rewardsAddress", - "type": "address" - }, - { - "internalType": "uint256", - "name": "amount", - "type": "uint256" - }, - { - "internalType": "bytes32", - "name": "quoteHash", - "type": "bytes32" - } - ], - "internalType": "struct IPaymentVault.DataPayment", - "name": "dataPayment", - "type": "tuple" + "internalType": "address", + "name": "rewardsAddress", + "type": "address" + }, + { + "internalType": "bytes32", + "name": "quoteHash", + "type": "bytes32" } ], "internalType": "struct IPaymentVault.PaymentVerification[]", @@ -344,8 +332,8 @@ "type": "function" } ], - "bytecode": "0x6080604052348015600f57600080fd5b50604051610db6380380610db6833981016040819052602c91607f565b6001600160a01b038216605257604051632d06160b60e21b815260040160405180910390fd5b600180546001600160a01b0319166001600160a01b039390931692909217909155600055600560035560b7565b60008060408385031215609157600080fd5b82516001600160a01b038116811460a757600080fd5b6020939093015192949293505050565b610cf0806100c66000396000f3fe608060405234801561001057600080fd5b506004361061007d5760003560e01c806380a38d971161005b57806380a38d9714610128578063b6c2141b14610148578063bcb2c1da1461015d578063c7170bb61461017d57600080fd5b80630716326d14610082578063474740b1146100e65780634ec42e8e146100fd575b600080fd5b6100bc610090366004610888565b60026020819052600091825260409091208054600182015491909201546001600160a01b039092169183565b604080516001600160a01b0390941684526020840192909252908201526060015b60405180910390f35b6100ef60005481565b6040519081526020016100dd565b600154610110906001600160a01b031681565b6040516001600160a01b0390911681526020016100dd565b61013b6101363660046108a1565b610186565b6040516100dd9190610918565b61015b61015636600461095b565b6101d3565b005b61017061016b3660046109c2565b6102c3565b6040516100dd9190610a2a565b6100ef60035481565b60408051600180825281830190925260609160009190602080830190803683370190505090506001816000815181106101c1576101c1610a7a565b60209081029190910101529392505050565b60005481908111156101f857604051630d67f41160e21b815260040160405180910390fd5b60005b818110156102bd573684848381811061021657610216610a7a565b60600291909101915061024a9050336102326020840184610aa5565b6001546001600160a01b03169190602085013561044e565b604080820135600090815260026020522081906102678282610ac2565b505060408101356020820180359061027f9084610aa5565b6001600160a01b03167ff998960b1c6f0e0e89b7bbe6b6fbf3e03e6f08eee5b8430877d8adb8e149d58060405160405180910390a4506001016101fb565b50505050565b6102cb6107b5565b60035482146102ed57604051637db491eb60e01b815260040160405180910390fd5b60006102f984846104a8565b905060005b60038110156104465760006002600084846003811061031f5761031f610a7a565b602090810291909101518101516040908101518352828201939093529082016000908120835160608101855281546001600160a01b031681526001820154938101939093526002015492820192909252915083836003811061038357610383610a7a565b60200201516020015160200151826020015114905060008484600381106103ac576103ac610a7a565b602002015160200151600001516001600160a01b031683600001516001600160a01b0316149050600060405180606001604052808787600381106103f2576103f2610a7a565b602002015160200151604001518152602001856020015181526020018480156104185750835b1515905290508087866003811061043157610431610a7a565b60200201525050600190920191506102fe9050565b505092915050565b604080516001600160a01b0385811660248301528416604482015260648082018490528251808303909101815260849091019091526020810180516001600160e01b03166323b872dd60e01b1790526102bd90859061060e565b6104b06107f4565b60005b8281101561060757815160209081015101518484838181106104d7576104d7610a7a565b9050610120020160c0016020013511156105315760208201805160408401528251905283838281811061050c5761050c610a7a565b905061012002018036038101906105239190610bd9565b8260005b60200201526105ff565b602080830151810151015184848381811061054e5761054e610a7a565b9050610120020160c00160200135111561059d576020820151604083015283838281811061057e5761057e610a7a565b905061012002018036038101906105959190610bd9565b826001610527565b604082015160209081015101518484838181106105bc576105bc610a7a565b9050610120020160c0016020013511156105ff578383828181106105e2576105e2610a7a565b905061012002018036038101906105f99190610bd9565b60408301525b6001016104b3565b5092915050565b60006106236001600160a01b0384168361067b565b905080516000141580156106485750808060200190518101906106469190610c69565b155b1561067657604051635274afe760e01b81526001600160a01b03841660048201526024015b60405180910390fd5b505050565b606061068983836000610690565b9392505050565b6060814710156106b55760405163cd78605960e01b815230600482015260240161066d565b600080856001600160a01b031684866040516106d19190610c8b565b60006040518083038185875af1925050503d806000811461070e576040519150601f19603f3d011682016040523d82523d6000602084013e610713565b606091505b509150915061072386838361072d565b9695505050505050565b6060826107425761073d82610789565b610689565b815115801561075957506001600160a01b0384163b155b1561078257604051639996b31560e01b81526001600160a01b038516600482015260240161066d565b5080610689565b8051156107995780518082602001fd5b604051630a12f52160e11b815260040160405180910390fd5b50565b60405180606001604052806003905b60408051606081018252600080825260208083018290529282015282526000199092019101816107c45790505090565b60405180606001604052806003905b61080b610821565b8152602001906001900390816108035790505090565b60405180604001604052806108656040518060c001604052806000815260200160008152602001600081526020016000815260200160008152602001600081525090565b815260408051606081018252600080825260208281018290529282015291015290565b60006020828403121561089a57600080fd5b5035919050565b600080602083850312156108b457600080fd5b823567ffffffffffffffff8111156108cb57600080fd5b8301601f810185136108dc57600080fd5b803567ffffffffffffffff8111156108f357600080fd5b85602060c08302840101111561090857600080fd5b6020919091019590945092505050565b602080825282518282018190526000918401906040840190835b81811015610950578351835260209384019390920191600101610932565b509095945050505050565b6000806020838503121561096e57600080fd5b823567ffffffffffffffff81111561098557600080fd5b8301601f8101851361099657600080fd5b803567ffffffffffffffff8111156109ad57600080fd5b85602060608302840101111561090857600080fd5b600080602083850312156109d557600080fd5b823567ffffffffffffffff8111156109ec57600080fd5b8301601f810185136109fd57600080fd5b803567ffffffffffffffff811115610a1457600080fd5b8560206101208302840101111561090857600080fd5b6101208101818360005b6003811015610a71578151805184526020810151602085015260408101511515604085015250606083019250602082019150600181019050610a34565b50505092915050565b634e487b7160e01b600052603260045260246000fd5b6001600160a01b03811681146107b257600080fd5b600060208284031215610ab757600080fd5b813561068981610a90565b8135610acd81610a90565b81546001600160a01b0319166001600160a01b039190911617815560208201356001820155604090910135600290910155565b6040805190810167ffffffffffffffff81118282101715610b3157634e487b7160e01b600052604160045260246000fd5b60405290565b60405160c0810167ffffffffffffffff81118282101715610b3157634e487b7160e01b600052604160045260246000fd5b600060608284031215610b7a57600080fd5b6040516060810167ffffffffffffffff81118282101715610bab57634e487b7160e01b600052604160045260246000fd5b6040529050808235610bbc81610a90565b815260208381013590820152604092830135920191909152919050565b600081830361012081128015610bee57600080fd5b506000610bf9610b00565b60c0831215610c06578182fd5b610c0e610b37565b853581526020808701359082015260408087013590820152606080870135908201526080808701359082015260a0808701359082018190528183529093509150610c5b8660c08701610b68565b602082015295945050505050565b600060208284031215610c7b57600080fd5b8151801515811461068957600080fd5b6000825160005b81811015610cac5760208186018101518583015201610c92565b50600092019182525091905056fea26469706673582212201a41add79cb171abb895d9581179301bd58160abb58ca4394c6b7d771da054a464736f6c634300081c0033", - "deployedBytecode": "0x608060405234801561001057600080fd5b506004361061007d5760003560e01c806380a38d971161005b57806380a38d9714610128578063b6c2141b14610148578063bcb2c1da1461015d578063c7170bb61461017d57600080fd5b80630716326d14610082578063474740b1146100e65780634ec42e8e146100fd575b600080fd5b6100bc610090366004610888565b60026020819052600091825260409091208054600182015491909201546001600160a01b039092169183565b604080516001600160a01b0390941684526020840192909252908201526060015b60405180910390f35b6100ef60005481565b6040519081526020016100dd565b600154610110906001600160a01b031681565b6040516001600160a01b0390911681526020016100dd565b61013b6101363660046108a1565b610186565b6040516100dd9190610918565b61015b61015636600461095b565b6101d3565b005b61017061016b3660046109c2565b6102c3565b6040516100dd9190610a2a565b6100ef60035481565b60408051600180825281830190925260609160009190602080830190803683370190505090506001816000815181106101c1576101c1610a7a565b60209081029190910101529392505050565b60005481908111156101f857604051630d67f41160e21b815260040160405180910390fd5b60005b818110156102bd573684848381811061021657610216610a7a565b60600291909101915061024a9050336102326020840184610aa5565b6001546001600160a01b03169190602085013561044e565b604080820135600090815260026020522081906102678282610ac2565b505060408101356020820180359061027f9084610aa5565b6001600160a01b03167ff998960b1c6f0e0e89b7bbe6b6fbf3e03e6f08eee5b8430877d8adb8e149d58060405160405180910390a4506001016101fb565b50505050565b6102cb6107b5565b60035482146102ed57604051637db491eb60e01b815260040160405180910390fd5b60006102f984846104a8565b905060005b60038110156104465760006002600084846003811061031f5761031f610a7a565b602090810291909101518101516040908101518352828201939093529082016000908120835160608101855281546001600160a01b031681526001820154938101939093526002015492820192909252915083836003811061038357610383610a7a565b60200201516020015160200151826020015114905060008484600381106103ac576103ac610a7a565b602002015160200151600001516001600160a01b031683600001516001600160a01b0316149050600060405180606001604052808787600381106103f2576103f2610a7a565b602002015160200151604001518152602001856020015181526020018480156104185750835b1515905290508087866003811061043157610431610a7a565b60200201525050600190920191506102fe9050565b505092915050565b604080516001600160a01b0385811660248301528416604482015260648082018490528251808303909101815260849091019091526020810180516001600160e01b03166323b872dd60e01b1790526102bd90859061060e565b6104b06107f4565b60005b8281101561060757815160209081015101518484838181106104d7576104d7610a7a565b9050610120020160c0016020013511156105315760208201805160408401528251905283838281811061050c5761050c610a7a565b905061012002018036038101906105239190610bd9565b8260005b60200201526105ff565b602080830151810151015184848381811061054e5761054e610a7a565b9050610120020160c00160200135111561059d576020820151604083015283838281811061057e5761057e610a7a565b905061012002018036038101906105959190610bd9565b826001610527565b604082015160209081015101518484838181106105bc576105bc610a7a565b9050610120020160c0016020013511156105ff578383828181106105e2576105e2610a7a565b905061012002018036038101906105f99190610bd9565b60408301525b6001016104b3565b5092915050565b60006106236001600160a01b0384168361067b565b905080516000141580156106485750808060200190518101906106469190610c69565b155b1561067657604051635274afe760e01b81526001600160a01b03841660048201526024015b60405180910390fd5b505050565b606061068983836000610690565b9392505050565b6060814710156106b55760405163cd78605960e01b815230600482015260240161066d565b600080856001600160a01b031684866040516106d19190610c8b565b60006040518083038185875af1925050503d806000811461070e576040519150601f19603f3d011682016040523d82523d6000602084013e610713565b606091505b509150915061072386838361072d565b9695505050505050565b6060826107425761073d82610789565b610689565b815115801561075957506001600160a01b0384163b155b1561078257604051639996b31560e01b81526001600160a01b038516600482015260240161066d565b5080610689565b8051156107995780518082602001fd5b604051630a12f52160e11b815260040160405180910390fd5b50565b60405180606001604052806003905b60408051606081018252600080825260208083018290529282015282526000199092019101816107c45790505090565b60405180606001604052806003905b61080b610821565b8152602001906001900390816108035790505090565b60405180604001604052806108656040518060c001604052806000815260200160008152602001600081526020016000815260200160008152602001600081525090565b815260408051606081018252600080825260208281018290529282015291015290565b60006020828403121561089a57600080fd5b5035919050565b600080602083850312156108b457600080fd5b823567ffffffffffffffff8111156108cb57600080fd5b8301601f810185136108dc57600080fd5b803567ffffffffffffffff8111156108f357600080fd5b85602060c08302840101111561090857600080fd5b6020919091019590945092505050565b602080825282518282018190526000918401906040840190835b81811015610950578351835260209384019390920191600101610932565b509095945050505050565b6000806020838503121561096e57600080fd5b823567ffffffffffffffff81111561098557600080fd5b8301601f8101851361099657600080fd5b803567ffffffffffffffff8111156109ad57600080fd5b85602060608302840101111561090857600080fd5b600080602083850312156109d557600080fd5b823567ffffffffffffffff8111156109ec57600080fd5b8301601f810185136109fd57600080fd5b803567ffffffffffffffff811115610a1457600080fd5b8560206101208302840101111561090857600080fd5b6101208101818360005b6003811015610a71578151805184526020810151602085015260408101511515604085015250606083019250602082019150600181019050610a34565b50505092915050565b634e487b7160e01b600052603260045260246000fd5b6001600160a01b03811681146107b257600080fd5b600060208284031215610ab757600080fd5b813561068981610a90565b8135610acd81610a90565b81546001600160a01b0319166001600160a01b039190911617815560208201356001820155604090910135600290910155565b6040805190810167ffffffffffffffff81118282101715610b3157634e487b7160e01b600052604160045260246000fd5b60405290565b60405160c0810167ffffffffffffffff81118282101715610b3157634e487b7160e01b600052604160045260246000fd5b600060608284031215610b7a57600080fd5b6040516060810167ffffffffffffffff81118282101715610bab57634e487b7160e01b600052604160045260246000fd5b6040529050808235610bbc81610a90565b815260208381013590820152604092830135920191909152919050565b600081830361012081128015610bee57600080fd5b506000610bf9610b00565b60c0831215610c06578182fd5b610c0e610b37565b853581526020808701359082015260408087013590820152606080870135908201526080808701359082015260a0808701359082018190528183529093509150610c5b8660c08701610b68565b602082015295945050505050565b600060208284031215610c7b57600080fd5b8151801515811461068957600080fd5b6000825160005b81811015610cac5760208186018101518583015201610c92565b50600092019182525091905056fea26469706673582212201a41add79cb171abb895d9581179301bd58160abb58ca4394c6b7d771da054a464736f6c634300081c0033", + "bytecode": "0x6080604052348015600f57600080fd5b50604051610dce380380610dce833981016040819052602c91607f565b6001600160a01b038216605257604051632d06160b60e21b815260040160405180910390fd5b600180546001600160a01b0319166001600160a01b039390931692909217909155600055600560035560b7565b60008060408385031215609157600080fd5b82516001600160a01b038116811460a757600080fd5b6020939093015192949293505050565b610d08806100c66000396000f3fe608060405234801561001057600080fd5b506004361061007d5760003560e01c806380a38d971161005b57806380a38d9714610128578063b6c2141b14610148578063c7170bb61461015d578063f69c32cd1461016657600080fd5b80630716326d14610082578063474740b1146100e65780634ec42e8e146100fd575b600080fd5b6100bc6100903660046108fc565b60026020819052600091825260409091208054600182015491909201546001600160a01b039092169183565b604080516001600160a01b0390941684526020840192909252908201526060015b60405180910390f35b6100ef60005481565b6040519081526020016100dd565b600154610110906001600160a01b031681565b6040516001600160a01b0390911681526020016100dd565b61013b610136366004610915565b610186565b6040516100dd919061098c565b61015b6101563660046109cf565b6101d3565b005b6100ef60035481565b610179610174366004610a36565b6102c3565b6040516100dd9190610a9d565b60408051600180825281830190925260609160009190602080830190803683370190505090506001816000815181106101c1576101c1610aed565b60209081029190910101529392505050565b60005481908111156101f857604051630d67f41160e21b815260040160405180910390fd5b60005b818110156102bd573684848381811061021657610216610aed565b60600291909101915061024a9050336102326020840184610b28565b6001546001600160a01b03169190602085013561045c565b604080820135600090815260026020522081906102678282610b45565b505060408101356020820180359061027f9084610b28565b6001600160a01b03167ff998960b1c6f0e0e89b7bbe6b6fbf3e03e6f08eee5b8430877d8adb8e149d58060405160405180910390a4506001016101fb565b50505050565b6102cb610838565b60035482146102ed57604051637db491eb60e01b815260040160405180910390fd5b60006102f984846104b6565b905060005b60038110156104545760006002600084846003811061031f5761031f610aed565b602090810291909101516040908101518352828201939093529082016000908120835160608101855281546001600160a01b0316815260018201549381018490526002909101549381018490529350911515919015159085856003811061038857610388610aed565b6020020151602001516001600160a01b031684600001516001600160a01b03161480156103d9575060008686600381106103c4576103c4610aed565b6020020151602001516001600160a01b031614155b9050600060405180606001604052808888600381106103fa576103fa610aed565b60200201516040015181526020018660200151815260200185801561041c5750845b80156104255750835b1515905290508088876003811061043e5761043e610aed565b60200201525050600190930192506102fe915050565b505092915050565b604080516001600160a01b0385811660248301528416604482015260648082018490528251808303909101815260849091019091526020810180516001600160e01b03166323b872dd60e01b1790526102bd908590610691565b6104be610877565b60005b8281101561068a576000600260008686858181106104e1576104e1610aed565b9050610100020160e0013581526020019081526020016000206040518060600160405290816000820160009054906101000a90046001600160a01b03166001600160a01b03166001600160a01b03168152602001600182015481526020016002820154815250509050600260008460006003811061056157610561610aed565b602002015160400151815260200190815260200160002060010154816020015111156105cd576020830180516040850152835190528484838181106105a8576105a8610aed565b905061010002018036038101906105bf9190610beb565b8360005b6020020152610681565b602080840151604090810151600090815260028352206001015490820151111561062c576020830151604084015284848381811061060d5761060d610aed565b905061010002018036038101906106249190610beb565b8360016105c3565b604080840151810151600090815260026020908152919020600101549082015111156106815784848381811061066457610664610aed565b9050610100020180360381019061067b9190610beb565b60408401525b506001016104c1565b5092915050565b60006106a66001600160a01b038416836106fe565b905080516000141580156106cb5750808060200190518101906106c99190610c81565b155b156106f957604051635274afe760e01b81526001600160a01b03841660048201526024015b60405180910390fd5b505050565b606061070c83836000610713565b9392505050565b6060814710156107385760405163cd78605960e01b81523060048201526024016106f0565b600080856001600160a01b031684866040516107549190610ca3565b60006040518083038185875af1925050503d8060008114610791576040519150601f19603f3d011682016040523d82523d6000602084013e610796565b606091505b50915091506107a68683836107b0565b9695505050505050565b6060826107c5576107c08261080c565b61070c565b81511580156107dc57506001600160a01b0384163b155b1561080557604051639996b31560e01b81526001600160a01b03851660048201526024016106f0565b508061070c565b80511561081c5780518082602001fd5b604051630a12f52160e11b815260040160405180910390fd5b50565b60405180606001604052806003905b60408051606081018252600080825260208083018290529282015282526000199092019101816108475790505090565b60405180606001604052806003905b61088e6108a4565b8152602001906001900390816108865790505090565b60405180606001604052806108e86040518060c001604052806000815260200160008152602001600081526020016000815260200160008152602001600081525090565b815260006020820181905260409091015290565b60006020828403121561090e57600080fd5b5035919050565b6000806020838503121561092857600080fd5b823567ffffffffffffffff81111561093f57600080fd5b8301601f8101851361095057600080fd5b803567ffffffffffffffff81111561096757600080fd5b85602060c08302840101111561097c57600080fd5b6020919091019590945092505050565b602080825282518282018190526000918401906040840190835b818110156109c45783518352602093840193909201916001016109a6565b509095945050505050565b600080602083850312156109e257600080fd5b823567ffffffffffffffff8111156109f957600080fd5b8301601f81018513610a0a57600080fd5b803567ffffffffffffffff811115610a2157600080fd5b85602060608302840101111561097c57600080fd5b60008060208385031215610a4957600080fd5b823567ffffffffffffffff811115610a6057600080fd5b8301601f81018513610a7157600080fd5b803567ffffffffffffffff811115610a8857600080fd5b8560208260081b840101111561097c57600080fd5b6101208101818360005b6003811015610ae4578151805184526020810151602085015260408101511515604085015250606083019250602082019150600181019050610aa7565b50505092915050565b634e487b7160e01b600052603260045260246000fd5b6001600160a01b038116811461083557600080fd5b8035610b2381610b03565b919050565b600060208284031215610b3a57600080fd5b813561070c81610b03565b8135610b5081610b03565b81546001600160a01b0319166001600160a01b039190911617815560208201356001820155604090910135600290910155565b6040516060810167ffffffffffffffff81118282101715610bb457634e487b7160e01b600052604160045260246000fd5b60405290565b60405160c0810167ffffffffffffffff81118282101715610bb457634e487b7160e01b600052604160045260246000fd5b600081830361010081128015610c0057600080fd5b506000610c0b610b83565b60c0831215610c18578182fd5b610c20610bba565b853581526020808701359082015260408087013590820152606080870135908201526080808701359082015260a080870135908201528082529250610c6760c08601610b18565b602082015260e09490940135604085015250919392505050565b600060208284031215610c9357600080fd5b8151801515811461070c57600080fd5b6000825160005b81811015610cc45760208186018101518583015201610caa565b50600092019182525091905056fea26469706673582212207d1a9d88b0ba14ca908470a69ea19a09d2c7617056be2605039bc4d121f4fc4b64736f6c634300081c0033", + "deployedBytecode": "0x608060405234801561001057600080fd5b506004361061007d5760003560e01c806380a38d971161005b57806380a38d9714610128578063b6c2141b14610148578063c7170bb61461015d578063f69c32cd1461016657600080fd5b80630716326d14610082578063474740b1146100e65780634ec42e8e146100fd575b600080fd5b6100bc6100903660046108fc565b60026020819052600091825260409091208054600182015491909201546001600160a01b039092169183565b604080516001600160a01b0390941684526020840192909252908201526060015b60405180910390f35b6100ef60005481565b6040519081526020016100dd565b600154610110906001600160a01b031681565b6040516001600160a01b0390911681526020016100dd565b61013b610136366004610915565b610186565b6040516100dd919061098c565b61015b6101563660046109cf565b6101d3565b005b6100ef60035481565b610179610174366004610a36565b6102c3565b6040516100dd9190610a9d565b60408051600180825281830190925260609160009190602080830190803683370190505090506001816000815181106101c1576101c1610aed565b60209081029190910101529392505050565b60005481908111156101f857604051630d67f41160e21b815260040160405180910390fd5b60005b818110156102bd573684848381811061021657610216610aed565b60600291909101915061024a9050336102326020840184610b28565b6001546001600160a01b03169190602085013561045c565b604080820135600090815260026020522081906102678282610b45565b505060408101356020820180359061027f9084610b28565b6001600160a01b03167ff998960b1c6f0e0e89b7bbe6b6fbf3e03e6f08eee5b8430877d8adb8e149d58060405160405180910390a4506001016101fb565b50505050565b6102cb610838565b60035482146102ed57604051637db491eb60e01b815260040160405180910390fd5b60006102f984846104b6565b905060005b60038110156104545760006002600084846003811061031f5761031f610aed565b602090810291909101516040908101518352828201939093529082016000908120835160608101855281546001600160a01b0316815260018201549381018490526002909101549381018490529350911515919015159085856003811061038857610388610aed565b6020020151602001516001600160a01b031684600001516001600160a01b03161480156103d9575060008686600381106103c4576103c4610aed565b6020020151602001516001600160a01b031614155b9050600060405180606001604052808888600381106103fa576103fa610aed565b60200201516040015181526020018660200151815260200185801561041c5750845b80156104255750835b1515905290508088876003811061043e5761043e610aed565b60200201525050600190930192506102fe915050565b505092915050565b604080516001600160a01b0385811660248301528416604482015260648082018490528251808303909101815260849091019091526020810180516001600160e01b03166323b872dd60e01b1790526102bd908590610691565b6104be610877565b60005b8281101561068a576000600260008686858181106104e1576104e1610aed565b9050610100020160e0013581526020019081526020016000206040518060600160405290816000820160009054906101000a90046001600160a01b03166001600160a01b03166001600160a01b03168152602001600182015481526020016002820154815250509050600260008460006003811061056157610561610aed565b602002015160400151815260200190815260200160002060010154816020015111156105cd576020830180516040850152835190528484838181106105a8576105a8610aed565b905061010002018036038101906105bf9190610beb565b8360005b6020020152610681565b602080840151604090810151600090815260028352206001015490820151111561062c576020830151604084015284848381811061060d5761060d610aed565b905061010002018036038101906106249190610beb565b8360016105c3565b604080840151810151600090815260026020908152919020600101549082015111156106815784848381811061066457610664610aed565b9050610100020180360381019061067b9190610beb565b60408401525b506001016104c1565b5092915050565b60006106a66001600160a01b038416836106fe565b905080516000141580156106cb5750808060200190518101906106c99190610c81565b155b156106f957604051635274afe760e01b81526001600160a01b03841660048201526024015b60405180910390fd5b505050565b606061070c83836000610713565b9392505050565b6060814710156107385760405163cd78605960e01b81523060048201526024016106f0565b600080856001600160a01b031684866040516107549190610ca3565b60006040518083038185875af1925050503d8060008114610791576040519150601f19603f3d011682016040523d82523d6000602084013e610796565b606091505b50915091506107a68683836107b0565b9695505050505050565b6060826107c5576107c08261080c565b61070c565b81511580156107dc57506001600160a01b0384163b155b1561080557604051639996b31560e01b81526001600160a01b03851660048201526024016106f0565b508061070c565b80511561081c5780518082602001fd5b604051630a12f52160e11b815260040160405180910390fd5b50565b60405180606001604052806003905b60408051606081018252600080825260208083018290529282015282526000199092019101816108475790505090565b60405180606001604052806003905b61088e6108a4565b8152602001906001900390816108865790505090565b60405180606001604052806108e86040518060c001604052806000815260200160008152602001600081526020016000815260200160008152602001600081525090565b815260006020820181905260409091015290565b60006020828403121561090e57600080fd5b5035919050565b6000806020838503121561092857600080fd5b823567ffffffffffffffff81111561093f57600080fd5b8301601f8101851361095057600080fd5b803567ffffffffffffffff81111561096757600080fd5b85602060c08302840101111561097c57600080fd5b6020919091019590945092505050565b602080825282518282018190526000918401906040840190835b818110156109c45783518352602093840193909201916001016109a6565b509095945050505050565b600080602083850312156109e257600080fd5b823567ffffffffffffffff8111156109f957600080fd5b8301601f81018513610a0a57600080fd5b803567ffffffffffffffff811115610a2157600080fd5b85602060608302840101111561097c57600080fd5b60008060208385031215610a4957600080fd5b823567ffffffffffffffff811115610a6057600080fd5b8301601f81018513610a7157600080fd5b803567ffffffffffffffff811115610a8857600080fd5b8560208260081b840101111561097c57600080fd5b6101208101818360005b6003811015610ae4578151805184526020810151602085015260408101511515604085015250606083019250602082019150600181019050610aa7565b50505092915050565b634e487b7160e01b600052603260045260246000fd5b6001600160a01b038116811461083557600080fd5b8035610b2381610b03565b919050565b600060208284031215610b3a57600080fd5b813561070c81610b03565b8135610b5081610b03565b81546001600160a01b0319166001600160a01b039190911617815560208201356001820155604090910135600290910155565b6040516060810167ffffffffffffffff81118282101715610bb457634e487b7160e01b600052604160045260246000fd5b60405290565b60405160c0810167ffffffffffffffff81118282101715610bb457634e487b7160e01b600052604160045260246000fd5b600081830361010081128015610c0057600080fd5b506000610c0b610b83565b60c0831215610c18578182fd5b610c20610bba565b853581526020808701359082015260408087013590820152606080870135908201526080808701359082015260a080870135908201528082529250610c6760c08601610b18565b602082015260e09490940135604085015250919392505050565b600060208284031215610c9357600080fd5b8151801515811461070c57600080fd5b6000825160005b81811015610cc45760208186018101518583015201610caa565b50600092019182525091905056fea26469706673582212207d1a9d88b0ba14ca908470a69ea19a09d2c7617056be2605039bc4d121f4fc4b64736f6c634300081c0033", "linkReferences": {}, "deployedLinkReferences": {} } diff --git a/evmlib/src/contract/payment_vault/handler.rs b/evmlib/src/contract/payment_vault/handler.rs index e1bdaec50f..1000d7d108 100644 --- a/evmlib/src/contract/payment_vault/handler.rs +++ b/evmlib/src/contract/payment_vault/handler.rs @@ -1,9 +1,7 @@ use crate::common::{Address, Amount, Calldata, TxHash}; use crate::contract::payment_vault::error::Error; +use crate::contract::payment_vault::interface::IPaymentVault; use crate::contract::payment_vault::interface::IPaymentVault::IPaymentVaultInstance; -use crate::contract::payment_vault::interface::{ - IPaymentVault, REQUIRED_PAYMENT_VERIFICATION_LENGTH, -}; use alloy::network::{Network, TransactionBuilder}; use alloy::providers::Provider; use alloy::transports::Transport; @@ -99,10 +97,6 @@ where .map(|v| v.into()) .collect(); - if payment_verifications.len() != REQUIRED_PAYMENT_VERIFICATION_LENGTH { - return Err(Error::PaymentVerificationLengthInvalid); - } - let results = self .contract .verifyPayment(payment_verifications) diff --git a/evmlib/src/contract/payment_vault/interface.rs b/evmlib/src/contract/payment_vault/interface.rs index 36ec3ee6b8..1e2e0f1e7c 100644 --- a/evmlib/src/contract/payment_vault/interface.rs +++ b/evmlib/src/contract/payment_vault/interface.rs @@ -3,8 +3,6 @@ use crate::quoting_metrics::QuotingMetrics; use alloy::primitives::FixedBytes; use alloy::sol; -pub const REQUIRED_PAYMENT_VERIFICATION_LENGTH: usize = 5; - sol!( #[allow(missing_docs)] #[derive(Debug)] @@ -17,7 +15,8 @@ impl From<(QuoteHash, QuotingMetrics, Address)> for IPaymentVault::PaymentVerifi fn from(value: (QuoteHash, QuotingMetrics, Address)) -> Self { Self { metrics: value.1.into(), - dataPayment: (value.0, value.2, Amount::ZERO).into(), + rewardsAddress: value.2, + quoteHash: value.0, } } } diff --git a/evmlib/tests/payment_vault.rs b/evmlib/tests/payment_vault.rs index 7578786c11..9d9a104eb0 100644 --- a/evmlib/tests/payment_vault.rs +++ b/evmlib/tests/payment_vault.rs @@ -15,7 +15,6 @@ use evmlib::common::{Amount, U256}; use evmlib::contract::network_token::NetworkToken; use evmlib::contract::payment_vault::handler::PaymentVaultHandler; use evmlib::contract::payment_vault::interface::IPaymentVault::DataPayment; -use evmlib::contract::payment_vault::interface::REQUIRED_PAYMENT_VERIFICATION_LENGTH; use evmlib::contract::payment_vault::{interface, MAX_TRANSFERS_PER_TRANSACTION}; use evmlib::quoting_metrics::QuotingMetrics; use evmlib::testnet::{deploy_data_payments_contract, deploy_network_token_contract, start_node}; @@ -137,7 +136,7 @@ async fn test_verify_payment() { let mut quote_payments = vec![]; - for _ in 0..REQUIRED_PAYMENT_VERIFICATION_LENGTH { + for _ in 0..5 { let quote_payment = random_quote_payment(); quote_payments.push(quote_payment); } From fd2059dfde8f1abdb337753f61ea80c4571b0d9a Mon Sep 17 00:00:00 2001 From: Warm Beer Date: Thu, 12 Dec 2024 16:55:17 +0100 Subject: [PATCH 32/33] fix: update get_quotes_by_peer --- ant-evm/src/data_payments.rs | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/ant-evm/src/data_payments.rs b/ant-evm/src/data_payments.rs index ddc1840998..48f904f8d4 100644 --- a/ant-evm/src/data_payments.rs +++ b/ant-evm/src/data_payments.rs @@ -77,9 +77,9 @@ impl ProofOfPayment { pub fn quotes_by_peer(&self, peer_id: &PeerId) -> Vec<&PaymentQuote> { self.peer_quotes .iter() - .filter_map(|(id, quote)| { - if let Ok(id) = id.to_peer_id() { - if id == *peer_id { + .filter_map(|(_id, quote)| { + if let Ok(quote_peer_id) = quote.peer_id() { + if *peer_id == quote_peer_id { return Some(quote); } } From 86fcaac067affabad3f9e0852a655b33bbe2131f Mon Sep 17 00:00:00 2001 From: Warm Beer Date: Thu, 12 Dec 2024 17:52:02 +0100 Subject: [PATCH 33/33] fix: clippy error --- evmlib/tests/payment_vault.rs | 8 ++------ 1 file changed, 2 insertions(+), 6 deletions(-) diff --git a/evmlib/tests/payment_vault.rs b/evmlib/tests/payment_vault.rs index 9d9a104eb0..41c5881cbb 100644 --- a/evmlib/tests/payment_vault.rs +++ b/evmlib/tests/payment_vault.rs @@ -14,7 +14,6 @@ use alloy::transports::http::{Client, Http}; use evmlib::common::{Amount, U256}; use evmlib::contract::network_token::NetworkToken; use evmlib::contract::payment_vault::handler::PaymentVaultHandler; -use evmlib::contract::payment_vault::interface::IPaymentVault::DataPayment; use evmlib::contract::payment_vault::{interface, MAX_TRANSFERS_PER_TRANSACTION}; use evmlib::quoting_metrics::QuotingMetrics; use evmlib::testnet::{deploy_data_payments_contract, deploy_network_token_contract, start_node}; @@ -158,11 +157,8 @@ async fn test_verify_payment() { .into_iter() .map(|v| interface::IPaymentVault::PaymentVerification { metrics: QuotingMetrics::default().into(), - dataPayment: DataPayment { - rewardsAddress: v.1, - amount: v.2, - quoteHash: v.0, - }, + rewardsAddress: v.1, + quoteHash: v.0, }) .collect();