From 41244a3ea666f925648aa752c9ac476486702473 Mon Sep 17 00:00:00 2001 From: SW van Heerden Date: Thu, 10 Aug 2023 14:23:15 +0200 Subject: [PATCH] feat: remove orphan validation and only validate on insertion (#5601) Description --- Currently, block validation is done in two steps orphan check and full check. An orphan check is done when a block is first inserted into the DB (all blocks are). But before this is done, the block must pass an internal integrity check (checking signatures, rangeproofs, etc) The blocks are then stored in the orphan pool. When they are added to the main chain, a full check is done on the blocks to ensure the mmr's are correct, no double spends + another integrity check. Bad blocks are also only tracked when syncing. This PR removes the orphan check and only does validation on the block when we add it to the main chain. But to ensure a node is not spammed by useless blocks, it makes sure the block received at least has a min difficulty. The orphan pool will clean out the oldest blocks and this stored amount is configurable via the config. As soon as we can track a block to the main chain, we will do a full Proof of work check. Then as when a chain/block proof of work is higher than the current chain the blocks will be added to the main chain. This is then when we do the full validation of the block. Changed up how the bad blocks are handled, and we now properly keep track of all bad blocks. When a block fails any part of the validation it will be added to the bad_block list. When we receive a new request for a block, this will be one of the first things we check, before we spent resources validation it again. Motivation and Context --- TARI-0003 A base node waste resources validating a block more than once. From testing, the most expensive tests to run are(in order): Rangeproofs, Metadatasignature, Scriptsignature, kernel signature. These tests are all run on orphan blocks. Its very easy to put junk data in these blocks and spam a node to keep wasting resources and validated these fake blocks. By using the tests added in #5599, this PR improves `add_block` time by almost 100%. How Has This Been Tested? --- Unit tests. What process can a PR reviewer use to test or verify this change? --- Checking that we have not changed or removed any required validation rules. --- Cargo.lock | 2 +- applications/minotari_node/src/bootstrap.rs | 1 + .../src/base_node/comms_interface/error.rs | 5 ++ .../comms_interface/inbound_handlers.rs | 84 +++++++++++++++---- .../core/src/base_node/service/initializer.rs | 6 ++ .../sync/header_sync/synchronizer.rs | 4 +- .../base_node/sync/header_sync/validator.rs | 39 +++++++-- base_layer/core/src/chain_storage/async_db.rs | 2 + .../src/chain_storage/blockchain_database.rs | 56 ++++++++++--- base_layer/core/src/proof_of_work/error.rs | 2 + base_layer/core/src/validation/error.rs | 2 - .../header/header_full_validator.rs | 2 +- base_layer/core/tests/helpers/nodes.rs | 4 +- .../core/tests/tests/node_comms_interface.rs | 15 +++- 14 files changed, 181 insertions(+), 43 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 9e6d2b6923..515f97860f 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -7357,4 +7357,4 @@ dependencies = [ "quick-error 2.0.1", "regex", "time", -] +] \ No newline at end of file diff --git a/applications/minotari_node/src/bootstrap.rs b/applications/minotari_node/src/bootstrap.rs index 7ee281f76c..9383ce6fa5 100644 --- a/applications/minotari_node/src/bootstrap.rs +++ b/applications/minotari_node/src/bootstrap.rs @@ -130,6 +130,7 @@ where B: BlockchainBackend + 'static self.mempool.clone(), self.rules.clone(), base_node_config.messaging_request_timeout, + self.randomx_factory.clone(), )) .add_initializer(MempoolServiceInitializer::new( self.mempool.clone(), diff --git a/base_layer/core/src/base_node/comms_interface/error.rs b/base_layer/core/src/base_node/comms_interface/error.rs index cc1f1b2314..ce27e93089 100644 --- a/base_layer/core/src/base_node/comms_interface/error.rs +++ b/base_layer/core/src/base_node/comms_interface/error.rs @@ -30,6 +30,7 @@ use crate::{ chain_storage::ChainStorageError, consensus::ConsensusManagerError, mempool::MempoolError, + proof_of_work::{monero_rx::MergeMineError, DifficultyError}, }; #[derive(Debug, Error)] @@ -70,4 +71,8 @@ pub enum CommsInterfaceError { InvalidRequest { request: &'static str, details: String }, #[error("Peer sent invalid full block {hash}: {details}")] InvalidFullBlock { hash: FixedHash, details: String }, + #[error("Invalid merge mined block: {0}")] + MergeMineError(#[from] MergeMineError), + #[error("Invalid difficulty: {0}")] + DifficultyError(#[from] DifficultyError), } diff --git a/base_layer/core/src/base_node/comms_interface/inbound_handlers.rs b/base_layer/core/src/base_node/comms_interface/inbound_handlers.rs index ceb6cc2252..0a84a991cf 100644 --- a/base_layer/core/src/base_node/comms_interface/inbound_handlers.rs +++ b/base_layer/core/src/base_node/comms_interface/inbound_handlers.rs @@ -46,13 +46,20 @@ use crate::{ }, metrics, }, - blocks::{Block, BlockBuilder, BlockHeader, ChainBlock, NewBlock, NewBlockTemplate}, + blocks::{Block, BlockBuilder, BlockHeader, BlockHeaderValidationError, ChainBlock, NewBlock, NewBlockTemplate}, chain_storage::{async_db::AsyncBlockchainDb, BlockAddResult, BlockchainBackend, ChainStorageError, PrunedOutput}, consensus::{ConsensusConstants, ConsensusManager}, mempool::Mempool, - proof_of_work::{Difficulty, PowAlgorithm}, + proof_of_work::{ + randomx_difficulty, + randomx_factory::RandomXFactory, + sha3x_difficulty, + Difficulty, + PowAlgorithm, + PowError, + }, transactions::aggregated_body::AggregateBody, - validation::helpers, + validation::{helpers, ValidationError}, }; const LOG_TARGET: &str = "c::bn::comms_interface::inbound_handler"; @@ -85,6 +92,7 @@ pub struct InboundNodeCommsHandlers { list_of_reconciling_blocks: Arc>>, outbound_nci: OutboundNodeCommsInterface, connectivity: ConnectivityRequester, + randomx_factory: RandomXFactory, } impl InboundNodeCommsHandlers @@ -98,6 +106,7 @@ where B: BlockchainBackend + 'static consensus_manager: ConsensusManager, outbound_nci: OutboundNodeCommsInterface, connectivity: ConnectivityRequester, + randomx_factory: RandomXFactory, ) -> Self { Self { block_event_sender, @@ -107,6 +116,7 @@ where B: BlockchainBackend + 'static list_of_reconciling_blocks: Arc::new(RwLock::new(HashSet::new())), outbound_nci, connectivity, + randomx_factory, } } @@ -438,15 +448,17 @@ where B: BlockchainBackend + 'static } // Lets check if the block exists before we try and ask for a complete block - if self.blockchain_db.block_exists(block_hash).await? { - debug!( - target: LOG_TARGET, - "Block with hash `{}` already stored", - block_hash.to_hex() - ); + if self.check_exists_and_not_bad_block(block_hash).await? { return Ok(()); } + // lets check that the difficulty at least matches the min required difficulty + // We cannot check the target difficulty as orphan blocks dont have a target difficulty. + // All we care here is that bad blocks are not free to make, and that they are more expensive to make then they + // are to validate. As soon as a block can be linked to the main chain, a proper full proof of work check will + // be done before any other validation. + self.check_min_block_difficulty(&new_block).await?; + { // we use a double lock to make sure we can only reconcile one unique block at a time. We may receive the // same block from multiple peer near simultaneously. We should only reconcile each unique block once. @@ -462,14 +474,10 @@ where B: BlockchainBackend + 'static } { let mut write_lock = self.list_of_reconciling_blocks.write().await; - if self.blockchain_db.block_exists(block_hash).await? { - debug!( - target: LOG_TARGET, - "Block with hash `{}` already stored", - block_hash.to_hex() - ); + if self.check_exists_and_not_bad_block(block_hash).await? { return Ok(()); } + if !write_lock.insert(block_hash) { debug!( target: LOG_TARGET, @@ -498,6 +506,51 @@ where B: BlockchainBackend + 'static Ok(()) } + async fn check_min_block_difficulty(&self, new_block: &NewBlock) -> Result<(), CommsInterfaceError> { + let constants = self.consensus_manager.consensus_constants(new_block.header.height); + let min_difficulty = constants.min_pow_difficulty(new_block.header.pow.pow_algo); + let achieved = match new_block.header.pow_algo() { + PowAlgorithm::RandomX => randomx_difficulty(&new_block.header, &self.randomx_factory)?, + PowAlgorithm::Sha3x => sha3x_difficulty(&new_block.header)?, + }; + if achieved < min_difficulty { + self.blockchain_db + .add_bad_block( + new_block.header.hash(), + self.blockchain_db.get_chain_metadata().await?.height_of_longest_chain(), + ) + .await?; + return Err(CommsInterfaceError::InvalidBlockHeader( + BlockHeaderValidationError::ProofOfWorkError(PowError::AchievedDifficultyBelowMin), + )); + } + Ok(()) + } + + async fn check_exists_and_not_bad_block(&self, block: FixedHash) -> Result { + if self.blockchain_db.block_exists(block).await? { + debug!( + target: LOG_TARGET, + "Block with hash `{}` already stored", + block.to_hex() + ); + return Ok(true); + } + if self.blockchain_db.bad_block_exists(block).await? { + debug!( + target: LOG_TARGET, + "Block with hash `{}` already validated as a bad block", + block.to_hex() + ); + return Err(CommsInterfaceError::ChainStorageError( + ChainStorageError::ValidationError { + source: ValidationError::BadBlockFound { hash: block.to_hex() }, + }, + )); + } + Ok(false) + } + async fn reconcile_and_add_block( &mut self, source_peer: NodeId, @@ -949,6 +1002,7 @@ impl Clone for InboundNodeCommsHandlers { list_of_reconciling_blocks: self.list_of_reconciling_blocks.clone(), outbound_nci: self.outbound_nci.clone(), connectivity: self.connectivity.clone(), + randomx_factory: self.randomx_factory.clone(), } } } diff --git a/base_layer/core/src/base_node/service/initializer.rs b/base_layer/core/src/base_node/service/initializer.rs index a097ab1040..0fa007a23c 100644 --- a/base_layer/core/src/base_node/service/initializer.rs +++ b/base_layer/core/src/base_node/service/initializer.rs @@ -51,6 +51,7 @@ use crate::{ chain_storage::{async_db::AsyncBlockchainDb, BlockchainBackend}, consensus::ConsensusManager, mempool::Mempool, + proof_of_work::randomx_factory::RandomXFactory, proto as shared_protos, proto::base_node as proto, }; @@ -65,6 +66,7 @@ pub struct BaseNodeServiceInitializer { mempool: Mempool, consensus_manager: ConsensusManager, service_request_timeout: Duration, + randomx_factory: RandomXFactory, } impl BaseNodeServiceInitializer @@ -77,6 +79,7 @@ where T: BlockchainBackend mempool: Mempool, consensus_manager: ConsensusManager, service_request_timeout: Duration, + randomx_factory: RandomXFactory, ) -> Self { Self { inbound_message_subscription_factory, @@ -84,6 +87,7 @@ where T: BlockchainBackend mempool, consensus_manager, service_request_timeout, + randomx_factory, } } @@ -175,6 +179,7 @@ where T: BlockchainBackend + 'static let blockchain_db = self.blockchain_db.clone(); let mempool = self.mempool.clone(); let consensus_manager = self.consensus_manager.clone(); + let randomx_factory = self.randomx_factory.clone(); context.spawn_when_ready(move |handles| async move { let dht = handles.expect_handle::(); @@ -190,6 +195,7 @@ where T: BlockchainBackend + 'static consensus_manager, outbound_nci.clone(), connectivity, + randomx_factory, ); let streams = BaseNodeStreams { diff --git a/base_layer/core/src/base_node/sync/header_sync/synchronizer.rs b/base_layer/core/src/base_node/sync/header_sync/synchronizer.rs index 1a471ead99..90a27ffa15 100644 --- a/base_layer/core/src/base_node/sync/header_sync/synchronizer.rs +++ b/base_layer/core/src/base_node/sync/header_sync/synchronizer.rs @@ -515,7 +515,7 @@ impl<'a, B: BlockchainBackend + 'static> HeaderSynchronizer<'a, B> { header.pow_algo(), header.hash().to_hex(), ); - self.header_validator.validate(header)?; + self.header_validator.validate(header).await?; } debug!( @@ -662,7 +662,7 @@ impl<'a, B: BlockchainBackend + 'static> HeaderSynchronizer<'a, B> { continue; } let current_height = header.height; - last_total_accumulated_difficulty = self.header_validator.validate(header)?; + last_total_accumulated_difficulty = self.header_validator.validate(header).await?; if has_switched_to_new_chain { // If we've switched to the new chain, we simply commit every COMMIT_EVERY_N_HEADERS headers diff --git a/base_layer/core/src/base_node/sync/header_sync/validator.rs b/base_layer/core/src/base_node/sync/header_sync/validator.rs index 6073119760..56f1601d47 100644 --- a/base_layer/core/src/base_node/sync/header_sync/validator.rs +++ b/base_layer/core/src/base_node/sync/header_sync/validator.rs @@ -27,12 +27,12 @@ use tari_utilities::{epoch_time::EpochTime, hex::Hex}; use crate::{ base_node::sync::BlockHeaderSyncError, - blocks::{BlockHeader, BlockHeaderAccumulatedData, ChainHeader}, + blocks::{BlockHeader, BlockHeaderAccumulatedData, BlockHeaderValidationError, ChainHeader}, chain_storage::{async_db::AsyncBlockchainDb, BlockchainBackend, ChainStorageError, TargetDifficulties}, common::rolling_vec::RollingVec, consensus::ConsensusManager, proof_of_work::{randomx_factory::RandomXFactory, PowAlgorithm}, - validation::{header::HeaderFullValidator, DifficultyCalculator, HeaderChainLinkedValidator}, + validation::{header::HeaderFullValidator, DifficultyCalculator, HeaderChainLinkedValidator, ValidationError}, }; const LOG_TARGET: &str = "c::bn::header_sync"; @@ -109,7 +109,7 @@ impl BlockHeaderSyncValidator { self.valid_headers().last() } - pub fn validate(&mut self, header: BlockHeader) -> Result { + pub async fn validate(&mut self, header: BlockHeader) -> Result { let state = self.state(); let constants = self.consensus_rules.consensus_constants(header.height); @@ -118,7 +118,7 @@ impl BlockHeaderSyncValidator { constants.max_pow_difficulty(header.pow_algo()), ); - let achieved_target = { + let result = { let txn = self.db.inner().db_read_access()?; self.validator.validate( &*txn, @@ -126,7 +126,30 @@ impl BlockHeaderSyncValidator { &state.previous_header, &state.timestamps, Some(target_difficulty), - )? + ) + }; + let achieved_target = match result { + Ok(achieved_target) => achieved_target, + // future timelimit validation can succeed at a later time. As the block is not yet valid, we discard it + // for now and ban the peer, but wont blacklist the block. + Err(e @ ValidationError::BlockHeaderError(BlockHeaderValidationError::InvalidTimestampFutureTimeLimit)) => { + return Err(e.into()) + }, + // We dont want to mark a block as bad for internal failures + Err( + e @ ValidationError::FatalStorageError(_) | + e @ ValidationError::NotEnoughTimestamps { .. } | + e @ ValidationError::AsyncTaskFailed(_), + ) => return Err(e.into()), + // We dont have to mark the block twice + Err(e @ ValidationError::BadBlockFound { .. }) => return Err(e.into()), + + Err(e) => { + let mut txn = self.db.write_transaction(); + txn.insert_bad_block(header.hash(), header.height); + txn.commit().await?; + return Err(e.into()); + }, }; // Header is valid, add this header onto the validation state for the next round @@ -299,11 +322,11 @@ mod test { validator.initialize_state(tip.hash()).await.unwrap(); assert!(validator.valid_headers().is_empty()); let next = BlockHeader::from_previous(tip.header()); - validator.validate(next).unwrap(); + validator.validate(next).await.unwrap(); assert_eq!(validator.valid_headers().len(), 1); let tip = validator.valid_headers().last().cloned().unwrap(); let next = BlockHeader::from_previous(tip.header()); - validator.validate(next).unwrap(); + validator.validate(next).await.unwrap(); assert_eq!(validator.valid_headers().len(), 2); } @@ -313,7 +336,7 @@ mod test { validator.initialize_state(tip.hash()).await.unwrap(); let mut next = BlockHeader::from_previous(tip.header()); next.height = 14; - let err = validator.validate(next).unwrap_err(); + let err = validator.validate(next).await.unwrap_err(); unpack_enum!(BlockHeaderSyncError::ValidationFailed(val_err) = err); unpack_enum!(ValidationError::BlockHeaderError(header_err) = val_err); unpack_enum!(BlockHeaderValidationError::InvalidHeight { actual, expected } = header_err); diff --git a/base_layer/core/src/chain_storage/async_db.rs b/base_layer/core/src/chain_storage/async_db.rs index 13e9e2a8d6..3cf0fe4025 100644 --- a/base_layer/core/src/chain_storage/async_db.rs +++ b/base_layer/core/src/chain_storage/async_db.rs @@ -225,6 +225,8 @@ impl AsyncBlockchainDb { make_async_fn!(bad_block_exists(block_hash: BlockHash) -> bool, "bad_block_exists"); + make_async_fn!(add_bad_block(hash: BlockHash, height: u64) -> (), "add_bad_block"); + make_async_fn!(fetch_block(height: u64, compact: bool) -> HistoricalBlock, "fetch_block"); make_async_fn!(fetch_blocks>(bounds: T, compact: bool) -> Vec, "fetch_blocks"); diff --git a/base_layer/core/src/chain_storage/blockchain_database.rs b/base_layer/core/src/chain_storage/blockchain_database.rs index f6aa26aaf5..8a18837cee 100644 --- a/base_layer/core/src/chain_storage/blockchain_database.rs +++ b/base_layer/core/src/chain_storage/blockchain_database.rs @@ -48,6 +48,7 @@ use crate::{ BlockAccumulatedData, BlockHeader, BlockHeaderAccumulatedData, + BlockHeaderValidationError, ChainBlock, ChainHeader, CompleteDeletedBitmap, @@ -95,6 +96,7 @@ use crate::{ DifficultyCalculator, HeaderChainLinkedValidator, InternalConsistencyValidator, + ValidationError, }, MutablePrunedOutputMmr, PrunedInputMmr, @@ -913,17 +915,17 @@ where B: BlockchainBackend if db.contains(&DbKey::BlockHash(block_hash))? { return Ok(BlockAddResult::BlockExists); } - // Perform orphan block validation. - if let Err(e) = self.validators.orphan.validate_internal_consistency(&block) { - warn!( - target: LOG_TARGET, - "Block #{} ({}) failed validation - {}", - &new_height, - block.hash().to_hex(), - e.to_string() - ); - return Err(e.into()); + if db.bad_block_exists(block_hash)? { + return Err(ChainStorageError::ValidationError { + source: ValidationError::BadBlockFound { + hash: block_hash.to_hex(), + }, + }); } + // the only fast check we can perform that is slightly expensive to fake is a min difficulty check, this is done + // as soon as we receive the block before we do any processing on it. A proper proof of work is done as soon as + // we can link it to the main chain. Full block validation only happens when the proof of work is higher than + // the main chain and we want to add the block to the main chain. let block_add_result = add_block( &mut *db, &self.config, @@ -1089,6 +1091,14 @@ where B: BlockchainBackend db.bad_block_exists(hash) } + /// Adds a block hash to the list of bad blocks so it wont get process again. + pub fn add_bad_block(&self, hash: BlockHash, height: u64) -> Result<(), ChainStorageError> { + let mut db = self.db_write_access()?; + let mut txn = DbTransaction::new(); + txn.insert_bad_block(hash, height); + db.write(txn) + } + /// Atomically commit the provided transaction to the database backend. This function does not update the metadata. pub fn commit(&self, txn: DbTransaction) -> Result<(), ChainStorageError> { let mut db = self.db_write_access()?; @@ -1934,6 +1944,7 @@ fn reorganize_chain( block_hash.to_hex(), e ); + txn.insert_bad_block(block.header().hash(), block.header().height); remove_orphan(backend, block_hash)?; info!(target: LOG_TARGET, "Restoring previous chain after failed reorg."); @@ -2172,7 +2183,30 @@ fn insert_orphan_and_find_new_tips( let timestamp = EpochTime::from(h.timestamp()); prev_timestamps.push(timestamp); } - validator.validate(db, &block.header, parent.header(), &prev_timestamps, None)?; + let result = validator.validate(db, &block.header, parent.header(), &prev_timestamps, None); + match result { + Ok(_) => {}, + // future timelimit validation can succeed at a later time. As the block is not yet valid, we discard it + // for now and ban the peer, but wont blacklist the block. + Err(e @ ValidationError::BlockHeaderError(BlockHeaderValidationError::InvalidTimestampFutureTimeLimit)) => { + return Err(e.into()) + }, + // We dont want to mark a block as bad for internal failures + Err( + e @ ValidationError::FatalStorageError(_) | + e @ ValidationError::NotEnoughTimestamps { .. } | + e @ ValidationError::AsyncTaskFailed(_), + ) => return Err(e.into()), + // We dont have to mark the block twice + Err(e @ ValidationError::BadBlockFound { .. }) => return Err(e.into()), + + Err(e) => { + let mut txn = DbTransaction::new(); + txn.insert_bad_block(block.header.hash(), block.header.height); + db.write(txn)?; + return Err(e.into()); + }, + }; let achieved_target_diff = difficulty_calculator.check_achieved_and_target_difficulty(db, &block.header)?; let accumulated_data = BlockHeaderAccumulatedData::builder(parent.accumulated_data()) diff --git a/base_layer/core/src/proof_of_work/error.rs b/base_layer/core/src/proof_of_work/error.rs index b9b7223618..d1daad5bd4 100644 --- a/base_layer/core/src/proof_of_work/error.rs +++ b/base_layer/core/src/proof_of_work/error.rs @@ -31,6 +31,8 @@ use crate::proof_of_work::Difficulty; pub enum PowError { #[error("ProofOfWorkFailed")] InvalidProofOfWork, + #[error("Achieved difficulty is below the minimum")] + AchievedDifficultyBelowMin, #[error("Target difficulty {target} not achieved. Achieved difficulty: {achieved}")] AchievedDifficultyTooLow { target: Difficulty, achieved: Difficulty }, #[error("Invalid target difficulty (expected: {expected}, got: {got})")] diff --git a/base_layer/core/src/validation/error.rs b/base_layer/core/src/validation/error.rs index 55be8e7dca..eba0bebc41 100644 --- a/base_layer/core/src/validation/error.rs +++ b/base_layer/core/src/validation/error.rs @@ -97,8 +97,6 @@ pub enum ValidationError { InvalidMinedHeight, #[error("Maximum transaction weight exceeded")] MaxTransactionWeightExceeded, - #[error("End of time: {0}")] - EndOfTimeError(String), #[error("Expected block height to be {expected}, but was {block_height}")] IncorrectHeight { expected: u64, block_height: u64 }, #[error("Expected block previous hash to be {expected}, but was {block_hash}")] diff --git a/base_layer/core/src/validation/header/header_full_validator.rs b/base_layer/core/src/validation/header/header_full_validator.rs index 02fe20753c..f700fb26a3 100644 --- a/base_layer/core/src/validation/header/header_full_validator.rs +++ b/base_layer/core/src/validation/header/header_full_validator.rs @@ -67,6 +67,7 @@ impl HeaderChainLinkedValidator for HeaderFullValidator ) -> Result { let constants = self.rules.consensus_constants(header.height); + check_not_bad_block(db, header.hash())?; check_blockchain_version(constants, header.version)?; check_timestamp_count(header, prev_timestamps, constants)?; @@ -75,7 +76,6 @@ impl HeaderChainLinkedValidator for HeaderFullValidator check_height(header, prev_header)?; check_prev_hash(header, prev_header)?; check_timestamp_ftl(header, &self.rules)?; - check_not_bad_block(db, header.hash())?; check_pow_data(header, &self.rules, db)?; let achieved_target = if let Some(target) = target_difficulty { diff --git a/base_layer/core/tests/helpers/nodes.rs b/base_layer/core/tests/helpers/nodes.rs index 3711a22139..f140ee9cdb 100644 --- a/base_layer/core/tests/helpers/nodes.rs +++ b/base_layer/core/tests/helpers/nodes.rs @@ -49,6 +49,7 @@ use tari_core::{ MempoolServiceInitializer, OutboundMempoolServiceInterface, }, + proof_of_work::randomx_factory::RandomXFactory, test_helpers::blockchain::{create_store_with_consensus_and_validators, TempDatabase}, validation::{ mocks::MockValidator, @@ -393,7 +394,7 @@ async fn setup_base_node_services( setup_comms_services(node_identity.clone(), peers, publisher, data_path).await; let mock_state_machine = MockBaseNodeStateMachine::new(); - + let randomx_factory = RandomXFactory::new(2); let handles = StackBuilder::new(shutdown.to_signal()) .add_initializer(RegisterHandle::new(dht)) .add_initializer(RegisterHandle::new(comms.connectivity())) @@ -407,6 +408,7 @@ async fn setup_base_node_services( mempool.clone(), consensus_manager, Duration::from_secs(60), + randomx_factory, )) .add_initializer(MempoolServiceInitializer::new(mempool.clone(), subscription_factory)) .add_initializer(mock_state_machine.get_initializer()) diff --git a/base_layer/core/tests/tests/node_comms_interface.rs b/base_layer/core/tests/tests/node_comms_interface.rs index 32c4487098..2d600de867 100644 --- a/base_layer/core/tests/tests/node_comms_interface.rs +++ b/base_layer/core/tests/tests/node_comms_interface.rs @@ -35,7 +35,7 @@ use tari_core::{ consensus::ConsensusManager, covenants::Covenant, mempool::{Mempool, MempoolConfig}, - proof_of_work::Difficulty, + proof_of_work::{randomx_factory::RandomXFactory, Difficulty}, test_helpers::{ blockchain::{create_store_with_consensus_and_validators_and_config, create_test_blockchain_db}, create_consensus_rules, @@ -72,7 +72,7 @@ async fn inbound_get_metadata() { let (request_sender, _) = reply_channel::unbounded(); let (block_sender, _) = mpsc::unbounded_channel(); let outbound_nci = OutboundNodeCommsInterface::new(request_sender, block_sender.clone()); - + let randomx_factory = RandomXFactory::new(2); let (connectivity, _) = create_connectivity_mock(); let inbound_nch = InboundNodeCommsHandlers::new( block_event_sender, @@ -81,6 +81,7 @@ async fn inbound_get_metadata() { consensus_manager, outbound_nci, connectivity, + randomx_factory, ); let block = store.fetch_block(0, true).unwrap().block().clone(); @@ -107,6 +108,7 @@ async fn inbound_fetch_kernel_by_excess_sig() { let (block_sender, _) = mpsc::unbounded_channel(); let outbound_nci = OutboundNodeCommsInterface::new(request_sender, block_sender.clone()); let (connectivity, _) = create_connectivity_mock(); + let randomx_factory = RandomXFactory::new(2); let inbound_nch = InboundNodeCommsHandlers::new( block_event_sender, store.clone().into(), @@ -114,6 +116,7 @@ async fn inbound_fetch_kernel_by_excess_sig() { consensus_manager, outbound_nci, connectivity, + randomx_factory, ); let block = store.fetch_block(0, true).unwrap().block().clone(); let sig = block.body.kernels()[0].excess_sig.clone(); @@ -140,6 +143,7 @@ async fn inbound_fetch_headers() { let (block_sender, _) = mpsc::unbounded_channel(); let outbound_nci = OutboundNodeCommsInterface::new(request_sender, block_sender); let (connectivity, _) = create_connectivity_mock(); + let randomx_factory = RandomXFactory::new(2); let inbound_nch = InboundNodeCommsHandlers::new( block_event_sender, store.clone().into(), @@ -147,6 +151,7 @@ async fn inbound_fetch_headers() { consensus_manager, outbound_nci, connectivity, + randomx_factory, ); let header = store.fetch_block(0, true).unwrap().header().clone(); @@ -171,6 +176,7 @@ async fn inbound_fetch_utxos() { let (block_sender, _) = mpsc::unbounded_channel(); let outbound_nci = OutboundNodeCommsInterface::new(request_sender, block_sender); let (connectivity, _) = create_connectivity_mock(); + let randomx_factory = RandomXFactory::new(2); let inbound_nch = InboundNodeCommsHandlers::new( block_event_sender, store.clone().into(), @@ -178,6 +184,7 @@ async fn inbound_fetch_utxos() { consensus_manager, outbound_nci, connectivity, + randomx_factory, ); let block = store.fetch_block(0, true).unwrap().block().clone(); let utxo_1 = block.body.outputs()[0].clone(); @@ -218,6 +225,7 @@ async fn inbound_fetch_blocks() { let (block_sender, _) = mpsc::unbounded_channel(); let outbound_nci = OutboundNodeCommsInterface::new(request_sender, block_sender); let (connectivity, _) = create_connectivity_mock(); + let randomx_factory = RandomXFactory::new(2); let inbound_nch = InboundNodeCommsHandlers::new( block_event_sender, store.clone().into(), @@ -225,6 +233,7 @@ async fn inbound_fetch_blocks() { consensus_manager, outbound_nci, connectivity, + randomx_factory, ); let block = store.fetch_block(0, true).unwrap().block().clone(); @@ -270,6 +279,7 @@ async fn inbound_fetch_blocks_before_horizon_height() { let (block_sender, _) = mpsc::unbounded_channel(); let outbound_nci = OutboundNodeCommsInterface::new(request_sender, block_sender); let (connectivity, _) = create_connectivity_mock(); + let randomx_factory = RandomXFactory::new(2); let inbound_nch = InboundNodeCommsHandlers::new( block_event_sender, store.clone().into(), @@ -277,6 +287,7 @@ async fn inbound_fetch_blocks_before_horizon_height() { consensus_manager.clone(), outbound_nci, connectivity, + randomx_factory, ); let script = script!(Nop); let amount = MicroMinotari(10_000);