sui_core/authority/
authority_test_utils.rs

1// Copyright (c) 2021, Facebook, Inc. and its affiliates
2// Copyright (c) Mysten Labs, Inc.
3// SPDX-License-Identifier: Apache-2.0
4
5use fastcrypto::hash::MultisetHash;
6use fastcrypto::traits::KeyPair;
7use sui_types::base_types::FullObjectRef;
8use sui_types::crypto::{AccountKeyPair, AuthorityKeyPair};
9use sui_types::utils::to_sender_signed_transaction;
10
11use super::shared_object_version_manager::AssignedVersions;
12use super::test_authority_builder::TestAuthorityBuilder;
13use super::*;
14
15#[cfg(test)]
16use super::shared_object_version_manager::{AssignedTxAndVersions, Schedulable};
17
18// =============================================================================
19// MFP (Mysticeti Fast Path) Test Helpers
20//
21// The MFP transaction flow is:
22//   1. Client signs transaction and submits to a validator.
23//   2. The validator validates transaction and submits it to consensus.
24//   3. Consensus finalizes the transaction and outputs it in a commit.
25//   4. Transactions in the commit are filtered, sequenced and processed. Then they are sent to execution.
26//
27// =============================================================================
28
29/// Validates a transaction.
30/// This is the MFP "voting" phase - similar to what happens when a validator
31/// receives a transaction before submitting to consensus.
32///
33/// Returns the verified transaction ready for consensus submission.
34pub fn vote_transaction(
35    authority: &AuthorityState,
36    transaction: Transaction,
37) -> Result<VerifiedTransaction, SuiError> {
38    let epoch_store = authority.load_epoch_store_one_call_per_task();
39    transaction.validity_check(&epoch_store.tx_validity_check_context())?;
40    let verified_tx = epoch_store
41        .verify_transaction_require_no_aliases(transaction)?
42        .into_tx();
43
44    // Validate the transaction.
45    authority.handle_vote_transaction(&epoch_store, verified_tx.clone())?;
46
47    Ok(verified_tx)
48}
49
50/// Creates a VerifiedExecutableTransaction from a signed transaction.
51/// This validates the transaction, votes on it, and creates an executable
52/// as if it came out of consensus.
53pub fn create_executable_transaction(
54    authority: &AuthorityState,
55    transaction: Transaction,
56) -> Result<VerifiedExecutableTransaction, SuiError> {
57    let epoch_store = authority.load_epoch_store_one_call_per_task();
58    let verified_tx = vote_transaction(authority, transaction)?;
59    Ok(VerifiedExecutableTransaction::new_from_consensus(
60        verified_tx,
61        epoch_store.epoch(),
62    ))
63}
64
65/// Submits a transaction to consensus for ordering and version assignment.
66/// This only simulates the consensus submission process by assigning versions
67/// to shared objects.
68///
69/// Returns the executable transaction (now certified by consensus) and assigned versions.
70/// The transaction is NOT automatically executed - use `execute_from_consensus` for that.
71pub async fn submit_to_consensus(
72    authority: &AuthorityState,
73    transaction: Transaction,
74) -> Result<(VerifiedExecutableTransaction, AssignedVersions), SuiError> {
75    let epoch_store = authority.load_epoch_store_one_call_per_task();
76
77    // First validate and vote
78    let verified_tx = vote_transaction(authority, transaction)?;
79
80    // Create executable - the transaction is now "certified" by consensus
81    let executable =
82        VerifiedExecutableTransaction::new_from_consensus(verified_tx, epoch_store.epoch());
83
84    // Assign shared object versions
85    let assigned_versions = authority
86        .epoch_store_for_testing()
87        .assign_shared_object_versions_for_tests(
88            authority.get_object_cache_reader().as_ref(),
89            &vec![executable.clone()],
90        )?;
91
92    let versions = assigned_versions
93        .into_map()
94        .get(&executable.key())
95        .cloned()
96        .unwrap_or_else(|| AssignedVersions::new(vec![], None));
97
98    Ok((executable, versions))
99}
100
101/// Executes a transaction that has already been sequenced through consensus.
102pub async fn execute_from_consensus(
103    authority: &AuthorityState,
104    executable: VerifiedExecutableTransaction,
105    assigned_versions: AssignedVersions,
106) -> (TransactionEffects, Option<ExecutionError>) {
107    let env = ExecutionEnv::new().with_assigned_versions(assigned_versions);
108    authority.execution_scheduler.enqueue(
109        vec![(executable.clone().into(), env.clone())],
110        &authority.epoch_store_for_testing(),
111    );
112
113    let (result, execution_error_opt) = authority
114        .try_execute_executable_for_test(&executable, env)
115        .await;
116    let effects = result.inner().data().clone();
117    (effects, execution_error_opt)
118}
119
120/// This is the primary test helper for executing transactions end-to-end.
121///
122/// Returns the executable transaction and signed effects.
123pub async fn submit_and_execute(
124    authority: &AuthorityState,
125    transaction: Transaction,
126) -> Result<(VerifiedExecutableTransaction, SignedTransactionEffects), SuiError> {
127    submit_and_execute_with_options(authority, None, transaction, false).await
128}
129
130/// Options:
131/// - `fullnode`: Optionally sync and execute on a fullnode as well
132/// - `with_shared`: Whether the transaction involves shared objects (triggers version assignment)
133pub async fn submit_and_execute_with_options(
134    authority: &AuthorityState,
135    fullnode: Option<&AuthorityState>,
136    transaction: Transaction,
137    with_shared: bool,
138) -> Result<(VerifiedExecutableTransaction, SignedTransactionEffects), SuiError> {
139    let (exec, effects, _) =
140        submit_and_execute_with_error(authority, fullnode, transaction, with_shared).await?;
141    Ok((exec, effects))
142}
143
144/// Complete MFP flow returning execution error if any.
145pub async fn submit_and_execute_with_error(
146    authority: &AuthorityState,
147    fullnode: Option<&AuthorityState>,
148    transaction: Transaction,
149    with_shared: bool,
150) -> Result<
151    (
152        VerifiedExecutableTransaction,
153        SignedTransactionEffects,
154        Option<ExecutionError>,
155    ),
156    SuiError,
157> {
158    let epoch_store = authority.load_epoch_store_one_call_per_task();
159
160    // Vote on the transaction.
161    let verified_tx = vote_transaction(authority, transaction)?;
162
163    // Create executable - transaction is now certified by consensus
164    let executable =
165        VerifiedExecutableTransaction::new_from_consensus(verified_tx, epoch_store.epoch());
166
167    // Assign shared object versions if needed
168    let assigned_versions = if with_shared {
169        let versions = authority
170            .epoch_store_for_testing()
171            .assign_shared_object_versions_for_tests(
172                authority.get_object_cache_reader().as_ref(),
173                &vec![executable.clone()],
174            )?;
175        versions
176            .into_map()
177            .get(&executable.key())
178            .cloned()
179            .unwrap_or_else(|| AssignedVersions::new(vec![], None))
180    } else {
181        AssignedVersions::new(vec![], None)
182    };
183
184    // State accumulator for validation
185    let state_acc =
186        GlobalStateHasher::new_for_tests(authority.get_global_state_hash_store().clone());
187    let include_wrapped_tombstone = !authority
188        .epoch_store_for_testing()
189        .protocol_config()
190        .simplified_unwrap_then_delete();
191    let mut state =
192        state_acc.accumulate_cached_live_object_set_for_testing(include_wrapped_tombstone);
193
194    // Execute
195    let env = ExecutionEnv::new().with_assigned_versions(assigned_versions.clone());
196    let (result, execution_error_opt) = authority
197        .try_execute_executable_for_test(&executable, env.clone())
198        .await;
199
200    // Validate state accumulation
201    let state_after =
202        state_acc.accumulate_cached_live_object_set_for_testing(include_wrapped_tombstone);
203    let effects_acc = state_acc.accumulate_effects(
204        &[result.inner().data().clone()],
205        epoch_store.protocol_config(),
206    );
207    state.union(&effects_acc);
208    assert_eq!(state_after.digest(), state.digest());
209
210    // Execute on fullnode if provided
211    if let Some(fullnode) = fullnode {
212        fullnode
213            .try_execute_executable_for_test(&executable, env)
214            .await;
215    }
216
217    Ok((executable, result.into_inner(), execution_error_opt))
218}
219
220/// Enqueues multiple transactions for execution after they've been through consensus.
221pub async fn enqueue_and_execute_all(
222    authority: &AuthorityState,
223    executables: Vec<(VerifiedExecutableTransaction, ExecutionEnv)>,
224) -> Result<Vec<TransactionEffects>, SuiError> {
225    authority.execution_scheduler.enqueue(
226        executables
227            .iter()
228            .map(|(exec, env)| (exec.clone().into(), env.clone()))
229            .collect(),
230        &authority.epoch_store_for_testing(),
231    );
232    let mut output = Vec::new();
233    for (exec, _) in executables {
234        let effects = authority.notify_read_effects("", *exec.digest()).await?;
235        output.push(effects);
236    }
237    Ok(output)
238}
239
240/// Submits a transaction to consensus and schedules for execution.
241/// Returns assigned versions. Execution happens asynchronously.
242pub async fn submit_and_schedule(
243    authority: &AuthorityState,
244    transaction: Transaction,
245) -> Result<AssignedVersions, SuiError> {
246    let (executable, versions) = submit_to_consensus(authority, transaction).await?;
247
248    let env = ExecutionEnv::new().with_assigned_versions(versions.clone());
249    authority.execution_scheduler().enqueue_transactions(
250        vec![(executable, env)],
251        &authority.epoch_store_for_testing(),
252    );
253
254    Ok(versions)
255}
256
257pub async fn init_state_validator_with_fullnode() -> (Arc<AuthorityState>, Arc<AuthorityState>) {
258    use sui_types::crypto::get_authority_key_pair;
259
260    let validator = TestAuthorityBuilder::new().build().await;
261    let fullnode_key_pair = get_authority_key_pair().1;
262    let fullnode = TestAuthorityBuilder::new()
263        .with_keypair(&fullnode_key_pair)
264        .build()
265        .await;
266    (validator, fullnode)
267}
268
269pub async fn init_state_with_committee(
270    genesis: &Genesis,
271    authority_key: &AuthorityKeyPair,
272) -> Arc<AuthorityState> {
273    TestAuthorityBuilder::new()
274        .with_genesis_and_keypair(genesis, authority_key)
275        .build()
276        .await
277}
278
279pub async fn init_state_with_ids<I: IntoIterator<Item = (SuiAddress, ObjectID)>>(
280    objects: I,
281) -> Arc<AuthorityState> {
282    let state = TestAuthorityBuilder::new().build().await;
283    for (address, object_id) in objects {
284        let obj = Object::with_id_owner_for_testing(object_id, address);
285        state.insert_genesis_object(obj).await;
286    }
287    state
288}
289
290pub async fn init_state_with_ids_and_versions<
291    I: IntoIterator<Item = (SuiAddress, ObjectID, SequenceNumber)>,
292>(
293    objects: I,
294) -> Arc<AuthorityState> {
295    let state = TestAuthorityBuilder::new().build().await;
296    for (address, object_id, version) in objects {
297        let obj = Object::with_id_owner_version_for_testing(
298            object_id,
299            version,
300            Owner::AddressOwner(address),
301        );
302        state.insert_genesis_object(obj).await;
303    }
304    state
305}
306
307pub async fn init_state_with_objects<I: IntoIterator<Item = Object>>(
308    objects: I,
309) -> Arc<AuthorityState> {
310    let dir = tempfile::TempDir::new().unwrap();
311    let network_config = sui_swarm_config::network_config_builder::ConfigBuilder::new(&dir).build();
312    let genesis = network_config.genesis;
313    let keypair = network_config.validator_configs[0]
314        .protocol_key_pair()
315        .copy();
316    init_state_with_objects_and_committee(objects, &genesis, &keypair).await
317}
318
319pub async fn init_state_with_objects_and_committee<I: IntoIterator<Item = Object>>(
320    objects: I,
321    genesis: &Genesis,
322    authority_key: &AuthorityKeyPair,
323) -> Arc<AuthorityState> {
324    let state = init_state_with_committee(genesis, authority_key).await;
325    for o in objects {
326        state.insert_genesis_object(o).await;
327    }
328    state
329}
330
331pub async fn init_state_with_object_id(
332    address: SuiAddress,
333    object: ObjectID,
334) -> Arc<AuthorityState> {
335    init_state_with_ids(std::iter::once((address, object))).await
336}
337
338pub async fn init_state_with_ids_and_expensive_checks<
339    I: IntoIterator<Item = (SuiAddress, ObjectID)>,
340>(
341    objects: I,
342    config: ExpensiveSafetyCheckConfig,
343) -> Arc<AuthorityState> {
344    let state = TestAuthorityBuilder::new()
345        .with_expensive_safety_checks(config)
346        .build()
347        .await;
348    for (address, object_id) in objects {
349        let obj = Object::with_id_owner_for_testing(object_id, address);
350        state.insert_genesis_object(obj).await;
351    }
352    state
353}
354
355pub fn init_transfer_transaction(
356    authority_state: &AuthorityState,
357    sender: SuiAddress,
358    secret: &AccountKeyPair,
359    recipient: SuiAddress,
360    object_ref: ObjectRef,
361    gas_object_ref: ObjectRef,
362    gas_budget: u64,
363    gas_price: u64,
364) -> VerifiedTransaction {
365    let data = TransactionData::new_transfer(
366        recipient,
367        FullObjectRef::from_fastpath_ref(object_ref),
368        sender,
369        gas_object_ref,
370        gas_budget,
371        gas_price,
372    );
373    let tx = to_sender_signed_transaction(data, secret);
374    authority_state
375        .epoch_store_for_testing()
376        .verify_transaction_require_no_aliases(tx)
377        .unwrap()
378        .into_tx()
379}
380
381#[cfg(test)]
382pub async fn submit_batch_to_consensus<C>(
383    authority: &AuthorityState,
384    transactions: &[Transaction],
385    consensus_handler: &mut crate::consensus_handler::ConsensusHandler<C>,
386    captured_transactions: &crate::consensus_test_utils::CapturedTransactions,
387) -> (Vec<Schedulable>, AssignedTxAndVersions)
388where
389    C: crate::checkpoints::CheckpointServiceNotify + Send + Sync + 'static,
390{
391    use crate::consensus_test_utils::TestConsensusCommit;
392    use sui_types::messages_consensus::ConsensusTransaction;
393
394    let consensus_transactions: Vec<ConsensusTransaction> = transactions
395        .iter()
396        .map(|tx| ConsensusTransaction::new_user_transaction_message(&authority.name, tx.clone()))
397        .collect();
398
399    let epoch_store = authority.epoch_store_for_testing();
400    let round = epoch_store.get_highest_pending_checkpoint_height() + 1;
401    let timestamp_ms = epoch_store.epoch_start_state().epoch_start_timestamp_ms();
402    let sub_dag_index = 0;
403
404    let commit =
405        TestConsensusCommit::new(consensus_transactions, round, timestamp_ms, sub_dag_index);
406
407    consensus_handler
408        .handle_consensus_commit_for_test(commit)
409        .await;
410
411    tokio::time::sleep(std::time::Duration::from_millis(100)).await;
412
413    let (scheduled_txns, assigned_tx_and_versions) = {
414        let mut captured = captured_transactions.lock();
415        assert!(
416            !captured.is_empty(),
417            "Expected transactions to be scheduled"
418        );
419        let (scheduled_txns, assigned_tx_and_versions, _) = captured.remove(0);
420        (scheduled_txns, assigned_tx_and_versions)
421    };
422
423    (scheduled_txns, assigned_tx_and_versions)
424}
425
426pub async fn assign_versions_and_schedule(
427    authority: &AuthorityState,
428    executable: &VerifiedExecutableTransaction,
429) -> AssignedVersions {
430    let assigned_versions = authority
431        .epoch_store_for_testing()
432        .assign_shared_object_versions_for_tests(
433            authority.get_object_cache_reader().as_ref(),
434            &vec![executable.clone()],
435        )
436        .unwrap();
437
438    let versions = assigned_versions
439        .into_map()
440        .get(&executable.key())
441        .cloned()
442        .unwrap_or_else(|| AssignedVersions::new(vec![], None));
443
444    let env = ExecutionEnv::new().with_assigned_versions(versions.clone());
445    authority.execution_scheduler().enqueue_transactions(
446        vec![(executable.clone(), env)],
447        &authority.epoch_store_for_testing(),
448    );
449
450    versions
451}
452
453/// Assigns shared object versions for an executable without scheduling for execution.
454/// This is used when you need version assignment but want to control execution separately.
455pub async fn assign_shared_object_versions(
456    authority: &AuthorityState,
457    executable: &VerifiedExecutableTransaction,
458) -> AssignedVersions {
459    let assigned_versions = authority
460        .epoch_store_for_testing()
461        .assign_shared_object_versions_for_tests(
462            authority.get_object_cache_reader().as_ref(),
463            &vec![executable.clone()],
464        )
465        .unwrap();
466
467    assigned_versions
468        .into_map()
469        .get(&executable.key())
470        .cloned()
471        .unwrap_or_else(|| AssignedVersions::new(vec![], None))
472}