From 6c88c0dcb43816367f7cbcf848695c1904c521f3 Mon Sep 17 00:00:00 2001 From: Ori Newman Date: Fri, 1 Sep 2023 15:05:13 +0300 Subject: [PATCH] Add pruning point list validations (#243) * Add pruning point list validations * Don't do DownloadHeadersProof on a relatively synced node * fmt * clippy * address review comments * Handle another case of finality conflict * Handle two TODOs * Address review comments * Use real consensus in IBD with headers proof when needed * acquire current consensus session only when needed * typo --- components/consensusmanager/src/session.rs | 18 +++- consensus/core/src/api/mod.rs | 16 ++++ consensus/core/src/config/params.rs | 4 + consensus/src/consensus/factory.rs | 2 + consensus/src/consensus/mod.rs | 33 +++++++ consensus/src/consensus/services.rs | 4 +- consensus/src/consensus/test_consensus.rs | 6 +- .../pipeline/virtual_processor/processor.rs | 31 +++++- consensus/src/processes/pruning.rs | 94 ++++++++++++++++++- consensus/src/processes/pruning_proof/mod.rs | 6 ++ protocol/flows/src/v5/ibd/flow.rs | 64 +++++++++---- simpa/src/main.rs | 8 +- simpa/src/simulator/network.rs | 11 ++- 13 files changed, 266 insertions(+), 31 deletions(-) diff --git a/components/consensusmanager/src/session.rs b/components/consensusmanager/src/session.rs index 31f22c452d..28b1d21843 100644 --- a/components/consensusmanager/src/session.rs +++ b/components/consensusmanager/src/session.rs @@ -10,7 +10,7 @@ use kaspa_consensus_core::{ blockstatus::BlockStatus, errors::consensus::ConsensusResult, header::Header, - pruning::{PruningPointProof, PruningPointTrustedData}, + pruning::{PruningPointProof, PruningPointTrustedData, PruningPointsList}, trusted::{ExternalGhostdagData, TrustedBlock}, tx::{Transaction, TransactionOutpoint, UtxoEntry}, BlockHashSet, ChainPath, Hash, @@ -350,6 +350,22 @@ impl ConsensusSessionOwned { ) -> ConsensusResult { self.clone().spawn_blocking(move |c| c.estimate_network_hashes_per_second(start_hash, window_size)).await } + + pub async fn async_validate_pruning_points(&self) -> ConsensusResult<()> { + self.clone().spawn_blocking(move |c| c.validate_pruning_points()).await + } + + pub async fn async_are_pruning_points_violating_finality(&self, pp_list: PruningPointsList) -> bool { + self.clone().spawn_blocking(move |c| c.are_pruning_points_violating_finality(pp_list)).await + } + + pub async fn async_creation_timestamp(&self) -> u64 { + self.clone().spawn_blocking(move |c| c.creation_timestamp()).await + } + + pub async fn async_finality_point(&self) -> Hash { + self.clone().spawn_blocking(move |c| c.finality_point()).await + } } pub type ConsensusProxy = ConsensusSessionOwned; diff --git a/consensus/core/src/api/mod.rs b/consensus/core/src/api/mod.rs index 6bc21677b5..b9662ed770 100644 --- a/consensus/core/src/api/mod.rs +++ b/consensus/core/src/api/mod.rs @@ -256,6 +256,22 @@ pub trait ConsensusApi: Send + Sync { fn estimate_network_hashes_per_second(&self, start_hash: Option, window_size: usize) -> ConsensusResult { unimplemented!() } + + fn validate_pruning_points(&self) -> ConsensusResult<()> { + unimplemented!() + } + + fn are_pruning_points_violating_finality(&self, pp_list: PruningPointsList) -> bool { + unimplemented!() + } + + fn creation_timestamp(&self) -> u64 { + unimplemented!() + } + + fn finality_point(&self) -> Hash { + unimplemented!() + } } pub type DynConsensus = Arc; diff --git a/consensus/core/src/config/params.rs b/consensus/core/src/config/params.rs index f0975cc3fd..f736887b8e 100644 --- a/consensus/core/src/config/params.rs +++ b/consensus/core/src/config/params.rs @@ -237,6 +237,10 @@ impl Params { pub fn default_rpc_port(&self) -> u16 { self.net.default_rpc_port() } + + pub fn finality_duration(&self) -> u64 { + self.target_time_per_block * self.finality_depth + } } impl From for Params { diff --git a/consensus/src/consensus/factory.rs b/consensus/src/consensus/factory.rs index a591820b77..0d66b61f2f 100644 --- a/consensus/src/consensus/factory.rs +++ b/consensus/src/consensus/factory.rs @@ -208,6 +208,7 @@ impl ConsensusFactory for Factory { session_lock.clone(), self.notification_root.clone(), self.counters.clone(), + entry.creation_timestamp, )); // We write the new active entry only once the instance was created successfully. @@ -235,6 +236,7 @@ impl ConsensusFactory for Factory { session_lock.clone(), self.notification_root.clone(), self.counters.clone(), + entry.creation_timestamp, )); (ConsensusInstance::new(session_lock, consensus.clone()), Arc::new(Ctl::new(self.management_store.clone(), db, consensus))) diff --git a/consensus/src/consensus/mod.rs b/consensus/src/consensus/mod.rs index d73429b6e9..1795236753 100644 --- a/consensus/src/consensus/mod.rs +++ b/consensus/src/consensus/mod.rs @@ -66,6 +66,7 @@ use crossbeam_channel::{ }; use itertools::Itertools; use kaspa_consensusmanager::{SessionLock, SessionReadGuard}; + use kaspa_database::prelude::StoreResultExtensions; use kaspa_hashes::Hash; use kaspa_muhash::MuHash; @@ -111,6 +112,9 @@ pub struct Consensus { // Config config: Arc, + + // Other + creation_timestamp: u64, } impl Deref for Consensus { @@ -128,6 +132,7 @@ impl Consensus { pruning_lock: SessionLock, notification_root: Arc, counters: Arc, + creation_timestamp: u64, ) -> Self { let params = &config.params; let perf_params = &config.perf; @@ -262,6 +267,7 @@ impl Consensus { notification_root, counters, config, + creation_timestamp, } } @@ -515,6 +521,20 @@ impl ConsensusApi for Consensus { self.virtual_processor.import_pruning_point_utxo_set(new_pruning_point, imported_utxo_multiset) } + fn validate_pruning_points(&self) -> ConsensusResult<()> { + let hst = self.storage.headers_selected_tip_store.read().get().unwrap().hash; + let pp_info = self.pruning_point_store.read().get().unwrap(); + if !self.services.pruning_point_manager.is_valid_pruning_point(pp_info.pruning_point, hst) { + return Err(ConsensusError::General("invalid pruning point candidate")); + } + + if !self.services.pruning_point_manager.are_pruning_points_in_valid_chain(pp_info, hst) { + return Err(ConsensusError::General("past pruning points do not form a valid chain")); + } + + Ok(()) + } + fn header_exists(&self, hash: Hash) -> bool { match self.statuses_store.read().get(hash).unwrap_option() { Some(status) => status.has_block_header(), @@ -727,4 +747,17 @@ impl ConsensusApi for Consensus { } } } + + fn are_pruning_points_violating_finality(&self, pp_list: PruningPointsList) -> bool { + self.virtual_processor.are_pruning_points_violating_finality(pp_list) + } + + fn creation_timestamp(&self) -> u64 { + self.creation_timestamp + } + + fn finality_point(&self) -> Hash { + self.virtual_processor + .virtual_finality_point(&self.virtual_stores.read().state.get().unwrap().ghostdag_data, self.pruning_point()) + } } diff --git a/consensus/src/consensus/services.rs b/consensus/src/consensus/services.rs index 347a965937..f0b791ba26 100644 --- a/consensus/src/consensus/services.rs +++ b/consensus/src/consensus/services.rs @@ -37,7 +37,8 @@ pub type DbSyncManager = SyncManager< DbStatusesStore, >; -pub type DbPruningPointManager = PruningPointManager; +pub type DbPruningPointManager = + PruningPointManager; pub type DbBlockDepthManager = BlockDepthManager; pub type DbParentsManager = ParentsManager>; @@ -153,6 +154,7 @@ impl ConsensusServices { storage.ghostdag_primary_store.clone(), storage.headers_store.clone(), storage.past_pruning_points_store.clone(), + storage.headers_selected_tip_store.clone(), ); let parents_manager = ParentsManager::new( diff --git a/consensus/src/consensus/test_consensus.rs b/consensus/src/consensus/test_consensus.rs index e75092b6a6..620722022a 100644 --- a/consensus/src/consensus/test_consensus.rs +++ b/consensus/src/consensus/test_consensus.rs @@ -50,7 +50,7 @@ impl TestConsensus { pub fn with_db(db: Arc, config: &Config, notification_sender: Sender) -> Self { let notification_root = Arc::new(ConsensusNotificationRoot::new(notification_sender)); let counters = Arc::new(ProcessingCounters::default()); - let consensus = Arc::new(Consensus::new(db, Arc::new(config.clone()), Default::default(), notification_root, counters)); + let consensus = Arc::new(Consensus::new(db, Arc::new(config.clone()), Default::default(), notification_root, counters, 0)); let block_builder = TestBlockBuilder::new(consensus.virtual_processor.clone()); Self { params: config.params.clone(), consensus, block_builder, db_lifetime: Default::default() } @@ -61,7 +61,7 @@ impl TestConsensus { let (db_lifetime, db) = create_temp_db!(ConnBuilder::default()); let notification_root = Arc::new(ConsensusNotificationRoot::new(notification_sender)); let counters = Arc::new(ProcessingCounters::default()); - let consensus = Arc::new(Consensus::new(db, Arc::new(config.clone()), Default::default(), notification_root, counters)); + let consensus = Arc::new(Consensus::new(db, Arc::new(config.clone()), Default::default(), notification_root, counters, 0)); let block_builder = TestBlockBuilder::new(consensus.virtual_processor.clone()); Self { consensus, block_builder, params: config.params.clone(), db_lifetime } @@ -73,7 +73,7 @@ impl TestConsensus { let (dummy_notification_sender, _) = async_channel::unbounded(); let notification_root = Arc::new(ConsensusNotificationRoot::new(dummy_notification_sender)); let counters = Arc::new(ProcessingCounters::default()); - let consensus = Arc::new(Consensus::new(db, Arc::new(config.clone()), Default::default(), notification_root, counters)); + let consensus = Arc::new(Consensus::new(db, Arc::new(config.clone()), Default::default(), notification_root, counters, 0)); let block_builder = TestBlockBuilder::new(consensus.virtual_processor.clone()); Self { consensus, block_builder, params: config.params.clone(), db_lifetime } diff --git a/consensus/src/pipeline/virtual_processor/processor.rs b/consensus/src/pipeline/virtual_processor/processor.rs index 488238f822..5a0c182de0 100644 --- a/consensus/src/pipeline/virtual_processor/processor.rs +++ b/consensus/src/pipeline/virtual_processor/processor.rs @@ -54,6 +54,7 @@ use kaspa_consensus_core::{ config::genesis::GenesisBlock, header::Header, merkle::calc_hash_merkle_root, + pruning::PruningPointsList, tx::{MutableTransaction, Transaction}, utxo::{ utxo_diff::UtxoDiff, @@ -316,7 +317,7 @@ impl VirtualStateProcessor { .expect("expecting an open unbounded channel"); } - pub(super) fn virtual_finality_point(&self, virtual_ghostdag_data: &GhostdagData, pruning_point: Hash) -> Hash { + pub(crate) fn virtual_finality_point(&self, virtual_ghostdag_data: &GhostdagData, pruning_point: Hash) -> Hash { let finality_point = self.depth_manager.calc_finality_point(virtual_ghostdag_data, pruning_point); if self.reachability_service.is_chain_ancestor_of(pruning_point, finality_point) { finality_point @@ -933,6 +934,34 @@ impl VirtualStateProcessor { Ok(()) } + + pub fn are_pruning_points_violating_finality(&self, pp_list: PruningPointsList) -> bool { + // Ideally we would want to check if the last known pruning point has the finality point + // in its chain, but in some cases it's impossible: let `lkp` be the last known pruning + // point from the list, and `fup` be the first unknown pruning point (the one following `lkp`). + // fup.blue_score - lkp.blue_score ≈ finality_depth (±k), so it's possible for `lkp` not to + // have the finality point in its past. So we have no choice but to check if `lkp` + // has `finality_point.finality_point` in its chain (in the worst case `fup` is one block + // above the current finality point, and in this case `lkp` will be a few blocks above the + // finality_point.finality_point), meaning this function can only detect finality violations + // in depth of 2*finality_depth, and can give false negatives for smaller finality violations. + let current_pp = self.pruning_point_store.read().pruning_point().unwrap(); + let vf = self.virtual_finality_point(&self.virtual_stores.read().state.get().unwrap().ghostdag_data, current_pp); + let vff = self.depth_manager.calc_finality_point(&self.ghostdag_primary_store.get_data(vf).unwrap(), current_pp); + + let last_known_pp = pp_list.iter().rev().find(|pp| match self.statuses_store.read().get(pp.hash).unwrap_option() { + Some(status) => status.is_valid(), + None => false, + }); + + if let Some(last_known_pp) = last_known_pp { + !self.reachability_service.is_chain_ancestor_of(vff, last_known_pp.hash) + } else { + // If no pruning point is known, there's definitely a finality violation + // (normally at least genesis should be known). + true + } + } } enum MergesetIncreaseResult { diff --git a/consensus/src/processes/pruning.rs b/consensus/src/processes/pruning.rs index 7f34eaa421..1d419c063d 100644 --- a/consensus/src/processes/pruning.rs +++ b/consensus/src/processes/pruning.rs @@ -1,4 +1,4 @@ -use std::sync::Arc; +use std::{collections::VecDeque, sync::Arc}; use super::reachability::ReachabilityResultExtensions; use crate::model::{ @@ -6,12 +6,15 @@ use crate::model::{ stores::{ ghostdag::{CompactGhostdagData, GhostdagStoreReader}, headers::HeaderStoreReader, + headers_selected_tip::HeadersSelectedTipStoreReader, past_pruning_points::PastPruningPointsStoreReader, pruning::PruningPointInfo, reachability::ReachabilityStoreReader, }, }; use kaspa_hashes::Hash; +use kaspa_utils::option::OptionExtensions; +use parking_lot::RwLock; #[derive(Clone)] pub struct PruningPointManager< @@ -19,6 +22,7 @@ pub struct PruningPointManager< T: ReachabilityStoreReader, U: HeaderStoreReader, V: PastPruningPointsStoreReader, + W: HeadersSelectedTipStoreReader, > { pruning_depth: u64, finality_depth: u64, @@ -28,10 +32,16 @@ pub struct PruningPointManager< ghostdag_store: Arc, headers_store: Arc, past_pruning_points_store: Arc, + header_selected_tip_store: Arc>, } -impl - PruningPointManager +impl< + S: GhostdagStoreReader, + T: ReachabilityStoreReader, + U: HeaderStoreReader, + V: PastPruningPointsStoreReader, + W: HeadersSelectedTipStoreReader, + > PruningPointManager { pub fn new( pruning_depth: u64, @@ -41,6 +51,7 @@ impl, headers_store: Arc, past_pruning_points_store: Arc, + header_selected_tip_store: Arc>, ) -> Self { Self { pruning_depth, @@ -50,6 +61,7 @@ impl= pp_bs + self.pruning_depth } + + pub fn is_valid_pruning_point(&self, pp_candidate: Hash, hst: Hash) -> bool { + if pp_candidate == self.genesis_hash { + return true; + } + if !self.reachability_service.is_chain_ancestor_of(pp_candidate, hst) { + return false; + } + + let hst_bs = self.ghostdag_store.get_blue_score(hst).unwrap(); + self.is_pruning_point_in_pruning_depth(hst_bs, pp_candidate) + } + + pub fn are_pruning_points_in_valid_chain(&self, pruning_info: PruningPointInfo, hst: Hash) -> bool { + // We want to validate that the past pruning points form a chain to genesis. Since + // each pruning point's header doesn't point to the previous pruning point, but to + // the pruning point from its POV, we can't just traverse from one pruning point to + // the next one by merely relying on the current pruning point header, but instead + // we rely on the fact that each pruning point is pointed by another known block or + // pruning point. + // So in the first stage we go over the selected chain and add to the queue of expected + // pruning points all the pruning points from the POV of some chain block. In the second + // stage we go over the past pruning points from recent to older, check that it's the head + // of the queue (by popping the queue), and add its header pruning point to the queue since + // we expect to see it later on the list. + // The first stage is important because the most recent pruning point is pointing to a few + // pruning points before, so the first few pruning points on the list won't be pointed by + // any other pruning point in the list, so we are compelled to check if it's refereced by + // the selected chain. + let mut expected_pps_queue = VecDeque::new(); + for current in self.reachability_service.backward_chain_iterator(hst, pruning_info.pruning_point, false) { + let current_header = self.headers_store.get_header(current).unwrap(); + if expected_pps_queue.back().is_none_or(|&&h| h != current_header.pruning_point) { + expected_pps_queue.push_back(current_header.pruning_point); + } + } + + for idx in (0..=pruning_info.index).rev() { + let pp = self.past_pruning_points_store.get(idx).unwrap(); + let pp_header = self.headers_store.get_header(pp).unwrap(); + let Some(expected_pp) = expected_pps_queue.pop_front() else { + // If we have less than expected pruning points. + return false; + }; + + if expected_pp != pp { + return false; + } + + if idx == 0 { + // The 0th pruning point should always be genesis, and no + // more pruning points should be expected below it. + if !expected_pps_queue.is_empty() || pp != self.genesis_hash { + return false; + } + break; + } + + // Add the pruning point from the POV of the current one if it's + // not already added. + match expected_pps_queue.back() { + Some(last_added_pp) => { + if *last_added_pp != pp_header.pruning_point { + expected_pps_queue.push_back(pp_header.pruning_point); + } + } + None => { + // expected_pps_queue should always have one block in the queue + // until we reach genesis. + return false; + } + } + } + + true + } } #[cfg(test)] diff --git a/consensus/src/processes/pruning_proof/mod.rs b/consensus/src/processes/pruning_proof/mod.rs index 8cf14c272b..593760450d 100644 --- a/consensus/src/processes/pruning_proof/mod.rs +++ b/consensus/src/processes/pruning_proof/mod.rs @@ -537,6 +537,12 @@ impl PruningProofManager { } } + if current_pp == self.genesis_hash { + // If the proof has better tips and the currnet pruning point is still + // genesis, we consider the proof state to be better. + return Ok(()); + } + for level in (0..=self.max_block_level).rev() { let level_idx = level as usize; match relations_read[level_idx].get_parents(current_pp).unwrap_option() { diff --git a/protocol/flows/src/v5/ibd/flow.rs b/protocol/flows/src/v5/ibd/flow.rs index ffddbe225e..add81ae842 100644 --- a/protocol/flows/src/v5/ibd/flow.rs +++ b/protocol/flows/src/v5/ibd/flow.rs @@ -14,7 +14,7 @@ use kaspa_consensus_core::{ BlockHashSet, }; use kaspa_consensusmanager::{spawn_blocking, ConsensusProxy, StagingConsensus}; -use kaspa_core::{debug, info, warn}; +use kaspa_core::{debug, info, time::unix_now, warn}; use kaspa_hashes::Hash; use kaspa_muhash::MuHash; use kaspa_p2p_lib::{ @@ -146,14 +146,30 @@ impl IbdFlow { return Ok(IbdType::Sync(highest_known_syncer_chain_hash)); } - // TODO: in this case we know a syncer chain block, but it violates our current finality. In some cases - // this info should possibly be used to reject the IBD despite having more blue work etc. + // If the pruning point is not in the chain of `highest_known_syncer_chain_hash`, it + // means it's in its antichain (because if `highest_known_syncer_chain_hash` was in + // the pruning point's past the pruning point itself would be + // `highest_known_syncer_chain_hash`). So it means there's a finality conflict. + // TODO: consider performing additional actions on finality conflicts in addition to disconnecting from the peer (e.g., banning, rpc notification) + return Ok(IbdType::None); } let hst_header = consensus.async_get_header(consensus.async_get_headers_selected_tip().await).await.unwrap(); if relay_header.blue_score >= hst_header.blue_score + self.ctx.config.pruning_depth && relay_header.blue_work > hst_header.blue_work { + if unix_now() > consensus.async_creation_timestamp().await + self.ctx.config.finality_duration() { + let fp = consensus.async_finality_point().await; + let fp_ts = consensus.async_get_header(fp).await?.timestamp; + if unix_now() < fp_ts + self.ctx.config.finality_duration() * 3 / 2 { + // We reject the headers proof if the node has a relatively up-to-date finality point and current + // consensus has matured for long enough (and not recently synced). This is mostly a spam-protector + // since subsequent checks identify these violations as well + // TODO: consider performing additional actions on finality conflicts in addition to disconnecting from the peer (e.g., banning, rpc notification) + return Ok(IbdType::None); + } + } + // The relayed block has sufficient blue score and blue work over the current header selected tip Ok(IbdType::DownloadHeadersProof) } else { @@ -169,16 +185,17 @@ impl IbdFlow { ) -> Result<(), ProtocolError> { info!("Starting IBD with headers proof with peer {}", self.router); - let session = staging.session().await; + let staging_session = staging.session().await; - let pruning_point = self.sync_and_validate_pruning_proof(&session).await?; - self.sync_headers(&session, syncer_virtual_selected_parent, pruning_point, relay_block).await?; - self.validate_staging_timestamps(&self.ctx.consensus().session().await, &session).await?; - self.sync_pruning_point_utxoset(&session, pruning_point).await?; + let pruning_point = self.sync_and_validate_pruning_proof(&staging_session).await?; + self.sync_headers(&staging_session, syncer_virtual_selected_parent, pruning_point, relay_block).await?; + staging_session.async_validate_pruning_points().await?; + self.validate_staging_timestamps(&self.ctx.consensus().session().await, &staging_session).await?; + self.sync_pruning_point_utxoset(&staging_session, pruning_point).await?; Ok(()) } - async fn sync_and_validate_pruning_proof(&mut self, consensus: &ConsensusProxy) -> Result { + async fn sync_and_validate_pruning_proof(&mut self, staging: &ConsensusProxy) -> Result { self.router.enqueue(make_message!(Payload::RequestPruningPointProof, RequestPruningPointProofMessage {})).await?; // Pruning proof generation and communication might take several minutes, so we allow a long 10 minute timeout @@ -186,11 +203,23 @@ impl IbdFlow { let proof: PruningPointProof = msg.try_into()?; debug!("received proof with overall {} headers", proof.iter().map(|l| l.len()).sum::()); + // Get a new session for current consensus (non staging) + let consensus = self.ctx.consensus().session().await; + + // The proof is validated in the context of current consensus let proof = consensus.clone().spawn_blocking(move |c| c.validate_pruning_proof(&proof).map(|()| proof)).await?; let proof_pruning_point = proof[0].last().expect("was just ensured by validation").hash; - // TODO: verify the proof pruning point is different than current consensus pruning point + if proof_pruning_point == self.ctx.config.genesis.hash { + return Err(ProtocolError::Other("the proof pruning point is the genesis block")); + } + + if proof_pruning_point == consensus.async_pruning_point().await { + return Err(ProtocolError::Other("the proof pruning point is the same as the current pruning point")); + } + + drop(consensus); self.router .enqueue(make_message!(Payload::RequestPruningPointAndItsAnticone, RequestPruningPointAndItsAnticoneMessage {})) @@ -207,7 +236,11 @@ impl IbdFlow { return Err(ProtocolError::Other("the first pruning point in the list is expected to be genesis")); } - // TODO: validate pruning points before importing + // Check if past pruning points violate finality of current consensus + if self.ctx.consensus().session().await.async_are_pruning_points_violating_finality(pruning_points.clone()).await { + // TODO: consider performing additional actions on finality conflicts in addition to disconnecting from the peer (e.g., banning, rpc notification) + return Err(ProtocolError::Other("pruning points are violating finality")); + } let msg = dequeue_with_timeout!(self.incoming_route, Payload::TrustedData)?; let pkg: TrustedDataPackage = msg.try_into()?; @@ -230,7 +263,7 @@ impl IbdFlow { let mut trusted_set = pkg.build_trusted_subdag(entries)?; if self.ctx.config.enable_sanity_checks { - trusted_set = consensus + trusted_set = staging .clone() .spawn_blocking(move |c| { let ref_proof = proof.clone(); @@ -261,7 +294,7 @@ impl IbdFlow { }) .await; } else { - trusted_set = consensus + trusted_set = staging .clone() .spawn_blocking(move |c| { c.apply_pruning_proof(proof, &trusted_set); @@ -283,12 +316,9 @@ impl IbdFlow { last_index = i; } // TODO: queue and join in batches - consensus.validate_and_insert_trusted_block(tb).await?; + staging.validate_and_insert_trusted_block(tb).await?; } info!("Done processing trusted blocks"); - - // TODO: make sure that the proof pruning point is not genesis - Ok(proof_pruning_point) } diff --git a/simpa/src/main.rs b/simpa/src/main.rs index 4281663884..123f21f724 100644 --- a/simpa/src/main.rs +++ b/simpa/src/main.rs @@ -19,7 +19,7 @@ use kaspa_consensus_core::{ BlockHashSet, BlockLevel, HashMapCustomHasher, }; use kaspa_consensus_notify::root::ConsensusNotificationRoot; -use kaspa_core::{info, task::service::AsyncService, task::tick::TickService, trace, warn}; +use kaspa_core::{info, task::service::AsyncService, task::tick::TickService, time::unix_now, trace, warn}; use kaspa_database::prelude::ConnBuilder; use kaspa_database::{create_temp_db, load_existing_db}; use kaspa_hashes::Hash; @@ -187,7 +187,8 @@ fn main() { }; let (dummy_notification_sender, _) = unbounded(); let notification_root = Arc::new(ConsensusNotificationRoot::new(dummy_notification_sender)); - let consensus = Arc::new(Consensus::new(db, config.clone(), Default::default(), notification_root, Default::default())); + let consensus = + Arc::new(Consensus::new(db, config.clone(), Default::default(), notification_root, Default::default(), unix_now())); (consensus, lifetime) } else { let until = if args.target_blocks.is_none() { config.genesis.timestamp + args.sim_time * 1000 } else { u64::MAX }; // milliseconds @@ -215,7 +216,8 @@ fn main() { let (_lifetime2, db2) = create_temp_db!(ConnBuilder::default().with_parallelism(num_cpus::get())); let (dummy_notification_sender, _) = unbounded(); let notification_root = Arc::new(ConsensusNotificationRoot::new(dummy_notification_sender)); - let consensus2 = Arc::new(Consensus::new(db2, config.clone(), Default::default(), notification_root, Default::default())); + let consensus2 = + Arc::new(Consensus::new(db2, config.clone(), Default::default(), notification_root, Default::default(), unix_now())); let handles2 = consensus2.run_processors(); rt.block_on(validate(&consensus, &consensus2, &config, args.delay, args.bps)); consensus2.shutdown(handles2); diff --git a/simpa/src/simulator/network.rs b/simpa/src/simulator/network.rs index f20b68d1d6..82e75178e8 100644 --- a/simpa/src/simulator/network.rs +++ b/simpa/src/simulator/network.rs @@ -1,5 +1,6 @@ use async_channel::unbounded; use kaspa_consensus_notify::root::ConsensusNotificationRoot; +use kaspa_core::time::unix_now; use std::sync::Arc; use std::thread::JoinHandle; @@ -75,8 +76,14 @@ impl KaspaNetworkSimulator { let (dummy_notification_sender, _) = unbounded(); let notification_root = Arc::new(ConsensusNotificationRoot::new(dummy_notification_sender)); - let consensus = - Arc::new(Consensus::new(db, self.config.clone(), Default::default(), notification_root, Default::default())); + let consensus = Arc::new(Consensus::new( + db, + self.config.clone(), + Default::default(), + notification_root, + Default::default(), + unix_now(), + )); let handles = consensus.run_processors(); let (sk, pk) = secp.generate_keypair(&mut rng); let miner_process = Box::new(Miner::new(