From 7ff723e0d41e33186f56931a3ad56dabd48a0146 Mon Sep 17 00:00:00 2001 From: Xun Li Date: Thu, 5 Oct 2023 14:30:43 -0700 Subject: [PATCH] Clean up ConsensusHandler --- .../authority/authority_per_epoch_store.rs | 186 ++++++++++-------- .../src/authority/authority_test_utils.rs | 62 +++--- crates/sui-core/src/consensus_handler.rs | 143 +++----------- crates/sui-core/src/lib.rs | 1 + .../sui-core/src/post_consensus_tx_reorder.rs | 44 +++++ .../src/unit_tests/consensus_tests.rs | 7 +- 6 files changed, 209 insertions(+), 234 deletions(-) create mode 100644 crates/sui-core/src/post_consensus_tx_reorder.rs diff --git a/crates/sui-core/src/authority/authority_per_epoch_store.rs b/crates/sui-core/src/authority/authority_per_epoch_store.rs index 1559eba8ee151..b5cdc099afeb3 100644 --- a/crates/sui-core/src/authority/authority_per_epoch_store.rs +++ b/crates/sui-core/src/authority/authority_per_epoch_store.rs @@ -12,7 +12,8 @@ use parking_lot::RwLock; use parking_lot::{Mutex, RwLockReadGuard, RwLockWriteGuard}; use rocksdb::Options; use serde::{Deserialize, Serialize}; -use std::collections::{HashMap, HashSet}; +use std::cmp::Ordering; +use std::collections::{BTreeSet, HashMap, HashSet}; use std::future::Future; use std::iter; use std::path::{Path, PathBuf}; @@ -41,7 +42,7 @@ use crate::authority::epoch_start_configuration::{EpochFlag, EpochStartConfigura use crate::authority::{AuthorityStore, ResolverWrapper}; use crate::checkpoints::{ BuilderCheckpointSummary, CheckpointCommitHeight, CheckpointServiceNotify, EpochStats, - PendingCheckpoint, + PendingCheckpoint, PendingCheckpointInfo, }; use crate::consensus_handler::{ SequencedConsensusTransaction, SequencedConsensusTransactionKey, @@ -50,12 +51,14 @@ use crate::consensus_handler::{ use crate::epoch::epoch_metrics::EpochMetrics; use crate::epoch::reconfiguration::ReconfigState; use crate::module_cache_metrics::ResolverMetrics; +use crate::post_consensus_tx_reorder::PostConsensusTxReorder; use crate::signature_verifier::*; use crate::stake_aggregator::{GenericMultiStakeAggregator, StakeAggregator}; use move_bytecode_utils::module_cache::SyncModuleCache; use mysten_common::sync::notify_once::NotifyOnce; use mysten_common::sync::notify_read::NotifyRead; use mysten_metrics::monitored_scope; +use narwhal_types::{Round, TimestampMs}; use prometheus::IntCounter; use std::str::FromStr; use sui_execution::{self, Executor}; @@ -1702,7 +1705,7 @@ impl AuthorityPerEpochStore { &self, transaction: SequencedConsensusTransaction, skipped_consensus_txns: &IntCounter, - ) -> Result { + ) -> Option { let _scope = monitored_scope("VerifyConsensusTransaction"); if self .is_consensus_message_processed(&transaction.transaction.key()) @@ -1714,7 +1717,7 @@ impl AuthorityPerEpochStore { "handle_consensus_transaction UserTransaction [skip]", ); skipped_consensus_txns.inc(); - return Err(()); + return None; } // Signatures are verified as part of narwhal payload verification in SuiTxValidator match &transaction.transaction { @@ -1728,7 +1731,7 @@ impl AuthorityPerEpochStore { }) => { if transaction.sender_authority() != data.summary.auth_sig().authority { warn!("CheckpointSignature authority {} does not match narwhal certificate source {}", data.summary.auth_sig().authority, transaction.certificate.origin() ); - return Err(()); + return None; } } SequencedConsensusTransactionKind::External(ConsensusTransaction { @@ -1741,7 +1744,7 @@ impl AuthorityPerEpochStore { authority, transaction.certificate.origin() ); - return Err(()); + return None; } } SequencedConsensusTransactionKind::External(ConsensusTransaction { @@ -1754,7 +1757,7 @@ impl AuthorityPerEpochStore { capabilities.authority, transaction.certificate.origin() ); - return Err(()); + return None; } } SequencedConsensusTransactionKind::External(ConsensusTransaction { @@ -1767,19 +1770,19 @@ impl AuthorityPerEpochStore { authority, transaction.certificate.origin() ); - return Err(()); + return None; } if !check_total_jwk_size(id, jwk) { warn!( "{:?} sent jwk that exceeded max size", transaction.sender_authority().concise() ); - return Err(()); + return None; } } SequencedConsensusTransactionKind::System(_) => {} } - Ok(VerifiedSequencedConsensusTransaction(transaction)) + Some(VerifiedSequencedConsensusTransaction(transaction)) } fn db_batch(&self) -> DBBatch { @@ -1796,37 +1799,102 @@ impl AuthorityPerEpochStore { C: CheckpointServiceNotify, >( self: &'a Arc, - transactions: &[VerifiedSequencedConsensusTransaction], - end_of_publish_transactions: Vec, + transactions: Vec, consensus_stats: &ExecutionIndicesWithStats, checkpoint_service: &Arc, object_store: impl ObjectStore, - ) -> SuiResult<( - Vec, - ConsensusCommitBatch<'a, C>, - )> { + commit_round: Round, + commit_timestamp: TimestampMs, + skipped_consensus_txns: &IntCounter, + ) -> SuiResult> { + let verified_transactions: Vec<_> = transactions + .into_iter() + .filter_map(|transaction| { + self.verify_consensus_transaction(transaction, skipped_consensus_txns) + }) + .collect(); + let roots: BTreeSet<_> = verified_transactions + .iter() + .filter_map(|transaction| transaction.0.transaction.executable_transaction_digest()) + .collect(); + let (end_of_publish_transactions, mut sequenced_transactions): (Vec<_>, Vec<_>) = + verified_transactions + .into_iter() + .partition(|transaction| transaction.0.is_end_of_publish()); + + PostConsensusTxReorder::reorder( + &mut sequenced_transactions, + self.protocol_config.consensus_transaction_ordering(), + ); + let mut batch = self.db_batch(); let (transactions_to_schedule, notifications, lock_and_final_round) = self .process_consensus_transactions( &mut batch, - transactions, + &sequenced_transactions, &end_of_publish_transactions, checkpoint_service, object_store, ) .await?; self.record_consensus_commit_stats(&mut batch, consensus_stats)?; - Ok(( - transactions_to_schedule, - ConsensusCommitBatch { - epoch_store: self.clone(), - checkpoint_service: checkpoint_service.clone(), - batch, - notifications, - end_of_publish_transactions, - lock_and_final_round, + + // The last block in this function notifies about new checkpoint if needed + // It's important that we use as_ref() here to make sure we are not dropping the lock. + // The lock needs to be held until the end of this function. + let final_checkpoint_round = lock_and_final_round.as_ref().map(|(_, r)| *r); + let final_checkpoint = match final_checkpoint_round.map(|r| r.cmp(&commit_round)) { + Some(Ordering::Less) => { + debug!( + "Not forming checkpoint for round {} above final checkpoint round {:?}", + commit_round, final_checkpoint_round + ); + return Ok(vec![]); + } + Some(Ordering::Equal) => true, + Some(Ordering::Greater) => false, + None => false, + }; + let pending_checkpoint = PendingCheckpoint { + roots: roots.into_iter().collect(), + details: PendingCheckpointInfo { + timestamp_ms: commit_timestamp, + last_of_epoch: final_checkpoint, + commit_height: commit_round, }, - )) + }; + + self.write_pending_checkpoint(&mut batch, &pending_checkpoint)?; + + batch.write()?; + + self.process_notifications(¬ifications, &end_of_publish_transactions); + + checkpoint_service.notify_checkpoint(&pending_checkpoint)?; + + if final_checkpoint { + info!( + epoch=?self.epoch(), + // Accessing lock_and_final_round on purpose so that the compiler ensures + // the lock is not yet dropped. + last_checkpoint_round=?lock_and_final_round.as_ref().map(|(_, r)| *r), + "Received 2f+1 EndOfPublish messages, notifying last checkpoint" + ); + self.record_end_of_message_quorum_time_metric(); + } + + Ok(transactions_to_schedule) + } + + #[cfg(any(test, feature = "test-utils"))] + fn get_highest_pending_checkpoint_height(&self) -> CheckpointCommitHeight { + self.tables + .pending_checkpoints + .unbounded_iter() + .skip_to_last() + .next() + .map(|(key, _)| key) + .unwrap_or_default() } // Caller is not required to set ExecutionIndices with the right semantics in @@ -1835,30 +1903,22 @@ impl AuthorityPerEpochStore { // process_consensus_transactions_and_commit_boundary(). #[cfg(any(test, feature = "test-utils"))] pub(crate) async fn process_consensus_transactions_for_tests( - &self, - transactions: Vec, + self: &Arc, + transactions: Vec, checkpoint_service: &Arc, object_store: impl ObjectStore, + skipped_consensus_txns: &IntCounter, ) -> SuiResult> { - let mut batch = self.db_batch(); - - let (transactions, end_of_publish_transactions): (Vec<_>, Vec<_>) = transactions - .into_iter() - .partition(|txn| !txn.0.is_end_of_publish()); - - let (certs, notifications, _lock) = self - .process_consensus_transactions( - &mut batch, - &transactions, - &end_of_publish_transactions, - checkpoint_service, - object_store, - ) - .await?; - batch.write()?; - - self.process_notifications(¬ifications, &end_of_publish_transactions); - Ok(certs) + self.process_consensus_transactions_and_commit_boundary( + transactions, + &ExecutionIndicesWithStats::default(), + checkpoint_service, + object_store, + self.get_highest_pending_checkpoint_height() + 1, + 0, + skipped_consensus_txns, + ) + .await } fn process_notifications( @@ -2468,36 +2528,6 @@ impl AuthorityPerEpochStore { } } -pub(crate) struct ConsensusCommitBatch<'a, C> { - epoch_store: Arc, - checkpoint_service: Arc, - batch: DBBatch, - notifications: Vec, - end_of_publish_transactions: Vec, - lock_and_final_round: Option<(parking_lot::RwLockWriteGuard<'a, ReconfigState>, u64)>, -} - -impl<'a, C: CheckpointServiceNotify> ConsensusCommitBatch<'a, C> { - pub fn commit(mut self, pending_checkpoint: PendingCheckpoint) -> SuiResult { - self.epoch_store - .write_pending_checkpoint(&mut self.batch, &pending_checkpoint)?; - - self.batch.write()?; - - self.epoch_store - .process_notifications(&self.notifications, &self.end_of_publish_transactions); - - self.checkpoint_service - .notify_checkpoint(&pending_checkpoint) - } - - pub fn final_checkpoint_round(&self) -> Option { - self.lock_and_final_round - .as_ref() - .map(|(_, final_round)| *final_round) - } -} - impl ExecutionComponents { fn new( protocol_config: &ProtocolConfig, diff --git a/crates/sui-core/src/authority/authority_test_utils.rs b/crates/sui-core/src/authority/authority_test_utils.rs index 78b34979f6ce4..6a66c51c6fe82 100644 --- a/crates/sui-core/src/authority/authority_test_utils.rs +++ b/crates/sui-core/src/authority/authority_test_utils.rs @@ -308,27 +308,21 @@ pub async fn send_consensus(authority: &AuthorityState, cert: &VerifiedCertifica ConsensusTransaction::new_certificate_message(&authority.name, cert.clone().into_inner()), ); - if let Ok(transaction) = authority + let certs = authority .epoch_store_for_testing() - .verify_consensus_transaction(transaction, &authority.metrics.skipped_consensus_txns) - { - let certs = authority - .epoch_store_for_testing() - .process_consensus_transactions_for_tests( - vec![transaction], - &Arc::new(CheckpointServiceNoop {}), - authority.db(), - ) - .await - .unwrap(); - - authority - .transaction_manager() - .enqueue(certs, &authority.epoch_store_for_testing()) - .unwrap(); - } else { - warn!("Failed to verify certificate: {:?}", cert); - } + .process_consensus_transactions_for_tests( + vec![transaction], + &Arc::new(CheckpointServiceNoop {}), + authority.db(), + &authority.metrics.skipped_consensus_txns, + ) + .await + .unwrap(); + + authority + .transaction_manager() + .enqueue(certs, &authority.epoch_store_for_testing()) + .unwrap(); } pub async fn send_consensus_no_execution(authority: &AuthorityState, cert: &VerifiedCertificate) { @@ -336,24 +330,18 @@ pub async fn send_consensus_no_execution(authority: &AuthorityState, cert: &Veri ConsensusTransaction::new_certificate_message(&authority.name, cert.clone().into_inner()), ); - if let Ok(transaction) = authority + // Call process_consensus_transaction() instead of handle_consensus_transaction(), to avoid actually executing cert. + // This allows testing cert execution independently. + authority .epoch_store_for_testing() - .verify_consensus_transaction(transaction, &authority.metrics.skipped_consensus_txns) - { - // Call process_consensus_transaction() instead of handle_consensus_transaction(), to avoid actually executing cert. - // This allows testing cert execution independently. - authority - .epoch_store_for_testing() - .process_consensus_transactions_for_tests( - vec![transaction], - &Arc::new(CheckpointServiceNoop {}), - &authority.db(), - ) - .await - .unwrap(); - } else { - warn!("Failed to verify certificate: {:?}", cert); - } + .process_consensus_transactions_for_tests( + vec![transaction], + &Arc::new(CheckpointServiceNoop {}), + &authority.db(), + &authority.metrics.skipped_consensus_txns, + ) + .await + .unwrap(); } pub fn build_test_modules_with_dep_addr( diff --git a/crates/sui-core/src/consensus_handler.rs b/crates/sui-core/src/consensus_handler.rs index e4ae87ef6cdd7..5ed658517faf1 100644 --- a/crates/sui-core/src/consensus_handler.rs +++ b/crates/sui-core/src/consensus_handler.rs @@ -6,9 +6,7 @@ use crate::authority::authority_per_epoch_store::{ }; use crate::authority::epoch_start_configuration::EpochStartConfigTrait; use crate::authority::AuthorityMetrics; -use crate::checkpoints::{CheckpointServiceNotify, PendingCheckpoint, PendingCheckpointInfo}; -use std::cmp::Ordering; - +use crate::checkpoints::CheckpointServiceNotify; use crate::scoring_decision::update_low_scoring_authorities; use crate::transaction_manager::TransactionManager; use arc_swap::ArcSwap; @@ -22,11 +20,10 @@ use narwhal_executor::{ExecutionIndices, ExecutionState}; use narwhal_types::{BatchAPI, CertificateAPI, ConsensusOutput, HeaderAPI}; use serde::{Deserialize, Serialize}; use std::collections::hash_map::DefaultHasher; -use std::collections::{BTreeSet, HashMap, HashSet}; +use std::collections::{HashMap, HashSet}; use std::hash::{Hash, Hasher}; use std::num::NonZeroUsize; use std::sync::Arc; -use sui_protocol_config::ConsensusTransactionOrdering; use sui_types::authenticator_state::ActiveJwk; use sui_types::base_types::{AuthorityName, EpochId, TransactionDigest}; use sui_types::storage::ObjectStore; @@ -163,11 +160,6 @@ impl Exe return; } - let mut sequenced_transactions = Vec::new(); - let mut end_of_publish_transactions = Vec::new(); - - let mut bytes = 0usize; - /* (serialized, transaction, output_cert) */ let mut transactions = vec![]; let timestamp = consensus_output.sub_dag.commit_timestamp(); @@ -240,6 +232,8 @@ impl Exe .consensus_committed_subdags .with_label_values(&[&leader_author.to_string()]) .inc(); + + let mut bytes = 0usize; for (cert, batches) in consensus_output .sub_dag .certificates @@ -301,9 +295,11 @@ impl Exe } } } + self.metrics + .consensus_handler_processed_bytes + .inc_by(bytes as u64); - let mut roots = BTreeSet::new(); - + let mut all_transactions = Vec::new(); { // We need a set here as well, since the processed_cache is a LRU cache and can drop // entries while we're iterating over the sequenced transactions. @@ -312,10 +308,6 @@ impl Exe for (seq, (serialized, transaction, output_cert)) in transactions.into_iter().enumerate() { - if let Some(digest) = transaction.executable_transaction_digest() { - roots.insert(digest); - } - let index = ExecutionIndices { last_committed_round: round, sub_dag_index: consensus_output.sub_dag.sub_dag_index, @@ -360,87 +352,23 @@ impl Exe continue; } - let Ok(verified_transaction) = self.epoch_store.verify_consensus_transaction( - sequenced_transaction, - &self.metrics.skipped_consensus_txns, - ) else { - continue; - }; - - if verified_transaction.0.is_end_of_publish() { - end_of_publish_transactions.push(verified_transaction); - } else { - sequenced_transactions.push(verified_transaction); - } + all_transactions.push(sequenced_transaction); } } - // TODO: make the reordering algorithm richer and depend on object hotness as well. - // Order transactions based on their gas prices. System transactions without gas price - // are put to the beginning of the sequenced_transactions vector. - if matches!( - self.epoch_store - .protocol_config() - .consensus_transaction_ordering(), - ConsensusTransactionOrdering::ByGasPrice - ) { - let _scope = monitored_scope("HandleConsensusOutput::order_by_gas_price"); - order_by_gas_price(&mut sequenced_transactions); - } - - // (!) Should not add new transactions to sequenced_transactions beyond this point - - self.metrics - .consensus_handler_processed_bytes - .inc_by(bytes as u64); - - let transactions_to_schedule = { - let (transactions_to_schedule, consensus_commit_batch) = self - .epoch_store - .process_consensus_transactions_and_commit_boundary( - &sequenced_transactions, - end_of_publish_transactions, - &self.last_consensus_stats, - &self.checkpoint_service, - &self.object_store, - ) - .await - .expect("Unrecoverable error in consensus handler"); - - // The last block in this function notifies about new checkpoint if needed - let final_checkpoint_round = consensus_commit_batch.final_checkpoint_round(); - let final_checkpoint = match final_checkpoint_round.map(|r| r.cmp(&round)) { - Some(Ordering::Less) => { - debug!( - "Not forming checkpoint for round {} above final checkpoint round {:?}", - round, final_checkpoint_round - ); - return; - } - Some(Ordering::Equal) => true, - Some(Ordering::Greater) => false, - None => false, - }; - let checkpoint = PendingCheckpoint { - roots: roots.into_iter().collect(), - details: PendingCheckpointInfo { - timestamp_ms: timestamp, - last_of_epoch: final_checkpoint, - commit_height: round, - }, - }; - - consensus_commit_batch - .commit(checkpoint) - .expect("Failed to commit consensus commit batch"); - - if final_checkpoint { - info!(epoch=?self.epoch(), "Received 2f+1 EndOfPublish messages, notifying last checkpoint"); - self.epoch_store.record_end_of_message_quorum_time_metric(); - } - - transactions_to_schedule - }; + let transactions_to_schedule = self + .epoch_store + .process_consensus_transactions_and_commit_boundary( + all_transactions, + &self.last_consensus_stats, + &self.checkpoint_service, + &self.object_store, + round, + timestamp, + &self.metrics.skipped_consensus_txns, + ) + .await + .expect("Unrecoverable error in consensus handler"); self.transaction_scheduler .schedule(transactions_to_schedule) @@ -452,23 +380,6 @@ impl Exe } } -fn order_by_gas_price(sequenced_transactions: &mut [VerifiedSequencedConsensusTransaction]) { - sequenced_transactions.sort_by_key(|txn| { - // Reverse order, so that transactions with higher gas price are put to the beginning. - std::cmp::Reverse({ - match &txn.0.transaction { - SequencedConsensusTransactionKind::External(ConsensusTransaction { - tracking_id: _, - kind: ConsensusTransactionKind::UserTransaction(cert), - }) => cert.gas_price(), - // Non-user transactions are considered to have gas price of MAX u64 and are put to the beginning. - // This way consensus commit prologue transactions will stay at the beginning. - _ => u64::MAX, - } - }) - }); -} - struct AsyncTransactionScheduler { sender: tokio::sync::mpsc::Sender>, } @@ -682,6 +593,7 @@ mod tests { use crate::authority::test_authority_builder::TestAuthorityBuilder; use crate::checkpoints::CheckpointServiceNoop; use crate::consensus_adapter::consensus_tests::{test_certificates, test_gas_objects}; + use crate::post_consensus_tx_reorder::PostConsensusTxReorder; use narwhal_config::AuthorityIdentifier; use narwhal_test_utils::latest_protocol_version; use narwhal_types::{ @@ -689,7 +601,8 @@ mod tests { }; use prometheus::Registry; use shared_crypto::intent::Intent; - use sui_protocol_config::SupportedProtocolVersions; + use std::collections::BTreeSet; + use sui_protocol_config::{ConsensusTransactionOrdering, SupportedProtocolVersions}; use sui_types::base_types::{random_object_ref, AuthorityName, SuiAddress}; use sui_types::committee::Committee; use sui_types::messages_consensus::{ @@ -846,7 +759,7 @@ mod tests { #[test] fn test_order_by_gas_price() { let mut v = vec![cap_txn(10), user_txn(42), user_txn(100), cap_txn(1)]; - order_by_gas_price(&mut v); + PostConsensusTxReorder::reorder(&mut v, ConsensusTransactionOrdering::ByGasPrice); assert_eq!( extract(v), vec![ @@ -867,7 +780,7 @@ mod tests { cap_txn(1), user_txn(1000), ]; - order_by_gas_price(&mut v); + PostConsensusTxReorder::reorder(&mut v, ConsensusTransactionOrdering::ByGasPrice); assert_eq!( extract(v), vec![ @@ -890,7 +803,7 @@ mod tests { cap_txn(1), eop_txn(11), ]; - order_by_gas_price(&mut v); + PostConsensusTxReorder::reorder(&mut v, ConsensusTransactionOrdering::ByGasPrice); assert_eq!( extract(v), vec![ diff --git a/crates/sui-core/src/lib.rs b/crates/sui-core/src/lib.rs index 9483c9f467e25..c4a8657952fa6 100644 --- a/crates/sui-core/src/lib.rs +++ b/crates/sui-core/src/lib.rs @@ -18,6 +18,7 @@ mod execution_driver; pub mod metrics; pub mod module_cache_metrics; pub mod narwhal_manager; +pub(crate) mod post_consensus_tx_reorder; pub mod quorum_driver; pub mod safe_client; mod scoring_decision; diff --git a/crates/sui-core/src/post_consensus_tx_reorder.rs b/crates/sui-core/src/post_consensus_tx_reorder.rs new file mode 100644 index 0000000000000..7bccea5d956ea --- /dev/null +++ b/crates/sui-core/src/post_consensus_tx_reorder.rs @@ -0,0 +1,44 @@ +// Copyright (c) Mysten Labs, Inc. +// SPDX-License-Identifier: Apache-2.0 + +use crate::consensus_handler::{ + SequencedConsensusTransactionKind, VerifiedSequencedConsensusTransaction, +}; +use mysten_metrics::monitored_scope; +use sui_protocol_config::ConsensusTransactionOrdering; +use sui_types::messages_consensus::{ConsensusTransaction, ConsensusTransactionKind}; + +pub struct PostConsensusTxReorder {} + +impl PostConsensusTxReorder { + pub fn reorder( + transactions: &mut [VerifiedSequencedConsensusTransaction], + kind: ConsensusTransactionOrdering, + ) { + // TODO: make the reordering algorithm richer and depend on object hotness as well. + // Order transactions based on their gas prices. System transactions without gas price + // are put to the beginning of the sequenced_transactions vector. + match kind { + ConsensusTransactionOrdering::ByGasPrice => Self::order_by_gas_price(transactions), + ConsensusTransactionOrdering::None => (), + } + } + + fn order_by_gas_price(transactions: &mut [VerifiedSequencedConsensusTransaction]) { + let _scope = monitored_scope("HandleConsensusOutput::order_by_gas_price"); + transactions.sort_by_key(|txn| { + // Reverse order, so that transactions with higher gas price are put to the beginning. + std::cmp::Reverse({ + match &txn.0.transaction { + SequencedConsensusTransactionKind::External(ConsensusTransaction { + tracking_id: _, + kind: ConsensusTransactionKind::UserTransaction(cert), + }) => cert.gas_price(), + // Non-user transactions are considered to have gas price of MAX u64 and are put to the beginning. + // This way consensus commit prologue transactions will stay at the beginning. + _ => u64::MAX, + } + }) + }) + } +} diff --git a/crates/sui-core/src/unit_tests/consensus_tests.rs b/crates/sui-core/src/unit_tests/consensus_tests.rs index 07cdf71219b7b..6c2a974107642 100644 --- a/crates/sui-core/src/unit_tests/consensus_tests.rs +++ b/crates/sui-core/src/unit_tests/consensus_tests.rs @@ -4,7 +4,7 @@ use super::*; use crate::authority::{authority_tests::init_state_with_objects, AuthorityState}; use crate::checkpoints::CheckpointServiceNoop; -use crate::consensus_handler::VerifiedSequencedConsensusTransaction; +use crate::consensus_handler::SequencedConsensusTransaction; use move_core_types::{account_address::AccountAddress, ident_str}; use narwhal_types::Transactions; use narwhal_types::TransactionsServer; @@ -124,11 +124,10 @@ async fn submit_transaction_to_consensus_adapter() { ) -> SuiResult { epoch_store .process_consensus_transactions_for_tests( - vec![VerifiedSequencedConsensusTransaction::new_test( - transaction.clone(), - )], + vec![SequencedConsensusTransaction::new_test(transaction.clone())], &Arc::new(CheckpointServiceNoop {}), self.0.db(), + &self.0.metrics.skipped_consensus_txns, ) .await?; Ok(())